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 moves the decoder of the settingswatcher under the mutex
to synchronize its usage and prevent data races because we now
use it in multiple places.

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 GetTenantClusterVersion doesn't find a value for
the version setting it will fatal because this means the
version is not set yet.

Release note: None
Epic: CRDB-10829
  • Loading branch information
healthy-pod committed Jan 19, 2023
1 parent 01032c2 commit d8fdf12
Show file tree
Hide file tree
Showing 5 changed files with 164 additions and 4 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
110 changes: 110 additions & 0 deletions pkg/ccl/serverccl/server_startup_guardrails_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
// 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 */
)

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 */
)

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())
}
}
13 changes: 13 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1483,6 +1483,19 @@ 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 := s.settingsWatcher.GetTenantClusterVersion(ctx, s.execCfg.DB).Version
if s.execCfg.Settings.Version.BinaryVersion().Less(tenantActiveVersion) {
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),
"use a tenant binary whose version is at least %v", tenantActiveVersion)
}

// 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: 2 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
41 changes: 37 additions & 4 deletions pkg/server/settingswatcher/settings_watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,14 @@ 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/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 @@ -40,14 +42,14 @@ type SettingsWatcher struct {
settings *cluster.Settings
f *rangefeed.Factory
stopper *stop.Stopper
dec RowDecoder
storage Storage

overridesMonitor OverridesMonitor

mu struct {
syncutil.Mutex

dec RowDecoder
updater settings.Updater
values map[string]settingsValue
overrides map[string]settings.EncodedValue
Expand Down Expand Up @@ -76,15 +78,16 @@ func New(
stopper *stop.Stopper,
storage Storage, // optional
) *SettingsWatcher {
return &SettingsWatcher{
sw := &SettingsWatcher{
clock: clock,
codec: codec,
settings: settingsToUpdate,
f: f,
stopper: stopper,
dec: MakeRowDecoder(codec),
storage: storage,
}
sw.mu.dec = MakeRowDecoder(codec)
return sw
}

// NewWithOverrides constructs a new SettingsWatcher which allows external
Expand Down Expand Up @@ -233,10 +236,12 @@ func (s *SettingsWatcher) Start(ctx context.Context) error {
func (s *SettingsWatcher) handleKV(
ctx context.Context, kv *roachpb.RangeFeedValue,
) rangefeedbuffer.Event {
name, val, tombstone, err := s.dec.DecodeRow(roachpb.KeyValue{
s.mu.Lock()
name, val, tombstone, err := s.mu.dec.DecodeRow(roachpb.KeyValue{
Key: kv.Key,
Value: kv.Value,
})
s.mu.Unlock()
if err != nil {
log.Warningf(ctx, "failed to decode settings row %v: %v", kv.Key, err)
return nil
Expand Down Expand Up @@ -440,3 +445,31 @@ func (s *SettingsWatcher) GetStorageClusterVersion() clusterversion.ClusterVersi
}
return s.mu.storageClusterVersion
}

// GetTenantClusterVersion reads the value of the "version" setting from the settings
// table in the tenant keyspace and returns it.
func (s *SettingsWatcher) GetTenantClusterVersion(
ctx context.Context, db *kv.DB,
) clusterversion.ClusterVersion {
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 := db.Get(context.Background(), key)
if err != nil {
log.Fatalf(ctx, "failed to get tenant cluster version row: %v", err)
}
if row.Value == nil {
log.Fatal(ctx, "got nil value for tenant cluster version row")
}
s.mu.Lock()
_, val, _, err := s.mu.dec.DecodeRow(roachpb.KeyValue{Key: row.Key, Value: *row.Value})
s.mu.Unlock()
if err != nil {
log.Fatalf(ctx, "failed to decode tenant cluster version row: %v", err)
}
var version clusterversion.ClusterVersion
if err := protoutil.Unmarshal([]byte(val.Value), &version); err != nil {
log.Fatalf(ctx, "failed to unmarshal tenant cluster version row: %v", err)
}
return version
}

0 comments on commit d8fdf12

Please sign in to comment.