From ebb812e3d71c9428bc1d2dce2594c033f2317262 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Thu, 18 Aug 2022 12:12:22 +0000 Subject: [PATCH] clusterversion: prevent upgrades from master versions This change defines a new "unstableVersionsAbove" point on the cluster version line, above which any cluster versions are considered unstable development-only versions which are still subject to change. Performing an upgrade to a version while it is still unstable leaves a cluster in a state where it persists a version that claims it has done that upgrade and all prior, however those upgrades are still subject to change by nature of being unstable. If it subsequently upgraded to a stable version, this could result in subtle and nearly impossible to detect issues, as being at or above a particular version is used to assume that all subsequent version upgrades _as released_ were run; on a cluster that ran an earlier iteration of an upgrade this does not hold. Thus to prevent clusters which upgrade to development versions from subsequently upgrading to a stable version, we offset all development versions -- those above the unstableVersionsAbove point -- into the far future by adding one million to their major version e.g. v22.x-y becomes 1000022.x-y. This means an attempt to subsequently "upgrade" to a stable version -- such as v22.2 -- will look like a downgrade and be forbidden. On the release branch, prior to starting to publish upgradable releases, the unstableVersionsAbove value should be set to invalidVersionKey to reflect that all version upgrades in that release branch are now considered to be stable, meaning they must be treated as immutable and append-only. Release note (ops change): clusters that are upgraded to an alpha or other manual build from the development branch will not be able to be subsequently upgraded to a release build. Release justification: high-priority change to existing functionality, to allow releasing alphas with known version upgrade bugs while ensuring they do not subsequently upgrade into stable version but silently corrupted clusters. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/backupccl/backup_test.go | 4 +- .../testdata/logic_test/crdb_internal_tenant | 4 +- pkg/clusterversion/cockroach_versions.go | 40 +++++++++++++++++-- pkg/clusterversion/key_string.go | 8 ++-- pkg/kv/kvserver/client_migration_test.go | 2 +- pkg/kv/kvserver/stores.go | 3 +- pkg/sql/crdb_internal_test.go | 2 +- .../testdata/logic_test/crdb_internal | 4 +- pkg/sql/ttl/ttljob/ttljob_test.go | 1 - ...efore_starting_an_upgrade_external_test.go | 3 +- .../wait_for_del_range_in_gc_job_test.go | 2 +- 13 files changed, 57 insertions(+), 20 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 72e07afa86b5..36ec622b9565 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -293,4 +293,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-62 set the active cluster version in the format '.' +version version 1000022.1-62 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index f53ad01bdb55..43bcf46e8b30 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -227,6 +227,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-62set the active cluster version in the format '.' +versionversion1000022.1-62set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index d46e1637b9ce..dde171b6f341 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8228,7 +8228,7 @@ func TestManifestTooNew(t *testing.T) { require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) // Bump the version and write it back out to make it look newer. - backupManifest.ClusterVersion = roachpb.Version{Major: 99, Minor: 1} + backupManifest.ClusterVersion = roachpb.Version{Major: math.MaxInt32, Minor: 1} manifestData, err = protoutil.Marshal(&backupManifest) require.NoError(t, err) require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) @@ -8238,7 +8238,7 @@ func TestManifestTooNew(t *testing.T) { require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) // Verify we reject it. - sqlDB.ExpectErr(t, "backup from version 99.1 is newer than current version", `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) + sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) // Bump the version down and write it back out to make it look older. backupManifest.ClusterVersion = roachpb.Version{Major: 20, Minor: 2, Internal: 2} diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 4108928e244e..3d6473ca810a 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -360,7 +360,7 @@ select crdb_internal.get_vmodule() ยท query T -select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''); +select regexp_replace(regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''), '10000', ''); ---- 22.1 @@ -453,7 +453,7 @@ select * from crdb_internal.gossip_alerts # Anyone can see the executable version. query T -select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''); +select regexp_replace(regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''), '10000', ''); ---- 22.1 diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index dd68b697b4e0..328224f65e51 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -302,8 +302,8 @@ const TODOPreV21_2 = V21_2 // previously referenced a < 22.1 version until that check/gate can be removed. const TODOPreV22_1 = V22_1 -// versionsSingleton lists all historical versions here in chronological order, -// with comments describing what backwards-incompatible features were +// rawVersionsSingleton lists all historical versions here in chronological +// order, with comments describing what backwards-incompatible features were // introduced. // // A roachpb.Version has the colloquial form MAJOR.MINOR[.PATCH][-INTERNAL], @@ -319,7 +319,11 @@ const TODOPreV22_1 = V22_1 // Such clusters would need to be wiped. As a result, do not bump the major or // minor version until we are absolutely sure that no new migrations will need // to be added (i.e., when cutting the final release candidate). -var versionsSingleton = keyedVersions{ +// +// rawVersionsSingleton is converted to versionsSingleton below, by adding a +// large number to every major if building from master, so as to ensure that +// master builds cannot be upgraded to release-branch builds. +var rawVersionsSingleton = keyedVersions{ { // V21_2 is CockroachDB v21.2. It's used for all v21.2.x patch releases. Key: V21_2, @@ -479,6 +483,36 @@ var versionsSingleton = 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 + + // 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 +) + +var versionsSingleton = func() keyedVersions { + if unstableVersionsAbove > invalidVersionKey { + 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 { + rawVersionsSingleton[i].Major += devOffset + } + } + } + return rawVersionsSingleton +}() + // TODO(irfansharif): clusterversion.binary{,MinimumSupported}Version // feels out of place. A "cluster version" and a "binary version" are two // separate concepts. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 540216642d36..3a25cd0a7915 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -8,6 +8,7 @@ func _() { // An "invalid array index" compiler error signifies that the constant values have changed. // Re-run the stringer command to generate them again. var x [1]struct{} + _ = x[invalidVersionKey - -1] _ = x[V21_2-0] _ = x[Start22_1-1] _ = x[ProbeRequest-2] @@ -47,13 +48,14 @@ func _() { _ = x[NoNonMVCCAddSSTable-36] } -const _Key_name = "V21_2Start22_1ProbeRequestEnableSpanConfigStoreEnableNewStoreRebalancerV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJobWaitedForDelRangeInGCJobRangefeedUseOneStreamPerNodeNoNonMVCCAddSSTable" +const _Key_name = "invalidVersionKeyV21_2Start22_1ProbeRequestEnableSpanConfigStoreEnableNewStoreRebalancerV22_1Start22_2LocalTimestampsPebbleFormatSplitUserKeysMarkedCompactedEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysTrigramInvertedIndexesRemoveGrantPrivilegeMVCCRangeTombstonesUpgradeSequenceToBeReferencedByIDSampledStmtDiagReqsAddSSTableTombstonesSystemPrivilegesTableEnablePredicateProjectionChangefeedAlterSystemSQLInstancesAddLocalitySystemExternalConnectionsTableAlterSystemStatementStatisticsAddIndexRecommendationsRoleIDSequenceAddSystemUserIDColumnSystemUsersIDColumnIsBackfilledSetSystemUsersUserIDColumnNotNullSQLSchemaTelemetryScheduledJobsSchemaChangeSupportsCreateFunctionDeleteRequestReturnKeyPebbleFormatPrePebblev1MarkedRoleOptionsTableHasIDColumnRoleOptionsIDColumnIsBackfilledSetRoleOptionsUserIDColumnNotNullUseDelRangeInGCJobWaitedForDelRangeInGCJobRangefeedUseOneStreamPerNodeNoNonMVCCAddSSTable" -var _Key_index = [...]uint16{0, 5, 14, 26, 47, 71, 76, 85, 100, 140, 174, 208, 230, 250, 269, 302, 321, 341, 362, 397, 431, 461, 514, 528, 549, 580, 613, 644, 678, 700, 729, 756, 787, 820, 838, 862, 890, 909} +var _Key_index = [...]uint16{0, 17, 22, 31, 43, 64, 88, 93, 102, 117, 157, 191, 225, 247, 267, 286, 319, 338, 358, 379, 414, 448, 478, 531, 545, 566, 597, 630, 661, 695, 717, 746, 773, 804, 837, 855, 879, 907, 926} func (i Key) String() string { + i -= -1 if i < 0 || i >= Key(len(_Key_index)-1) { - return "Key(" + strconv.FormatInt(int64(i), 10) + ")" + return "Key(" + strconv.FormatInt(int64(i+-1), 10) + ")" } return _Key_name[_Key_index[i]:_Key_index[i+1]] } diff --git a/pkg/kv/kvserver/client_migration_test.go b/pkg/kv/kvserver/client_migration_test.go index ddb17fca56a6..6efacf3226ad 100644 --- a/pkg/kv/kvserver/client_migration_test.go +++ b/pkg/kv/kvserver/client_migration_test.go @@ -55,7 +55,7 @@ func TestStorePurgeOutdatedReplicas(t *testing.T) { t.Run(fmt.Sprintf("with-initial-version=%t", withInitialVersion), func(t *testing.T) { const numStores = 3 ctx := context.Background() - migrationVersion := roachpb.Version{Major: 42} + migrationVersion := roachpb.Version{Major: 1000042} storeKnobs := &kvserver.StoreTestingKnobs{ DisableEagerReplicaRemoval: true, diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index fde593a1c248..60cdd1b6d8fc 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -13,6 +13,7 @@ package kvserver import ( "context" "fmt" + math "math" "sync" "unsafe" @@ -364,7 +365,7 @@ func SynthesizeClusterVersionFromEngines( origin string } - maxPossibleVersion := roachpb.Version{Major: 999999} // Sort above any real version. + maxPossibleVersion := roachpb.Version{Major: math.MaxInt32} // Sort above any real version. minStoreVersion := originVersion{ Version: maxPossibleVersion, origin: "(no store)", diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 6b184003a87f..3208886a1118 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -923,7 +923,7 @@ func TestIsAtLeastVersion(t *testing.T) { errorRE string }{ {version: "21.2", expected: "true"}, - {version: "99.2", expected: "false"}, + {version: "1000099.2", expected: "false"}, {version: "foo", errorRE: ".*invalid version.*"}, } { query := fmt.Sprintf("SELECT crdb_internal.is_at_least_version('%s')", tc.version) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 3e790d4f2350..46f1f3a07cbc 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -534,7 +534,7 @@ select crdb_internal.get_vmodule() query T select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''); ---- -22.1 +1000022.1 query ITTT colnames select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', ''), e':\\d+', ':') as value from crdb_internal.node_runtime_info @@ -693,7 +693,7 @@ select * from crdb_internal.node_inflight_trace_spans query T select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', ''); ---- -22.1 +1000022.1 user root diff --git a/pkg/sql/ttl/ttljob/ttljob_test.go b/pkg/sql/ttl/ttljob/ttljob_test.go index d4e1de778bff..cdc9fdd81878 100644 --- a/pkg/sql/ttl/ttljob/ttljob_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_test.go @@ -420,7 +420,6 @@ func TestRowLevelTTLJobMultipleNodes(t *testing.T) { tableName, ) const rowsPerRange = 10 - const expiredRowsPerRange = rowsPerRange / 2 splitPoints := make([]serverutils.SplitPoint, len(splitAts)) for i, splitAt := range splitAts { newLeaseHolderIdx := (leaseHolderIdx + 1 + i) % numNodes diff --git a/pkg/upgrade/upgrades/precondition_before_starting_an_upgrade_external_test.go b/pkg/upgrade/upgrades/precondition_before_starting_an_upgrade_external_test.go index c2cf38c85d25..413bc086e754 100644 --- a/pkg/upgrade/upgrades/precondition_before_starting_an_upgrade_external_test.go +++ b/pkg/upgrade/upgrades/precondition_before_starting_an_upgrade_external_test.go @@ -14,6 +14,7 @@ import ( "context" gosql "database/sql" "encoding/hex" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -107,7 +108,7 @@ func TestPreconditionBeforeStartingAnUpgrade(t *testing.T) { "There exists invalid descriptors as listed below. Fix these descriptors before attempting to upgrade again.\n"+ "Invalid descriptor: defaultdb.public.t (104) because 'relation \"t\" (104): invalid depended-on-by relation back reference: referenced descriptor ID 53: referenced descriptor not found'\n"+ "Invalid descriptor: defaultdb.public.temp_tbl (104) because 'no matching name info found in non-dropped relation \"t\"'", - err.Error()) + strings.ReplaceAll(err.Error(), "1000022", "22")) // The cluster version should remain at `v0`. tdb.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v0.String()}}) }) diff --git a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go index f8c98867e5cd..41e6101f5258 100644 --- a/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go +++ b/pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go @@ -94,7 +94,7 @@ SELECT count(*) WHERE job_type = 'SCHEMA CHANGE GC' AND status = 'paused'`, [][]string{{"2"}}) - tdb.ExpectErr(t, `verifying precondition for version 22.1-\d+: `+ + tdb.ExpectErr(t, `verifying precondition for version \d*22.1-\d+: `+ `paused GC jobs prevent upgrading GC job behavior: \[\d+ \d+]`, "SET CLUSTER SETTING version = crdb_internal.node_executable_version()")