From 6e889e423a178f50e313dea63f445c26e11d495e Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 25 Nov 2020 20:53:49 -0500 Subject: [PATCH] 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. (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 | 20 ---- 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 | 25 +---- 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, 73 insertions(+), 227 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 8f7abb95205a..0b71843341ea 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -66,7 +66,6 @@ type VersionKey int const ( _ VersionKey = iota - 1 // want first named one to start at zero Version19_1 - VersionContainsEstimatesCounter VersionNamespaceTableWithSchemas VersionAuthLocalAndTrustRejectMethods @@ -122,25 +121,6 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: Version19_1, Version: roachpb.Version{Major: 19, Minor: 1}, }, - { - // 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. - Key: VersionContainsEstimatesCounter, - Version: roachpb.Version{Major: 19, Minor: 2, Internal: 2}, - }, { // VersionNamespaceTableWithSchemas is https://github.com/cockroachdb/cockroach/pull/41977 // 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..0cc1a52f4307 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -792,26 +792,11 @@ 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()) - } + // 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 } // 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 {