Skip to content

Commit

Permalink
Merge #111396
Browse files Browse the repository at this point in the history
111396: *: delete old version gates r=healthy-pod a=healthy-pod

Release note: None
Epic: none

Fixes #96762
Fixes #96761 
Fixes #96765

Co-authored-by: healthy-pod <ahmad@cockroachlabs.com>
  • Loading branch information
craig[bot] and healthy-pod committed Sep 29, 2023
2 parents 727060f + 51a913b commit 52ae7d5
Show file tree
Hide file tree
Showing 29 changed files with 29 additions and 462 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/changefeedccl/cdceval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
deps = [
"//pkg/ccl/changefeedccl/cdcevent",
"//pkg/ccl/changefeedccl/changefeedbase",
"//pkg/clusterversion",
"//pkg/jobs/jobspb",
"//pkg/roachpb",
"//pkg/security/username",
Expand Down
7 changes: 0 additions & 7 deletions pkg/ccl/changefeedccl/cdceval/validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ import (
"context"

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -80,12 +79,6 @@ func normalizeAndValidateSelectForTarget(
}
}()

if !execCfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2EnablePredicateProjectionChangefeed) {
return nil, errors.Newf(
`filters and projections not supported until upgrade to version %s or higher is finalized`,
clusterversion.TODODelete_V22_2EnablePredicateProjectionChangefeed.String())
}

// This really shouldn't happen as it's enforced by sql.y.
if len(sc.From.Tables) != 1 {
return nil, pgerror.Newf(pgcode.Syntax,
Expand Down
1 change: 0 additions & 1 deletion pkg/cloud/amazon/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ go_library(
"//pkg/cloud/externalconn",
"//pkg/cloud/externalconn/connectionpb",
"//pkg/cloud/externalconn/utils",
"//pkg/clusterversion",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
5 changes: 0 additions & 5 deletions pkg/cloud/amazon/aws_kms.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kms"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -196,10 +195,6 @@ func MakeAWSKMS(ctx context.Context, uri string, env cloud.KMSEnv) (cloud.KMS, e
}

if kmsURIParams.roleProvider != (roleProvider{}) {
if !env.ClusterSettings().Version.IsActive(ctx, clusterversion.TODODelete_V22_2SupportAssumeRoleAuth) {
return nil, errors.New("cannot authenticate to KMS via assume role until cluster has fully upgraded to 22.2")
}

// If there are delegate roles in the assume-role chain, we create a session
// for each role in order for it to fetch the credentials from the next role
// in the chain.
Expand Down
5 changes: 0 additions & 5 deletions pkg/cloud/amazon/s3_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudpb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -566,10 +565,6 @@ func newClient(
}

if conf.assumeRoleProvider.roleARN != "" {
if !settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SupportAssumeRoleAuth) {
return s3Client{}, "", errors.New("cannot authenticate to cloud storage via assume role until cluster has fully upgraded to 22.2")
}

for _, delegateProvider := range conf.delegateRoleProviders {
intermediateCreds := stscreds.NewCredentials(sess, delegateProvider.roleARN, withExternalID(delegateProvider.externalID))
opts.Config.Credentials = intermediateCreds
Expand Down
1 change: 0 additions & 1 deletion pkg/cloud/cloudprivilege/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ go_library(
deps = [
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/clusterversion",
"//pkg/sql",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down
6 changes: 0 additions & 6 deletions pkg/cloud/cloudprivilege/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudpb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -59,11 +58,6 @@ func CheckDestinationPrivileges(ctx context.Context, p sql.PlanHookState, to []s
// If the resource being used is an External Connection, check that the user
// has adequate privileges.
if conf.Provider == cloudpb.ExternalStorageProvider_external {
if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SystemExternalConnectionsTable) {
return pgerror.Newf(pgcode.FeatureNotSupported,
"version %v must be finalized to backup to an External Connection",
clusterversion.ByKey(clusterversion.TODODelete_V22_2SystemExternalConnectionsTable))
}
ecPrivilege := &syntheticprivilege.ExternalConnectionPrivilege{
ConnectionName: conf.ExternalConnectionConfig.Name,
}
Expand Down
1 change: 0 additions & 1 deletion pkg/cloud/gcp/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
"//pkg/cloud/externalconn",
"//pkg/cloud/externalconn/connectionpb",
"//pkg/cloud/externalconn/utils",
"//pkg/clusterversion",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
5 changes: 0 additions & 5 deletions pkg/cloud/gcp/gcp_kms.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
kms "cloud.google.com/go/kms/apiv1"
kmspb "cloud.google.com/go/kms/apiv1/kmspb"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/errors"
"google.golang.org/api/option"
"google.golang.org/protobuf/types/known/wrapperspb"
Expand Down Expand Up @@ -127,10 +126,6 @@ func MakeGCSKMS(ctx context.Context, uri string, env cloud.KMSEnv) (cloud.KMS, e
if kmsURIParams.assumeRole == "" {
opts = append(opts, credentialsOpt...)
} else {
if !env.ClusterSettings().Version.IsActive(ctx, clusterversion.TODODelete_V22_2SupportAssumeRoleAuth) {
return nil, errors.New("cannot authenticate to KMS via assume role until cluster has fully upgraded to 22.2")
}

assumeOpt, err := createImpersonateCredentials(ctx, kmsURIParams.assumeRole, kmsURIParams.delegateRoles, kms.DefaultAuthScopes(), credentialsOpt...)
if err != nil {
return nil, cloud.KMSInaccessible(errors.Wrapf(err, "failed to assume role"))
Expand Down
5 changes: 0 additions & 5 deletions pkg/cloud/gcp/gcs_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudpb"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -182,10 +181,6 @@ func makeGCSStorage(
if conf.AssumeRole == "" {
opts = append(opts, credentialsOpt...)
} else {
if !args.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2SupportAssumeRoleAuth) {
return nil, errors.New("cannot authenticate to cloud storage via assume role until cluster has fully upgraded to 22.2")
}

assumeOpt, err := createImpersonateCredentials(ctx, conf.AssumeRole, conf.AssumeRoleDelegates, []string{scope}, credentialsOpt...)
if err != nil {
return nil, errors.Wrapf(err, "failed to assume role")
Expand Down
184 changes: 0 additions & 184 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,100 +174,12 @@ const (
// to succeed without actually performing the update.
VPrimordialMax

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

// v22.2 versions.
//
// TODODelete_V22_2Start demarcates work towards CockroachDB v22.2.
TODODelete_V22_2Start

// TODODelete_V22_2PebbleFormatSplitUserKeysMarkedCompacted updates the Pebble format
// version that recombines all user keys that may be split across multiple
// files into a single table.
TODODelete_V22_2PebbleFormatSplitUserKeysMarkedCompacted
// TODODelete_V22_2EnsurePebbleFormatVersionRangeKeys is the first step of a two-part
// migration that bumps Pebble's format major version to a version that
// supports range keys.
TODODelete_V22_2EnsurePebbleFormatVersionRangeKeys
// TODODelete_V22_2EnablePebbleFormatVersionRangeKeys is the second of a two-part migration
// and is used as the feature gate for use of range keys. Any node at this
// version is guaranteed to reside in a cluster where all nodes support range
// keys at the Pebble layer.
TODODelete_V22_2EnablePebbleFormatVersionRangeKeys
// TODODelete_V22_2RemoveGrantPrivilege is the last step to migrate from the GRANT privilege to WITH GRANT OPTION.
TODODelete_V22_2RemoveGrantPrivilege
// TODODelete_V22_2MVCCRangeTombstones enables the use of MVCC range tombstones.
TODODelete_V22_2MVCCRangeTombstones
// TODODelete_V22_2SystemPrivilegesTable adds system.privileges table.
TODODelete_V22_2SystemPrivilegesTable
// TODODelete_V22_2EnablePredicateProjectionChangefeed indicates that changefeeds support
// predicates and projections.
TODODelete_V22_2EnablePredicateProjectionChangefeed
// TODODelete_V22_2SystemExternalConnectionsTable adds system.external_connections table.
TODODelete_V22_2SystemExternalConnectionsTable
// TODODelete_V22_2RoleIDSequence is the version where the system.role_id_sequence exists.
TODODelete_V22_2RoleIDSequence
// TODODelete_V22_2AddSystemUserIDColumn is the version where the system.users table has
// a user_id column for writes only.
TODODelete_V22_2AddSystemUserIDColumn
// TODODelete_V22_2SystemUsersIDColumnIsBackfilled is the version where all users in the system.users table
// have ids.
TODODelete_V22_2SystemUsersIDColumnIsBackfilled
// TODODelete_V22_2SetSystemUsersUserIDColumnNotNull sets the user_id column in system.users to not null.
TODODelete_V22_2SetSystemUsersUserIDColumnNotNull
// Permanent_V22_2SQLSchemaTelemetryScheduledJobs adds an automatic schedule for SQL schema
// telemetry logging jobs.
//
// This is a permanent migration which should exist forever.
Permanent_V22_2SQLSchemaTelemetryScheduledJobs
// TODODelete_V22_2SchemaChangeSupportsCreateFunction adds support of CREATE FUNCTION
// statement.
TODODelete_V22_2SchemaChangeSupportsCreateFunction
// TODODelete_V22_2PebbleFormatPrePebblev1Marked performs a Pebble-level migration and
// upgrades the Pebble format major version to FormatPrePebblev1Marked. This
// migration occurs at the per-store level and is twofold:
// - Each store is first bumped to a Pebble format major version that raises
// the minimum supported sstable format to (Pebble,v1) (block properties). New
// tables generated by Pebble (via compactions / flushes), and tables written
// for ingestion will be at table format version (Pebble,v1).
// - Each store is then instructed to mark all existing tables that are
// pre-Pebblev1 for a low-priority compaction. In a future release of
// Cockroach (likely 23.1), a blocking migration will be run to
// rewrite-compact on any remaining marked tables.
TODODelete_V22_2PebbleFormatPrePebblev1Marked
// TODODelete_V22_2RoleOptionsTableHasIDColumn is the version where the role options table
// has ids.
TODODelete_V22_2RoleOptionsTableHasIDColumn
// TODODelete_V22_2RoleOptionsIDColumnIsBackfilled is the version where ids in the role options
// table are backfilled.
TODODelete_V22_2RoleOptionsIDColumnIsBackfilled
// TODODelete_V22_2SetRoleOptionsUserIDColumnNotNull is the version where the role
// options table id column cannot be null. This is the final step
// of the system.role_options table migration.
TODODelete_V22_2SetRoleOptionsUserIDColumnNotNull
// TODODelete_V22_2RangefeedUseOneStreamPerNode changes rangefeed implementation to use 1 RPC stream per node.
TODODelete_V22_2RangefeedUseOneStreamPerNode
// TODODelete_V22_2TTLDistSQL uses DistSQL to distribute TTL SELECT/DELETE statements to
// leaseholder nodes.
TODODelete_V22_2TTLDistSQL
// TODODelete_V22_2PrioritizeSnapshots adds prioritization to sender snapshots. When this
// version is enabled, the receiver will look at the priority of snapshots
// using the fields added in 22.2.
TODODelete_V22_2PrioritizeSnapshots
// TODODelete_V22_2EnableLeaseUpgrade version gates a change in the lease transfer protocol
// whereby we only ever transfer expiration-based leases (and have
// recipients later upgrade them to the more efficient epoch based ones).
// This was done to limit the effects of ill-advised lease transfers since
// the incoming leaseholder would need to recognize itself as such within a
// few seconds. This needs version gating so that in mixed-version clusters,
// as part of lease transfers, we don't start sending out expiration based
// leases to nodes that (i) don't expect them for certain keyspans, and (ii)
// don't know to upgrade them to efficient epoch-based ones.
TODODelete_V22_2EnableLeaseUpgrade
// TODODelete_V22_2SupportAssumeRoleAuth is the version where assume role authorization is
// supported in cloud storage and KMS.
TODODelete_V22_2SupportAssumeRoleAuth

// V22_2 is CockroachDB v22.2. It's used for all v22.2.x patch releases.
V22_2
Expand Down Expand Up @@ -635,108 +547,12 @@ var rawVersionsSingleton = keyedVersions{
Key: VPrimordialMax,
Version: roachpb.Version{Major: 0, Minor: 0, Internal: 424242},
},
{
Key: TODODelete_V22_1,
Version: roachpb.Version{Major: 22, Minor: 1},
},

// v22.2 versions. Internal versions must be even.
{
Key: TODODelete_V22_2Start,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 2},
},
{
Key: TODODelete_V22_2PebbleFormatSplitUserKeysMarkedCompacted,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 6},
},
{
Key: TODODelete_V22_2EnsurePebbleFormatVersionRangeKeys,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 8},
},
{
Key: TODODelete_V22_2EnablePebbleFormatVersionRangeKeys,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 10},
},
{
Key: TODODelete_V22_2RemoveGrantPrivilege,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 14},
},
{
Key: TODODelete_V22_2MVCCRangeTombstones,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 16},
},
{
Key: TODODelete_V22_2SystemPrivilegesTable,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 24},
},
{
Key: TODODelete_V22_2EnablePredicateProjectionChangefeed,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 26},
},
{
Key: TODODelete_V22_2SystemExternalConnectionsTable,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 30},
},
{
Key: TODODelete_V22_2RoleIDSequence,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 34},
},
{
Key: TODODelete_V22_2AddSystemUserIDColumn,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 36},
},
{
Key: TODODelete_V22_2SystemUsersIDColumnIsBackfilled,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 38},
},
{
Key: TODODelete_V22_2SetSystemUsersUserIDColumnNotNull,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 40},
},
{
Key: Permanent_V22_2SQLSchemaTelemetryScheduledJobs,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 42},
},
{
Key: TODODelete_V22_2SchemaChangeSupportsCreateFunction,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 44},
},
{
Key: TODODelete_V22_2PebbleFormatPrePebblev1Marked,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 48},
},
{
Key: TODODelete_V22_2RoleOptionsTableHasIDColumn,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 50},
},
{
Key: TODODelete_V22_2RoleOptionsIDColumnIsBackfilled,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 52},
},
{
Key: TODODelete_V22_2SetRoleOptionsUserIDColumnNotNull,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 54},
},
{
Key: TODODelete_V22_2RangefeedUseOneStreamPerNode,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 60},
},
{
Key: TODODelete_V22_2TTLDistSQL,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 68},
},
{
Key: TODODelete_V22_2PrioritizeSnapshots,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 70},
},
{
Key: TODODelete_V22_2EnableLeaseUpgrade,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 72},
},
{
Key: TODODelete_V22_2SupportAssumeRoleAuth,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 74},
},
{
Key: V22_2,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 0},
Expand Down
4 changes: 1 addition & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
Expand Down Expand Up @@ -223,8 +222,7 @@ func (ds *DistSender) RangeFeedSpans(

rl := newCatchupScanRateLimiter(&ds.st.SV)

if ds.st.Version.IsActive(ctx, clusterversion.TODODelete_V22_2RangefeedUseOneStreamPerNode) &&
enableMuxRangeFeed && cfg.useMuxRangeFeed {
if enableMuxRangeFeed && cfg.useMuxRangeFeed {
return muxRangeFeed(ctx, cfg, spans, ds, rr, rl, eventCh)
}

Expand Down
Loading

0 comments on commit 52ae7d5

Please sign in to comment.