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 23, 2023
1 parent 01032c2 commit a9cefae
Show file tree
Hide file tree
Showing 9 changed files with 196 additions and 16 deletions.
2 changes: 2 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 Down Expand Up @@ -88,6 +89,7 @@ go_test(
"@com_github_lib_pq//:pq",
"@com_github_prometheus_client_model//go",
"@com_github_prometheus_common//expfmt",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
"@org_golang_x_crypto//bcrypt",
],
Expand Down
116 changes: 116 additions & 0 deletions pkg/ccl/serverccl/server_startup_guardrails_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
// 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"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/assert"
)

// 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: major, Minor: minor}
}

var tests = []struct {
storageBinaryVersion roachpb.Version
storageBinaryMinSupportedVersion roachpb.Version
tenantBinaryVersion roachpb.Version
tenantBinaryMinSupportedVersion roachpb.Version
expErrMatch string // empty if expecting a nil error
}{
{
storageBinaryVersion: v(21, 2),
storageBinaryMinSupportedVersion: v(21, 1),
tenantBinaryVersion: v(21, 2),
tenantBinaryMinSupportedVersion: v(21, 1),
expErrMatch: "",
},
{
storageBinaryVersion: v(21, 2),
storageBinaryMinSupportedVersion: v(20, 2),
tenantBinaryVersion: v(21, 1),
tenantBinaryMinSupportedVersion: v(20, 2),
expErrMatch: "preventing SQL server from starting because its binary version is too low for the tenant active version: " +
"server binary version = 21.1, tenant active version = 21.2",
},
}

for i, test := range tests {
storageSettings := cluster.MakeTestingClusterSettingsWithVersions(
test.storageBinaryVersion,
test.storageBinaryMinSupportedVersion,
false, /* initializeVersion */
)

// The active version of this server should be equal to its binary version. We ensure this is
// true with an assertion below. This is needed because in some test cases we want to ensure
// the active version of this server is greater than the binary version of the tenant. By knowing
// that the SBV is higher than TBV and SLV is equal to SBV we can be sure that SLV is higher than
// TBV.
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{}),
},
},
})

// Ensure that SLV is equal to SBV.
assert.True(t, s.ClusterSettings().Version.ActiveVersion(context.Background()).Version == test.storageBinaryVersion,
"invalid test state: SLV not equal to SBV")

tenantSettings := cluster.MakeTestingClusterSettingsWithVersions(
test.tenantBinaryVersion,
test.tenantBinaryMinSupportedVersion,
true, /* initializeVersion */
)

// The tenant will be created with an active version equal to SLV.
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())
}
}
16 changes: 16 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1483,6 +1483,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
4 changes: 3 additions & 1 deletion pkg/server/settings_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/settingswatcher"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -143,8 +144,9 @@ func initializeCachedSettings(
ctx context.Context, codec keys.SQLCodec, updater settings.Updater, kvs []roachpb.KeyValue,
) error {
dec := settingswatcher.MakeRowDecoder(codec)
var alloc *tree.DatumAlloc
for _, kv := range kvs {
settings, val, _, err := dec.DecodeRow(kv)
settings, val, _, err := dec.DecodeRow(kv, alloc)
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
50 changes: 42 additions & 8 deletions 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 All @@ -35,13 +38,14 @@ import (
// SettingsWatcher is used to watch for cluster settings changes with a
// rangefeed.
type SettingsWatcher struct {
clock *hlc.Clock
codec keys.SQLCodec
settings *cluster.Settings
f *rangefeed.Factory
stopper *stop.Stopper
dec RowDecoder
storage Storage
clock *hlc.Clock
codec keys.SQLCodec
settings *cluster.Settings
f *rangefeed.Factory
stopper *stop.Stopper
dec RowDecoder
storage Storage
allocForHandleKV tree.DatumAlloc

overridesMonitor OverridesMonitor

Expand Down Expand Up @@ -236,7 +240,7 @@ func (s *SettingsWatcher) handleKV(
name, val, tombstone, err := s.dec.DecodeRow(roachpb.KeyValue{
Key: kv.Key,
Value: kv.Value,
})
}, &s.allocForHandleKV)
if err != nil {
log.Warningf(ctx, "failed to decode settings row %v: %v", kv.Key, err)
return nil
Expand Down Expand Up @@ -440,3 +444,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
}
4 changes: 3 additions & 1 deletion pkg/server/settingswatcher/settings_watcher_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down

0 comments on commit a9cefae

Please sign in to comment.