Skip to content

Commit

Permalink
upgrade: move most of the upgrade pkg to roachpb.Version
Browse files Browse the repository at this point in the history
A lot of functions and structures were using the more complicated
clusterversion.ClusterVersion where roachpb.Version is all they need.
ClusterVersion is not plain data, so it was awkward to use it as map
keys and such.

Release note: None
Epic: None
  • Loading branch information
andreimatei committed Nov 16, 2022
1 parent 1db137d commit b258d20
Show file tree
Hide file tree
Showing 19 changed files with 181 additions and 199 deletions.
18 changes: 8 additions & 10 deletions pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,25 +252,23 @@ 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,
) 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,
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
13 changes: 6 additions & 7 deletions pkg/server/version_cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -419,16 +419,15 @@ 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,
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
14 changes: 5 additions & 9 deletions pkg/upgrade/system_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,22 +124,18 @@ type SystemUpgrade struct {
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,
) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", cv), nil)
return m.fn(ctx, cv, d)
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)
}
15 changes: 5 additions & 10 deletions pkg/upgrade/tenant_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,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 @@ -100,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,
) error {
ctx = logtags.AddTag(ctx, fmt.Sprintf("upgrade=%s", cv), nil)
return m.fn(ctx, cv, d)
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
14 changes: 7 additions & 7 deletions pkg/upgrade/upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ package upgrade
import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

// Upgrade defines a program to be executed once every node in the cluster is
Expand Down Expand Up @@ -61,7 +61,7 @@ import (
// [3]: truncatedStateMigration
// [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go
type Upgrade interface {
ClusterVersion() clusterversion.ClusterVersion
Version() roachpb.Version
Name() string
internal() // restrict implementations to this package
}
Expand All @@ -72,25 +72,25 @@ type JobDeps interface {

// GetUpgrade returns the upgrade associated with the cluster version
// if one exists.
GetUpgrade(key clusterversion.ClusterVersion) (Upgrade, bool)
GetUpgrade(key roachpb.Version) (Upgrade, bool)

// SystemDeps returns a handle to upgrade dependencies on a system tenant.
SystemDeps() SystemDeps
}

type upgrade struct {
description string
cv clusterversion.ClusterVersion
v roachpb.Version
}

// ClusterVersion makes SystemUpgrade an Upgrade.
func (m *upgrade) ClusterVersion() clusterversion.ClusterVersion {
return m.cv
func (m *upgrade) Version() roachpb.Version {
return m.v
}

// Name returns a human-readable name for this upgrade.
func (m *upgrade) Name() string {
return fmt.Sprintf("Upgrade to %s: %q", m.cv.String(), m.description)
return fmt.Sprintf("Upgrade to %s: %q", m.v.String(), m.description)
}

func (m *upgrade) internal() {}
1 change: 1 addition & 0 deletions pkg/upgrade/upgradejob/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/security/username",
"//pkg/settings/cluster",
"//pkg/sql",
Expand Down
Loading

0 comments on commit b258d20

Please sign in to comment.