Skip to content

Commit

Permalink
clusterversion: require env var to do poison dev upgrades
Browse files Browse the repository at this point in the history
Previously the offsetting of all in-development versions ensured that upgrading to one
of these would mark the cluster as untrusted, dev-version-only, however the fact we did
not offset already released versions meant that one could perform such an upgrade easily,
by simply starting a dev binary in a stable release data directory, as upgrades happen by
default automatically. This could lead to an inadvertent and irreversible conversion of a
cluster to dev versions.

This changes the behavior to default to offsetting _all_ versions, not just the the new ones,
which has the effect of also offset the version _from which_ a binary is willing to upgrade.
This significantly reduces the risk of inadvertently upgrading a cluster to a dev version,
as by default, the dev version will refuse to start in a release-version's data directory.

In some cases however it is useful to start a custom or development build in an existing
data directory, e.g. a snapshot collected from production. For these cases, the env var
COCKROACH_UPGRADE_TO_DEV_VERSION can be used to only offset the second defined version
and above, meaning that the first version, which is typically the minBinaryVersion, is
left alone, and that binary thus considers itself backwards compatible with that older
release version and will thus be willing to start in / join that existing cluster.

Release note: none.

Release justification: bug fix in new functionality.
  • Loading branch information
dt committed Sep 8, 2022
1 parent dadda8f commit 5630389
Show file tree
Hide file tree
Showing 8 changed files with 38 additions and 20 deletions.
1 change: 1 addition & 0 deletions pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
deps = [
"//pkg/roachpb",
"//pkg/settings",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/protoutil",
Expand Down
33 changes: 23 additions & 10 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,10 @@

package clusterversion

import "github.com/cockroachdb/cockroach/pkg/roachpb"
import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
)

// Key is a unique identifier for a version of CockroachDB.
type Key int
Expand Down Expand Up @@ -472,28 +475,38 @@ var rawVersionsSingleton = keyedVersions{
}

const (
// unstableVersionsAbove is a cluster version Key above which any upgrades in
// this version are considered unstable development-only versions if it is not
// negative, and upgrading to them should permanently move a cluster to
// development versions. On master it should be the minted version of the last
// release, while on release branches it can be set to invalidVersionKey to
// disable marking any versions as development versions.
unstableVersionsAbove = V22_1
// developmentBranch should be toggled to false on a release branch once the
// set of versions becomes append-only and associated upgrade implementations
// are frozen. It is always true on the main development branch.
developmentBranch = true

// finalVersion should be set on a release branch to the minted final cluster
// version key, e.g. to V22_2 on the release-22.2 branch once it is minted.
// Setting it has the effect of ensuring no versions are subsequently added.
finalVersion = invalidVersionKey
)

// devVersionsAbove is the version key above which all versions are offset to be
// development version when developmentBranch is true. By default this is all
// versions, by setting this to -1, but an env var can override this, to leave
// the first version un-offset. Doing so means that that version, which is
// generally minBinaryVersion as well, is unchanged, and thus allows upgrading a
// stable release data-dir to a dev version if desired.
var devVersionsAbove Key = func() Key {
if envutil.EnvOrDefaultBool("COCKROACH_UPGRADE_TO_DEV_VERSION", false) {
return invalidVersionKey + 1
}
return invalidVersionKey
}()

var versionsSingleton = func() keyedVersions {
if unstableVersionsAbove > invalidVersionKey {
if developmentBranch {
const devOffset = 1000000
// Throw every version above the last release (which will be none on a release
// branch) 1 million major versions into the future, so any "upgrade" to a
// release branch build will be a downgrade and thus blocked.
for i := range rawVersionsSingleton {
if rawVersionsSingleton[i].Key > unstableVersionsAbove {
if rawVersionsSingleton[i].Key > devVersionsAbove {
rawVersionsSingleton[i].Major += devOffset
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,7 @@ func upgradeNodes(

binary := uploadVersion(ctx, t, c, c.Node(node), newVersion)
settings := install.MakeClusterSettings(install.BinaryOption(binary))
settings.Env = append(settings.Env, "COCKROACH_UPGRADE_TO_DEV_VERSION=true")
c.Start(ctx, t.L(), startOpts, settings, c.Node(node))
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/client_migration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,8 +240,8 @@ func TestMigrateWaitsForApplication(t *testing.T) {
blockApplicationCh := make(chan struct{})

// We're going to be migrating from startV to endV.
startV := roachpb.Version{Major: 41}
endV := roachpb.Version{Major: 42}
startV := roachpb.Version{Major: 1000041}
endV := roachpb.Version{Major: 1000042}

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1207,9 +1207,9 @@ func (t *logicTest) newCluster(
tempStorageDiskLimit := int64(512 << 20) /* 512 MiB */
// MVCC range tombstones are only available in 22.2 or newer.
supportsMVCCRangeTombstones := (t.cfg.BootstrapVersion.Equal(roachpb.Version{}) ||
!t.cfg.BootstrapVersion.Less(roachpb.Version{Major: 22, Minor: 2})) &&
!t.cfg.BootstrapVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull))) &&
(t.cfg.BinaryVersion.Equal(roachpb.Version{}) ||
!t.cfg.BinaryVersion.Less(roachpb.Version{Major: 22, Minor: 2}))
!t.cfg.BinaryVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull)))
ignoreMVCCRangeTombstoneErrors := supportsMVCCRangeTombstones &&
(globalMVCCRangeTombstone || useMVCCRangeTombstonesForPointDeletes)

Expand Down Expand Up @@ -1708,7 +1708,7 @@ CREATE DATABASE test; USE test;
t.Fatal(err)
}

if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(roachpb.Version{Major: 22, Minor: 2}) {
if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(clusterversion.ByKey(clusterversion.SetSystemUsersUserIDColumnNotNull)) {
// Hacky way to create user with an ID if we're on a
// bootstrapped binary less than 22.2. The version gate
// causes the regular CREATE USER to fail since it will not
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/logictestbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/clusterversion",
"//pkg/roachpb",
"//pkg/util",
],
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/logictest/logictestbase/logictestbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util"
)
Expand Down Expand Up @@ -462,8 +463,8 @@ var LogicTestConfigs = []TestClusterConfig{
Name: "local-mixed-22.1-22.2",
NumNodes: 1,
OverrideDistSQLMode: "off",
BootstrapVersion: roachpb.Version{Major: 22, Minor: 1},
BinaryVersion: roachpb.Version{Major: 22, Minor: 2},
BootstrapVersion: clusterversion.ByKey(clusterversion.V22_1),
BinaryVersion: clusterversion.ByKey(clusterversion.PrioritizeSnapshots), //TODO: switch to 22.2.
DisableUpgrade: true,
DeclarativeCorpusCollection: true,
},
Expand Down
7 changes: 4 additions & 3 deletions pkg/upgrade/upgrades/builtins_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,13 +46,14 @@ func TestIsAtLeastVersionBuiltin(t *testing.T) {
)
defer tc.Stopper().Stop(ctx)

v := clusterversion.ByKey(clusterversion.Start22_2).String()
// Check that the builtin returns false when comparing against 22.1-2
// version because we are still on 22.1-0.
sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"false"}})
sqlDB.CheckQueryResults(t, "SELECT crdb_internal.is_at_least_version('"+v+"')", [][]string{{"false"}})

// Run the upgrade.
sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", clusterversion.ByKey(clusterversion.Start22_2).String())
sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", v)

// It should now return true.
sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('22.1-2')", [][]string{{"true"}})
sqlDB.CheckQueryResultsRetry(t, "SELECT crdb_internal.is_at_least_version('"+v+"')", [][]string{{"true"}})
}

0 comments on commit 5630389

Please sign in to comment.