From 81070225a0e8ebb40e2fbb4deb70433f675d5fbd Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 20:53:49 -0500 Subject: [PATCH 1/5] clusterversion,*: remove VersionContainsEstimatesCounter This, and all surrounding migration code and tests, are now safe to remove. It mostly served as documentation, which we've moved to the field itself. Part of #47447. Fixes #56401. (While here, Let's also tell git that `versionkey_string.go` is a generated file.) Release note: None --- pkg/clusterversion/.gitattributes | 1 + pkg/clusterversion/cockroach_versions.go | 21 ---- pkg/clusterversion/versionkey_string.go | 57 ++++++----- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 2 - .../kvserver/batcheval/cmd_end_transaction.go | 3 - .../kvserver/batcheval/cmd_recompute_stats.go | 7 -- .../replica_application_state_machine.go | 39 +------- pkg/kv/kvserver/replica_proposal.go | 23 +---- pkg/kv/kvserver/replica_test.go | 97 +------------------ pkg/storage/BUILD.bazel | 1 - pkg/storage/enginepb/mvcc.pb.go | 30 ++++-- pkg/storage/enginepb/mvcc.proto | 16 ++- pkg/storage/mvcc.go | 2 - 13 files changed, 71 insertions(+), 228 deletions(-) create mode 100644 pkg/clusterversion/.gitattributes diff --git a/pkg/clusterversion/.gitattributes b/pkg/clusterversion/.gitattributes new file mode 100644 index 000000000000..a7a30eb6d584 --- /dev/null +++ b/pkg/clusterversion/.gitattributes @@ -0,0 +1 @@ +versionkey_string.go binary diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 0ecedb516ba4..7ba8009d4c73 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -121,23 +121,6 @@ const ( // v20.1 versions. // - // VersionContainsEstimatesCounter is - // https://github.com/cockroachdb/cockroach/pull/37583. - // - // MVCCStats.ContainsEstimates has been migrated from boolean to a - // counter so that the consistency checker and splits can reset it by - // returning -ContainsEstimates, avoiding racing with other operations - // that want to also change it. - // - // The migration maintains the invariant that raft commands with - // ContainsEstimates zero or one want the bool behavior (i.e. 1+1=1). - // Before the cluster version is active, at proposal time we'll refuse - // any negative ContainsEstimates plus we clamp all others to {0,1}. - // When the version is active, and ContainsEstimates is positive, we - // multiply it by 2 (i.e. we avoid 1). Downstream of raft, we use old - // behavior for ContainsEstimates=1 and the additive behavior for - // anything else. - VersionContainsEstimatesCounter // VersionNamespaceTableWithSchemas is // https://github.com/cockroachdb/cockroach/pull/41977 // @@ -261,10 +244,6 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: Version19_1, Version: roachpb.Version{Major: 19, Minor: 1}, }, - { - Key: VersionContainsEstimatesCounter, - Version: roachpb.Version{Major: 19, Minor: 2, Internal: 2}, - }, { Key: VersionNamespaceTableWithSchemas, Version: roachpb.Version{Major: 19, Minor: 2, Internal: 5}, diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go index dc241f02e31f..469008264afb 100644 --- a/pkg/clusterversion/versionkey_string.go +++ b/pkg/clusterversion/versionkey_string.go @@ -9,38 +9,37 @@ func _() { // Re-run the stringer command to generate them again. var x [1]struct{} _ = x[Version19_1-0] - _ = x[VersionContainsEstimatesCounter-1] - _ = x[VersionNamespaceTableWithSchemas-2] - _ = x[VersionAuthLocalAndTrustRejectMethods-3] - _ = x[VersionStart20_2-4] - _ = x[VersionGeospatialType-5] - _ = x[VersionEnums-6] - _ = x[VersionRangefeedLeases-7] - _ = x[VersionAlterColumnTypeGeneral-8] - _ = x[VersionAlterSystemJobsAddCreatedByColumns-9] - _ = x[VersionAddScheduledJobsTable-10] - _ = x[VersionUserDefinedSchemas-11] - _ = x[VersionNoOriginFKIndexes-12] - _ = x[VersionClientRangeInfosOnBatchResponse-13] - _ = x[VersionNodeMembershipStatus-14] - _ = x[VersionRangeStatsRespHasDesc-15] - _ = x[VersionMinPasswordLength-16] - _ = x[VersionAbortSpanBytes-17] - _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-18] - _ = x[VersionMaterializedViews-19] - _ = x[VersionBox2DType-20] - _ = x[VersionLeasedDatabaseDescriptors-21] - _ = x[VersionUpdateScheduledJobsSchema-22] - _ = x[VersionCreateLoginPrivilege-23] - _ = x[VersionHBAForNonTLS-24] - _ = x[Version20_2-25] - _ = x[VersionStart21_1-26] - _ = x[VersionEmptyArraysInInvertedIndexes-27] + _ = x[VersionNamespaceTableWithSchemas-1] + _ = x[VersionAuthLocalAndTrustRejectMethods-2] + _ = x[VersionStart20_2-3] + _ = x[VersionGeospatialType-4] + _ = x[VersionEnums-5] + _ = x[VersionRangefeedLeases-6] + _ = x[VersionAlterColumnTypeGeneral-7] + _ = x[VersionAlterSystemJobsAddCreatedByColumns-8] + _ = x[VersionAddScheduledJobsTable-9] + _ = x[VersionUserDefinedSchemas-10] + _ = x[VersionNoOriginFKIndexes-11] + _ = x[VersionClientRangeInfosOnBatchResponse-12] + _ = x[VersionNodeMembershipStatus-13] + _ = x[VersionRangeStatsRespHasDesc-14] + _ = x[VersionMinPasswordLength-15] + _ = x[VersionAbortSpanBytes-16] + _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-17] + _ = x[VersionMaterializedViews-18] + _ = x[VersionBox2DType-19] + _ = x[VersionLeasedDatabaseDescriptors-20] + _ = x[VersionUpdateScheduledJobsSchema-21] + _ = x[VersionCreateLoginPrivilege-22] + _ = x[VersionHBAForNonTLS-23] + _ = x[Version20_2-24] + _ = x[VersionStart21_1-25] + _ = x[VersionEmptyArraysInInvertedIndexes-26] } -const _VersionKey_name = "Version19_1VersionContainsEstimatesCounterVersionNamespaceTableWithSchemasVersionAuthLocalAndTrustRejectMethodsVersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1VersionEmptyArraysInInvertedIndexes" +const _VersionKey_name = "Version19_1VersionNamespaceTableWithSchemasVersionAuthLocalAndTrustRejectMethodsVersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1VersionEmptyArraysInInvertedIndexes" -var _VersionKey_index = [...]uint16{0, 11, 42, 74, 111, 127, 148, 160, 182, 211, 252, 280, 305, 329, 367, 394, 422, 446, 467, 538, 562, 578, 610, 642, 669, 688, 699, 715, 750} +var _VersionKey_index = [...]uint16{0, 11, 43, 80, 96, 117, 129, 151, 180, 221, 249, 274, 298, 336, 363, 391, 415, 436, 507, 531, 547, 579, 611, 638, 657, 668, 684, 719} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index a8439dd853b7..0a0919d2256f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -13,7 +13,6 @@ package batcheval import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -175,7 +174,6 @@ func EvalAddSSTable( stats.Subtract(skippedKVStats) stats.ContainsEstimates = 0 } else { - _ = clusterversion.VersionContainsEstimatesCounter // see for info on ContainsEstimates migration stats.ContainsEstimates++ } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 87a86164c507..01c0c22930b0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -1020,9 +1020,6 @@ func splitTriggerHelper( } deltaPostSplitLeft := h.DeltaPostSplitLeft() - if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionContainsEstimatesCounter) { - deltaPostSplitLeft.ContainsEstimates = 0 - } return deltaPostSplitLeft, pd, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go index c1974eb1fc12..258764aa0e33 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_recompute_stats.go @@ -13,7 +13,6 @@ package batcheval import ( "context" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -102,12 +101,6 @@ func RecomputeStats( // stats for timeseries ranges (which go cold and the approximate stats are // wildly overcounting) and this is paced by the consistency checker, but it // means some extra engine churn. - if !cArgs.EvalCtx.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionContainsEstimatesCounter) { - // We are running with the older version of MVCCStats.ContainsEstimates - // which was a boolean, so we should keep it in {0,1} and not reset it - // to avoid racing with another command that sets it to true. - delta.ContainsEstimates = currentStats.ContainsEstimates - } cArgs.Stats.Add(delta) } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 9d6157546ac7..ad02664c0565 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -15,7 +15,6 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -796,46 +795,12 @@ func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( } res := cmd.replicatedResult() - // Detect whether the incoming stats contain estimates that resulted from the - // evaluation of a command under the 19.1 cluster version. These were either - // evaluated on a 19.1 node (where ContainsEstimates is a bool, which maps - // to 0 and 1 in 19.2+) or on a 19.2 node which hadn't yet had its cluster - // version bumped. - // - // 19.2 nodes will never emit a ContainsEstimates outside of 0 or 1 until - // the cluster version is active (during command evaluation). When the - // version is active, they will never emit odd positive numbers (1, 3, ...). - // - // As a result, we can pinpoint exactly when the proposer of this command - // has used the old cluster version: it's when the incoming - // ContainsEstimates is 1. If so, we need to assume that an old node is processing - // the same commands (as `true + true = true`), so make sure that `1 + 1 = 1`. - _ = clusterversion.VersionContainsEstimatesCounter // see for info on ContainsEstimates migration - deltaStats := res.Delta.ToStats() - if deltaStats.ContainsEstimates == 1 && b.state.Stats.ContainsEstimates == 1 { - deltaStats.ContainsEstimates = 0 - } - // Special-cased MVCC stats handling to exploit commutativity of stats delta // upgrades. Thanks to commutativity, the spanlatch manager does not have to // serialize on the stats key. + deltaStats := res.Delta.ToStats() b.state.Stats.Add(deltaStats) - // Exploit the fact that a split will result in a full stats - // recomputation to reset the ContainsEstimates flag. - // If we were running the new VersionContainsEstimatesCounter cluster version, - // the consistency checker will be able to reset the stats itself, and splits - // will as a side effect also remove estimates from both the resulting left and right hand sides. - // - // TODO(tbg): this can be removed in v20.2 and not earlier. - // Consider the following scenario: - // - all nodes are running 19.2 - // - all nodes rebooted into 20.1 - // - cluster version bumped, but node1 doesn't receive the gossip update for that - // node1 runs a split that should emit ContainsEstimates=-1, but it clamps it to 0/1 because it - // doesn't know that 20.1 is active. - if res.Split != nil && deltaStats.ContainsEstimates == 0 { - b.state.Stats.ContainsEstimates = 0 - } + if res.State != nil && res.State.UsingAppliedStateKey && !b.state.UsingAppliedStateKey { b.migrateToAppliedStateKey = true } diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index c64563f16f52..388bb2e50111 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -792,26 +792,9 @@ func (r *Replica) evaluateProposal( res.Replicated.Timestamp = ba.Timestamp res.Replicated.Delta = ms.ToStatsDelta() - _ = clusterversion.VersionContainsEstimatesCounter // see for info on ContainsEstimates migration - if r.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionContainsEstimatesCounter) { - // Encode that this command (and any that follow) uses regular arithmetic for ContainsEstimates - // by making sure ContainsEstimates is > 1. - // This will be interpreted during command application. - if res.Replicated.Delta.ContainsEstimates > 0 { - res.Replicated.Delta.ContainsEstimates *= 2 - } - } else { - // This range may still need to have its commands processed by nodes which treat ContainsEstimates - // as a bool, so clamp it to {0,1}. This enables use of bool semantics in command application. - if res.Replicated.Delta.ContainsEstimates > 1 { - res.Replicated.Delta.ContainsEstimates = 1 - } else if res.Replicated.Delta.ContainsEstimates < 0 { - // The caller should have checked the cluster version. At the - // time of writing, this is only RecomputeStats and the split - // trigger, which both have the check, but better safe than sorry. - log.Fatalf(ctx, "cannot propose negative ContainsEstimates "+ - "without VersionContainsEstimatesCounter in %s", ba.Summary()) - } + // This is the result of a migration. See the field for more details. + if res.Replicated.Delta.ContainsEstimates > 0 { + res.Replicated.Delta.ContainsEstimates *= 2 } // If the cluster version doesn't track abort span size in MVCCStats, we diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 9106f856b470..fd31e9e01bcb 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -12730,16 +12729,12 @@ func TestReplicateQueueProcessOne(t *testing.T) { } // TestContainsEstimatesClamp tests the massaging of ContainsEstimates -// before proposing a raft command. -// - If the proposing node's version is lower than the VersionContainsEstimatesCounter, -// ContainsEstimates must be clamped to {0,1}. -// - Otherwise, it should always be >1 and an even number. +// before proposing a raft command. It should always be >1 and an even number. +// See the comment on ContainEstimates to understand why. func TestContainsEstimatesClampProposal(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _ = clusterversion.VersionContainsEstimatesCounter // see for details on the ContainsEstimates migration - someRequestToProposal := func(tc *testContext, ctx context.Context) *ProposalData { cmdIDKey := kvserverbase.CmdIDKey("some-cmdid-key") var ba roachpb.BatchRequest @@ -12757,23 +12752,6 @@ func TestContainsEstimatesClampProposal(t *testing.T) { // any number >1. defer setMockPutWithEstimates(2)() - t.Run("Pre-VersionContainsEstimatesCounter", func(t *testing.T) { - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - cfg := TestStoreConfig(nil) - version := clusterversion.VersionByKey(clusterversion.VersionContainsEstimatesCounter - 1) - cfg.Settings = cluster.MakeTestingClusterSettingsWithVersions(version, version, false /* initializeVersion */) - var tc testContext - tc.StartWithStoreConfigAndVersion(t, stopper, cfg, version) - - proposal := someRequestToProposal(&tc, ctx) - - if proposal.command.ReplicatedEvalResult.Delta.ContainsEstimates != 1 { - t.Error("Expected ContainsEstimates to be 1, was", proposal.command.ReplicatedEvalResult.Delta.ContainsEstimates) - } - }) - t.Run("VersionContainsEstimatesCounter", func(t *testing.T) { ctx := context.Background() stopper := stop.NewStopper() @@ -12790,77 +12768,6 @@ func TestContainsEstimatesClampProposal(t *testing.T) { } -// TestContainsEstimatesClampApplication tests that if the ContainsEstimates -// delta from a proposed command is 1 (and the replica state ContainsEstimates <= 1), -// ContainsEstimates will be kept 1 in the replica state. This is because -// ContainsEstimates==1 in a proposed command means that the proposer may run -// with a version older than VersionContainsEstimatesCounter, in which ContainsEstimates -// is a bool. -func TestContainsEstimatesClampApplication(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - _ = clusterversion.VersionContainsEstimatesCounter // see for details on the ContainsEstimates migration - - ctx := context.Background() - stopper := stop.NewStopper() - defer stopper.Stop(ctx) - tc := testContext{} - tc.Start(t, stopper) - - // We will stage and apply 2 batches with a command that has ContainsEstimates=1 - // and expect that ReplicaState.Stats.ContainsEstimates will not become greater than 1. - applyBatch := func() { - tc.repl.raftMu.Lock() - defer tc.repl.raftMu.Unlock() - sm := tc.repl.getStateMachine() - batch := sm.NewBatch(false /* ephemeral */) - rAppbatch := batch.(*replicaAppBatch) - - lease, _ := tc.repl.GetLease() - - cmd := replicatedCmd{ - ctx: ctx, - ent: &raftpb.Entry{ - // Term: 1, - Index: rAppbatch.state.RaftAppliedIndex + 1, - Type: raftpb.EntryNormal, - }, - decodedRaftEntry: decodedRaftEntry{ - idKey: makeIDKey(), - raftCmd: kvserverpb.RaftCommand{ - ProposerLeaseSequence: rAppbatch.state.Lease.Sequence, - ReplicatedEvalResult: kvserverpb.ReplicatedEvalResult{ - Timestamp: tc.Clock().Now(), - IsLeaseRequest: true, - State: &kvserverpb.ReplicaState{ - Lease: &lease, - }, - Delta: enginepb.MVCCStatsDelta{ - ContainsEstimates: 1, - }, - }, - }, - }, - } - - _, err := rAppbatch.Stage(apply.Command(&cmd)) - if err != nil { - t.Fatal(err) - } - - if err := batch.ApplyToStateMachine(ctx); err != nil { - t.Fatal(err) - } - } - - applyBatch() - assert.Equal(t, int64(1), tc.repl.State().ReplicaState.Stats.ContainsEstimates) - - applyBatch() - assert.Equal(t, int64(1), tc.repl.State().ReplicaState.Stats.ContainsEstimates) -} - // setMockPutWithEstimates mocks the Put command (could be any) to simulate a command // that touches ContainsEstimates, in order to test request proposal behavior. func setMockPutWithEstimates(containsEstimatesDelta int64) (undo func()) { diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 3fbf2fa9c72d..3aed6a80c529 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -74,7 +74,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", diff --git a/pkg/storage/enginepb/mvcc.pb.go b/pkg/storage/enginepb/mvcc.pb.go index 512dea633fba..5fd5ae396b52 100644 --- a/pkg/storage/enginepb/mvcc.pb.go +++ b/pkg/storage/enginepb/mvcc.pb.go @@ -78,7 +78,7 @@ type MVCCMetadata struct { func (m *MVCCMetadata) Reset() { *m = MVCCMetadata{} } func (*MVCCMetadata) ProtoMessage() {} func (*MVCCMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_883f89000dcabd92, []int{0} + return fileDescriptor_mvcc_33d1719bb1dfaf1f, []int{0} } func (m *MVCCMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -118,7 +118,7 @@ type MVCCMetadata_SequencedIntent struct { func (m *MVCCMetadata_SequencedIntent) Reset() { *m = MVCCMetadata_SequencedIntent{} } func (*MVCCMetadata_SequencedIntent) ProtoMessage() {} func (*MVCCMetadata_SequencedIntent) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_883f89000dcabd92, []int{0, 0} + return fileDescriptor_mvcc_33d1719bb1dfaf1f, []int{0, 0} } func (m *MVCCMetadata_SequencedIntent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -162,7 +162,7 @@ func (m *MVCCMetadataSubsetForMergeSerialization) Reset() { } func (*MVCCMetadataSubsetForMergeSerialization) ProtoMessage() {} func (*MVCCMetadataSubsetForMergeSerialization) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_883f89000dcabd92, []int{1} + return fileDescriptor_mvcc_33d1719bb1dfaf1f, []int{1} } func (m *MVCCMetadataSubsetForMergeSerialization) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -221,8 +221,20 @@ type MVCCStats struct { // contains_estimates indicates that the MVCCStats object contains values // which have been estimated. This means that the stats should not be used // where complete accuracy is required, and instead should be recomputed - // when necessary. See clusterversion.VersionContainsEstimatesCounter for - // details about the migration from bool to int64. + // when necessary. + // + // This field used to be a bool; in #37583 we migrated it to int64. See #37120 + // for the motivation for doing so. If zero, it's "false". If non-zero, it's + // "true". + // + // As a result of the migration, our usage of this int64 field is now a bit + // involved. When ContainsEstimates is 0 or 1, we behave like a boolean + // (i.e. `1+1=1` aka `true+true=true`). Downstream of raft, we use the + // boolean behavior for ContainsEstimates=1 and the additive behavior for + // anything else. If non-zero, we encode the fact that we're allowed to use + // regular arithmetic for this field by making sure it contains a value >1 (we + // multiply it by 2, and thus avoiding 1). This is then interpreted during + // command application. ContainsEstimates int64 `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates" json:"contains_estimates"` // last_update_nanos is a timestamp at which the ages were last // updated. See the comment on MVCCStats. @@ -280,7 +292,7 @@ func (m *MVCCStats) Reset() { *m = MVCCStats{} } func (m *MVCCStats) String() string { return proto.CompactTextString(m) } func (*MVCCStats) ProtoMessage() {} func (*MVCCStats) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_883f89000dcabd92, []int{2} + return fileDescriptor_mvcc_33d1719bb1dfaf1f, []int{2} } func (m *MVCCStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -334,7 +346,7 @@ func (m *MVCCStatsLegacyRepresentation) Reset() { *m = MVCCStatsLegacyRe func (m *MVCCStatsLegacyRepresentation) String() string { return proto.CompactTextString(m) } func (*MVCCStatsLegacyRepresentation) ProtoMessage() {} func (*MVCCStatsLegacyRepresentation) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc_883f89000dcabd92, []int{3} + return fileDescriptor_mvcc_33d1719bb1dfaf1f, []int{3} } func (m *MVCCStatsLegacyRepresentation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2277,9 +2289,9 @@ var ( ErrIntOverflowMvcc = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/enginepb/mvcc.proto", fileDescriptor_mvcc_883f89000dcabd92) } +func init() { proto.RegisterFile("storage/enginepb/mvcc.proto", fileDescriptor_mvcc_33d1719bb1dfaf1f) } -var fileDescriptor_mvcc_883f89000dcabd92 = []byte{ +var fileDescriptor_mvcc_33d1719bb1dfaf1f = []byte{ // 780 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x95, 0x31, 0x4f, 0xdb, 0x4c, 0x18, 0xc7, 0xe3, 0x37, 0x01, 0x9c, 0x4b, 0x48, 0xc0, 0x2f, 0xd2, 0x1b, 0x85, 0xb7, 0x4e, 0x0a, diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index 8941657f3d32..c6cc5fd9685b 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -141,8 +141,20 @@ message MVCCStats { // contains_estimates indicates that the MVCCStats object contains values // which have been estimated. This means that the stats should not be used // where complete accuracy is required, and instead should be recomputed - // when necessary. See clusterversion.VersionContainsEstimatesCounter for - // details about the migration from bool to int64. + // when necessary. + // + // This field used to be a bool; in #37583 we migrated it to int64. See #37120 + // for the motivation for doing so. If zero, it's "false". If non-zero, it's + // "true". + // + // As a result of the migration, our usage of this int64 field is now a bit + // involved. When ContainsEstimates is 0 or 1, we behave like a boolean + // (i.e. `1+1=1` aka `true+true=true`). Downstream of raft, we use the + // boolean behavior for ContainsEstimates=1 and the additive behavior for + // anything else. If non-zero, we encode the fact that we're allowed to use + // regular arithmetic for this field by making sure it contains a value >1 (we + // multiply it by 2, and thus avoiding 1). This is then interpreted during + // command application. optional int64 contains_estimates = 14 [(gogoproto.nullable) = false]; // last_update_nanos is a timestamp at which the ages were last diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index b45c62034e49..8fdfa9d8fa86 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -22,7 +22,6 @@ import ( "sync" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -283,7 +282,6 @@ func updateStatsOnMerge(key roachpb.Key, valSize, nowNanos int64) enginepb.MVCCS sys := isSysLocal(key) ms.AgeTo(nowNanos) - _ = clusterversion.VersionContainsEstimatesCounter // see for info on ContainsEstimates migration ms.ContainsEstimates = 1 if sys { From 86859a783aad6e81eaf0271364a7d17867499c7d Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 21:18:39 -0500 Subject: [PATCH 2/5] sql,clusterversion: remove VersionAuthLocalAndTrustRejectMethods It's an old cluster version, introduced in the 19.2 release cycle. It's now safe to remove. Part of #47447. Fixes #56398. Release note: None --- pkg/ccl/gssapiccl/gssapi.go | 3 +- pkg/clusterversion/cockroach_versions.go | 20 --------- pkg/clusterversion/versionkey_string.go | 56 ++++++++++++------------ pkg/sql/pgwire/auth_methods.go | 12 +++-- pkg/sql/pgwire/hba_conf.go | 28 ++---------- 5 files changed, 37 insertions(+), 82 deletions(-) diff --git a/pkg/ccl/gssapiccl/gssapi.go b/pkg/ccl/gssapiccl/gssapi.go index 8790b78b9b61..f8ee5d797666 100644 --- a/pkg/ccl/gssapiccl/gssapi.go +++ b/pkg/ccl/gssapiccl/gssapi.go @@ -21,7 +21,6 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" @@ -207,5 +206,5 @@ func checkEntry(entry hba.Entry) error { } func init() { - pgwire.RegisterAuthMethod("gss", authGSS, clusterversion.Version19_1, hba.ConnHostSSL, checkEntry) + pgwire.RegisterAuthMethod("gss", authGSS, hba.ConnHostSSL, checkEntry) } diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 7ba8009d4c73..5733b56f237f 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -115,10 +115,6 @@ type VersionKey int const ( _ VersionKey = iota - 1 // want first named one to start at zero - // Version19_1 is CockroachDB v19.1. It's used for all v19.1.x patch - // releases. - Version19_1 - // v20.1 versions. // // VersionNamespaceTableWithSchemas is @@ -128,14 +124,6 @@ const ( // 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 - // VersionAuthLocalAndTrustRejectMethods introduces the HBA rule - // prefix 'local' and auth methods 'trust' and 'reject', for use - // in server.host_based_authentication.configuration. - // - // A separate cluster version ensures the new syntax is not - // introduced while previous-version nodes are still running, as - // this would block any new SQL client. - VersionAuthLocalAndTrustRejectMethods // TODO(irfansharif): The versions above can/should all be removed. They // were orinally introduced in v20.1. There are inflight PRs to do so @@ -240,18 +228,10 @@ const ( // minor version until we are absolutely sure that no new migrations will need // to be added (i.e., when cutting the final release candidate). var versionsSingleton = keyedVersions([]keyedVersion{ - { - Key: Version19_1, - Version: roachpb.Version{Major: 19, Minor: 1}, - }, { Key: VersionNamespaceTableWithSchemas, Version: roachpb.Version{Major: 19, Minor: 2, Internal: 5}, }, - { - Key: VersionAuthLocalAndTrustRejectMethods, - Version: roachpb.Version{Major: 19, Minor: 2, Internal: 8}, - }, // v20.2 versions. { diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go index 469008264afb..61075b57e639 100644 --- a/pkg/clusterversion/versionkey_string.go +++ b/pkg/clusterversion/versionkey_string.go @@ -8,38 +8,36 @@ 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[Version19_1-0] - _ = x[VersionNamespaceTableWithSchemas-1] - _ = x[VersionAuthLocalAndTrustRejectMethods-2] - _ = x[VersionStart20_2-3] - _ = x[VersionGeospatialType-4] - _ = x[VersionEnums-5] - _ = x[VersionRangefeedLeases-6] - _ = x[VersionAlterColumnTypeGeneral-7] - _ = x[VersionAlterSystemJobsAddCreatedByColumns-8] - _ = x[VersionAddScheduledJobsTable-9] - _ = x[VersionUserDefinedSchemas-10] - _ = x[VersionNoOriginFKIndexes-11] - _ = x[VersionClientRangeInfosOnBatchResponse-12] - _ = x[VersionNodeMembershipStatus-13] - _ = x[VersionRangeStatsRespHasDesc-14] - _ = x[VersionMinPasswordLength-15] - _ = x[VersionAbortSpanBytes-16] - _ = x[VersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-17] - _ = x[VersionMaterializedViews-18] - _ = x[VersionBox2DType-19] - _ = x[VersionLeasedDatabaseDescriptors-20] - _ = x[VersionUpdateScheduledJobsSchema-21] - _ = x[VersionCreateLoginPrivilege-22] - _ = x[VersionHBAForNonTLS-23] - _ = x[Version20_2-24] - _ = x[VersionStart21_1-25] - _ = x[VersionEmptyArraysInInvertedIndexes-26] + _ = 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 = "Version19_1VersionNamespaceTableWithSchemasVersionAuthLocalAndTrustRejectMethodsVersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1VersionEmptyArraysInInvertedIndexes" +const _VersionKey_name = "VersionNamespaceTableWithSchemasVersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionAbortSpanBytesVersionAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableVersionMaterializedViewsVersionBox2DTypeVersionLeasedDatabaseDescriptorsVersionUpdateScheduledJobsSchemaVersionCreateLoginPrivilegeVersionHBAForNonTLSVersion20_2VersionStart21_1VersionEmptyArraysInInvertedIndexes" -var _VersionKey_index = [...]uint16{0, 11, 43, 80, 96, 117, 129, 151, 180, 221, 249, 274, 298, 336, 363, 391, 415, 436, 507, 531, 547, 579, 611, 638, 657, 668, 684, 719} +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) { diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index 9856a8f8cadc..6cbc28f95f8d 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -16,7 +16,6 @@ import ( "crypto/tls" "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" @@ -41,27 +40,26 @@ func loadDefaultMethods() { // // Care should be taken by administrators to only accept this auth // method over secure connections, e.g. those encrypted using SSL. - RegisterAuthMethod("password", authPassword, clusterversion.Version19_1, hba.ConnAny, nil) + RegisterAuthMethod("password", authPassword, hba.ConnAny, nil) // The "cert" method requires a valid client certificate for the // user attempting to connect. // // This method is only usable over SSL connections. - RegisterAuthMethod("cert", authCert, clusterversion.Version19_1, hba.ConnHostSSL, nil) + RegisterAuthMethod("cert", authCert, hba.ConnHostSSL, nil) // The "cert-password" method requires either a valid client // certificate for the connecting user, or, if no cert is provided, // a cleartext password. - RegisterAuthMethod("cert-password", authCertPassword, clusterversion.Version19_1, hba.ConnAny, nil) + RegisterAuthMethod("cert-password", authCertPassword, hba.ConnAny, nil) // The "reject" method rejects any connection attempt that matches // the current rule. - RegisterAuthMethod("reject", authReject, clusterversion.VersionAuthLocalAndTrustRejectMethods, hba.ConnAny, nil) + RegisterAuthMethod("reject", authReject, hba.ConnAny, nil) // The "trust" method accepts any connection attempt that matches // the current rule. - RegisterAuthMethod("trust", authTrust, clusterversion.VersionAuthLocalAndTrustRejectMethods, hba.ConnAny, nil) - + RegisterAuthMethod("trust", authTrust, hba.ConnAny, nil) } // AuthMethod defines a method for authentication of a connection. diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go index 270ae5399802..998d10159790 100644 --- a/pkg/sql/pgwire/hba_conf.go +++ b/pkg/sql/pgwire/hba_conf.go @@ -143,13 +143,6 @@ func checkHBASyntaxBeforeUpdatingSetting(values *settings.Values, s string) erro switch entry.ConnType { case hba.ConnHostAny: case hba.ConnLocal: - if vh != nil && - !vh.IsActive(context.TODO(), clusterversion.VersionAuthLocalAndTrustRejectMethods) { - return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, - `authentication rule type 'local' requires all nodes to be upgraded to %s`, - clusterversion.VersionByKey(clusterversion.VersionAuthLocalAndTrustRejectMethods), - ) - } case hba.ConnHostSSL, hba.ConnHostNoSSL: if vh != nil && !vh.IsActive(context.TODO(), clusterversion.VersionHBAForNonTLS) { @@ -198,13 +191,6 @@ func checkHBASyntaxBeforeUpdatingSetting(values *settings.Values, s string) erro "unknown auth method %q", entry.Method.Value), "Supported methods: %s", listRegisteredMethods()) } - // Verify that the cluster setting is at least the required version. - if vh != nil && !vh.IsActive(context.TODO(), method.minReqVersion) { - return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, - `authentication method '%s' requires all nodes to be upgraded to %s`, - entry.Method.Value, - clusterversion.VersionByKey(method.minReqVersion)) - } // Run the per-method validation. if check := hbaCheckHBAEntries[entry.Method.Value]; check != nil { if err := check(entry); err != nil { @@ -238,14 +224,14 @@ func ParseAndNormalize(val string) (*hba.Conf, error) { // Lookup and cache the auth methods. for i := range conf.Entries { method := conf.Entries[i].Method.Value - methodEntry, ok := hbaAuthMethods[method] + methodInfo, ok := hbaAuthMethods[method] if !ok { // TODO(knz): Determine if an error should be reported // upon unknown auth methods. // See: https://github.com/cockroachdb/cockroach/issues/43716 return nil, errors.Errorf("unknown auth method %s", method) } - conf.Entries[i].MethodFn = methodEntry.methodInfo + conf.Entries[i].MethodFn = methodInfo } return conf, nil @@ -325,11 +311,10 @@ func (s *Server) GetAuthenticationConfiguration() *hba.Conf { func RegisterAuthMethod( method string, fn AuthMethod, - minReqVersion clusterversion.VersionKey, validConnTypes hba.ConnType, checkEntry CheckHBAEntry, ) { - hbaAuthMethods[method] = authMethodEntry{methodInfo{validConnTypes, fn}, minReqVersion} + hbaAuthMethods[method] = methodInfo{validConnTypes, fn} if checkEntry != nil { hbaCheckHBAEntries[method] = checkEntry } @@ -347,15 +332,10 @@ func listRegisteredMethods() string { } var ( - hbaAuthMethods = map[string]authMethodEntry{} + hbaAuthMethods = map[string]methodInfo{} hbaCheckHBAEntries = map[string]CheckHBAEntry{} ) -type authMethodEntry struct { - methodInfo - minReqVersion clusterversion.VersionKey -} - type methodInfo struct { validConnTypes hba.ConnType fn AuthMethod From 93812e96ad6e6f41785276e6a7cea0b3b8804f7d Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 10:17:02 -0500 Subject: [PATCH 3/5] migration,server: plumb in initial version to the migration manager It'll let us return early, and makes the manager "more functional" in its behavior. We should also be using the ClusterVersion type when talking about migrations, so we make that change here. Release note: None --- pkg/migration/BUILD.bazel | 1 + pkg/migration/manager.go | 17 +++++++++++------ pkg/server/server_sql.go | 4 ++-- pkg/sql/exec_util.go | 3 ++- pkg/sql/set_cluster_setting.go | 17 +++++++++++++---- 5 files changed, 29 insertions(+), 13 deletions(-) diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index d669c27d2ae4..c02fd0142384 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/migration", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/kv", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index 7846f7883932..282186286f6c 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -24,6 +24,7 @@ package migration import ( "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -188,14 +189,18 @@ func NewManager( } } -// MigrateTo runs the set of migrations required to upgrade the cluster version -// to the provided target version. -// -// TODO(irfansharif): Do something real here. -func (m *Manager) MigrateTo(ctx context.Context, targetV roachpb.Version) error { +// Migrate runs the set of migrations required to upgrade the cluster version +// from the current version to the target one. +func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVersion) error { + if from == to { + // Nothing to do here. + return nil + } + // TODO(irfansharif): Should we inject every ctx here with specific labels // for each migration, so they log distinctly? ctx = logtags.AddTag(ctx, "migration-mgr", nil) + log.Infof(ctx, "migrating cluster from %s to %s", from, to) // TODO(irfansharif): We'll need to acquire a lease here and refresh it // throughout during the migration to ensure mutual exclusion. @@ -242,7 +247,7 @@ func (m *Manager) MigrateTo(ctx context.Context, targetV roachpb.Version) error // defining migrations is that they'd only care about introducing the next // version key within pkg/clusterversion, and registering a corresponding // migration for it here. - var vs = []roachpb.Version{targetV} + var vs = []roachpb.Version{to.Version} for _, version := range vs { h := &Helper{Manager: m} diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 962bc97d51b8..d8ed7906e785 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -649,8 +649,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { cfg.circularInternalExecutor, cfg.db, ) - execCfg.VersionUpgradeHook = func(ctx context.Context, targetV roachpb.Version) error { - return migrationMgr.MigrateTo(ctx, targetV) + execCfg.VersionUpgradeHook = func(ctx context.Context, from, to clusterversion.ClusterVersion) error { + return migrationMgr.Migrate(ctx, from, to) } } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index d5432f8a89a0..0e067069bcaa 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" @@ -740,7 +741,7 @@ type ExecutorConfig struct { // VersionUpgradeHook is called after validating a `SET CLUSTER SETTING // version` but before executing it. It can carry out arbitrary migrations // that allow us to eventually remove legacy code. - VersionUpgradeHook func(ctx context.Context, to roachpb.Version) error + VersionUpgradeHook func(ctx context.Context, from, to clusterversion.ClusterVersion) error } // Organization returns the value of cluster.organization. diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index d2029d28a248..2811ce56e9d3 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/hintdetail" @@ -51,7 +52,7 @@ type setClusterSettingNode struct { // versionUpgradeHook is called after validating a `SET CLUSTER SETTING // version` but before executing it. It can carry out arbitrary migrations // that allow us to eventually remove legacy code. - versionUpgradeHook func(ctx context.Context, to roachpb.Version) error + versionUpgradeHook func(ctx context.Context, from, to clusterversion.ClusterVersion) error } func checkPrivilegesForSetting(ctx context.Context, p *planner, name string, action string) error { @@ -244,12 +245,20 @@ func (n *setClusterSettingNode) startExec(params runParams) error { } if isSetVersion { + var from, to clusterversion.ClusterVersion + + fromVersionVal := []byte(string(*prev.(*tree.DString))) + if err := protoutil.Unmarshal(fromVersionVal, &from); err != nil { + return err + } + + targetVersionStr := string(*value.(*tree.DString)) + to.Version = roachpb.MustParseVersion(targetVersionStr) + // toSettingString already validated the input, and checked to // see that we are allowed to transition. Let's call into our // upgrade hook to run migrations, if any. - versionStr := string(*value.(*tree.DString)) - targetVersion := roachpb.MustParseVersion(versionStr) - if err := n.versionUpgradeHook(ctx, targetVersion); err != nil { + if err := n.versionUpgradeHook(ctx, from, to); err != nil { return err } } From e8c3b906d93ff05b37d596915b07060fbd10e7a8 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 10:49:58 -0500 Subject: [PATCH 4/5] clusterversion: rename proto pkg name for cleaner import path Release note: None --- pkg/clusterversion/cluster_version.pb.go | 34 ++++++++++++------------ pkg/clusterversion/cluster_version.proto | 2 +- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/pkg/clusterversion/cluster_version.pb.go b/pkg/clusterversion/cluster_version.pb.go index 4e109feac915..95952071b650 100644 --- a/pkg/clusterversion/cluster_version.pb.go +++ b/pkg/clusterversion/cluster_version.pb.go @@ -33,7 +33,7 @@ type ClusterVersion struct { func (m *ClusterVersion) Reset() { *m = ClusterVersion{} } func (*ClusterVersion) ProtoMessage() {} func (*ClusterVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_cluster_version_1bc4b027714f9c3f, []int{0} + return fileDescriptor_cluster_version_d76e19b7fb1f16de, []int{0} } func (m *ClusterVersion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -59,7 +59,7 @@ func (m *ClusterVersion) XXX_DiscardUnknown() { var xxx_messageInfo_ClusterVersion proto.InternalMessageInfo func init() { - proto.RegisterType((*ClusterVersion)(nil), "cockroach.base.ClusterVersion") + proto.RegisterType((*ClusterVersion)(nil), "cockroach.clusterversion.ClusterVersion") } func (this *ClusterVersion) Equal(that interface{}) bool { if that == nil { @@ -330,23 +330,23 @@ var ( ) func init() { - proto.RegisterFile("clusterversion/cluster_version.proto", fileDescriptor_cluster_version_1bc4b027714f9c3f) + proto.RegisterFile("clusterversion/cluster_version.proto", fileDescriptor_cluster_version_d76e19b7fb1f16de) } -var fileDescriptor_cluster_version_1bc4b027714f9c3f = []byte{ - // 214 bytes of a gzipped FileDescriptorProto +var fileDescriptor_cluster_version_d76e19b7fb1f16de = []byte{ + // 212 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x49, 0xce, 0x29, 0x2d, 0x2e, 0x49, 0x2d, 0x2a, 0x4b, 0x2d, 0x2a, 0xce, 0xcc, 0xcf, 0xd3, 0x87, 0x72, 0xe3, 0xa1, 0x7c, - 0xbd, 0x82, 0xa2, 0xfc, 0x92, 0x7c, 0x21, 0xbe, 0xe4, 0xfc, 0xe4, 0xec, 0xa2, 0xfc, 0xc4, 0xe4, - 0x0c, 0xbd, 0xa4, 0xc4, 0xe2, 0x54, 0x29, 0x31, 0x30, 0xbb, 0x20, 0x49, 0x3f, 0x37, 0xb5, 0x24, - 0x31, 0x25, 0xb1, 0x24, 0x11, 0xa2, 0x4e, 0x4a, 0x24, 0x3d, 0x3f, 0x3d, 0x1f, 0xcc, 0xd4, 0x07, - 0xb1, 0x20, 0xa2, 0x4a, 0xb9, 0x5c, 0x7c, 0xce, 0x10, 0x63, 0xc3, 0x20, 0xa6, 0x0a, 0x79, 0x73, - 0xf1, 0x25, 0x26, 0x97, 0x64, 0x96, 0xa5, 0xc2, 0xec, 0x91, 0x60, 0x52, 0x60, 0xd4, 0xe0, 0x36, - 0x92, 0xd2, 0x43, 0x58, 0x04, 0xb5, 0x42, 0x0f, 0xaa, 0xc7, 0x89, 0xe3, 0xc4, 0x3d, 0x79, 0x86, - 0x0b, 0xf7, 0xe4, 0x19, 0x83, 0x78, 0x21, 0x7a, 0xa1, 0x12, 0x56, 0x1c, 0x33, 0x16, 0xc8, 0x33, - 0xbc, 0x58, 0x20, 0xcf, 0xe8, 0xc5, 0xc2, 0xc1, 0x28, 0xc0, 0xe4, 0x64, 0x70, 0xe2, 0xa1, 0x1c, - 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, - 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0xc5, - 0x87, 0xea, 0xe9, 0x24, 0x36, 0xb0, 0x3b, 0x8d, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0x11, 0xdd, - 0x63, 0x6c, 0x0d, 0x01, 0x00, 0x00, + 0xbd, 0x82, 0xa2, 0xfc, 0x92, 0x7c, 0x21, 0x89, 0xe4, 0xfc, 0xe4, 0xec, 0xa2, 0xfc, 0xc4, 0xe4, + 0x0c, 0x3d, 0x54, 0xf5, 0x52, 0x62, 0x60, 0xd1, 0x82, 0x24, 0xfd, 0xdc, 0xd4, 0x92, 0xc4, 0x94, + 0xc4, 0x92, 0x44, 0x88, 0x0e, 0x29, 0x91, 0xf4, 0xfc, 0xf4, 0x7c, 0x30, 0x53, 0x1f, 0xc4, 0x82, + 0x88, 0x2a, 0xe5, 0x72, 0xf1, 0x39, 0x43, 0xf4, 0x87, 0x41, 0xf4, 0x0b, 0x79, 0x73, 0xf1, 0x25, + 0x26, 0x97, 0x64, 0x96, 0xa5, 0xc2, 0x6c, 0x94, 0x60, 0x52, 0x60, 0xd4, 0xe0, 0x36, 0x92, 0xd2, + 0x43, 0x58, 0x09, 0xb5, 0x42, 0x0f, 0xaa, 0xc7, 0x89, 0xe3, 0xc4, 0x3d, 0x79, 0x86, 0x0b, 0xf7, + 0xe4, 0x19, 0x83, 0x78, 0x21, 0x7a, 0xa1, 0x12, 0x56, 0x1c, 0x33, 0x16, 0xc8, 0x33, 0xbc, 0x58, + 0x20, 0xcf, 0xe8, 0xc5, 0xc2, 0xc1, 0x28, 0xc0, 0xe4, 0x64, 0x70, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, + 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, + 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x87, 0xea, + 0x9d, 0x24, 0x36, 0xb0, 0x3b, 0x8d, 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0x7d, 0xac, 0xaa, 0x79, + 0x17, 0x01, 0x00, 0x00, } diff --git a/pkg/clusterversion/cluster_version.proto b/pkg/clusterversion/cluster_version.proto index 85407e62a50b..e3abd5db8e5a 100644 --- a/pkg/clusterversion/cluster_version.proto +++ b/pkg/clusterversion/cluster_version.proto @@ -9,7 +9,7 @@ // licenses/APL.txt. syntax = "proto3"; -package cockroach.base; +package cockroach.clusterversion; option go_package = "clusterversion"; import "roachpb/metadata.proto"; From 3783e636ba093e2bfa49af87a8a0fd96281bd3a8 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 10:24:17 -0500 Subject: [PATCH 5/5] migration: promote clusterversion to first-class citizen... in the external facing API for pkg/migration. The migration package concerns itself with migrations between cluster versions, so we should have our API reflect as much. The proto changes are safe, we haven't had a major release with the previous proto definitions. Release note: None --- pkg/migration/manager.go | 14 +++-- pkg/migration/migrations.go | 10 ++-- pkg/server/migration.go | 19 +++---- pkg/server/migration_test.go | 20 ++++--- pkg/server/serverpb/BUILD.bazel | 1 + pkg/server/serverpb/migration.pb.go | 86 ++++++++++++++--------------- pkg/server/serverpb/migration.proto | 6 +- pkg/server/version_cluster_test.go | 2 +- 8 files changed, 83 insertions(+), 75 deletions(-) diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index 282186286f6c..adc931f12cf4 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -247,9 +247,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // defining migrations is that they'd only care about introducing the next // version key within pkg/clusterversion, and registering a corresponding // migration for it here. - var vs = []roachpb.Version{to.Version} + var clusterVersions = []clusterversion.ClusterVersion{to} - for _, version := range vs { + for _, clusterVersion := range clusterVersions { h := &Helper{Manager: m} // Push out the version gate to every node in the cluster. Each node @@ -260,7 +260,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe { // First sanity check that we'll actually be able to perform the // cluster version bump, cluster-wide. - req := &serverpb.ValidateTargetClusterVersionRequest{Version: &version} + req := &serverpb.ValidateTargetClusterVersionRequest{ + ClusterVersion: &clusterVersion, + } err := h.EveryNode(ctx, "validate-cv", func(ctx context.Context, client serverpb.MigrationClient) error { _, err := client.ValidateTargetClusterVersion(ctx, req) return err @@ -270,7 +272,9 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe } } { - req := &serverpb.BumpClusterVersionRequest{Version: &version} + req := &serverpb.BumpClusterVersionRequest{ + ClusterVersion: &clusterVersion, + } err := h.EveryNode(ctx, "bump-cv", func(ctx context.Context, client serverpb.MigrationClient) error { _, err := client.BumpClusterVersion(ctx, req) return err @@ -285,7 +289,7 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // TODO(irfansharif): We'll want to be able to override which migration // is retrieved here within tests. We could make the registry be a part // of the manager, and all tests to provide their own. - _ = Registry[version] + _ = Registry[clusterVersion] } return nil diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index cf3941ea0d53..e3b0be47ed53 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -10,12 +10,12 @@ package migration -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import "github.com/cockroachdb/cockroach/pkg/clusterversion" -// Registry defines the global mapping between a version, and the associated -// migration. The migration is only executed after a cluster-wide bump of the -// version gate. -var Registry = make(map[roachpb.Version]Migration) +// Registry defines the global mapping between a cluster version, and the +// associated migration. The migration is only executed after a cluster-wide +// bump of the version gate. +var Registry = make(map[clusterversion.ClusterVersion]Migration) func init() { // TODO(irfansharif): We'll want to register individual migrations with diff --git a/pkg/server/migration.go b/pkg/server/migration.go index 513cf243693a..1690fc401cf3 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -14,7 +14,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -40,22 +39,22 @@ var _ serverpb.MigrationServer = &migrationServer{} func (m *migrationServer) ValidateTargetClusterVersion( ctx context.Context, req *serverpb.ValidateTargetClusterVersionRequest, ) (*serverpb.ValidateTargetClusterVersionResponse, error) { - targetVersion := *req.Version + targetCV := req.ClusterVersion versionSetting := m.server.ClusterSettings().Version // We're validating the following: // // node's minimum supported version <= target version <= node's binary version - if targetVersion.Less(versionSetting.BinaryMinSupportedVersion()) { - msg := fmt.Sprintf("target version %s less than binary's min supported version %s", - targetVersion, versionSetting.BinaryMinSupportedVersion()) + if targetCV.Less(versionSetting.BinaryMinSupportedVersion()) { + msg := fmt.Sprintf("target cluster version %s less than binary's min supported version %s", + targetCV, versionSetting.BinaryMinSupportedVersion()) log.Warningf(ctx, "%s", msg) return nil, errors.Newf("%s", redact.Safe(msg)) } - if versionSetting.BinaryVersion().Less(targetVersion) { - msg := fmt.Sprintf("binary version %s less than target version %s", - versionSetting.BinaryVersion(), targetVersion) + if versionSetting.BinaryVersion().Less(targetCV.Version) { + msg := fmt.Sprintf("binary version %s less than target cluster version %s", + versionSetting.BinaryVersion(), targetCV) log.Warningf(ctx, "%s", msg) return nil, errors.Newf("%s", redact.Safe(msg)) } @@ -82,10 +81,10 @@ func (m *migrationServer) BumpClusterVersion( return nil, err } - newCV := clusterversion.ClusterVersion{Version: *req.Version} + newCV := *req.ClusterVersion if err := func() error { - if !prevCV.Version.Less(*req.Version) { + if !prevCV.Less(newCV.Version) { // Nothing to do. return nil } diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go index e5643a1bb614..0056a4d8985e 100644 --- a/pkg/server/migration_test.go +++ b/pkg/server/migration_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -29,36 +30,39 @@ func TestValidateTargetClusterVersion(t *testing.T) { v := func(major, minor int32) roachpb.Version { return roachpb.Version{Major: major, Minor: minor} } + cv := func(major, minor int32) clusterversion.ClusterVersion { + return clusterversion.ClusterVersion{Version: v(major, minor)} + } var tests = []struct { binaryVersion roachpb.Version binaryMinSupportedVersion roachpb.Version - targetVersion roachpb.Version + targetClusterVersion clusterversion.ClusterVersion expErrMatch string // empty if expecting a nil error }{ { binaryVersion: v(20, 2), binaryMinSupportedVersion: v(20, 1), - targetVersion: v(20, 1), + targetClusterVersion: cv(20, 1), expErrMatch: "", }, { binaryVersion: v(20, 2), binaryMinSupportedVersion: v(20, 1), - targetVersion: v(20, 2), + targetClusterVersion: cv(20, 2), expErrMatch: "", }, { binaryVersion: v(20, 2), binaryMinSupportedVersion: v(20, 1), - targetVersion: v(21, 1), - expErrMatch: "binary version.*less than target version", + targetClusterVersion: cv(21, 1), + expErrMatch: "binary version.*less than target cluster version", }, { binaryVersion: v(20, 2), binaryMinSupportedVersion: v(20, 1), - targetVersion: v(19, 2), - expErrMatch: "target version.*less than binary's min supported version", + targetClusterVersion: cv(19, 2), + expErrMatch: "target cluster version.*less than binary's min supported version", }, } @@ -82,7 +86,7 @@ func TestValidateTargetClusterVersion(t *testing.T) { migrationServer := s.MigrationServer().(*migrationServer) req := &serverpb.ValidateTargetClusterVersionRequest{ - Version: &test.targetVersion, + ClusterVersion: &test.targetClusterVersion, } _, err := migrationServer.ValidateTargetClusterVersion(context.Background(), req) if !testutils.IsError(err, test.expErrMatch) { diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index 5b4e3351ba0a..f06b0ed0e583 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -18,6 +18,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/build", + "//pkg/clusterversion", "//pkg/config/zonepb", "//pkg/gossip", "//pkg/jobs/jobspb", diff --git a/pkg/server/serverpb/migration.pb.go b/pkg/server/serverpb/migration.pb.go index 4bcdaab18c07..76f178fb6d27 100644 --- a/pkg/server/serverpb/migration.pb.go +++ b/pkg/server/serverpb/migration.pb.go @@ -6,7 +6,7 @@ package serverpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import clusterversion "github.com/cockroachdb/cockroach/pkg/clusterversion" import ( context "context" @@ -29,14 +29,14 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // ValidateTargetClusterVersion is used to verify that the target node is // running a binary that's able to support the specified cluster version. type ValidateTargetClusterVersionRequest struct { - Version *roachpb.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + ClusterVersion *clusterversion.ClusterVersion `protobuf:"bytes,1,opt,name=cluster_version,json=clusterVersion,proto3" json:"cluster_version,omitempty"` } func (m *ValidateTargetClusterVersionRequest) Reset() { *m = ValidateTargetClusterVersionRequest{} } func (m *ValidateTargetClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionRequest) ProtoMessage() {} func (*ValidateTargetClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_37957468907239c5, []int{0} + return fileDescriptor_migration_23f3ba74c87bf5d5, []int{0} } func (m *ValidateTargetClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -70,7 +70,7 @@ func (m *ValidateTargetClusterVersionResponse) Reset() { *m = ValidateTa func (m *ValidateTargetClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionResponse) ProtoMessage() {} func (*ValidateTargetClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_37957468907239c5, []int{1} + return fileDescriptor_migration_23f3ba74c87bf5d5, []int{1} } func (m *ValidateTargetClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -98,14 +98,14 @@ var xxx_messageInfo_ValidateTargetClusterVersionResponse proto.InternalMessageIn // BumpClusterVersionRequest is used to inform a given node of a cluster version // bump. type BumpClusterVersionRequest struct { - Version *roachpb.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + ClusterVersion *clusterversion.ClusterVersion `protobuf:"bytes,1,opt,name=cluster_version,json=clusterVersion,proto3" json:"cluster_version,omitempty"` } func (m *BumpClusterVersionRequest) Reset() { *m = BumpClusterVersionRequest{} } func (m *BumpClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionRequest) ProtoMessage() {} func (*BumpClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_37957468907239c5, []int{2} + return fileDescriptor_migration_23f3ba74c87bf5d5, []int{2} } func (m *BumpClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -138,7 +138,7 @@ func (m *BumpClusterVersionResponse) Reset() { *m = BumpClusterVersionRe func (m *BumpClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionResponse) ProtoMessage() {} func (*BumpClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_37957468907239c5, []int{3} + return fileDescriptor_migration_23f3ba74c87bf5d5, []int{3} } func (m *BumpClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -388,11 +388,11 @@ func (m *ValidateTargetClusterVersionRequest) MarshalTo(dAtA []byte) (int, error _ = i var l int _ = l - if m.Version != nil { + if m.ClusterVersion != nil { dAtA[i] = 0xa i++ - i = encodeVarintMigration(dAtA, i, uint64(m.Version.Size())) - n1, err := m.Version.MarshalTo(dAtA[i:]) + i = encodeVarintMigration(dAtA, i, uint64(m.ClusterVersion.Size())) + n1, err := m.ClusterVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } @@ -434,11 +434,11 @@ func (m *BumpClusterVersionRequest) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Version != nil { + if m.ClusterVersion != nil { dAtA[i] = 0xa i++ - i = encodeVarintMigration(dAtA, i, uint64(m.Version.Size())) - n2, err := m.Version.MarshalTo(dAtA[i:]) + i = encodeVarintMigration(dAtA, i, uint64(m.ClusterVersion.Size())) + n2, err := m.ClusterVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } @@ -480,8 +480,8 @@ func (m *ValidateTargetClusterVersionRequest) Size() (n int) { } var l int _ = l - if m.Version != nil { - l = m.Version.Size() + if m.ClusterVersion != nil { + l = m.ClusterVersion.Size() n += 1 + l + sovMigration(uint64(l)) } return n @@ -502,8 +502,8 @@ func (m *BumpClusterVersionRequest) Size() (n int) { } var l int _ = l - if m.Version != nil { - l = m.Version.Size() + if m.ClusterVersion != nil { + l = m.ClusterVersion.Size() n += 1 + l + sovMigration(uint64(l)) } return n @@ -562,7 +562,7 @@ func (m *ValidateTargetClusterVersionRequest) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ClusterVersion", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -586,10 +586,10 @@ func (m *ValidateTargetClusterVersionRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Version == nil { - m.Version = &roachpb.Version{} + if m.ClusterVersion == nil { + m.ClusterVersion = &clusterversion.ClusterVersion{} } - if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ClusterVersion.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -695,7 +695,7 @@ func (m *BumpClusterVersionRequest) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ClusterVersion", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -719,10 +719,10 @@ func (m *BumpClusterVersionRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Version == nil { - m.Version = &roachpb.Version{} + if m.ClusterVersion == nil { + m.ClusterVersion = &clusterversion.ClusterVersion{} } - if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ClusterVersion.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -903,27 +903,27 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_37957468907239c5) + proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_23f3ba74c87bf5d5) } -var fileDescriptor_migration_37957468907239c5 = []byte{ - // 279 bytes of a gzipped FileDescriptorProto +var fileDescriptor_migration_23f3ba74c87bf5d5 = []byte{ + // 278 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x87, 0x50, 0x05, 0x49, 0xfa, 0xb9, 0x99, 0xe9, 0x45, 0x89, 0x25, 0x99, 0xf9, 0x79, 0x7a, 0x05, 0x45, 0xf9, 0x25, 0xf9, 0x42, 0x92, 0xc9, 0xf9, 0xc9, 0xd9, 0x45, 0xf9, 0x89, - 0xc9, 0x19, 0x7a, 0x10, 0x35, 0x7a, 0x30, 0xa5, 0x52, 0x62, 0x60, 0x61, 0x90, 0x9e, 0xd4, 0x92, - 0xc4, 0x94, 0xc4, 0x92, 0x44, 0x88, 0x16, 0xa5, 0x68, 0x2e, 0xe5, 0xb0, 0xc4, 0x9c, 0xcc, 0x94, - 0xc4, 0x92, 0xd4, 0x90, 0xc4, 0xa2, 0xf4, 0xd4, 0x12, 0xe7, 0x9c, 0xd2, 0xe2, 0x92, 0xd4, 0xa2, - 0xb0, 0xd4, 0xa2, 0xe2, 0xcc, 0xfc, 0xbc, 0xa0, 0xd4, 0xc2, 0xd2, 0xd4, 0xe2, 0x12, 0x21, 0x13, - 0x2e, 0xf6, 0x32, 0x88, 0x88, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0xb7, 0x91, 0x94, 0x1e, 0xc2, 0x2e, - 0xa8, 0xd1, 0x7a, 0x30, 0x3d, 0x30, 0xa5, 0x4a, 0x6a, 0x5c, 0x2a, 0xf8, 0x0d, 0x2f, 0x2e, 0xc8, - 0xcf, 0x2b, 0x4e, 0x55, 0x0a, 0xe4, 0x92, 0x74, 0x2a, 0xcd, 0x2d, 0xa0, 0xa6, 0xd5, 0x32, 0x5c, - 0x52, 0xd8, 0x8c, 0x84, 0x58, 0x68, 0xb4, 0x95, 0x89, 0x8b, 0xd3, 0x17, 0x16, 0x78, 0x42, 0x0b, - 0x19, 0xb9, 0x64, 0xf0, 0xb9, 0x53, 0xc8, 0x4e, 0x0f, 0x67, 0xc0, 0xea, 0x11, 0x11, 0x7a, 0x52, - 0xf6, 0x64, 0xeb, 0x87, 0x06, 0x10, 0x83, 0x50, 0x33, 0x23, 0x97, 0x10, 0xa6, 0x87, 0x84, 0x4c, - 0xf0, 0x98, 0x8c, 0x33, 0x48, 0xa5, 0x4c, 0x49, 0xd4, 0x05, 0x73, 0x85, 0x93, 0xd6, 0x89, 0x87, - 0x72, 0x0c, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x78, 0x24, 0xc7, 0x78, 0xe3, 0x91, 0x1c, 0xe3, 0x83, - 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, 0x5c, 0x78, 0x2c, 0xc7, 0x70, 0xe3, 0xb1, 0x1c, 0x43, - 0x14, 0x07, 0xcc, 0xa0, 0x24, 0x36, 0x70, 0x02, 0x33, 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0xf8, - 0x76, 0x3a, 0x9f, 0xb6, 0x02, 0x00, 0x00, + 0xc9, 0x19, 0x7a, 0x10, 0x35, 0x7a, 0x30, 0xa5, 0x52, 0x2a, 0xc9, 0x39, 0xa5, 0xc5, 0x25, 0x60, + 0x5e, 0x71, 0x66, 0x7e, 0x9e, 0x3e, 0x94, 0x1b, 0x0f, 0xe5, 0x43, 0x0c, 0x50, 0xaa, 0xe0, 0x52, + 0x0e, 0x4b, 0xcc, 0xc9, 0x4c, 0x49, 0x2c, 0x49, 0x0d, 0x49, 0x2c, 0x4a, 0x4f, 0x2d, 0x71, 0x86, + 0x28, 0x0b, 0x83, 0xa8, 0x0a, 0x4a, 0x2d, 0x2c, 0x4d, 0x2d, 0x2e, 0x11, 0x0a, 0xe4, 0xe2, 0x47, + 0xd3, 0x2f, 0xc1, 0xa8, 0xc0, 0xa8, 0xc1, 0x6d, 0xa4, 0xa1, 0x87, 0x70, 0x01, 0xaa, 0x85, 0x7a, + 0x68, 0x26, 0xf1, 0x25, 0xa3, 0xf0, 0x95, 0xd4, 0xb8, 0x54, 0xf0, 0xdb, 0x5c, 0x5c, 0x90, 0x9f, + 0x57, 0x9c, 0xaa, 0x94, 0xc7, 0x25, 0xe9, 0x54, 0x9a, 0x5b, 0x40, 0x37, 0x77, 0xc9, 0x70, 0x49, + 0x61, 0xb3, 0x0f, 0xe2, 0x1a, 0xa3, 0xad, 0x4c, 0x5c, 0x9c, 0xbe, 0xb0, 0x48, 0x10, 0x5a, 0xc8, + 0xc8, 0x25, 0x83, 0xcf, 0x13, 0x42, 0x76, 0x7a, 0x38, 0x23, 0x48, 0x8f, 0x88, 0x70, 0x97, 0xb2, + 0x27, 0x5b, 0x3f, 0x34, 0xf4, 0x18, 0x84, 0x9a, 0x19, 0xb9, 0x84, 0x30, 0x3d, 0x24, 0x64, 0x82, + 0xc7, 0x64, 0x9c, 0xe1, 0x2d, 0x65, 0x4a, 0xa2, 0x2e, 0x98, 0x2b, 0x9c, 0xb4, 0x4e, 0x3c, 0x94, + 0x63, 0x38, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x1b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, + 0x92, 0x63, 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, 0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, + 0x38, 0x60, 0x06, 0x25, 0xb1, 0x81, 0x93, 0xa6, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0x08, 0xa5, + 0xf0, 0xc3, 0xfe, 0x02, 0x00, 0x00, } diff --git a/pkg/server/serverpb/migration.proto b/pkg/server/serverpb/migration.proto index 590ed76a6a64..34ff08feb849 100644 --- a/pkg/server/serverpb/migration.proto +++ b/pkg/server/serverpb/migration.proto @@ -12,12 +12,12 @@ syntax = "proto3"; package cockroach.server.serverpb; option go_package = "serverpb"; -import "roachpb/metadata.proto"; +import "clusterversion/cluster_version.proto"; // ValidateTargetClusterVersion is used to verify that the target node is // running a binary that's able to support the specified cluster version. message ValidateTargetClusterVersionRequest { - roachpb.Version version = 1; + clusterversion.ClusterVersion cluster_version = 1; } // ValidateTargetClusterVersionResponse is the response to a @@ -28,7 +28,7 @@ message ValidateTargetClusterVersionResponse { // BumpClusterVersionRequest is used to inform a given node of a cluster version // bump. message BumpClusterVersionRequest { - roachpb.Version version = 1; + clusterversion.ClusterVersion cluster_version = 1; } // BumpClusterVersionResponse is the response to an BumpClusterVersionRequest. diff --git a/pkg/server/version_cluster_test.go b/pkg/server/version_cluster_test.go index 99fc0c44480f..f751e95773fe 100644 --- a/pkg/server/version_cluster_test.go +++ b/pkg/server/version_cluster_test.go @@ -445,7 +445,7 @@ func TestClusterVersionMixedVersionTooOld(t *testing.T) { return err } - if !testutils.IsError(err, fmt.Sprintf("binary version %s less than target version %s", v0s, v1s)) { + if !testutils.IsError(err, fmt.Sprintf("binary version %s less than target cluster version %s", v0s, v1s)) { t.Fatal(i, err) } return nil