Skip to content

Commit

Permalink
clusterversion,storage,backupccl: remove pre-22.1 gates, bump minVersion
Browse files Browse the repository at this point in the history
Release justification: bump min supported version
Release note: None
  • Loading branch information
celiala committed Aug 30, 2022
1 parent f4b491f commit a8bf7e3
Show file tree
Hide file tree
Showing 10 changed files with 39 additions and 170 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/backupdest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_library(
"//pkg/ccl/backupccl/backuputils",
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/clusterversion",
"//pkg/jobs/jobspb",
"//pkg/roachpb",
"//pkg/security/username",
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/backupccl/backupdest/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudpb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
Expand Down Expand Up @@ -168,8 +167,7 @@ func ResolveDest(
if err != nil {
return ResolvedDestination{}, err
}
if exists && !dest.Exists && chosenSuffix != "" &&
execCfg.Settings.Version.IsActive(ctx, clusterversion.Start22_1) {
if exists && !dest.Exists && chosenSuffix != "" {
// We disallow a user from writing a full backup to a path in a collection containing an
// existing backup iff we're 99.9% confident this backup was planned on a 22.1 node.
return ResolvedDestination{},
Expand Down
48 changes: 1 addition & 47 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,25 +156,6 @@ type Key int
const (
invalidVersionKey Key = iota - 1 // want first named one to start at zero

// V21_2 is CockroachDB v21.2. It's used for all v21.2.x patch releases.
V21_2

// v22.1 versions.
//
// Start22_1 demarcates work towards CockroachDB v22.1.
Start22_1

// ProbeRequest is the version at which roachpb.ProbeRequest was introduced.
// This version must be active before any ProbeRequest is issued on the
// cluster.
ProbeRequest
// EnableSpanConfigStore enables the use of the span configs infrastructure
// in KV.
EnableSpanConfigStore
// EnableNewStoreRebalancer enables the new store rebalancer introduced in
// 22.1.
EnableNewStoreRebalancer

// V22_1 is CockroachDB v22.1. It's used for all v22.1.x patch releases.
V22_1

Expand Down Expand Up @@ -304,10 +285,6 @@ const (
// *************************************************
)

// TODOPreV21_2 is an alias for V21_2 for use in any version gate/check that
// previously referenced a < 21.2 version until that check/gate can be removed.
const TODOPreV21_2 = V21_2

// TODOPreV22_1 is an alias for V22_1 for use in any version gate/check that
// previously referenced a < 22.1 version until that check/gate can be removed.
const TODOPreV22_1 = V22_1
Expand All @@ -334,29 +311,6 @@ const TODOPreV22_1 = V22_1
// 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,
Version: roachpb.Version{Major: 21, Minor: 2},
},

// v22.1 versions. Internal versions must be even.
{
Key: Start22_1,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2},
},
{
Key: ProbeRequest,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 26},
},
{
Key: EnableSpanConfigStore,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 40},
},
{
Key: EnableNewStoreRebalancer,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 96},
},
{
Key: V22_1,
Version: roachpb.Version{Major: 22, Minor: 1},
Expand Down Expand Up @@ -545,7 +499,7 @@ var (
// version than binaryMinSupportedVersion, then the binary will exit with
// an error. This typically trails the current release by one (see top-level
// comment).
binaryMinSupportedVersion = ByKey(V21_2)
binaryMinSupportedVersion = ByKey(V22_1)

// binaryVersion is the version of this binary.
//
Expand Down
79 changes: 37 additions & 42 deletions pkg/clusterversion/key_string.go

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

6 changes: 0 additions & 6 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -145,11 +144,6 @@ func (p *planner) AlterPrimaryKey(
if col.IsNullable() {
return pgerror.Newf(pgcode.InvalidSchemaDefinition, "cannot use nullable column %q in primary key", col.GetName())
}
if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.Start22_1) {
if col.IsVirtual() {
return pgerror.Newf(pgcode.FeatureNotSupported, "cannot use virtual column %q in primary key", col.GetName())
}
}
}

// Validate if the end result is the same as the current
Expand Down
9 changes: 0 additions & 9 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
Expand Down Expand Up @@ -1998,14 +1997,6 @@ func NewTableDesc(

for i := range desc.Columns {
if _, ok := primaryIndexColumnSet[desc.Columns[i].Name]; ok {
if !st.Version.IsActive(ctx, clusterversion.Start22_1) {
if desc.Columns[i].Virtual {
return nil, pgerror.Newf(
pgcode.FeatureNotSupported,
"cannot use virtual column %q in primary key", desc.Columns[i].Name,
)
}
}
desc.Columns[i].Nullable = false
}
}
Expand Down
33 changes: 0 additions & 33 deletions pkg/sql/schema_changer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
Expand All @@ -34,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -8256,37 +8254,6 @@ DROP VIEW IF EXISTS v
wg.Wait()
}

func TestVirtualColumnNotAllowedInPkeyBefore22_1(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

params, _ := tests.CreateTestServerParams()
params.Knobs.Server = &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
BinaryVersionOverride: clusterversion.ByKey(clusterversion.V21_2),
}

s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)

_, err := sqlDB.Exec(`CREATE TABLE t (a INT NOT NULL AS (1+1) VIRTUAL, PRIMARY KEY (a))`)
require.Error(t, err)
require.Equal(t, "pq: cannot use virtual column \"a\" in primary key", err.Error())

_, err = sqlDB.Exec(`CREATE TABLE t (a INT NOT NULL AS (1+1) VIRTUAL PRIMARY KEY)`)
require.Error(t, err)
require.Equal(t, "pq: cannot use virtual column \"a\" in primary key", err.Error())

_, err = sqlDB.Exec(`CREATE TABLE t (a INT PRIMARY KEY, b INT NOT NULL AS (1+1) VIRTUAL)`)
require.NoError(t, err)

_, err = sqlDB.Exec(`ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (b)`)
require.Error(t, err)
require.Equal(t, "pq: cannot use virtual column \"b\" in primary key", err.Error())
}

// TestColumnBackfillProcessingDoesNotHoldLockOnJobsTable is a
// regression test to ensure that when the column backfill progresses
// to the next backfill chunk and it needs to update its progress, it
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -245,13 +244,6 @@ func checkForEarlyExit(b BuildCtx, tbl *scpb.Table, t alterPrimaryKeySpec) {
panic(pgerror.Newf(pgcode.InvalidSchemaDefinition, "cannot use nullable column "+
"%q in primary key", col.Column))
}

if !b.EvalCtx().Settings.Version.IsActive(b, clusterversion.Start22_1) {
if colTypeElem.IsVirtual {
panic(pgerror.Newf(pgcode.FeatureNotSupported, "cannot use virtual column %q "+
"in primary key", col.Column))
}
}
}
}

Expand Down
17 changes: 0 additions & 17 deletions pkg/storage/min_version_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,23 +96,6 @@ func TestSetMinVersion(t *testing.T) {
defer p.Close()
require.Equal(t, pebble.FormatMostCompatible, p.db.FormatMajorVersion())

// The earliest supported Cockroach version advances the pebble version.
err = p.SetMinVersion(clusterversion.ByKey(clusterversion.V21_2))
require.NoError(t, err)
require.Equal(t, pebble.FormatSetWithDelete, p.db.FormatMajorVersion())

// Setting the same min version twice is okay.
err = p.SetMinVersion(clusterversion.ByKey(clusterversion.V21_2))
require.NoError(t, err)
require.Equal(t, pebble.FormatSetWithDelete, p.db.FormatMajorVersion())

// Advancing the store cluster version to another cluster version
// that does not advance the Pebble format major version should
// leave the format major version unchanged.
err = p.SetMinVersion(clusterversion.ByKey(clusterversion.Start22_1))
require.NoError(t, err)
require.Equal(t, pebble.FormatSetWithDelete, p.db.FormatMajorVersion())

// Advancing the store cluster version to TODOPreV22_1
// should also advance the store's format major version.
err = p.SetMinVersion(clusterversion.ByKey(clusterversion.TODOPreV22_1))
Expand Down
4 changes: 0 additions & 4 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -1859,10 +1859,6 @@ func (p *Pebble) SetMinVersion(version roachpb.Version) error {
if formatVers < pebble.FormatSplitUserKeysMarked {
formatVers = pebble.FormatSplitUserKeysMarked
}
case !version.Less(clusterversion.ByKey(clusterversion.TODOPreV21_2)):
if formatVers < pebble.FormatSetWithDelete {
formatVers = pebble.FormatSetWithDelete
}
}
if p.db.FormatMajorVersion() < formatVers {
if err := p.db.RatchetFormatMajorVersion(formatVers); err != nil {
Expand Down

0 comments on commit a8bf7e3

Please sign in to comment.