Skip to content

Commit

Permalink
migration: promote clusterversion to first-class citizen...
Browse files Browse the repository at this point in the history
in the external facing API for pkg/migration.

The migration package concerns itself with migrations between cluster
versions, so we should have our API reflect as much. The proto changes
are safe, we haven't had a major release with the previous proto
definitions.

Release note: None
  • Loading branch information
irfansharif committed Nov 27, 2020
1 parent e8c3b90 commit 3783e63
Show file tree
Hide file tree
Showing 8 changed files with 83 additions and 75 deletions.
14 changes: 9 additions & 5 deletions pkg/migration/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,9 +247,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe
// defining migrations is that they'd only care about introducing the next
// version key within pkg/clusterversion, and registering a corresponding
// migration for it here.
var vs = []roachpb.Version{to.Version}
var clusterVersions = []clusterversion.ClusterVersion{to}

for _, version := range vs {
for _, clusterVersion := range clusterVersions {
h := &Helper{Manager: m}

// Push out the version gate to every node in the cluster. Each node
Expand All @@ -260,7 +260,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe
{
// First sanity check that we'll actually be able to perform the
// cluster version bump, cluster-wide.
req := &serverpb.ValidateTargetClusterVersionRequest{Version: &version}
req := &serverpb.ValidateTargetClusterVersionRequest{
ClusterVersion: &clusterVersion,
}
err := h.EveryNode(ctx, "validate-cv", func(ctx context.Context, client serverpb.MigrationClient) error {
_, err := client.ValidateTargetClusterVersion(ctx, req)
return err
Expand All @@ -270,7 +272,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe
}
}
{
req := &serverpb.BumpClusterVersionRequest{Version: &version}
req := &serverpb.BumpClusterVersionRequest{
ClusterVersion: &clusterVersion,
}
err := h.EveryNode(ctx, "bump-cv", func(ctx context.Context, client serverpb.MigrationClient) error {
_, err := client.BumpClusterVersion(ctx, req)
return err
Expand All @@ -285,7 +289,7 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe
// TODO(irfansharif): We'll want to be able to override which migration
// is retrieved here within tests. We could make the registry be a part
// of the manager, and all tests to provide their own.
_ = Registry[version]
_ = Registry[clusterVersion]
}

return nil
Expand Down
10 changes: 5 additions & 5 deletions pkg/migration/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,12 +10,12 @@

package migration

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

// Registry defines the global mapping between a version, and the associated
// migration. The migration is only executed after a cluster-wide bump of the
// version gate.
var Registry = make(map[roachpb.Version]Migration)
// Registry defines the global mapping between a cluster version, and the
// associated migration. The migration is only executed after a cluster-wide
// bump of the version gate.
var Registry = make(map[clusterversion.ClusterVersion]Migration)

func init() {
// TODO(irfansharif): We'll want to register individual migrations with
Expand Down
19 changes: 9 additions & 10 deletions pkg/server/migration.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand All @@ -40,22 +39,22 @@ var _ serverpb.MigrationServer = &migrationServer{}
func (m *migrationServer) ValidateTargetClusterVersion(
ctx context.Context, req *serverpb.ValidateTargetClusterVersionRequest,
) (*serverpb.ValidateTargetClusterVersionResponse, error) {
targetVersion := *req.Version
targetCV := req.ClusterVersion
versionSetting := m.server.ClusterSettings().Version

// We're validating the following:
//
// node's minimum supported version <= target version <= node's binary version
if targetVersion.Less(versionSetting.BinaryMinSupportedVersion()) {
msg := fmt.Sprintf("target version %s less than binary's min supported version %s",
targetVersion, versionSetting.BinaryMinSupportedVersion())
if targetCV.Less(versionSetting.BinaryMinSupportedVersion()) {
msg := fmt.Sprintf("target cluster version %s less than binary's min supported version %s",
targetCV, versionSetting.BinaryMinSupportedVersion())
log.Warningf(ctx, "%s", msg)
return nil, errors.Newf("%s", redact.Safe(msg))
}

if versionSetting.BinaryVersion().Less(targetVersion) {
msg := fmt.Sprintf("binary version %s less than target version %s",
versionSetting.BinaryVersion(), targetVersion)
if versionSetting.BinaryVersion().Less(targetCV.Version) {
msg := fmt.Sprintf("binary version %s less than target cluster version %s",
versionSetting.BinaryVersion(), targetCV)
log.Warningf(ctx, "%s", msg)
return nil, errors.Newf("%s", redact.Safe(msg))
}
Expand All @@ -82,10 +81,10 @@ func (m *migrationServer) BumpClusterVersion(
return nil, err
}

newCV := clusterversion.ClusterVersion{Version: *req.Version}
newCV := *req.ClusterVersion

if err := func() error {
if !prevCV.Version.Less(*req.Version) {
if !prevCV.Less(newCV.Version) {
// Nothing to do.
return nil
}
Expand Down
20 changes: 12 additions & 8 deletions pkg/server/migration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"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/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand All @@ -29,36 +30,39 @@ func TestValidateTargetClusterVersion(t *testing.T) {
v := func(major, minor int32) roachpb.Version {
return roachpb.Version{Major: major, Minor: minor}
}
cv := func(major, minor int32) clusterversion.ClusterVersion {
return clusterversion.ClusterVersion{Version: v(major, minor)}
}

var tests = []struct {
binaryVersion roachpb.Version
binaryMinSupportedVersion roachpb.Version
targetVersion roachpb.Version
targetClusterVersion clusterversion.ClusterVersion
expErrMatch string // empty if expecting a nil error
}{
{
binaryVersion: v(20, 2),
binaryMinSupportedVersion: v(20, 1),
targetVersion: v(20, 1),
targetClusterVersion: cv(20, 1),
expErrMatch: "",
},
{
binaryVersion: v(20, 2),
binaryMinSupportedVersion: v(20, 1),
targetVersion: v(20, 2),
targetClusterVersion: cv(20, 2),
expErrMatch: "",
},
{
binaryVersion: v(20, 2),
binaryMinSupportedVersion: v(20, 1),
targetVersion: v(21, 1),
expErrMatch: "binary version.*less than target version",
targetClusterVersion: cv(21, 1),
expErrMatch: "binary version.*less than target cluster version",
},
{
binaryVersion: v(20, 2),
binaryMinSupportedVersion: v(20, 1),
targetVersion: v(19, 2),
expErrMatch: "target version.*less than binary's min supported version",
targetClusterVersion: cv(19, 2),
expErrMatch: "target cluster version.*less than binary's min supported version",
},
}

Expand All @@ -82,7 +86,7 @@ func TestValidateTargetClusterVersion(t *testing.T) {

migrationServer := s.MigrationServer().(*migrationServer)
req := &serverpb.ValidateTargetClusterVersionRequest{
Version: &test.targetVersion,
ClusterVersion: &test.targetClusterVersion,
}
_, err := migrationServer.ValidateTargetClusterVersion(context.Background(), req)
if !testutils.IsError(err, test.expErrMatch) {
Expand Down
1 change: 1 addition & 0 deletions pkg/server/serverpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/clusterversion",
"//pkg/config/zonepb",
"//pkg/gossip",
"//pkg/jobs/jobspb",
Expand Down
86 changes: 43 additions & 43 deletions pkg/server/serverpb/migration.pb.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 3783e63

Please sign in to comment.