Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
91929: startupmigrations: remove useless guard in migration r=andreimatei a=andreimatei

The migration writing the initial value of the cluster version had a conditional about whether it's performing the migration for the system tenant. The migration only runs as the system tenant, courtesy of the clusterWide setting [1]. This patch this replaces the conditional with an assertion.

[1] https://github.com/cockroachdb/cockroach/blob/fd42d376fb5ec42e9d9a30fcdcb210a8a79ef0e2/pkg/startupmigrations/migrations.go#L108

Release note: None
Epic: None

91931: startupmigrations: remove no-op SET CLUSTER SETTING version r=andreimatei a=andreimatei

The migration writing the initial value of the cluster version setting was also performing a `SET CLUSTER SETTING version = v`, where v was the version is had just written in the settings table. This statement was a no-op, because it amounted to an update from v to v [1]. This patch removes the statement.

[1] https://github.com/cockroachdb/cockroach/blob/f87728fee6d16e2a352ac76e7ae9225930032ce6/pkg/upgrade/upgrademanager/manager.go#L167-L171

91998: upgrade: minor cleanups r=andreimatei a=andreimatei

See individual commits.
Extracted from #91627

Release note: None
Epic: None

Co-authored-by: Andrei Matei <[email protected]>
  • Loading branch information
craig[bot] and andreimatei committed Nov 16, 2022
4 parents e3a8850 + 351d89f + b350828 + b258d20 commit 7c7622f
Show file tree
Hide file tree
Showing 40 changed files with 239 additions and 296 deletions.
22 changes: 10 additions & 12 deletions pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,28 +252,26 @@ func TestTenantUpgradeFailure(t *testing.T) {
TestingKnobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
UpgradeManager: &upgrade.TestingKnobs{
ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion {
return []clusterversion.ClusterVersion{{Version: v1}, {Version: v2}}
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{v1, v2}
},
RegistryOverride: func(cv clusterversion.ClusterVersion) (upgrade.Upgrade, bool) {
switch cv.Version {
RegistryOverride: func(v roachpb.Version) (upgrade.Upgrade, bool) {
switch v {
case v1:
return upgrade.NewTenantUpgrade("testing", clusterversion.ClusterVersion{
Version: v1,
},
return upgrade.NewTenantUpgrade("testing",
v1,
upgrades.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps, _ *jobs.Job,
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
return nil
}), true
case v2:
return upgrade.NewTenantUpgrade("testing next", clusterversion.ClusterVersion{
Version: v2,
},
return upgrade.NewTenantUpgrade("testing next",
v2,
upgrades.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps, _ *jobs.Job,
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
tenantStopperChannel <- struct{}{}
return nil
Expand Down
13 changes: 0 additions & 13 deletions pkg/clusterversion/clusterversion.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,19 +265,6 @@ func (cv ClusterVersion) SafeFormat(p redact.SafePrinter, _ rune) {
p.Print(cv.Version)
}

// PrettyPrint returns the value in a format that makes it apparent whether or
// not it is a fence version.
func (cv ClusterVersion) PrettyPrint() string {
// If we're a version greater than v20.2 and have an odd internal version,
// we're a fence version. See fenceVersionFor in pkg/upgrade to understand
// what these are.
fenceVersion := !cv.Version.LessEq(roachpb.Version{Major: 20, Minor: 2}) && (cv.Internal%2) == 1
if !fenceVersion {
return cv.String()
}
return redact.Sprintf("%s%s", cv.String(), "(fence)").StripMarkers()
}

// ClusterVersionImpl implements the settings.ClusterVersionImpl interface.
func (cv ClusterVersion) ClusterVersionImpl() {}

Expand Down
10 changes: 5 additions & 5 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -638,16 +638,16 @@ func ByKey(key Key) roachpb.Version {

// ListBetween returns the list of cluster versions in the range
// (from, to].
func ListBetween(from, to ClusterVersion) []ClusterVersion {
func ListBetween(from, to roachpb.Version) []roachpb.Version {
return listBetweenInternal(from, to, versionsSingleton)
}

func listBetweenInternal(from, to ClusterVersion, vs keyedVersions) []ClusterVersion {
var cvs []ClusterVersion
func listBetweenInternal(from, to roachpb.Version, vs keyedVersions) []roachpb.Version {
var cvs []roachpb.Version
for _, keyedV := range vs {
// Read: "from < keyedV <= to".
if from.Less(keyedV.Version) && keyedV.Version.LessEq(to.Version) {
cvs = append(cvs, ClusterVersion{Version: keyedV.Version})
if from.Less(keyedV.Version) && keyedV.Version.LessEq(to) {
cvs = append(cvs, keyedV.Version)
}
}
return cvs
Expand Down
26 changes: 13 additions & 13 deletions pkg/clusterversion/cockroach_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,27 +119,27 @@ func TestGetVersionsBetween(t *testing.T) {
Version: roachpb.Version{Major: int32(i)},
})
}
cv := func(major int32) ClusterVersion {
return ClusterVersion{Version: roachpb.Version{Major: major}}
v := func(major int32) roachpb.Version {
return roachpb.Version{Major: major}
}
list := func(first, last int32) []ClusterVersion {
var cvs []ClusterVersion
list := func(first, last int32) []roachpb.Version {
var cvs []roachpb.Version
for i := first; i <= last; i++ {
cvs = append(cvs, cv(i))
cvs = append(cvs, v(i))
}
return cvs
}

var tests = []struct {
from, to ClusterVersion
exp []ClusterVersion
from, to roachpb.Version
exp []roachpb.Version
}{
{cv(5), cv(8), list(6, 8)},
{cv(1), cv(1), []ClusterVersion{}},
{cv(7), cv(7), []ClusterVersion{}},
{cv(1), cv(5), list(3, 5)},
{cv(6), cv(12), list(7, 9)},
{cv(4), cv(5), list(5, 5)},
{v(5), v(8), list(6, 8)},
{v(1), v(1), []roachpb.Version{}},
{v(7), v(7), []roachpb.Version{}},
{v(1), v(5), list(3, 5)},
{v(6), v(12), list(7, 9)},
{v(4), v(5), list(5, 5)},
}

for _, test := range tests {
Expand Down
14 changes: 14 additions & 0 deletions pkg/roachpb/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package roachpb

import (
"fmt"
"strconv"
"strings"

Expand Down Expand Up @@ -60,6 +61,19 @@ func (v Version) SafeFormat(p redact.SafePrinter, _ rune) {
p.Printf("%d.%d-%d", v.Major, v.Minor, v.Internal)
}

// PrettyPrint returns the value in a format that makes it apparent whether or
// not it is a fence version.
func (v Version) PrettyPrint() string {
// If we're a version greater than v20.2 and have an odd internal version,
// we're a fence version. See fenceVersionFor in pkg/upgrade to understand
// what these are.
fenceVersion := !v.LessEq(Version{Major: 20, Minor: 2}) && (v.Internal%2) == 1
if !fenceVersion {
return v.String()
}
return fmt.Sprintf("%v(fence)", v)
}

// ParseVersion parses a Version from a string of the form
// "<major>.<minor>-<internal>" where the "-<internal>" is optional. We don't
// use the Patch component, so it is always zero.
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1055,10 +1055,10 @@ func TestAdminAPIEvents(t *testing.T) {
{"create_database", false, 0, false, 3},
{"drop_table", false, 0, false, 2},
{"create_table", false, 0, false, 3},
{"set_cluster_setting", false, 0, false, 4},
{"set_cluster_setting", false, 0, false, 3},
// We use limit=true with no limit here because otherwise the
// expCount will mess up the expected total count below.
{"set_cluster_setting", true, 0, true, 4},
{"set_cluster_setting", true, 0, true, 3},
{"create_table", true, 0, false, 3},
{"create_table", true, -1, false, 3},
{"create_table", true, 2, false, 2},
Expand Down
16 changes: 7 additions & 9 deletions pkg/server/version_cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
Expand Down Expand Up @@ -420,19 +419,18 @@ func TestClusterVersionMixedVersionTooOld(t *testing.T) {
// Inject an upgrade which would run to upgrade the cluster.
// We'll validate that we never create a job for this upgrade.
UpgradeManager: &upgrade.TestingKnobs{
ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion {
return []clusterversion.ClusterVersion{to}
ListBetweenOverride: func(from, to roachpb.Version) []roachpb.Version {
return []roachpb.Version{to}
},
RegistryOverride: func(cv clusterversion.ClusterVersion) (upgrade.Upgrade, bool) {
if !cv.Version.Equal(v1) {
RegistryOverride: func(cv roachpb.Version) (upgrade.Upgrade, bool) {
if !cv.Equal(v1) {
return nil, false
}
return upgrade.NewTenantUpgrade("testing", clusterversion.ClusterVersion{
Version: v1,
},
return upgrade.NewTenantUpgrade("testing",
v1,
upgrades.NoPrecondition,
func(
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps, _ *jobs.Job,
ctx context.Context, version clusterversion.ClusterVersion, deps upgrade.TenantDeps,
) error {
return nil
}), true
Expand Down
10 changes: 4 additions & 6 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1342,18 +1342,16 @@ func (t *logicTest) newCluster(

// If we're injecting fake versions, hook up logic to simulate the end
// version existing.
from := clusterversion.ClusterVersion{Version: cfg.BootstrapVersion}
to := clusterversion.ClusterVersion{Version: cfg.BinaryVersion}
if len(clusterversion.ListBetween(from, to)) == 0 {
if len(clusterversion.ListBetween(cfg.BootstrapVersion, cfg.BinaryVersion)) == 0 {
mm, ok := nodeParams.Knobs.UpgradeManager.(*upgrade.TestingKnobs)
if !ok {
mm = &upgrade.TestingKnobs{}
nodeParams.Knobs.UpgradeManager = mm
}
mm.ListBetweenOverride = func(
from, to clusterversion.ClusterVersion,
) []clusterversion.ClusterVersion {
return []clusterversion.ClusterVersion{to}
from, to roachpb.Version,
) []roachpb.Version {
return []roachpb.Version{to}
}
}
}
Expand Down
51 changes: 14 additions & 37 deletions pkg/startupmigrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,10 @@ func initializeClusterSecret(ctx context.Context, r runner) error {
}

func populateVersionSetting(ctx context.Context, r runner) error {
if !r.codec.ForSystemTenant() {
log.Fatalf(ctx, "populateVersionSetting can only run for the system tenant")
}

var v roachpb.Version
if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
return txn.GetProto(ctx, keys.BootstrapVersionKey, &v)
Expand Down Expand Up @@ -729,46 +733,19 @@ func populateVersionSetting(ctx context.Context, r runner) error {
return err
}

// If this is the system tenant, add the host cluster version override for
// all tenants. This override is used by secondary tenants to observe the
// host cluster version number and ensure that secondary tenants don't
// upgrade to a version beyond the host cluster version. As mentioned above,
// don't retry on conflict.
if r.codec.ForSystemTenant() {
// Tenant ID 0 indicates that we're overriding the value for all
// tenants.
tenantID := tree.NewDInt(0)
if err := r.execAsRoot(
ctx,
"insert-setting",
fmt.Sprintf(`INSERT INTO system.tenant_settings (tenant_id, name, value, "last_updated", "value_type") VALUES (%d, 'version', x'%x', now(), 'm') ON CONFLICT(tenant_id, name) DO NOTHING`, tenantID, b),
); err != nil {
return err
}
}

// NB: We have to run with retry here due to the following "race" condition:
// - We're attempting to the set the cluster version at startup.
// - Setting the cluster version requires all nodes to be up and running, in
// order to push out all relevant version gates.
// - This list of "all nodes" is gathered by looking at all the liveness
// records in KV.
// - When starting a multi-node cluster all at once, nodes other than the
// one being bootstrapped join the cluster using the join RPC.
// - The join RPC results in the creation of a liveness record for the
// joining node, except it starts off in an expired state (leaving it to
// the joining node to heartbeat it for the very first time).
//
// Attempting to set the cluster version at startup, while there also may be
// other nodes trying to join, could then result in failures where the
// migration infrastructure find expired liveness records and gives up. To
// that end we'll simply retry, expecting the joining nodes to "come live"
// before long.
if err := r.execAsRootWithRetry(
ctx, "set-setting", "SET CLUSTER SETTING version = $1", v.String(),
// Add the host cluster version override for all tenants. This override is
// used by secondary tenants to observe the host cluster version number and
// ensure that secondary tenants don't upgrade to a version beyond the host
// cluster version. As mentioned above, don't retry on conflict.
tenantID := tree.NewDInt(0) // Tenant ID 0 indicates that we're overriding the value for all tenants.
if err := r.execAsRoot(
ctx,
"insert-setting",
fmt.Sprintf(`INSERT INTO system.tenant_settings (tenant_id, name, value, "last_updated", "value_type") VALUES (%d, 'version', x'%x', now(), 'm') ON CONFLICT(tenant_id, name) DO NOTHING`, tenantID, b),
); err != nil {
return err
}

return nil
}

Expand Down
17 changes: 6 additions & 11 deletions pkg/upgrade/system_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -122,25 +121,21 @@ type SystemUpgrade struct {

// SystemUpgradeFunc is used to perform kv-level upgrades. It should only be
// run from the system tenant.
type SystemUpgradeFunc func(context.Context, clusterversion.ClusterVersion, SystemDeps, *jobs.Job) error
type SystemUpgradeFunc func(context.Context, clusterversion.ClusterVersion, SystemDeps) error

// NewSystemUpgrade constructs a SystemUpgrade.
func NewSystemUpgrade(
description string, cv clusterversion.ClusterVersion, fn SystemUpgradeFunc,
) *SystemUpgrade {
func NewSystemUpgrade(description string, v roachpb.Version, fn SystemUpgradeFunc) *SystemUpgrade {
return &SystemUpgrade{
upgrade: upgrade{
description: description,
cv: cv,
v: v,
},
fn: fn,
}
}

// Run kickstarts the actual upgrade process for system-level upgrades.
func (m *SystemUpgrade) Run(
ctx context.Context, cv clusterversion.ClusterVersion, d SystemDeps, job *jobs.Job,
) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", cv), nil)
return m.fn(ctx, cv, d, job)
func (m *SystemUpgrade) Run(ctx context.Context, v roachpb.Version, d SystemDeps) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", v), nil)
return m.fn(ctx, clusterversion.ClusterVersion{Version: v}, d)
}
23 changes: 12 additions & 11 deletions pkg/upgrade/tenant_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,13 @@ type TenantDeps struct {

// TenantUpgradeFunc is used to perform sql-level upgrades. It may be run from
// any tenant.
type TenantUpgradeFunc func(context.Context, clusterversion.ClusterVersion, TenantDeps, *jobs.Job) error
//
// NOTE: The upgrade func runs inside a job, and the job can in principle be
// used to checkpoint the upgrade's progress ergonomically. The tenant func used
// to take a reference to the job running it for this purpose, but it was
// removed because it was no longer used and because of testing complications.
// It can be added back, though, if some upgrade needs it again.
type TenantUpgradeFunc func(context.Context, clusterversion.ClusterVersion, TenantDeps) error

// PreconditionFunc is a function run without isolation before attempting an
// upgrade that includes this upgrade. It is used to verify that the
Expand All @@ -77,15 +83,12 @@ var _ Upgrade = (*TenantUpgrade)(nil)

// NewTenantUpgrade constructs a TenantUpgrade.
func NewTenantUpgrade(
description string,
cv clusterversion.ClusterVersion,
precondition PreconditionFunc,
fn TenantUpgradeFunc,
description string, v roachpb.Version, precondition PreconditionFunc, fn TenantUpgradeFunc,
) *TenantUpgrade {
m := &TenantUpgrade{
upgrade: upgrade{
description: description,
cv: cv,
v: v,
},
fn: fn,
precondition: precondition,
Expand All @@ -94,11 +97,9 @@ func NewTenantUpgrade(
}

// Run kick-starts the actual upgrade process for tenant-level upgrades.
func (m *TenantUpgrade) Run(
ctx context.Context, cv clusterversion.ClusterVersion, d TenantDeps, job *jobs.Job,
) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", cv), nil)
return m.fn(ctx, cv, d, job)
func (m *TenantUpgrade) Run(ctx context.Context, v roachpb.Version, d TenantDeps) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", v), nil)
return m.fn(ctx, clusterversion.ClusterVersion{Version: v}, d)
}

// Precondition runs the precondition check if there is one and reports
Expand Down
6 changes: 3 additions & 3 deletions pkg/upgrade/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ package upgrade

import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

// TestingKnobs are knobs to inject behavior into the upgrade manager which
Expand All @@ -22,10 +22,10 @@ type TestingKnobs struct {
// ListBetweenOverride injects an override for `clusterversion.ListBetween()
// in order to run upgrades corresponding to versions which do not
// actually exist.
ListBetweenOverride func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion
ListBetweenOverride func(from, to roachpb.Version) []roachpb.Version

// RegistryOverride is used to inject upgrades for specific cluster versions.
RegistryOverride func(cv clusterversion.ClusterVersion) (Upgrade, bool)
RegistryOverride func(v roachpb.Version) (Upgrade, bool)
}

// ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs.
Expand Down
Loading

0 comments on commit 7c7622f

Please sign in to comment.