From 6093dd363df78f4c821012cade0a59775c26a0f1 Mon Sep 17 00:00:00 2001 From: healthy-pod Date: Mon, 9 Jan 2023 12:56:08 -0800 Subject: [PATCH] multitenant: add SQL server startup guardrails This code change prevents a SQL server from starting if its binary version is less than the tenant's active version. It makes `RowDecoder.DecodeRow` thread safe by making it accept a `*tree.DatumAlloc` and use a new alloc if its nil. The check runs after we run permanent upgrades because that's when the version setting is written to the settings table of the system tenant (and we want to apply this check to both system and secondary tenants). To guard against future changes to where we set the version, if GetClusterVersionFromStorage doesn't find a value for the version setting it will return an error because this means the version is not set yet. Release note: None Epic: CRDB-10829 --- pkg/ccl/serverccl/BUILD.bazel | 3 + .../server_startup_guardrails_test.go | 123 ++++++++++++++++++ pkg/clusterversion/cockroach_versions.go | 15 ++- pkg/clusterversion/cockroach_versions_test.go | 16 ++- pkg/server/server_sql.go | 20 +++ pkg/server/settings_cache.go | 2 +- pkg/server/settingswatcher/BUILD.bazel | 3 + pkg/server/settingswatcher/row_decoder.go | 11 +- .../row_decoder_external_test.go | 6 +- .../settingswatcher/settings_watcher.go | 33 ++++- .../settings_watcher_external_test.go | 4 +- pkg/sql/catalog/bootstrap/BUILD.bazel | 3 +- ...evious_release.go => previous_releases.go} | 15 +++ pkg/sql/sem/eval/testing_knobs.go | 9 +- pkg/sql/tenant_creation.go | 44 ++++--- 15 files changed, 274 insertions(+), 33 deletions(-) create mode 100644 pkg/ccl/serverccl/server_startup_guardrails_test.go rename pkg/sql/catalog/bootstrap/{previous_release.go => previous_releases.go} (99%) diff --git a/pkg/ccl/serverccl/BUILD.bazel b/pkg/ccl/serverccl/BUILD.bazel index 0916e052002c..f036904a3cc2 100644 --- a/pkg/ccl/serverccl/BUILD.bazel +++ b/pkg/ccl/serverccl/BUILD.bazel @@ -39,6 +39,7 @@ go_test( "role_authentication_test.go", "server_controller_test.go", "server_sql_test.go", + "server_startup_guardrails_test.go", "tenant_decommissioned_host_test.go", "tenant_vars_test.go", ], @@ -50,6 +51,7 @@ go_test( "//pkg/ccl", "//pkg/ccl/kvccl", "//pkg/ccl/utilccl/licenseccl", + "//pkg/clusterversion", "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", @@ -65,6 +67,7 @@ go_test( "//pkg/sql", "//pkg/sql/lexbase", "//pkg/sql/sem/catconstants", + "//pkg/sql/sem/eval", "//pkg/sql/sqlinstance/instancestorage", "//pkg/sql/tests", "//pkg/testutils", diff --git a/pkg/ccl/serverccl/server_startup_guardrails_test.go b/pkg/ccl/serverccl/server_startup_guardrails_test.go new file mode 100644 index 000000000000..fb0571484fc1 --- /dev/null +++ b/pkg/ccl/serverccl/server_startup_guardrails_test.go @@ -0,0 +1,123 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package serverccl + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +// TestServerStartupGuardrails ensures that a SQL server will fail to start if +// its binary version (TBV) is less than the tenant's logical version (TLV). +func TestServerStartupGuardrails(t *testing.T) { + defer leaktest.AfterTest(t)() + + v := func(major, minor int32) roachpb.Version { + return roachpb.Version{Major: clusterversion.DevOffset + major, Minor: minor} + } + + tests := []struct { + storageBinaryVersion roachpb.Version + storageBinaryMinSupportedVersion roachpb.Version + tenantBinaryVersion roachpb.Version + tenantBinaryMinSupportedVersion roachpb.Version + TenantLogicalVersionKey clusterversion.Key + expErrMatch string // empty if expecting a nil error + }{ + // First test case ensures that a tenant server can start if the server binary + // version is not too low for the tenant logical version. + { + storageBinaryVersion: v(22, 2), + storageBinaryMinSupportedVersion: v(22, 1), + tenantBinaryVersion: v(22, 2), + tenantBinaryMinSupportedVersion: v(22, 2), + TenantLogicalVersionKey: clusterversion.V22_2, + expErrMatch: "", + }, + // Second test case ensures that a tenant server is prevented from starting if + // its binary version is too low for the current tenant logical version. + { + storageBinaryVersion: v(22, 2), + storageBinaryMinSupportedVersion: v(22, 1), + tenantBinaryVersion: v(22, 1), + tenantBinaryMinSupportedVersion: v(21, 2), + TenantLogicalVersionKey: clusterversion.V22_2, + expErrMatch: fmt.Sprintf("preventing SQL server from starting because its binary version is too low for the tenant active version: "+ + "server binary version = %v, tenant active version = %v", v(22, 1), v(22, 2)), + }, + } + + for i, test := range tests { + storageSettings := cluster.MakeTestingClusterSettingsWithVersions( + test.storageBinaryVersion, + test.storageBinaryMinSupportedVersion, + false, /* initializeVersion */ + ) + + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + // Disable the default test tenant, since we create one explicitly + // below. + DisableDefaultTestTenant: true, + Settings: storageSettings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: test.storageBinaryVersion, + DisableAutomaticVersionUpgrade: make(chan struct{}), + }, + SQLEvalContext: &eval.TestingKnobs{ + TenantLogicalVersionKeyOverride: test.TenantLogicalVersionKey, + }, + }, + }) + + tenantSettings := cluster.MakeTestingClusterSettingsWithVersions( + test.tenantBinaryVersion, + test.tenantBinaryMinSupportedVersion, + true, /* initializeVersion */ + ) + + // The tenant will be created with an active version equal to the version + // corresponding to TenantLogicalVersionKey. Tenant creation is expected + // to succeed for all test cases but server creation is expected to succeed + // only if tenantBinaryVersion is at least equal to the version corresponding + // to TenantLogicalVersionKey. + tenantServer, err := s.StartTenant(context.Background(), + base.TestTenantArgs{ + Settings: tenantSettings, + TenantID: serverutils.TestTenantID(), + TestingKnobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: test.tenantBinaryVersion, + DisableAutomaticVersionUpgrade: make(chan struct{}), + }, + }, + }) + + if !testutils.IsError(err, test.expErrMatch) { + t.Fatalf("test %d: got error %s, wanted error matching '%s'", i, err, test.expErrMatch) + } + + // Only attempt to stop the tenant if it was started successfully. + if err == nil { + tenantServer.Stopper().Stop(context.Background()) + } + s.Stopper().Stop(context.Background()) + } +} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index cd1c15e13695..160d8d30af71 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -422,6 +422,9 @@ func (k Key) String() string { // previously referenced a < 22.1 version until that check/gate can be removed. const TODOPreV22_1 = V22_1 +// Offset every version +1M major versions into the future if this is a dev branch. +const DevOffset = 1000000 + // rawVersionsSingleton lists all historical versions here in chronological // order, with comments describing what backwards-incompatible features were // introduced. @@ -750,7 +753,6 @@ var versionsSingleton = func() keyedVersions { // which conceptually is actually back down to 2 -- then back to to 1000003, // then on to 1000004, etc. skipFirst := allowUpgradeToDev - const devOffset = 1000000 first := true for i := range rawVersionsSingleton { // VPrimordial versions are not offset; they don't matter for the logic @@ -763,7 +765,7 @@ var versionsSingleton = func() keyedVersions { first = false continue } - rawVersionsSingleton[i].Major += devOffset + rawVersionsSingleton[i].Major += DevOffset } } return rawVersionsSingleton @@ -777,6 +779,10 @@ var versionsSingleton = func() keyedVersions { // simply need to check is that the cluster has upgraded to 23.1. var V23_1 = versionsSingleton[len(versionsSingleton)-1].Key +const ( + BinaryMinSupportedVersionKey = V22_1 +) + // TODO(irfansharif): clusterversion.binary{,MinimumSupported}Version // feels out of place. A "cluster version" and a "binary version" are two // separate concepts. @@ -786,12 +792,13 @@ var ( // version than binaryMinSupportedVersion, then the binary will exit with // an error. This typically trails the current release by one (see top-level // comment). - binaryMinSupportedVersion = ByKey(V22_1) + binaryMinSupportedVersion = ByKey(BinaryMinSupportedVersionKey) + BinaryVersionKey = V23_1 // binaryVersion is the version of this binary. // // This is the version that a new cluster will use when created. - binaryVersion = versionsSingleton[len(versionsSingleton)-1].Version + binaryVersion = ByKey(BinaryVersionKey) ) func init() { diff --git a/pkg/clusterversion/cockroach_versions_test.go b/pkg/clusterversion/cockroach_versions_test.go index 2c98637208f2..1552b9ab28a6 100644 --- a/pkg/clusterversion/cockroach_versions_test.go +++ b/pkg/clusterversion/cockroach_versions_test.go @@ -90,7 +90,7 @@ func TestClusterVersionPrettyPrint(t *testing.T) { } } - var tests = []struct { + tests := []struct { cv ClusterVersion exp string }{ @@ -130,7 +130,7 @@ func TestGetVersionsBetween(t *testing.T) { return cvs } - var tests = []struct { + tests := []struct { from, to roachpb.Version exp []roachpb.Version }{ @@ -155,3 +155,15 @@ func TestGetVersionsBetween(t *testing.T) { } } } + +// TestEnsureConsistentBinaryVersion ensures that BinaryVersionKey maps to a +// version equal to binaryVersion. +func TestEnsureConsistentBinaryVersion(t *testing.T) { + require.Equal(t, ByKey(BinaryVersionKey), binaryVersion) +} + +// TestEnsureConsistentMinBinaryVersion ensures that BinaryMinSupportedVersionKey +// maps to a version equal to binaryMinSupportedVersion. +func TestEnsureConsistentMinBinaryVersion(t *testing.T) { + require.Equal(t, ByKey(BinaryMinSupportedVersionKey), binaryMinSupportedVersion) +} diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 2a3ffb5a8161..3faa257033d1 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/blobs/blobspb" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -1520,6 +1521,25 @@ func (s *SQLServer) preStart( log.Infof(ctx, "done ensuring all necessary startup migrations have run") + // Prevent the server from starting if its binary version is too low + // for the current tenant cluster version. + // This check needs to run after the "version" setting is set in the + // "system.settings" table of this tenant. This includes both system + // and secondary tenants. + var tenantActiveVersion clusterversion.ClusterVersion + if err := s.execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + tenantActiveVersion, err = s.settingsWatcher.GetClusterVersionFromStorage(ctx, txn) + return err + }); err != nil { + return err + } + if s.execCfg.Settings.Version.BinaryVersion().Less(tenantActiveVersion.Version) { + return errors.WithHintf(errors.Newf("preventing SQL server from starting because its binary version "+ + "is too low for the tenant active version: server binary version = %v, tenant active version = %v", + s.execCfg.Settings.Version.BinaryVersion(), tenantActiveVersion.Version), + "use a tenant binary whose version is at least %v", tenantActiveVersion.Version) + } + // Delete all orphaned table leases created by a prior instance of this // node. This also uses SQL. s.leaseMgr.DeleteOrphanedLeases(ctx, orphanedLeasesTimeThresholdNanos) diff --git a/pkg/server/settings_cache.go b/pkg/server/settings_cache.go index 1827571636ad..0067a1a06bd8 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -144,7 +144,7 @@ func initializeCachedSettings( ) error { dec := settingswatcher.MakeRowDecoder(codec) for _, kv := range kvs { - settings, val, _, err := dec.DecodeRow(kv) + settings, val, _, err := dec.DecodeRow(kv, nil /* alloc */) if err != nil { return errors.Wrap(err, `while decoding settings data -this likely indicates the settings table structure or encoding has been altered; diff --git a/pkg/server/settingswatcher/BUILD.bazel b/pkg/server/settingswatcher/BUILD.bazel index a92eadf5a7e7..5fd05663112e 100644 --- a/pkg/server/settingswatcher/BUILD.bazel +++ b/pkg/server/settingswatcher/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", @@ -25,6 +26,7 @@ go_library( "//pkg/sql/rowenc/valueside", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/protoutil", @@ -58,6 +60,7 @@ go_test( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/sem/tree", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/server/settingswatcher/row_decoder.go b/pkg/server/settingswatcher/row_decoder.go index c2169e11a113..55272626d740 100644 --- a/pkg/server/settingswatcher/row_decoder.go +++ b/pkg/server/settingswatcher/row_decoder.go @@ -24,9 +24,9 @@ import ( ) // RowDecoder decodes rows from the settings table. +// It is safe for concurrent use. type RowDecoder struct { codec keys.SQLCodec - alloc tree.DatumAlloc columns []catalog.Column decoder valueside.Decoder } @@ -45,9 +45,12 @@ func MakeRowDecoder(codec keys.SQLCodec) RowDecoder { // present, the setting key will be returned but the value will be zero and the // tombstone bool will be set. func (d *RowDecoder) DecodeRow( - kv roachpb.KeyValue, + kv roachpb.KeyValue, alloc *tree.DatumAlloc, ) (setting string, val settings.EncodedValue, tombstone bool, _ error) { // First we need to decode the setting name field from the index key. + if alloc == nil { + alloc = &tree.DatumAlloc{} + } { types := []*types.T{d.columns[0].GetType()} nameRow := make([]rowenc.EncDatum, 1) @@ -55,7 +58,7 @@ func (d *RowDecoder) DecodeRow( if err != nil { return "", settings.EncodedValue{}, false, errors.Wrap(err, "failed to decode key") } - if err := nameRow[0].EnsureDecoded(types[0], &d.alloc); err != nil { + if err := nameRow[0].EnsureDecoded(types[0], alloc); err != nil { return "", settings.EncodedValue{}, false, err } setting = string(tree.MustBeDString(nameRow[0].Datum)) @@ -70,7 +73,7 @@ func (d *RowDecoder) DecodeRow( return "", settings.EncodedValue{}, false, err } - datums, err := d.decoder.Decode(&d.alloc, bytes) + datums, err := d.decoder.Decode(alloc, bytes) if err != nil { return "", settings.EncodedValue{}, false, err } diff --git a/pkg/server/settingswatcher/row_decoder_external_test.go b/pkg/server/settingswatcher/row_decoder_external_test.go index b718daa4b004..5cc751e1be32 100644 --- a/pkg/server/settingswatcher/row_decoder_external_test.go +++ b/pkg/server/settingswatcher/row_decoder_external_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/settingswatcher" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -70,13 +71,14 @@ func TestRowDecoder(t *testing.T) { rows, err := tc.Server(0).DB().Scan(ctx, k, k.PrefixEnd(), 0 /* maxRows */) require.NoError(t, err) dec := settingswatcher.MakeRowDecoder(keys.SystemSQLCodec) + var alloc *tree.DatumAlloc for _, row := range rows { kv := roachpb.KeyValue{ Key: row.Key, Value: *row.Value, } - k, val, tombstone, err := dec.DecodeRow(kv) + k, val, tombstone, err := dec.DecodeRow(kv, alloc) require.NoError(t, err) require.False(t, tombstone) if exp, ok := toSet[k]; ok { @@ -88,7 +90,7 @@ func TestRowDecoder(t *testing.T) { // Test the tombstone logic while we're here. { kv.Value.Reset() - tombstoneK, val, tombstone, err := dec.DecodeRow(kv) + tombstoneK, val, tombstone, err := dec.DecodeRow(kv, alloc) require.NoError(t, err) require.True(t, tombstone) require.Equal(t, k, tombstoneK) diff --git a/pkg/server/settingswatcher/settings_watcher.go b/pkg/server/settingswatcher/settings_watcher.go index e2ed6632df44..f98df151e706 100644 --- a/pkg/server/settingswatcher/settings_watcher.go +++ b/pkg/server/settingswatcher/settings_watcher.go @@ -17,12 +17,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -114,7 +117,7 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { EndKey: settingsTablePrefix.PrefixEnd(), } s.resetUpdater() - var initialScan = struct { + initialScan := struct { ch chan struct{} done bool err error @@ -232,10 +235,11 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { func (s *SettingsWatcher) handleKV( ctx context.Context, kv *roachpb.RangeFeedValue, ) rangefeedbuffer.Event { + var alloc tree.DatumAlloc name, val, tombstone, err := s.dec.DecodeRow(roachpb.KeyValue{ Key: kv.Key, Value: kv.Value, - }) + }, &alloc) if err != nil { log.Warningf(ctx, "failed to decode settings row %v: %v", kv.Key, err) return nil @@ -439,3 +443,28 @@ func (s *SettingsWatcher) GetStorageClusterVersion() clusterversion.ClusterVersi } return s.mu.storageClusterVersion } + +// GetClusterVersionFromStorage reads the cluster version from the storage via +// the given transaction. +func (s *SettingsWatcher) GetClusterVersionFromStorage( + ctx context.Context, txn *kv.Txn, +) (clusterversion.ClusterVersion, error) { + indexPrefix := s.codec.IndexPrefix(keys.SettingsTableID, uint32(1)) + key := encoding.EncodeUvarintAscending(encoding.EncodeStringAscending(indexPrefix, "version"), uint64(0)) + row, err := txn.Get(ctx, key) + if err != nil { + return clusterversion.ClusterVersion{}, err + } + if row.Value == nil { + return clusterversion.ClusterVersion{}, errors.New("got nil value for tenant cluster version row") + } + _, val, _, err := s.dec.DecodeRow(roachpb.KeyValue{Key: row.Key, Value: *row.Value}, nil /* alloc */) + if err != nil { + return clusterversion.ClusterVersion{}, err + } + var version clusterversion.ClusterVersion + if err := protoutil.Unmarshal([]byte(val.Value), &version); err != nil { + return clusterversion.ClusterVersion{}, err + } + return version, nil +} diff --git a/pkg/server/settingswatcher/settings_watcher_external_test.go b/pkg/server/settingswatcher/settings_watcher_external_test.go index 80fcb7b5f076..bae03e6fc6d7 100644 --- a/pkg/server/settingswatcher/settings_watcher_external_test.go +++ b/pkg/server/settingswatcher/settings_watcher_external_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -488,12 +489,13 @@ func TestStaleRowsDoNotCauseSettingsToRegress(t *testing.T) { rows, err := s.DB().Scan(ctx, k, k.PrefixEnd(), 0 /* maxRows */) require.NoError(t, err) dec := settingswatcher.MakeRowDecoder(codec) + var alloc tree.DatumAlloc for _, r := range rows { rkv := roachpb.KeyValue{Key: r.Key} if r.Value != nil { rkv.Value = *r.Value } - name, _, _, err := dec.DecodeRow(rkv) + name, _, _, err := dec.DecodeRow(rkv, &alloc) require.NoError(t, err) if name == fakeSettingName { rkv.Key, err = codec.StripTenantPrefix(rkv.Key) diff --git a/pkg/sql/catalog/bootstrap/BUILD.bazel b/pkg/sql/catalog/bootstrap/BUILD.bazel index 9534b9d61733..b22601abf30f 100644 --- a/pkg/sql/catalog/bootstrap/BUILD.bazel +++ b/pkg/sql/catalog/bootstrap/BUILD.bazel @@ -6,11 +6,12 @@ go_library( srcs = [ "kv_writer.go", "metadata.go", - "previous_release.go", + "previous_releases.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/config/zonepb", "//pkg/keys", "//pkg/kv", diff --git a/pkg/sql/catalog/bootstrap/previous_release.go b/pkg/sql/catalog/bootstrap/previous_releases.go similarity index 99% rename from pkg/sql/catalog/bootstrap/previous_release.go rename to pkg/sql/catalog/bootstrap/previous_releases.go index 99ad4d282836..37cd318138b2 100644 --- a/pkg/sql/catalog/bootstrap/previous_release.go +++ b/pkg/sql/catalog/bootstrap/previous_releases.go @@ -16,12 +16,27 @@ import ( "encoding/hex" "sort" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" ) +var initialValuesFnByKey = map[clusterversion.Key]initialValuesFn{ + clusterversion.V22_2: InitialValuesForTenantV222, +} + +type initialValuesFn = func( + codec keys.SQLCodec, + defaultZoneConfig *zonepb.ZoneConfig, + defaultSystemZoneConfig *zonepb.ZoneConfig, +) (kvs []roachpb.KeyValue, splits []roachpb.RKey, _ error) + +func GetInitialValuesFn(key clusterversion.Key) initialValuesFn { + return initialValuesFnByKey[key] +} + // InitialValuesForTenantV222 returns the initial values as produced by the // binary for the 22.2 release, for a non-system tenant. The only difference // is that the contents of the system.zones table are overwritten with the new diff --git a/pkg/sql/sem/eval/testing_knobs.go b/pkg/sql/sem/eval/testing_knobs.go index 5136add032a2..18da3b1068b4 100644 --- a/pkg/sql/sem/eval/testing_knobs.go +++ b/pkg/sql/sem/eval/testing_knobs.go @@ -10,7 +10,10 @@ package eval -import "github.com/cockroachdb/cockroach/pkg/base" +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" +) // TestingKnobs contains test knobs. type TestingKnobs struct { @@ -39,6 +42,10 @@ type TestingKnobs struct { ForceProductionValues bool CallbackGenerators map[string]*CallbackValueGenerator + + // We use clusterversion.Key rather than a roachpb.Version because it will be used + // to get initial values to use during bootstrap. + TenantLogicalVersionKeyOverride clusterversion.Key } var _ base.ModuleTestingKnobs = &TestingKnobs{} diff --git a/pkg/sql/tenant_creation.go b/pkg/sql/tenant_creation.go index 9b61a0d36daf..7e6e25d57dcd 100644 --- a/pkg/sql/tenant_creation.go +++ b/pkg/sql/tenant_creation.go @@ -42,6 +42,10 @@ import ( "github.com/cockroachdb/redact" ) +const ( + tenantCreationMinSupportedVersionKey = clusterversion.V22_2 +) + // CreateTenant implements the tree.TenantOperator interface. func (p *planner) CreateTenant( ctx context.Context, parameters string, @@ -133,33 +137,43 @@ func (p *planner) createTenantInternal( codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(tenantID)) var kvs []roachpb.KeyValue var splits []roachpb.RKey - const minVersion = clusterversion.V22_2 - curVersion := clusterversion.V23_1 - if p.EvalContext().Settings.Version.IsActive(ctx, curVersion) { - // The cluster is running the latest version. - // Use this version to create the tenant and bootstrap it using the host - // cluster's bootstrapping logic. - tenantVersion.Version = clusterversion.ByKey(curVersion) - schema := bootstrap.MakeMetadataSchema( + + processNonActiveVersionInitialValues := func(versionKey clusterversion.Key) error { + tenantVersion.Version = clusterversion.ByKey(versionKey) + kvs, splits, err = bootstrap.GetInitialValuesFn(versionKey)( codec, initialTenantZoneConfig, /* defaultZoneConfig */ initialTenantZoneConfig, /* defaultSystemZoneConfig */ ) - kvs, splits = schema.GetInitialValues() - } else { + return err + } + + TenantLogicalVersionKeyOverride := p.EvalContext().TestingKnobs.TenantLogicalVersionKeyOverride + if TenantLogicalVersionKeyOverride != 0 { + // An override was passed using testing knobs. Use this override to get + // the initial values and bootstrap the tenant using them. + if err = processNonActiveVersionInitialValues(TenantLogicalVersionKeyOverride); err != nil { + return tid, err + } + } else if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.BinaryVersionKey) { // The cluster is not running the latest version. // Use the previous major version to create the tenant and bootstrap it // just like the previous major version binary would, using hardcoded // initial values. - tenantVersion.Version = clusterversion.ByKey(minVersion) - kvs, splits, err = bootstrap.InitialValuesForTenantV222( + if err = processNonActiveVersionInitialValues(tenantCreationMinSupportedVersionKey); err != nil { + return tid, err + } + } else { + // The cluster is running the latest version. + // Use this version to create the tenant and bootstrap it using the host + // cluster's bootstrapping logic. + tenantVersion.Version = clusterversion.ByKey(clusterversion.BinaryVersionKey) + schema := bootstrap.MakeMetadataSchema( codec, initialTenantZoneConfig, /* defaultZoneConfig */ initialTenantZoneConfig, /* defaultSystemZoneConfig */ ) - if err != nil { - return tid, err - } + kvs, splits = schema.GetInitialValues() } {