diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 4b598856880f..aa30dc4b924c 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -325,4 +325,4 @@ trace.snapshot.rate duration 0s if non-zero, interval at which background trace trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez application trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. application ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] application -version version 1000023.2-4 set the active cluster version in the format '.' application +version version 1000023.2-upgrading-to-1000024.1-step-004 set the active cluster version in the format '.' application diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 1277b50facca..1781b53e66b4 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -276,6 +276,6 @@
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracezServerless/Dedicated/Self-Hosted
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.Serverless/Dedicated/Self-Hosted
ui.display_timezone
enumerationetc/utcthe timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1]Serverless/Dedicated/Self-Hosted -
version
version1000023.2-4set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted +
version
version1000023.2-upgrading-to-1000024.1-step-004set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted diff --git a/pkg/ccl/backupccl/restore_planning_test.go b/pkg/ccl/backupccl/restore_planning_test.go index 2c7c05b9146d..aeba80b182a8 100644 --- a/pkg/ccl/backupccl/restore_planning_test.go +++ b/pkg/ccl/backupccl/restore_planning_test.go @@ -132,10 +132,10 @@ func TestBackupManifestVersionCompatibility(t *testing.T) { }, { name: "alpha-restore", - backupVersion: roachpb.Version{Major: 100022, Minor: 2, Internal: 14}, + backupVersion: roachpb.Version{Major: 1000022, Minor: 2, Internal: 14}, clusterVersion: roachpb.Version{Major: 23, Minor: 1}, minimumSupportedVersion: roachpb.Version{Major: 22, Minor: 2}, - expectedError: "backup from version 100022.2-14 is newer than current version 23.1", + expectedError: "backup from version 1000022.2-upgrading-to-1000023.1-step-014 is newer than current version 23.1", }, { name: "old-backup", diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 6681f065c214..c884875dc4cc 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -421,18 +421,16 @@ node_id component field value 0 UI URI / statement error unsupported within a virtual cluster -SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '') as server_version FROM crdb_internal.gossip_nodes WHERE node_id = 1 +SELECT node_id FROM crdb_internal.gossip_nodes WHERE node_id = 1 statement error unsupported within a virtual cluster -SELECT node_id, epoch, regexp_replace(expiration, '^\d+\.\d+,\d+$', '') as expiration, draining, decommissioning, membership FROM crdb_internal.gossip_liveness WHERE node_id = 1 +SELECT node_id FROM crdb_internal.gossip_liveness WHERE node_id = 1 statement error unsupported within a virtual cluster -SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '') as server_version, regexp_replace(go_version, '^go.+$', '') as go_version -FROM crdb_internal.kv_node_status WHERE node_id = 1 +SELECT node_id FROM crdb_internal.kv_node_status WHERE node_id = 1 statement error unsupported within a virtual cluster -SELECT node_id, store_id, attrs, used -FROM crdb_internal.kv_store_status WHERE node_id = 1 +SELECT node_id FROM crdb_internal.kv_store_status WHERE node_id = 1 query TT SELECT * FROM crdb_internal.regions ORDER BY 1 diff --git a/pkg/ccl/storageccl/engineccl/BUILD.bazel b/pkg/ccl/storageccl/engineccl/BUILD.bazel index 1c66dec6382d..67a691858a7d 100644 --- a/pkg/ccl/storageccl/engineccl/BUILD.bazel +++ b/pkg/ccl/storageccl/engineccl/BUILD.bazel @@ -50,6 +50,7 @@ go_test( deps = [ "//pkg/base", "//pkg/ccl/baseccl", + "//pkg/ccl/securityccl/fipsccl", "//pkg/ccl/storageccl/engineccl/enginepbccl", "//pkg/clusterversion", "//pkg/keys", diff --git a/pkg/ccl/storageccl/engineccl/ctr_stream_test.go b/pkg/ccl/storageccl/engineccl/ctr_stream_test.go index 73571b2d2345..992b98b438e1 100644 --- a/pkg/ccl/storageccl/engineccl/ctr_stream_test.go +++ b/pkg/ccl/storageccl/engineccl/ctr_stream_test.go @@ -11,10 +11,12 @@ package engineccl import ( "context" "crypto/rand" + "encoding/binary" "fmt" "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/ccl/securityccl/fipsccl" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -173,3 +175,68 @@ func TestFileCipherStreamCreator(t *testing.T) { t.Fatalf("%s\n%s", strings.Join(diff, "\n"), data) } } + +// Running non-fips mode: +// ./dev bench pkg/ccl/storageccl/engineccl -f FileCipherStream --stream-output --ignore-cache +// Running fips mode (be sure to look for fips=true in the output): +// ./dev test-binaries --cross=crosslinuxfips pkg/ccl/storageccl/engineccl && mkdir -p fipsbin && tar xf bin/test_binaries.tar.gz -C fipsbin && docker run -v $PWD/fipsbin:/fipsbin -it redhat/ubi9 /fipsbin/pkg/ccl/storageccl/engineccl/bin/engineccl_test -test.run '^$' -test.bench FileCipherStream +func BenchmarkFileCipherStream(b *testing.B) { + isFips := fipsccl.IsFIPSReady() + for _, keySize := range []int{128, 192, 256} { + for _, blockSize := range []int{16, 1024, 10240} { + b.Run(fmt.Sprintf("fips=%t/key=%d/block=%d/", isFips, keySize, blockSize), func(b *testing.B) { + keyBytes := make([]byte, keySize/8) + if _, err := rand.Read(keyBytes); err != nil { + panic(err) + } + var encType enginepbccl.EncryptionType + switch keySize { + case 128: + encType = enginepbccl.EncryptionType_AES128_CTR + case 192: + encType = enginepbccl.EncryptionType_AES192_CTR + case 256: + encType = enginepbccl.EncryptionType_AES256_CTR + default: + panic("unknown key size") + } + key := &enginepbccl.SecretKey{ + Info: &enginepbccl.KeyInfo{ + EncryptionType: encType, + }, + Key: keyBytes, + } + nonce := make([]byte, ctrNonceSize) + if _, err := rand.Read(nonce); err != nil { + panic(err) + } + initCounterBytes := make([]byte, 4) + if _, err := rand.Read(initCounterBytes); err != nil { + panic(err) + } + // Endianness doesn't matter for converting this random number to an int. + initCounter := binary.LittleEndian.Uint32(initCounterBytes) + blockStream, err := newCTRBlockCipherStream(key, nonce, initCounter) + if err != nil { + panic(err) + } + + stream := fileCipherStream{blockStream} + + // Benchmarks are fun! We're just going to encrypt a bunch of zeros + // and re-encrypt over the previous output because that doesn't matter + // to the speed :) + // + // TODO(bdarnell): The offset argument to stream.Encrypt *does* matter, + // specifically whether the data is aligned to the CTR block size or not. + data := make([]byte, blockSize) + b.SetBytes(int64(blockSize)) + b.ResetTimer() + + for i := 0; i < b.N; i++ { + stream.Encrypt(0, data) + } + }) + } + } +} diff --git a/pkg/clusterversion/BUILD.bazel b/pkg/clusterversion/BUILD.bazel index fa1f58656157..a1534de04e79 100644 --- a/pkg/clusterversion/BUILD.bazel +++ b/pkg/clusterversion/BUILD.bazel @@ -16,7 +16,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/clusterversion", visibility = ["//visibility:public"], deps = [ - "//pkg/build", "//pkg/roachpb", "//pkg/settings", "//pkg/testutils/skip", diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 997ba29d807a..377a049f78eb 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -10,10 +10,7 @@ package clusterversion -import ( - "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/roachpb" -) +import "github.com/cockroachdb/cockroach/pkg/roachpb" // Key is a unique identifier for a version of CockroachDB. type Key int @@ -425,27 +422,21 @@ func (k Key) IsFinal() bool { return k.Version().IsFinal() } -// ReleaseSeries returns the final version for the release series the Key -// belongs to. Specifically: -// - if the key corresponds to a final version (e.g. 23.2), the result is the -// same version; e.g. V23_2.ReleaseSeries() is v23.2. -// - if the key corresponds to a transitional upgrade version (e.g. v23.2-8), -// the result is the next final version (e.g. v24.1). -// -// Note that the result does not have any DevOffset applied. -func (k Key) ReleaseSeries() roachpb.Version { - // Find the first key >= k that is a final version. - for k := k; k < numKeys; k++ { - if k.IsFinal() { - return removeDevOffset(k.Version()) - } - } - // k is a dev version in the latest release series. - major, minor := build.BranchReleaseSeries() - return roachpb.Version{ - Major: int32(major), - Minor: int32(minor), - } +// ReleaseSeries returns the release series the Key. Specifically: +// - if the key corresponds to a final version (e.g. V23_2), the result has the +// same major/minor; +// - if the key corresponds to a transitional upgrade version (e.g. +// V23_2SomeFeature with version 23.2-x), the result is the next series +// (e.g. 24.1). +// +// The key must be in the range [MinSupported, Latest]. +// +// Note that the release series won't have the DevOffset applied, even if the +// version has it. +func (k Key) ReleaseSeries() roachpb.ReleaseSeries { + // Note: TestReleaseSeries ensures that this works for all valid Keys. + s, _ := removeDevOffset(k.Version()).ReleaseSeries() + return s } func (k Key) String() string { diff --git a/pkg/clusterversion/cockroach_versions_test.go b/pkg/clusterversion/cockroach_versions_test.go index f8341210e5d5..a12f8d1c6010 100644 --- a/pkg/clusterversion/cockroach_versions_test.go +++ b/pkg/clusterversion/cockroach_versions_test.go @@ -145,12 +145,11 @@ func TestClusterVersionPrettyPrint(t *testing.T) { cv ClusterVersion exp string }{ - {cv(19, 2, 1, 5), "19.2-5"}, - {cv(20, 1, 0, 4), "20.1-4"}, - {cv(20, 2, 0, 7), "20.2-7(fence)"}, - {cv(20, 2, 0, 4), "20.2-4"}, - {cv(20, 2, 1, 5), "20.2-5(fence)"}, - {cv(20, 2, 1, 4), "20.2-4"}, + {cv(20, 1, 0, 4), "20.1-upgrading-to-20.2-step-004"}, + {cv(20, 2, 0, 7), "20.2-upgrading-to-21.1-step-007(fence)"}, + {cv(20, 2, 0, 4), "20.2-upgrading-to-21.1-step-004"}, + {cv(22, 2, 1, 5), "22.2-upgrading-to-23.1-step-005(fence)"}, + {cv(22, 2, 1, 4), "22.2-upgrading-to-23.1-step-004"}, } for _, test := range tests { if actual := test.cv.PrettyPrint(); actual != test.exp { @@ -160,13 +159,25 @@ func TestClusterVersionPrettyPrint(t *testing.T) { } func TestReleaseSeries(t *testing.T) { + // Verify that the ReleaseSeries call works on all keys. + for k := Latest; k > 0; k-- { + if k.Version().Major > 0 { + require.NotEqual(t, roachpb.ReleaseSeries{}, k.ReleaseSeries()) + } else { + require.Equal(t, roachpb.ReleaseSeries{}, k.ReleaseSeries()) + } + } + + // Verify the latest version. require.Equal(t, fmt.Sprintf("v%s", Latest.ReleaseSeries()), build.BinaryVersionPrefix()) - if Latest.IsFinal() { - require.True(t, Latest.Version() == Latest.ReleaseSeries()) - } else { - require.True(t, removeDevOffset(Latest.Version()).Less(Latest.ReleaseSeries())) + + // Verify the ReleaseSeries results down to MinSupported. + expected := Latest.ReleaseSeries() + for k := Latest; k >= MinSupported; k-- { + if k.IsFinal() { + v := removeDevOffset(k.Version()) + expected = roachpb.ReleaseSeries{Major: v.Major, Minor: v.Minor} + } + require.Equalf(t, expected, k.ReleaseSeries(), "version: %s", k) } - require.Equal(t, PreviousRelease.ReleaseSeries(), removeDevOffset(PreviousRelease.Version())) - require.Equal(t, (PreviousRelease - 1).ReleaseSeries(), removeDevOffset(PreviousRelease.Version())) - require.Equal(t, MinSupported.ReleaseSeries(), removeDevOffset(MinSupported.Version())) } diff --git a/pkg/clusterversion/dev_offset.go b/pkg/clusterversion/dev_offset.go index 4cbc59782352..250725d3882f 100644 --- a/pkg/clusterversion/dev_offset.go +++ b/pkg/clusterversion/dev_offset.go @@ -76,7 +76,7 @@ var devOffsetKeyStart = func() Key { // DevOffset is the offset applied to major versions into the future if this is // a dev branch. -const DevOffset = 1_000_000 +const DevOffset = roachpb.VersionMajorDevOffset // maybeApplyDevOffset applies DevOffset to the major version, if appropriate. func maybeApplyDevOffset(key Key, v roachpb.Version) roachpb.Version { diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/helper_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/helper_test.go index 30b4e9baec62..7c509d949751 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/helper_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/helper_test.go @@ -33,7 +33,7 @@ func TestClusterVersionAtLeast(t *testing.T) { name: "invalid minVersion", currentVersion: "23.1", minVersion: "v23.1", - expectedErr: `invalid version v23.1: strconv.ParseInt: parsing "v23": invalid syntax`, + expectedErr: `invalid version v23.1`, }, { name: "cluster version is behind", diff --git a/pkg/cmd/sql-bootstrap-data/main.go b/pkg/cmd/sql-bootstrap-data/main.go index e553c93e8b21..e225387a5fcc 100644 --- a/pkg/cmd/sql-bootstrap-data/main.go +++ b/pkg/cmd/sql-bootstrap-data/main.go @@ -55,7 +55,7 @@ func main() { } // writeDataFile creates a file in pkg/sql/catalog/bootstrap/data. -func writeDataFile(version roachpb.Version, filenameSuffix, data string) { +func writeDataFile(version roachpb.ReleaseSeries, filenameSuffix, data string) { filename := filepath.Join( "pkg", "sql", "catalog", "bootstrap", "data", fmt.Sprintf("%d_%d_%s", version.Major, version.Minor, filenameSuffix), diff --git a/pkg/roachpb/version.go b/pkg/roachpb/version.go index 3c604c1be3c6..290c4cfdd3b0 100644 --- a/pkg/roachpb/version.go +++ b/pkg/roachpb/version.go @@ -12,8 +12,8 @@ package roachpb import ( "fmt" + "regexp" "strconv" - "strings" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -54,16 +54,36 @@ func (v Version) AtLeast(otherV Version) bool { return !v.Less(otherV) } -// String implements the fmt.Stringer interface. +// String implements the fmt.Stringer interface. The result is of the form +// "23.2" for final versions and "23.2-upgrading-to-24.1-step-004" for +// transitional internal versions during upgrade. func (v Version) String() string { return redact.StringWithoutMarkers(v) } +// VersionMajorDevOffset is an offset we apply to major version numbers during +// development; see clusterversion.DevOffset for more information. +const VersionMajorDevOffset = 1_000_000 + // SafeFormat implements the redact.SafeFormatter interface. func (v Version) SafeFormat(p redact.SafePrinter, _ rune) { if v.IsFinal() { p.Printf("%d.%d", v.Major, v.Minor) return } - p.Printf("%d.%d-%d", v.Major, v.Minor, v.Internal) + // If the version is offset, remove the offset and add it back to the result. We want + // 1000023.1-upgrading-to-1000023.2-step-002, not 1000023.1-upgrading-to-23.2-step-002. + noOffsetVersion := v + if v.Major > VersionMajorDevOffset { + noOffsetVersion.Major -= VersionMajorDevOffset + } + if s, ok := noOffsetVersion.ReleaseSeries(); ok { + if v.Major > VersionMajorDevOffset { + s.Major += VersionMajorDevOffset + } + p.Printf("%d.%d-upgrading-to-%d.%d-step-%03d", v.Major, v.Minor, s.Major, s.Minor, v.Internal) + } else { + // This shouldn't happen in practice. + p.Printf("%d.%d-upgrading-step-%03d", v.Major, v.Minor, v.Internal) + } } // IsFinal returns true if this is a final version (as opposed to a transitional @@ -87,39 +107,45 @@ func (v Version) PrettyPrint() string { return fmt.Sprintf("%v(fence)", v) } -// ParseVersion parses a Version from a string of the form -// ".-" where the "-" is optional. We don't -// use the Patch component, so it is always zero. -func ParseVersion(s string) (Version, error) { - var c Version - dotParts := strings.Split(s, ".") +var ( + verPattern = regexp.MustCompile( + `^(?P[0-9]+)\.(?P[0-9]+)(|(-|-upgrading(|-to-[0-9]+.[0-9]+)-step-)(?P[0-9]+))$`, + ) + verPatternMajorIdx = verPattern.SubexpIndex("major") + verPatternMinorIdx = verPattern.SubexpIndex("minor") + verPatternInternalIdx = verPattern.SubexpIndex("internal") +) - if len(dotParts) != 2 { +// ParseVersion parses a Version from a string of one of the forms: +// - "." +// - ".-upgrading-to-.-step-" +// - ".-" (older version of the above) +// +// We don't use the Patch component, so it is always zero. +func ParseVersion(s string) (Version, error) { + matches := verPattern.FindStringSubmatch(s) + if matches == nil { return Version{}, errors.Errorf("invalid version %s", s) } - parts := append(dotParts[:1], strings.Split(dotParts[1], "-")...) - if len(parts) == 2 { - parts = append(parts, "0") + var err error + toInt := func(s string) int32 { + if err != nil || s == "" { + return 0 + } + var n int64 + n, err = strconv.ParseInt(s, 10, 32) + return int32(n) } - - if len(parts) != 3 { - return c, errors.Errorf("invalid version %s", s) + v := Version{ + Major: toInt(matches[verPatternMajorIdx]), + Minor: toInt(matches[verPatternMinorIdx]), + Internal: toInt(matches[verPatternInternalIdx]), } - - ints := make([]int64, len(parts)) - for i := range parts { - var err error - if ints[i], err = strconv.ParseInt(parts[i], 10, 32); err != nil { - return c, errors.Wrapf(err, "invalid version %s", s) - } + if err != nil { + return Version{}, errors.Wrapf(err, "invalid version %s", s) } - - c.Major = int32(ints[0]) - c.Minor = int32(ints[1]) - c.Internal = int32(ints[2]) - - return c, nil + return v, nil } // MustParseVersion calls ParseVersion and panics on error. @@ -130,3 +156,59 @@ func MustParseVersion(s string) Version { } return v } + +// ReleaseSeries is just the major.minor part of a Version. +type ReleaseSeries struct { + Major int32 + Minor int32 +} + +func (s ReleaseSeries) String() string { + return fmt.Sprintf("%d.%d", s.Major, s.Minor) +} + +// Successor returns the next release series, if known. This is only guaranteed +// to work for versions from the minimum supported series up to the previous +// series. +func (s ReleaseSeries) Successor() (_ ReleaseSeries, ok bool) { + res, ok := successorSeries[s] + return res, ok +} + +// successorSeries stores the successor for each series. We are only concerned +// with versions within our compatibility window, but there is no harm in +// populating more if they are known. +// +// When this map is updated, the expected result in TestReleaseSeriesSuccessor +// needs to be updated. Also note that clusterversion tests ensure that this map +// contains all necessary versions. +var successorSeries = map[ReleaseSeries]ReleaseSeries{ + {20, 1}: {20, 2}, + {20, 2}: {21, 1}, + {21, 1}: {21, 2}, + {21, 2}: {22, 1}, + {22, 1}: {22, 2}, + {22, 2}: {23, 1}, + {23, 1}: {23, 2}, + {23, 2}: {24, 1}, +} + +// ReleaseSeries obtains the release series for the given version. Specifically: +// - if the version is final (Internal=0), the ReleaseSeries has the same major/minor. +// - if the version is a transitional version during upgrade (e.g. v23.1-8), +// the result is the next final version (e.g. v23.1). +// +// For non-final versions (which indicate an update to the next series), this +// requires knowledge of the next series; unknown non-final versions will return +// ok=false. +// +// Note that if the version has the clusterversion.DevOffset applied, the +// resulting release series will have it too. +func (v Version) ReleaseSeries() (s ReleaseSeries, ok bool) { + base := ReleaseSeries{v.Major, v.Minor} + if v.IsFinal() { + return base, true + } + s, ok = base.Successor() + return s, ok +} diff --git a/pkg/roachpb/version_test.go b/pkg/roachpb/version_test.go index 162443cb5971..48f13306de0e 100644 --- a/pkg/roachpb/version_test.go +++ b/pkg/roachpb/version_test.go @@ -11,11 +11,37 @@ package roachpb import ( + "strings" "testing" "github.com/kr/pretty" + "github.com/stretchr/testify/require" ) +func TestParseVersion(t *testing.T) { + testData := []struct { + s string + v Version + roundtrip bool + }{ + {s: "23.1", v: Version{Major: 23, Minor: 1}, roundtrip: true}, + {s: "23.1-upgrading-to-23.2-step-004", v: Version{Major: 23, Minor: 1, Internal: 4}, roundtrip: true}, + {s: "1000023.1-upgrading-to-1000023.2-step-004", v: Version{Major: 1000023, Minor: 1, Internal: 4}, roundtrip: true}, + {s: "23.1-4", v: Version{Major: 23, Minor: 1, Internal: 4}}, + {s: "23.1-upgrading-step-004", v: Version{Major: 23, Minor: 1, Internal: 4}}, + } + for _, tc := range testData { + t.Run("", func(t *testing.T) { + v, err := ParseVersion(tc.s) + require.NoError(t, err) + require.Equal(t, tc.v, v) + if tc.roundtrip { + require.Equal(t, tc.s, v.String()) + } + }) + } +} + func TestVersionCmp(t *testing.T) { v := func(major, minor, patch, internal int32) Version { return Version{ @@ -60,3 +86,45 @@ func TestVersionCmp(t *testing.T) { }) } } + +func TestReleaseSeriesSuccessor(t *testing.T) { + r := ReleaseSeries{20, 1} + var seq []string + for ok := true; ok; r, ok = r.Successor() { + seq = append(seq, r.String()) + } + expected := "20.1, 20.2, 21.1, 21.2, 22.1, 22.2, 23.1, 23.2, 24.1" + require.Equal(t, expected, strings.Join(seq, ", ")) +} + +func TestReleaseSeries(t *testing.T) { + testCases := []struct { + v Version + s ReleaseSeries + }{ + { + v: Version{Major: 22, Minor: 2, Internal: 0}, + s: ReleaseSeries{Major: 22, Minor: 2}, + }, + { + v: Version{Major: 22, Minor: 2, Internal: 8}, + s: ReleaseSeries{Major: 23, Minor: 1}, + }, + { + v: Version{Major: 23, Minor: 1, Internal: 0}, + s: ReleaseSeries{Major: 23, Minor: 1}, + }, + { + v: Version{Major: 23, Minor: 1, Internal: 2}, + s: ReleaseSeries{Major: 23, Minor: 2}, + }, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + res, ok := tc.v.ReleaseSeries() + require.True(t, ok) + require.Equal(t, tc.s, res) + }) + } +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 89b8df09135f..b0b55850189e 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -667,7 +667,7 @@ node_id component field value 1 UI URI / query ITTTTT colnames -SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '') as server_version FROM crdb_internal.gossip_nodes WHERE node_id = 1 +SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-upgrading-to-\d+\.\d+-step-\d+)?$', '') as server_version FROM crdb_internal.gossip_nodes WHERE node_id = 1 ---- node_id network address attrs locality server_version 1 tcp 127.0.0.1: [] region=test,dc=dc1 @@ -679,7 +679,7 @@ node_id epoch expiration draining decommissioning membership 1 false false active query ITTTTTT colnames -SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-\d+)?$', '') as server_version, regexp_replace(go_version, '^go.+$', '') as go_version +SELECT node_id, network, regexp_replace(address, '\d+$', '') as address, attrs, locality, regexp_replace(server_version, '^\d+\.\d+(-upgrading-to-\d+\.\d+-step-\d+)?$', '') as server_version, regexp_replace(go_version, '^go.+$', '') as go_version FROM crdb_internal.kv_node_status WHERE node_id = 1 ---- node_id network address attrs locality server_version go_version diff --git a/pkg/sql/show_cluster_setting_test.go b/pkg/sql/show_cluster_setting_test.go index d8ef812831eb..b6f5ef8c4df6 100644 --- a/pkg/sql/show_cluster_setting_test.go +++ b/pkg/sql/show_cluster_setting_test.go @@ -38,26 +38,26 @@ func TestCheckClusterSettingValuesAreEquivalent(t *testing.T) { exp string }{ { // 0 - local: encode(t, "22.2-10"), - kv: encode(t, "22.2-10"), + local: encode(t, "22.2-upgrading-to-23.1-step-010"), + kv: encode(t, "22.2-upgrading-to-23.1-step-010"), }, { // 1 - local: encode(t, "22.2-12"), - kv: encode(t, "22.2-11"), - exp: "value differs between local setting (22.2-12) and KV (22.2-11)", + local: encode(t, "22.2-upgrading-to-23.1-step-012"), + kv: encode(t, "22.2-upgrading-to-23.1-step-011"), + exp: "value differs between local setting (22.2-upgrading-to-23.1-step-012) and KV (22.2-upgrading-to-23.1-step-011)", }, { // 2 - local: encode(t, "22.2-11"), - kv: encode(t, "22.2-10"), + local: encode(t, "22.2-upgrading-to-23.1-step-011"), + kv: encode(t, "22.2-upgrading-to-23.1-step-010"), }, { // 3 - local: encode(t, "22.2-11"), + local: encode(t, "22.2-upgrading-to-23.1-step-011"), kv: []byte("abc"), - exp: "value differs between local setting (22.2-11) and KV ([97 98 99])", + exp: "value differs between local setting (22.2-upgrading-to-23.1-step-011) and KV ([97 98 99])", }, { // 4 - kv: encode(t, "22.2-11"), - exp: "value differs between local setting ([]) and KV (22.2-11)", + kv: encode(t, "22.2-upgrading-to-23.1-step-011"), + exp: "value differs between local setting ([]) and KV (22.2-upgrading-to-23.1-step-011)", }, } { t.Run("", func(t *testing.T) {