Skip to content

Commit

Permalink
clusterversion,*: remove VersionContainsEstimatesCounter
Browse files Browse the repository at this point in the history
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 cockroachdb#47447. Fixes cockroachdb#56401.

(While here, Let's also tell git that `versionkey_string.go` is a
generated file.)

Release note: None
  • Loading branch information
irfansharif committed Nov 26, 2020
1 parent 6206935 commit eae9904
Show file tree
Hide file tree
Showing 13 changed files with 73 additions and 227 deletions.
1 change: 1 addition & 0 deletions pkg/clusterversion/.gitattributes
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
versionkey_string.go binary
20 changes: 0 additions & 20 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,6 @@ type VersionKey int
const (
_ VersionKey = iota - 1 // want first named one to start at zero
Version19_1
VersionContainsEstimatesCounter
VersionNamespaceTableWithSchemas
VersionAuthLocalAndTrustRejectMethods

Expand Down Expand Up @@ -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
//
Expand Down
57 changes: 28 additions & 29 deletions pkg/clusterversion/versionkey_string.go

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

2 changes: 0 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -175,7 +174,6 @@ func EvalAddSSTable(
stats.Subtract(skippedKVStats)
stats.ContainsEstimates = 0
} else {
_ = clusterversion.VersionContainsEstimatesCounter // see for info on ContainsEstimates migration
stats.ContainsEstimates++
}

Expand Down
3 changes: 0 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -1020,9 +1020,6 @@ func splitTriggerHelper(
}

deltaPostSplitLeft := h.DeltaPostSplitLeft()
if !rec.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionContainsEstimatesCounter) {
deltaPostSplitLeft.ContainsEstimates = 0
}
return deltaPostSplitLeft, pd, nil
}

Expand Down
7 changes: 0 additions & 7 deletions pkg/kv/kvserver/batcheval/cmd_recompute_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}

Expand Down
39 changes: 2 additions & 37 deletions pkg/kv/kvserver/replica_application_state_machine.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}
Expand Down
25 changes: 5 additions & 20 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
97 changes: 2 additions & 95 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand All @@ -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()
Expand All @@ -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()) {
Expand Down
1 change: 0 additions & 1 deletion pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
Loading

0 comments on commit eae9904

Please sign in to comment.