From 32307340f401a2449d2d98f6f89a4efff6f20172 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Tue, 1 Dec 2020 18:07:55 -0500 Subject: [PATCH] clusterversion: reduce stuttering in package API This package was extracted out of pkg/settings/cluster in #45455. Now that we're coupling it tightly with our migrations infrastructure (pkg/migration), let's remove some unnecessary stuttering for ease of use. ```diff - if .Version.IsActive(ctx, clusterversion.VersionSomething) { + if .Version.IsActive(ctx, clusterversion.Something) { ``` All the diffs here are just mechanical goland-renames. Release note: None --- pkg/ccl/backupccl/restore_job.go | 2 +- pkg/ccl/backupccl/restore_planning.go | 2 +- pkg/ccl/importccl/import_stmt.go | 2 +- pkg/clusterversion/.gitattributes | 2 +- pkg/clusterversion/BUILD.bazel | 2 +- pkg/clusterversion/clusterversion.go | 8 +- pkg/clusterversion/cockroach_versions.go | 204 +++++++++--------- pkg/clusterversion/cockroach_versions_test.go | 4 +- pkg/clusterversion/key_string.go | 47 ++++ pkg/clusterversion/keyed_versions.go | 4 +- pkg/clusterversion/setting.go | 2 +- pkg/clusterversion/versionkey_string.go | 47 ---- pkg/jobs/jobs.go | 2 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 2 +- .../kvserver/batcheval/cmd_end_transaction.go | 2 +- pkg/kv/kvserver/merge_queue.go | 2 +- pkg/kv/kvserver/replica_consistency.go | 4 +- pkg/kv/kvserver/replica_gossip.go | 2 +- pkg/kv/kvserver/replica_proposal.go | 2 +- pkg/kv/kvserver/replica_send.go | 2 +- pkg/migration/manager.go | 2 +- pkg/migration/migrations.go | 2 +- pkg/server/server.go | 2 +- pkg/server/server_sql.go | 8 +- pkg/sql/alter_column_type.go | 4 +- pkg/sql/alter_role.go | 4 +- pkg/sql/alter_table.go | 2 +- pkg/sql/catalog/bootstrap/metadata.go | 2 +- pkg/sql/catalog/catalogkv/namespace.go | 2 +- pkg/sql/catalog/lease/lease.go | 12 +- pkg/sql/crdb_internal.go | 2 +- pkg/sql/create_index.go | 2 +- pkg/sql/create_role.go | 2 +- pkg/sql/create_schema.go | 4 +- pkg/sql/create_table.go | 12 +- pkg/sql/create_table_test.go | 6 +- pkg/sql/create_type.go | 2 +- pkg/sql/create_view.go | 2 +- pkg/sql/descriptor.go | 2 +- pkg/sql/drop_index.go | 2 +- pkg/sql/pgwire/hba_conf.go | 4 +- pkg/sql/reparent_database.go | 4 +- pkg/sql/sem/builtins/builtins.go | 2 +- pkg/sql/sqlliveness/sqlliveness.go | 2 +- pkg/sqlmigrations/migrations.go | 26 +-- 45 files changed, 227 insertions(+), 231 deletions(-) create mode 100644 pkg/clusterversion/key_string.go delete mode 100644 pkg/clusterversion/versionkey_string.go diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 9364a7f576f9..586b7abd360a 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1153,7 +1153,7 @@ func (r *restoreResumer) Resume( details := r.job.Details().(jobspb.RestoreDetails) p := execCtx.(sql.JobExecContext) r.versionAtLeast20_2 = p.ExecCfg().Settings.Version.IsActive( - ctx, clusterversion.VersionLeasedDatabaseDescriptors) + ctx, clusterversion.LeasedDatabaseDescriptors) backupManifests, latestBackupManifest, sqlDescs, err := loadBackupSQLDescs( ctx, p, details, details.Encryption, diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 173d111788a0..69784cd3cda1 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1663,7 +1663,7 @@ func doRestorePlan( // // TODO(ajwerner): Remove this version check in 21.1. canResetModTime := p.ExecCfg().Settings.Version.IsActive( - ctx, clusterversion.VersionLeasedDatabaseDescriptors) + ctx, clusterversion.LeasedDatabaseDescriptors) if err := RewriteTableDescs( tables, descriptorRewrites, intoDB, canResetModTime, ); err != nil { diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 6b18239a5c19..7a0d7e10bf91 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -1027,7 +1027,7 @@ func prepareNewTableDescsForIngestion( } // TODO(ajwerner): Remove this in 21.1. canResetModTime := p.ExecCfg().Settings.Version.IsActive( - ctx, clusterversion.VersionLeasedDatabaseDescriptors) + ctx, clusterversion.LeasedDatabaseDescriptors) if err := backupccl.RewriteTableDescs( newMutableTableDescriptors, tableRewrites, "", canResetModTime, ); err != nil { diff --git a/pkg/clusterversion/.gitattributes b/pkg/clusterversion/.gitattributes index a7a30eb6d584..97091d3e2936 100644 --- a/pkg/clusterversion/.gitattributes +++ b/pkg/clusterversion/.gitattributes @@ -1 +1 @@ -versionkey_string.go binary +key_string.go binary diff --git a/pkg/clusterversion/BUILD.bazel b/pkg/clusterversion/BUILD.bazel index 6bb7f9c907fb..4e3738f166af 100644 --- a/pkg/clusterversion/BUILD.bazel +++ b/pkg/clusterversion/BUILD.bazel @@ -6,10 +6,10 @@ go_library( "cluster_version.pb.go", "clusterversion.go", "cockroach_versions.go", + "key_string.go", "keyed_versions.go", "setting.go", "testutils.go", - "versionkey_string.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/clusterversion", visibility = ["//visibility:public"], diff --git a/pkg/clusterversion/clusterversion.go b/pkg/clusterversion/clusterversion.go index 14e42a90feb2..d4a14d7499dd 100644 --- a/pkg/clusterversion/clusterversion.go +++ b/pkg/clusterversion/clusterversion.go @@ -99,7 +99,7 @@ type Handle interface { // outbound requests. When receiving these "new" inbound requests, despite // not seeing the latest active version, node2 is aware that the sending // node has, and it will too, eventually. - IsActive(context.Context, VersionKey) bool + IsActive(context.Context, Key) bool // BinaryVersion returns the build version of this binary. BinaryVersion() roachpb.Version @@ -206,7 +206,7 @@ func (v *handleImpl) SetActiveVersion(ctx context.Context, cv ClusterVersion) er } // IsActive implements the Handle interface. -func (v *handleImpl) IsActive(ctx context.Context, key VersionKey) bool { +func (v *handleImpl) IsActive(ctx context.Context, key Key) bool { return version.isActive(ctx, v.sv, key) } @@ -228,8 +228,8 @@ func (cv ClusterVersion) IsActiveVersion(v roachpb.Version) bool { // IsActive returns true if the features of the supplied version are active at // the running version. -func (cv ClusterVersion) IsActive(versionKey VersionKey) bool { - v := VersionByKey(versionKey) +func (cv ClusterVersion) IsActive(versionKey Key) bool { + v := ByKey(versionKey) return cv.IsActiveVersion(v) } diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 486782f5545d..4911baaceced 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -12,8 +12,8 @@ package clusterversion import "github.com/cockroachdb/cockroach/pkg/roachpb" -// VersionKey is a unique identifier for a version of CockroachDB. -type VersionKey int +// Key is a unique identifier for a version of CockroachDB. +type Key int // Version constants. You'll want to add a new one in the following cases: // @@ -80,12 +80,12 @@ type VersionKey int // (b) When cutting a major release branch. When cutting release-20.2 for // example, you'll want to introduce the following to `master`. // -// (i) Version20_2 (keyed to v20.2.0-0}) -// (ii) VersionStart21_1 (keyed to v20.2.0-1}) +// (i) V20_2 (keyed to v20.2.0-0}) +// (ii) Start21_1 (keyed to v20.2.0-1}) // // You'll then want to backport (i) to the release branch itself (i.e. // release-20.2). You'll also want to bump binaryMinSupportedVersion. In the -// example above, you'll set it to Version20_2. This indicates that the +// example above, you'll set it to V20_2. This indicates that the // minimum binary version required in a cluster with with nodes running // v21.1 binaries (including pre-release alphas) is v20.2, i.e. that an // upgrade into such a binary must start out from at least v20.2 nodes. @@ -106,26 +106,26 @@ type VersionKey int // You'll want to delete versions from this list after cutting a major release. // Once the development for 21.1 begins, after step (ii) from above, all // versions introduced in the previous release can be removed (everything prior -// to Version20_2 in our example). +// to V20_2 in our example). // // When removing a version, you'll want to remove its associated runtime checks. // All "is active" checks for the key will always evaluate to true. You'll also // want to delete the constant and remove its entry in the `versionsSingleton` // block below. // -//go:generate stringer -type=VersionKey +//go:generate stringer -type=Key const ( - _ VersionKey = iota - 1 // want first named one to start at zero + _ Key = iota - 1 // want first named one to start at zero // v20.1 versions. // - // VersionNamespaceTableWithSchemas is + // NamespaceTableWithSchemas is // https://github.com/cockroachdb/cockroach/pull/41977 // // It represents the migration to a new system.namespace table that has an // added parentSchemaID column. In addition to the new column, the table is // no longer in the system config range -- implying it is no longer gossiped. - VersionNamespaceTableWithSchemas + NamespaceTableWithSchemas // TODO(irfansharif): The versions above can/should all be removed. They // were orinally introduced in v20.1. There are inflight PRs to do so @@ -133,81 +133,77 @@ const ( // v20.2 versions. // - // VersionStart20_2 demarcates work towards CockroachDB v20.2. - VersionStart20_2 - // VersionGeospatialType enables the use of Geospatial features. - VersionGeospatialType - // VersionEnums enables the use of ENUM types. - VersionEnums - // VersionRangefeedLeases is the enablement of leases uses rangefeeds. All - // nodes with this versions will have rangefeeds enabled on all system - // ranges. Once this version is finalized, gossip is not needed in the - // schema lease subsystem. Nodes which start with this version finalized - // will not pass gossip to the SQL layer. - VersionRangefeedLeases - // VersionAlterColumnTypeGeneral enables the use of alter column type for + // Start20_2 demarcates work towards CockroachDB v20.2. + Start20_2 + // GeospatialType enables the use of Geospatial features. + GeospatialType + // Enums enables the use of ENUM types. + Enums + // RangefeedLeases is the enablement of leases uses rangefeeds. All nodes + // with this versions will have rangefeeds enabled on all system ranges. + // Once this version is finalized, gossip is not needed in the schema lease + // subsystem. Nodes which start with this version finalized will not pass + // gossip to the SQL layer. + RangefeedLeases + // AlterColumnTypeGeneral enables the use of alter column type for // conversions that require the column data to be rewritten. - VersionAlterColumnTypeGeneral - // VersionAlterSystemJobsTable is a version which modified system.jobs + AlterColumnTypeGeneral + // AlterSystemJobsTable is a version which modified system.jobs table. + AlterSystemJobsAddCreatedByColumns + // AddScheduledJobsTable is a version which adds system.scheduled_jobs // table. - VersionAlterSystemJobsAddCreatedByColumns - // VersionAddScheduledJobsTable is a version which adds - // system.scheduled_jobs table. - VersionAddScheduledJobsTable - // VersionUserDefinedSchemas enables the creation of user defined schemas. - VersionUserDefinedSchemas - // VersionNoOriginFKIndexes allows for foreign keys to no longer need - // indexes on the origin side of the relationship. - VersionNoOriginFKIndexes - // VersionClientRangeInfosOnBatchResponse moves the response RangeInfos from + AddScheduledJobsTable + // UserDefinedSchemas enables the creation of user defined schemas. + UserDefinedSchemas + // NoOriginFKIndexes allows for foreign keys to no longer need indexes on + // the origin side of the relationship. + NoOriginFKIndexes + // ClientRangeInfosOnBatchResponse moves the response RangeInfos from // individual response headers to the batch header. - VersionClientRangeInfosOnBatchResponse - // VersionNodeMembershipStatus gates the usage of the MembershipStatus enum - // in the Liveness proto. See comment on proto definition for more details. - VersionNodeMembershipStatus - // VersionRangeStatsRespHasDesc adds the RangeStatsResponse.RangeInfo field. - VersionRangeStatsRespHasDesc - // VersionMinPasswordLength adds the server.user_login.min_password_length - // setting. - VersionMinPasswordLength - // VersionAbortSpanBytes adds a field to MVCCStats + ClientRangeInfosOnBatchResponse + // NodeMembershipStatus gates the usage of the MembershipStatus enum in the + // Liveness proto. See comment on proto definition for more details. + NodeMembershipStatus + // RangeStatsRespHasDesc adds the RangeStatsResponse.RangeInfo field. + RangeStatsRespHasDesc + // MinPasswordLength adds the server.user_login.min_password_length setting. + MinPasswordLength + // AbortSpanBytes adds a field to MVCCStats // (MVCCStats.AbortSpanBytes) that tracks the size of a range's abort span. - VersionAbortSpanBytes - // VersionAlterSystemJobsTableAddLeaseColumn is a version which modified + AbortSpanBytes + // AlterSystemJobsTableAddLeaseColumn is a version which modified // system.jobs table. - VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable - // VersionMaterializedViews enables the use of materialized views. - VersionMaterializedViews - // VersionBox2DType enables the use of the box2d type. - VersionBox2DType - // VersionLeasedDatabasedDescriptors enables leased database descriptors. - // Now that we unconditionally use leased descriptors in 21.1 and the main - // usages of this version gate have been removed, this version remains to - // gate a few miscellaneous database descriptor changes. - VersionLeasedDatabaseDescriptors - // VersionUpdateScheduledJobsSchema drops schedule_changes and adds + AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable + // MaterializedViews enables the use of materialized views. + MaterializedViews + // Box2DType enables the use of the box2d type. + Box2DType + // LeasedDatabasedDescriptors enables leased database descriptors. Now that + // we unconditionally use leased descriptors in 21.1 and the main usages of + // this version gate have been removed, this version remains to gate a few + // miscellaneous database descriptor changes. + LeasedDatabaseDescriptors + // UpdateScheduledJobsSchema drops schedule_changes and adds // schedule_status. - VersionUpdateScheduledJobsSchema - // VersionCreateLoginPrivilege is when CREATELOGIN/NOCREATELOGIN are - // introduced. + UpdateScheduledJobsSchema + // CreateLoginPrivilege is when CREATELOGIN/NOCREATELOGIN are introduced. // // It represents adding authn principal management via CREATELOGIN role // option. - VersionCreateLoginPrivilege - // VersionHBAForNonTLS is when the 'hostssl' and 'hostnossl' HBA configs are + CreateLoginPrivilege + // HBAForNonTLS is when the 'hostssl' and 'hostnossl' HBA configs are // introduced. - VersionHBAForNonTLS - // Version20_2 is CockroachDB v20.2. It's used for all v20.2.x patch - // releases. - Version20_2 + HBAForNonTLS + // V20_2 is CockroachDB v20.2. It's used for all v20.2.x patch releases. + V20_2 // v21.1 versions. // - // VersionStart21_1 demarcates work towards CockroachDB v21.1. - VersionStart21_1 - // VersionEmptyArraysInInvertedIndexes is when empty arrays are added to - // array inverted indexes. - VersionEmptyArraysInInvertedIndexes + // Start21_1 demarcates work towards CockroachDB v21.1. + Start21_1 + // EmptyArraysInInvertedIndexes is when empty arrays are added to array + // inverted indexes. + EmptyArraysInInvertedIndexes // Step (1): Add new versions here. ) @@ -231,107 +227,107 @@ const ( // to be added (i.e., when cutting the final release candidate). var versionsSingleton = keyedVersions([]keyedVersion{ { - Key: VersionNamespaceTableWithSchemas, + Key: NamespaceTableWithSchemas, Version: roachpb.Version{Major: 19, Minor: 2, Internal: 5}, }, // v20.2 versions. { - Key: VersionStart20_2, + Key: Start20_2, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 1}, }, { - Key: VersionGeospatialType, + Key: GeospatialType, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 2}, }, { - Key: VersionEnums, + Key: Enums, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 3}, }, { - Key: VersionRangefeedLeases, + Key: RangefeedLeases, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 4}, }, { - Key: VersionAlterColumnTypeGeneral, + Key: AlterColumnTypeGeneral, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 5}, }, { - Key: VersionAlterSystemJobsAddCreatedByColumns, + Key: AlterSystemJobsAddCreatedByColumns, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 6}, }, { - Key: VersionAddScheduledJobsTable, + Key: AddScheduledJobsTable, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 7}, }, { - Key: VersionUserDefinedSchemas, + Key: UserDefinedSchemas, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 8}, }, { - Key: VersionNoOriginFKIndexes, + Key: NoOriginFKIndexes, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 9}, }, { - Key: VersionClientRangeInfosOnBatchResponse, + Key: ClientRangeInfosOnBatchResponse, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 10}, }, { - Key: VersionNodeMembershipStatus, + Key: NodeMembershipStatus, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 11}, }, { - Key: VersionRangeStatsRespHasDesc, + Key: RangeStatsRespHasDesc, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 12}, }, { - Key: VersionMinPasswordLength, + Key: MinPasswordLength, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 13}, }, { - Key: VersionAbortSpanBytes, + Key: AbortSpanBytes, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 14}, }, { - Key: VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable, + Key: AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 15}, }, { - Key: VersionMaterializedViews, + Key: MaterializedViews, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 16}, }, { - Key: VersionBox2DType, + Key: Box2DType, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 17}, }, { - Key: VersionLeasedDatabaseDescriptors, + Key: LeasedDatabaseDescriptors, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 18}, }, { - Key: VersionUpdateScheduledJobsSchema, + Key: UpdateScheduledJobsSchema, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 19}, }, { - Key: VersionCreateLoginPrivilege, + Key: CreateLoginPrivilege, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 20}, }, { - Key: VersionHBAForNonTLS, + Key: HBAForNonTLS, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 21}, }, { - Key: Version20_2, + Key: V20_2, Version: roachpb.Version{Major: 20, Minor: 2}, }, // v21.1 versions. Internal versions defined here-on-forth must be even. { - Key: VersionStart21_1, + Key: Start21_1, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 2}, }, { - Key: VersionEmptyArraysInInvertedIndexes, + Key: EmptyArraysInInvertedIndexes, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 4}, }, @@ -346,7 +342,7 @@ var ( // this binary. If this binary is started using a store marked with an older // version than binaryMinSupportedVersion, then the binary will exit with // an error. - binaryMinSupportedVersion = VersionByKey(Version20_2) + binaryMinSupportedVersion = ByKey(V20_2) // binaryVersion is the version of this binary. // @@ -354,19 +350,19 @@ var ( binaryVersion = versionsSingleton[len(versionsSingleton)-1].Version ) -// VersionByKey returns the roachpb.Version for a given key. +// ByKey returns the roachpb.Version for a given key. // It is a fatal error to use an invalid key. -func VersionByKey(key VersionKey) roachpb.Version { +func ByKey(key Key) roachpb.Version { return versionsSingleton.MustByKey(key) } -// GetVersionsBetween returns the list of cluster versions in the range +// ListBetween returns the list of cluster versions in the range // (from, to]. -func GetVersionsBetween(from, to ClusterVersion) []ClusterVersion { - return getVersionBetweenInternal(from, to, versionsSingleton) +func ListBetween(from, to ClusterVersion) []ClusterVersion { + return listBetweenInternal(from, to, versionsSingleton) } -func getVersionBetweenInternal(from, to ClusterVersion, vs keyedVersions) []ClusterVersion { +func listBetweenInternal(from, to ClusterVersion, vs keyedVersions) []ClusterVersion { var cvs []ClusterVersion for _, keyedV := range vs { // Read: "from < keyedV <= to". diff --git a/pkg/clusterversion/cockroach_versions_test.go b/pkg/clusterversion/cockroach_versions_test.go index abd0d0a6d6eb..e22bf880cb1c 100644 --- a/pkg/clusterversion/cockroach_versions_test.go +++ b/pkg/clusterversion/cockroach_versions_test.go @@ -89,7 +89,7 @@ func TestGetVersionsBetween(t *testing.T) { var vs keyedVersions for i := 3; i < 10; i++ { vs = append(vs, keyedVersion{ - Key: VersionKey(42), + Key: Key(42), Version: roachpb.Version{Major: int32(i)}, }) } @@ -117,7 +117,7 @@ func TestGetVersionsBetween(t *testing.T) { } for _, test := range tests { - actual := getVersionBetweenInternal(test.from, test.to, vs) + actual := listBetweenInternal(test.from, test.to, vs) if len(actual) != len(test.exp) { t.Errorf("expected %d versions, got %d", len(test.exp), len(actual)) } diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go new file mode 100644 index 000000000000..0abcd5ce0670 --- /dev/null +++ b/pkg/clusterversion/key_string.go @@ -0,0 +1,47 @@ +// Code generated by "stringer -type=Key"; DO NOT EDIT. + +package clusterversion + +import "strconv" + +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[NamespaceTableWithSchemas-0] + _ = x[Start20_2-1] + _ = x[GeospatialType-2] + _ = x[Enums-3] + _ = x[RangefeedLeases-4] + _ = x[AlterColumnTypeGeneral-5] + _ = x[AlterSystemJobsAddCreatedByColumns-6] + _ = x[AddScheduledJobsTable-7] + _ = x[UserDefinedSchemas-8] + _ = x[NoOriginFKIndexes-9] + _ = x[ClientRangeInfosOnBatchResponse-10] + _ = x[NodeMembershipStatus-11] + _ = x[RangeStatsRespHasDesc-12] + _ = x[MinPasswordLength-13] + _ = x[AbortSpanBytes-14] + _ = x[AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-15] + _ = x[MaterializedViews-16] + _ = x[Box2DType-17] + _ = x[LeasedDatabaseDescriptors-18] + _ = x[UpdateScheduledJobsSchema-19] + _ = x[CreateLoginPrivilege-20] + _ = x[HBAForNonTLS-21] + _ = x[V20_2-22] + _ = x[Start21_1-23] + _ = x[EmptyArraysInInvertedIndexes-24] +} + +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeLeasedDatabaseDescriptorsUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexes" + +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 211, 231, 252, 269, 283, 347, 364, 373, 398, 423, 443, 455, 460, 469, 497} + +func (i Key) String() string { + if i < 0 || i >= Key(len(_Key_index)-1) { + return "Key(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _Key_name[_Key_index[i]:_Key_index[i+1]] +} diff --git a/pkg/clusterversion/keyed_versions.go b/pkg/clusterversion/keyed_versions.go index 2ed8b3eff617..3f4ae584fe14 100644 --- a/pkg/clusterversion/keyed_versions.go +++ b/pkg/clusterversion/keyed_versions.go @@ -23,7 +23,7 @@ import ( // keyedVersion associates a key to a version. type keyedVersion struct { - Key VersionKey + Key Key roachpb.Version } @@ -31,7 +31,7 @@ type keyedVersion struct { type keyedVersions []keyedVersion // MustByKey asserts that the version specified by this key exists, and returns it. -func (kv keyedVersions) MustByKey(k VersionKey) roachpb.Version { +func (kv keyedVersions) MustByKey(k Key) roachpb.Version { key := int(k) if key >= len(kv) || key < 0 { log.Fatalf(context.Background(), "version with key %d does not exist, have:\n%s", diff --git a/pkg/clusterversion/setting.go b/pkg/clusterversion/setting.go index 945ba931d3f3..3d47498e5f2f 100644 --- a/pkg/clusterversion/setting.go +++ b/pkg/clusterversion/setting.go @@ -142,7 +142,7 @@ func (cv *clusterVersionSetting) activeVersionOrEmpty( // isActive returns true if the features of the supplied version key are active // at the running version. See comment on Handle.IsActive for intended usage. func (cv *clusterVersionSetting) isActive( - ctx context.Context, sv *settings.Values, versionKey VersionKey, + ctx context.Context, sv *settings.Values, versionKey Key, ) bool { return cv.activeVersion(ctx, sv).IsActive(versionKey) } diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go deleted file mode 100644 index 61075b57e639..000000000000 --- a/pkg/clusterversion/versionkey_string.go +++ /dev/null @@ -1,47 +0,0 @@ -// Code generated by "stringer -type=VersionKey"; DO NOT EDIT. - -package clusterversion - -import "strconv" - -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[VersionNamespaceTableWithSchemas-0] - _ = x[VersionStart20_2-1] - _ = x[VersionGeospatialType-2] - _ = x[VersionEnums-3] - _ = x[VersionRangefeedLeases-4] - _ = x[VersionAlterColumnTypeGeneral-5] - _ = x[VersionAlterSystemJobsAddCreatedByColumns-6] - _ = x[VersionAddScheduledJobsTable-7] - _ = x[VersionUserDefinedSchemas-8] - _ = x[VersionNoOriginFKIndexes-9] - _ = x[VersionClientRangeInfosOnBatchResponse-10] - _ = x[VersionNodeMembershipStatus-11] - _ = x[VersionRangeStatsRespHasDesc-12] - _ = x[VersionMinPasswordLength-13] - _ = x[VersionAbortSpanBytes-14] - _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-15] - _ = x[VersionMaterializedViews-16] - _ = x[VersionBox2DType-17] - _ = x[VersionLeasedDatabaseDescriptors-18] - _ = x[VersionUpdateScheduledJobsSchema-19] - _ = x[VersionCreateLoginPrivilege-20] - _ = x[VersionHBAForNonTLS-21] - _ = x[Version20_2-22] - _ = x[VersionStart21_1-23] - _ = x[VersionEmptyArraysInInvertedIndexes-24] -} - -const _VersionKey_name = "VersionNamespaceTableWithSchemasVersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1VersionEmptyArraysInInvertedIndexes" - -var _VersionKey_index = [...]uint16{0, 32, 48, 69, 81, 103, 132, 173, 201, 226, 250, 288, 315, 343, 367, 388, 459, 483, 499, 531, 563, 590, 609, 620, 636, 671} - -func (i VersionKey) String() string { - if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { - return "VersionKey(" + strconv.FormatInt(int64(i), 10) + ")" - } - return _VersionKey_name[_VersionKey_index[i]:_VersionKey_index[i+1]] -} diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index 542ad6a406a4..e56beab8374a 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -716,7 +716,7 @@ func (j *Job) load(ctx context.Context) error { if err := j.runInTxn(ctx, func(ctx context.Context, txn *kv.Txn) error { const newStmt = "SELECT payload, progress, created_by_type, created_by_id FROM system.jobs WHERE id = $1" const oldStmt = "SELECT payload, progress FROM system.jobs WHERE id = $1" - hasCreatedBy := j.registry.settings.Version.IsActive(ctx, clusterversion.VersionAlterSystemJobsAddCreatedByColumns) + hasCreatedBy := j.registry.settings.Version.IsActive(ctx, clusterversion.AlterSystemJobsAddCreatedByColumns) stmt := oldStmt if hasCreatedBy { stmt = newStmt diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index a85ed133787c..911781236adc 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -1549,7 +1549,7 @@ func (ds *DistSender) sendPartialBatch( // the br. if ba.ReturnRangeInfo && len(reply.RangeInfos) == 0 && - !ds.st.Version.IsActive(ctx, clusterversion.VersionClientRangeInfosOnBatchResponse) { + !ds.st.Version.IsActive(ctx, clusterversion.ClientRangeInfosOnBatchResponse) { // All the responses have the same RangeInfos in them, so just look at the // first one. firstRes := reply.Responses[0].GetInner() diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 01c0c22930b0..df228fd6dae3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -885,7 +885,7 @@ func splitTriggerHelper( return enginepb.MVCCStats{}, result.Result{}, err } - if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionAbortSpanBytes) { + if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.AbortSpanBytes) { // Since the stats here is used to seed the initial state for the RHS // replicas, we need to be careful about zero-ing out the abort span // bytes if the cluster version introducing it is not yet active. Not diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 0f1af5509771..c628e8b12b99 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -180,7 +180,7 @@ func (mq *mergeQueue) requestRangeStats( RequestHeader: roachpb.RequestHeader{Key: key}, }) - if !mq.store.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionRangeStatsRespHasDesc) { + if !mq.store.ClusterSettings().Version.IsActive(ctx, clusterversion.RangeStatsRespHasDesc) { ba.Header.ReturnRangeInfo = true } diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 1afde8723ac4..f0dd22c55f17 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -233,10 +233,10 @@ func (r *Replica) CheckConsistency( // are consistent. Verify this only for clusters that started out on 19.1 or // higher. if !v.Less(roachpb.Version{Major: 19, Minor: 1}) { - // If version >= 19.1 but < VersionAbortSpanBytes, we want to ignore any delta + // If version >= 19.1 but < AbortSpanBytes, we want to ignore any delta // in AbortSpanBytes when comparing stats since older versions will not be // tracking abort span bytes. - if v.Less(clusterversion.VersionByKey(clusterversion.VersionAbortSpanBytes)) { + if v.Less(clusterversion.ByKey(clusterversion.AbortSpanBytes)) { delta.AbortSpanBytes = 0 haveDelta = delta != enginepb.MVCCStats{} } diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index c9e6cc01cb0d..e914e40dab23 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -177,7 +177,7 @@ func (r *Replica) MaybeGossipNodeLiveness(ctx context.Context, span roachpb.Span continue } } - if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionNodeMembershipStatus) { + if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.NodeMembershipStatus) { // We can't transmit liveness records with a backwards incompatible // representation unless we're told by the user that there are no // pre-v20.1 nodes around. We should never get here. diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 16b6f4a56e74..dee979f062fa 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -800,7 +800,7 @@ func (r *Replica) evaluateProposal( // If the cluster version doesn't track abort span size in MVCCStats, we // zero it out to prevent inconsistencies in MVCCStats across nodes in a // possibly mixed-version cluster. - if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionAbortSpanBytes) { + if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.AbortSpanBytes) { res.Replicated.Delta.AbortSpanBytes = 0 } diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 81bbf2c0fa89..82ad6aad7628 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -135,7 +135,7 @@ func (r *Replica) maybeAddRangeInfoToResponse( desc, lease := r.GetDescAndLease(ctx) br.RangeInfos = []roachpb.RangeInfo{{Desc: desc, Lease: lease}} - if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionClientRangeInfosOnBatchResponse) { + if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.ClientRangeInfosOnBatchResponse) { // Also set the RangeInfo on the individual responses, for compatibility // with 20.1. for _, r := range br.Responses { diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index b29244962001..25488e7ec2a1 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -208,7 +208,7 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // TODO(irfansharif): We'll need to create a system table to store // in-progress state of long running migrations, for introspection. - clusterVersions := clusterversion.GetVersionsBetween(from, to) + clusterVersions := clusterversion.ListBetween(from, to) if len(clusterVersions) == 0 { // We're attempt to migrate to something that's not defined in cluster // versions. This only happens in tests, when we're exercising version diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index e3b0be47ed53..dfae1ee82155 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -21,5 +21,5 @@ func init() { // TODO(irfansharif): We'll want to register individual migrations with // specific internal cluster versions here. // - // Registry[clusterversion.VersionByKey(clusterversion.VersionWhatever)] = WhateverMigration + // Registry[clusterversion.ByKey(clusterversion.VersionWhatever)] = WhateverMigration } diff --git a/pkg/server/server.go b/pkg/server/server.go index 7c3e1ac98e42..07714a904497 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -2000,7 +2000,7 @@ func (s *sqlServer) startServeSQL( func (s *Server) Decommission( ctx context.Context, targetStatus livenesspb.MembershipStatus, nodeIDs []roachpb.NodeID, ) error { - if !s.st.Version.IsActive(ctx, clusterversion.VersionNodeMembershipStatus) { + if !s.st.Version.IsActive(ctx, clusterversion.NodeMembershipStatus) { if targetStatus.Decommissioned() { // In mixed-version cluster settings, we need to ensure that we're // on-the-wire compatible with nodes only familiar with the boolean diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index a89648b6f0a8..6fa9ecf4449e 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -785,11 +785,11 @@ func (s *sqlServer) preStart( // Since we don't record this version anywhere, we do the next-best thing // and pass a lower-bound on the bootstrap version. We know that no tenants // could have been created before the start of the v20.2 dev cycle, so we - // pass VersionStart20_2. bootstrapVersion is only used to avoid performing + // pass Start20_2. bootstrapVersion is only used to avoid performing // superfluous but necessarily idempotent SQL migrations, so at worst, we're // doing more work than strictly necessary during the first time that the // migrations are run. - bootstrapVersion = clusterversion.VersionByKey(clusterversion.VersionStart20_2) + bootstrapVersion = clusterversion.ByKey(clusterversion.Start20_2) } // Run startup migrations (note: these depend on jobs subsystem running). @@ -809,8 +809,8 @@ func (s *sqlServer) preStart( // This clause exists to support sqlmigrations tests which intentionally // inject a binary version below the one which includes the relevant // migration. In this case we won't start the sqlliveness subsystem. - (!s.execCfg.Settings.Version.BinaryVersion().Less(clusterversion.VersionByKey( - clusterversion.VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable))) { + (!s.execCfg.Settings.Version.BinaryVersion().Less(clusterversion.ByKey( + clusterversion.AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable))) { s.sqlLivenessProvider.Start(ctx) } // Start the async migration to upgrade namespace entries from the old diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index 968e73679c32..89b07770c291 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -162,11 +162,11 @@ func alterColumnTypeGeneral( // general alter column type conversions. if !params.p.ExecCfg().Settings.Version.IsActive( params.ctx, - clusterversion.VersionAlterColumnTypeGeneral, + clusterversion.AlterColumnTypeGeneral, ) { return pgerror.Newf(pgcode.FeatureNotSupported, "version %v must be finalized to run this alter column type", - clusterversion.VersionAlterColumnTypeGeneral) + clusterversion.AlterColumnTypeGeneral) } if !params.SessionData().AlterColumnTypeGeneralEnabled { return pgerror.WithCandidateCode( diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index f5bac7a42ff3..d6da56fc68e8 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -87,12 +87,12 @@ func (p *planner) AlterRoleNode( func (p *planner) checkPasswordOptionConstraints( ctx context.Context, roleOptions roleoption.List, newUser bool, ) error { - if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.VersionCreateLoginPrivilege) { + if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.CreateLoginPrivilege) { // TODO(knz): Remove this condition in 21.1. if roleOptions.Contains(roleoption.CREATELOGIN) || roleOptions.Contains(roleoption.NOCREATELOGIN) { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, `granting CREATELOGIN or NOCREATELOGIN requires all nodes to be upgraded to %s`, - clusterversion.VersionByKey(clusterversion.VersionCreateLoginPrivilege)) + clusterversion.ByKey(clusterversion.CreateLoginPrivilege)) } } diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 920d5f1557aa..e79bc10afa81 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -534,7 +534,7 @@ func (n *alterTableNode) startExec(params runParams) error { // Since we are able to drop indexes used by foreign keys on the origin side, // the drop index codepaths aren't going to remove dependent FKs, so we // need to do that here. - if params.p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.VersionNoOriginFKIndexes) { + if params.p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.NoOriginFKIndexes) { // We update the FK's slice in place here. sliceIdx := 0 for i := range n.tableDesc.OutboundFKs { diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 6ecf2c285127..3bd9518253e5 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -269,7 +269,7 @@ func LookupSystemTableDescriptorID( } if settings != nil && - !settings.Version.IsActive(ctx, clusterversion.VersionNamespaceTableWithSchemas) && + !settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) && tableName == systemschema.NamespaceTableName { return systemschema.DeprecatedNamespaceTable.ID } diff --git a/pkg/sql/catalog/catalogkv/namespace.go b/pkg/sql/catalog/catalogkv/namespace.go index 69d2ffb55d8f..5840c3bd75c6 100644 --- a/pkg/sql/catalog/catalogkv/namespace.go +++ b/pkg/sql/catalog/catalogkv/namespace.go @@ -129,7 +129,7 @@ func MakeObjectNameKey( name string, ) catalogkeys.DescriptorKey { // TODO(solon): This if condition can be removed in 20.2 - if !settings.Version.IsActive(ctx, clusterversion.VersionNamespaceTableWithSchemas) { + if !settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) { return catalogkeys.NewDeprecatedTableKey(parentID, name) } var key catalogkeys.DescriptorKey diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 7caf31da086b..d170aec84145 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -1653,7 +1653,7 @@ func (m *Manager) findDescriptorState(id descpb.ID, create bool) *descriptorStat // RefreshLeases starts a goroutine that refreshes the lease manager // leases for descriptors received in the latest system configuration via gossip or // rangefeeds. This function must be passed a non-nil gossip if -// VersionRangefeedLeases is not active. +// RangefeedLeases is not active. func (m *Manager) RefreshLeases( ctx context.Context, s *stop.Stopper, db *kv.DB, g gossip.OptionalGossip, ) { @@ -1718,7 +1718,7 @@ func (m *Manager) watchForUpdates( descUpdateCh chan *descpb.Descriptor, ) { useRangefeeds := m.testingKnobs.AlwaysUseRangefeeds || - m.storage.settings.Version.IsActive(ctx, clusterversion.VersionRangefeedLeases) + m.storage.settings.Version.IsActive(ctx, clusterversion.RangefeedLeases) if useRangefeeds { m.watchForRangefeedUpdates(ctx, s, db, descUpdateCh) return @@ -1761,8 +1761,8 @@ func (m *Manager) watchForGossipUpdates( ) { rawG, err := g.OptionalErr(47150) if err != nil { - if v := clusterversion.VersionRangefeedLeases; !m.storage.settings.Version.IsActive(ctx, v) { - log.Fatalf(ctx, "required gossip until %v is active: %v", clusterversion.VersionRangefeedLeases, err) + if v := clusterversion.RangefeedLeases; !m.storage.settings.Version.IsActive(ctx, v) { + log.Fatalf(ctx, "required gossip until %v is active: %v", clusterversion.RangefeedLeases, err) } return } @@ -1771,7 +1771,7 @@ func (m *Manager) watchForGossipUpdates( descKeyPrefix := m.storage.codec.TablePrefix(uint32(systemschema.DescriptorTable.ID)) // TODO(ajwerner): Add a mechanism to unregister this channel upon // return. NB: this call is allowed to bypass OptionalGossip because - // we'll never get here after VersionRangefeedLeases. + // we'll never get here after RangefeedLeases. gossipUpdateC := rawG.RegisterSystemConfigChannel() filter := gossip.MakeSystemConfigDeltaFilter(descKeyPrefix) @@ -1955,7 +1955,7 @@ func (m *Manager) waitForRangefeedsToBeUsable(ctx context.Context, s *stop.Stopp select { case <-timer.C: timer.Read = true - if m.storage.settings.Version.IsActive(ctx, clusterversion.VersionRangefeedLeases) { + if m.storage.settings.Version.IsActive(ctx, clusterversion.RangefeedLeases) { close(upgradeChan) return } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index efe91ad2ecbe..c95e8a7f78e2 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2632,7 +2632,7 @@ func getAllNames( ctx context.Context, txn *kv.Txn, executor *InternalExecutor, ) (map[descpb.ID]NamespaceKey, error) { namespace := map[descpb.ID]NamespaceKey{} - if executor.s.cfg.Settings.Version.IsActive(ctx, clusterversion.VersionNamespaceTableWithSchemas) { + if executor.s.cfg.Settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) { rows, err := executor.Query( ctx, "get-all-names", txn, `SELECT id, "parentID", "parentSchemaID", name FROM system.namespace`, diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 1e8c64fb55a0..5203792c7c65 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -420,7 +420,7 @@ func (n *createIndexNode) startExec(params runParams) error { } encodingVersion := descpb.SecondaryIndexFamilyFormatVersion - if params.p.EvalContext().Settings.Version.IsActive(params.ctx, clusterversion.VersionEmptyArraysInInvertedIndexes) { + if params.p.EvalContext().Settings.Version.IsActive(params.ctx, clusterversion.EmptyArraysInInvertedIndexes) { encodingVersion = descpb.EmptyArraysInInvertedIndexesVersion } indexDesc.Version = encodingVersion diff --git a/pkg/sql/create_role.go b/pkg/sql/create_role.go index 8e1c86ee3bfb..65adc3cd1d43 100644 --- a/pkg/sql/create_role.go +++ b/pkg/sql/create_role.go @@ -322,7 +322,7 @@ func (p *planner) checkPasswordAndGetHash( } st := p.ExecCfg().Settings - if st.Version.IsActive(ctx, clusterversion.VersionMinPasswordLength) { + if st.Version.IsActive(ctx, clusterversion.MinPasswordLength) { if minLength := security.MinPasswordLength.Get(&st.SV); minLength >= 1 && int64(len(password)) < minLength { return hashedPassword, errors.WithHintf(security.ErrPasswordTooShort, "Passwords must be %d characters or longer.", minLength) diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index 9c8b779b4534..b9b73a07d001 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -96,10 +96,10 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema } // Ensure that the cluster version is high enough to create the schema. - if !params.p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.VersionUserDefinedSchemas) { + if !params.p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.UserDefinedSchemas) { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, `creating schemas requires all nodes to be upgraded to %s`, - clusterversion.VersionByKey(clusterversion.VersionUserDefinedSchemas)) + clusterversion.ByKey(clusterversion.UserDefinedSchemas)) } // Create the ID. diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 24cb213835b4..7b0baedb1fb5 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -80,10 +80,10 @@ type createTableRun struct { // minimumTypeUsageVersions defines the minimum version needed for a new // data type. -var minimumTypeUsageVersions = map[types.Family]clusterversion.VersionKey{ - types.GeographyFamily: clusterversion.VersionGeospatialType, - types.GeometryFamily: clusterversion.VersionGeospatialType, - types.Box2DFamily: clusterversion.VersionBox2DType, +var minimumTypeUsageVersions = map[types.Family]clusterversion.Key{ + types.GeographyFamily: clusterversion.GeospatialType, + types.GeometryFamily: clusterversion.GeospatialType, + types.Box2DFamily: clusterversion.Box2DType, } // isTypeSupportedInVersion returns whether a given type is supported in the given version. @@ -787,7 +787,7 @@ func ResolveFK( // Check if the version is high enough to stop creating origin indexes. if evalCtx.Settings != nil && - !evalCtx.Settings.Version.IsActive(ctx, clusterversion.VersionNoOriginFKIndexes) { + !evalCtx.Settings.Version.IsActive(ctx, clusterversion.NoOriginFKIndexes) { // Search for an index on the origin table that matches. If one doesn't exist, // we create one automatically if the table to alter is new or empty. We also // search if an index for the set of columns was created in this transaction. @@ -1248,7 +1248,7 @@ func NewTableDesc( // server setup before the cluster version has been initialized. version := st.Version.ActiveVersionOrEmpty(ctx) if version != (clusterversion.ClusterVersion{}) { - if version.IsActive(clusterversion.VersionEmptyArraysInInvertedIndexes) { + if version.IsActive(clusterversion.EmptyArraysInInvertedIndexes) { indexEncodingVersion = descpb.EmptyArraysInInvertedIndexesVersion } } diff --git a/pkg/sql/create_table_test.go b/pkg/sql/create_table_test.go index 436370540c6a..406ef49db139 100644 --- a/pkg/sql/create_table_test.go +++ b/pkg/sql/create_table_test.go @@ -26,18 +26,18 @@ func TestIsTypeSupportedInVersion(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { - v clusterversion.VersionKey + v clusterversion.Key t *types.T ok bool }{ - {clusterversion.VersionGeospatialType, types.Geometry, true}, + {clusterversion.GeospatialType, types.Geometry, true}, } for _, tc := range testCases { t.Run(fmt.Sprintf("%s:%s", tc.v, tc.t.SQLString()), func(t *testing.T) { ok, err := isTypeSupportedInVersion( - clusterversion.ClusterVersion{Version: clusterversion.VersionByKey(tc.v)}, + clusterversion.ClusterVersion{Version: clusterversion.ByKey(tc.v)}, tc.t, ) require.NoError(t, err) diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go index eacecab21c42..00c7cce8acb7 100644 --- a/pkg/sql/create_type.go +++ b/pkg/sql/create_type.go @@ -275,7 +275,7 @@ func (p *planner) createEnumWithID( enumType enumType, ) error { // Make sure that all nodes in the cluster are able to recognize ENUM types. - if !p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.VersionEnums) { + if !p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.Enums) { return pgerror.Newf(pgcode.FeatureNotSupported, "not all nodes are the correct version for ENUM type creation") } diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 10f13311ac14..d06c6b44beb2 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -180,7 +180,7 @@ func (n *createViewNode) startExec(params runParams) error { if n.materialized { // Ensure all nodes are the correct version. - if !params.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.VersionMaterializedViews) { + if !params.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.MaterializedViews) { return pgerror.New(pgcode.FeatureNotSupported, "all nodes are not the correct version to use materialized views") } diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index 4de9af60e797..3f13196f144c 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -68,7 +68,7 @@ func (p *planner) createDatabase( // TODO(solon): This conditional can be removed in 20.2. Every database // is created with a public schema for cluster version >= 20.1, so we can remove // the `shouldCreatePublicSchema` logic as well. - if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.VersionNamespaceTableWithSchemas) { + if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) { shouldCreatePublicSchema = false } diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index 8f513f42897d..0de870939c35 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -356,7 +356,7 @@ func (p *planner) dropIndexByName( // If the we aren't at a high enough version to drop indexes on the origin // side then we have to attempt to delete them. - if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.VersionNoOriginFKIndexes) { + if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.NoOriginFKIndexes) { // Index for updating the FK slices in place when removing FKs. sliceIdx := 0 for i := range tableDesc.OutboundFKs { diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go index c1b8816e4f61..629b7e2608d6 100644 --- a/pkg/sql/pgwire/hba_conf.go +++ b/pkg/sql/pgwire/hba_conf.go @@ -145,10 +145,10 @@ func checkHBASyntaxBeforeUpdatingSetting(values *settings.Values, s string) erro case hba.ConnLocal: case hba.ConnHostSSL, hba.ConnHostNoSSL: if vh != nil && - !vh.IsActive(context.TODO(), clusterversion.VersionHBAForNonTLS) { + !vh.IsActive(context.TODO(), clusterversion.HBAForNonTLS) { return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, `authentication rule type 'hostssl'/'hostnossl' requires all nodes to be upgraded to %s`, - clusterversion.VersionByKey(clusterversion.VersionHBAForNonTLS), + clusterversion.ByKey(clusterversion.HBAForNonTLS), ) } diff --git a/pkg/sql/reparent_database.go b/pkg/sql/reparent_database.go index 6fcac10a72f5..dfe46555f08a 100644 --- a/pkg/sql/reparent_database.go +++ b/pkg/sql/reparent_database.go @@ -54,10 +54,10 @@ func (p *planner) ReparentDatabase( } // Ensure that the cluster version is high enough to create the schema. - if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.VersionUserDefinedSchemas) { + if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.UserDefinedSchemas) { return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, `creating schemas requires all nodes to be upgraded to %s`, - clusterversion.VersionByKey(clusterversion.VersionUserDefinedSchemas)) + clusterversion.ByKey(clusterversion.UserDefinedSchemas)) } if string(n.Name) == p.CurrentDatabase() { diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 1aa5e9bb4e8a..ca15c0bd6ac8 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -6611,7 +6611,7 @@ func arrayNumInvertedIndexEntries( v := descpb.SecondaryIndexFamilyFormatVersion if version == tree.DNull { if ctx.Settings.Version.IsActive( - ctx.Context, clusterversion.VersionEmptyArraysInInvertedIndexes, + ctx.Context, clusterversion.EmptyArraysInInvertedIndexes, ) { v = descpb.EmptyArraysInInvertedIndexesVersion } diff --git a/pkg/sql/sqlliveness/sqlliveness.go b/pkg/sql/sqlliveness/sqlliveness.go index 505d0e4727b4..9d1a97a345c2 100644 --- a/pkg/sql/sqlliveness/sqlliveness.go +++ b/pkg/sql/sqlliveness/sqlliveness.go @@ -87,7 +87,7 @@ type Reader interface { func IsActive(ctx context.Context, settings *cluster.Settings) bool { return settings.Version.IsActive( ctx, - clusterversion.VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable, + clusterversion.AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable, ) } diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index d0f615472afe..98d6743aeea2 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -261,7 +261,7 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // upgraded to the 20.1 betas with the problem. name: "create new system.namespace table v2", workFn: createNewSystemNamespaceDescriptor, - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionNamespaceTableWithSchemas), + includedInBootstrap: clusterversion.ByKey(clusterversion.NamespaceTableWithSchemas), newDescriptorIDs: staticIDs(keys.NamespaceTableID), }, { @@ -269,7 +269,7 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // StartSystemNamespaceMigration post-finalization-style migration. name: "migrate system.namespace_deprecated entries into system.namespace", // workFn: migrateSystemNamespace, - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionNamespaceTableWithSchemas), + includedInBootstrap: clusterversion.ByKey(clusterversion.NamespaceTableWithSchemas), }, { // Introduced in v20.1, baked into v20.2. @@ -289,22 +289,22 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // Introduced in v20.2. name: "add created_by columns to system.jobs", workFn: alterSystemJobsAddCreatedByColumns, - includedInBootstrap: clusterversion.VersionByKey( - clusterversion.VersionAlterSystemJobsAddCreatedByColumns), + includedInBootstrap: clusterversion.ByKey( + clusterversion.AlterSystemJobsAddCreatedByColumns), }, { // Introduced in v20.2. name: "create new system.scheduled_jobs table", workFn: createScheduledJobsTable, - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionAddScheduledJobsTable), + includedInBootstrap: clusterversion.ByKey(clusterversion.AddScheduledJobsTable), newDescriptorIDs: staticIDs(keys.ScheduledJobsTableID), }, { // Introduced in v20.2. name: "add new sqlliveness table and claim columns to system.jobs", workFn: alterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable, - includedInBootstrap: clusterversion.VersionByKey( - clusterversion.VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable), + includedInBootstrap: clusterversion.ByKey( + clusterversion.AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable), }, { // Introduced in v20.2. @@ -313,14 +313,14 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // NB: no dedicated cluster version was introduced for this table at the // time (4272248e573cbaa4fac436b0ea07195fcd648845). The below is the first // cluster version that was added after the system.tenants table. - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionAlterColumnTypeGeneral), + includedInBootstrap: clusterversion.ByKey(clusterversion.AlterColumnTypeGeneral), newDescriptorIDs: staticIDs(keys.TenantsTableID), }, { // Introduced in v20.2. name: "alter scheduled jobs", workFn: alterSystemScheduledJobsFixTableSchema, - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionUpdateScheduledJobsSchema), + includedInBootstrap: clusterversion.ByKey(clusterversion.UpdateScheduledJobsSchema), }, { // Introduced in v20.2. @@ -331,7 +331,7 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // Introduced in v20.2. name: "mark non-terminal schema change jobs with a pre-20.1 format version as failed", workFn: markDeprecatedSchemaChangeJobsFailed, - includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionLeasedDatabaseDescriptors), + includedInBootstrap: clusterversion.ByKey(clusterversion.LeasedDatabaseDescriptors), }, } @@ -719,7 +719,7 @@ var systemNamespaceMigrationEnabled = settings.RegisterBoolSetting( func (m *Manager) StartSystemNamespaceMigration( ctx context.Context, bootstrapVersion roachpb.Version, ) error { - if !bootstrapVersion.Less(clusterversion.VersionByKey(clusterversion.VersionNamespaceTableWithSchemas)) { + if !bootstrapVersion.Less(clusterversion.ByKey(clusterversion.NamespaceTableWithSchemas)) { // Our bootstrap version is equal to or greater than 20.1, where no old // namespace table is created: we can skip this migration. return nil @@ -738,7 +738,7 @@ func (m *Manager) StartSystemNamespaceMigration( if !systemNamespaceMigrationEnabled.Get(&m.settings.SV) { continue } - if m.settings.Version.IsActive(ctx, clusterversion.VersionNamespaceTableWithSchemas) { + if m.settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) { break } } @@ -792,7 +792,7 @@ func (m *Manager) StartSystemNamespaceMigration( // Only entries that do not exist in the new table are copied. // // New database and table entries continue to be written to the deprecated -// namespace table until VersionNamespaceTableWithSchemas is active. This means +// namespace table until NamespaceTableWithSchemas is active. This means // that an additional migration will be necessary in 20.2 to catch any new // entries which may have been missed by this one. In the meantime, namespace // lookups fall back to the deprecated table if a name is not found in the new