Skip to content

Commit

Permalink
multitenant: add SQL server startup guardrails
Browse files Browse the repository at this point in the history
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
  • Loading branch information
healthy-pod committed Jan 31, 2023
1 parent 91bdcdd commit d8170d9
Show file tree
Hide file tree
Showing 14 changed files with 258 additions and 30 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/serverccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
Expand All @@ -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",
Expand All @@ -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",
Expand Down
121 changes: 121 additions & 0 deletions pkg/ccl/serverccl/server_startup_guardrails_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
// 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())
}
}
15 changes: 11 additions & 4 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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
Expand All @@ -763,7 +765,7 @@ var versionsSingleton = func() keyedVersions {
first = false
continue
}
rawVersionsSingleton[i].Major += devOffset
rawVersionsSingleton[i].Major += DevOffset
}
}
return rawVersionsSingleton
Expand All @@ -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.
Expand All @@ -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() {
Expand Down
16 changes: 16 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1520,6 +1520,22 @@ 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.
tenantActiveVersion, err := s.settingsWatcher.GetClusterVersionFromStorage(ctx, s.execCfg.DB.Get)
if 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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/settings_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
if err != nil {
return errors.Wrap(err, `while decoding settings data
-this likely indicates the settings table structure or encoding has been altered;
Expand Down
3 changes: 3 additions & 0 deletions pkg/server/settingswatcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
11 changes: 7 additions & 4 deletions pkg/server/settingswatcher/row_decoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand All @@ -45,17 +45,20 @@ 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)
_, _, err := rowenc.DecodeIndexKey(d.codec, types, nameRow, nil, kv.Key)
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))
Expand All @@ -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
}
Expand Down
6 changes: 4 additions & 2 deletions pkg/server/settingswatcher/row_decoder_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 {
Expand All @@ -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)
Expand Down
36 changes: 35 additions & 1 deletion pkg/server/settingswatcher/settings_watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -439,3 +443,33 @@ func (s *SettingsWatcher) GetStorageClusterVersion() clusterversion.ClusterVersi
}
return s.mu.storageClusterVersion
}

// GetFn is a function to retrieve data from the kv store. It is implemented
// by both (*kv.DB).Get and (*kv.Txn).Get.
type GetFn = func(context.Context, interface{}) (kv.KeyValue, error)

// GetClusterVersionFromStorage reads the cluster version from the storage via
// the GetFn.
func (s *SettingsWatcher) GetClusterVersionFromStorage(
ctx context.Context, get GetFn,
) (clusterversion.ClusterVersion, error) {
indexPrefix := s.codec.IndexPrefix(keys.SettingsTableID, uint32(1))
key := encoding.EncodeUvarintAscending(encoding.EncodeStringAscending(indexPrefix, "version"), uint64(0))
log.Infof(ctx, "getting tenant cluster version using key: '%v'", roachpb.Key(key))
row, err := 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
}
Loading

0 comments on commit d8170d9

Please sign in to comment.