Skip to content

Commit

Permalink
Merge #99017
Browse files Browse the repository at this point in the history
99017: kvserver: reduce `SysBytes` MVCC stats race during merges r=erikgrinaker a=erikgrinaker

During a range merge, we subsume the RHS and ship its MVCC stats via the merge trigger to add them to the LHS stats. Since the RHS range ID-local keys aren't present in the merged range, the merge trigger computed these and subtracted them from the given stats. However, this could race with a lease request, which ignores latches and writes to the range ID-local keyspace, resulting in incorrect `SysBytes` MVCC stats.

This patch instead computes the range ID-local MVCC stats during subsume and sends them via a new `RangeIDLocalMVCCStats` field. This still doesn't guarantee that they're consistent with the RHS's in-memory stats, since the latch-ignoring lease request can update these independently of the subsume request's engine snapshot. However, it substantially reduces the likelihood of this race.

While it would be possible to prevent this race entirely by introducing additional synchronization between lease requests and merge application, this would likely come with significant additional complexity, which doesn't seem worth it just to avoid `SysBytes` being a few bytes wrong. The main fallout is a log message when the consistency checker detects the stats mismatch, and potential test flake. This PR therefore settles for best-effort prevention.

Resolves #93896.
Resolves #94876.
Resolves #99010.

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
  • Loading branch information
craig[bot] and erikgrinaker committed Mar 27, 2023
2 parents 2bd2c80 + 2d855d3 commit 3c3d2a5
Show file tree
Hide file tree
Showing 6 changed files with 49 additions and 27 deletions.
17 changes: 0 additions & 17 deletions pkg/kv/kvnemesis/env.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
gosql "database/sql"
"fmt"
"regexp"
"time"

"github.com/cockroachdb/cockroach-go/v2/crdb"
Expand Down Expand Up @@ -69,22 +68,6 @@ func (e *Env) CheckConsistency(ctx context.Context, span roachpb.Span) []error {
if err := rows.Scan(&rangeID, &key, &status, &detail); err != nil {
return []error{err}
}
// TODO(erikgrinaker): There's a known issue that can result in a 10-byte
// discrepancy in SysBytes. This hasn't been investigated, but it's not
// critical so we ignore it for now. See:
// https://github.com/cockroachdb/cockroach/issues/93896
if status == kvpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_INCORRECT.String() {
m := regexp.MustCompile(`.*\ndelta \(stats-computed\): \{(.*)\}`).FindStringSubmatch(detail)
if len(m) > 1 {
delta := m[1]
// Strip out LastUpdateNanos and all zero-valued fields.
delta = regexp.MustCompile(`LastUpdateNanos:\d+`).ReplaceAllString(delta, "")
delta = regexp.MustCompile(`\S+:0\b`).ReplaceAllString(delta, "")
if regexp.MustCompile(`^\s*SysBytes:10\s*$`).MatchString(delta) {
continue
}
}
}
switch status {
case kvpb.CheckConsistencyResponse_RANGE_INDETERMINATE.String():
// Can't do anything, so let it slide.
Expand Down
8 changes: 8 additions & 0 deletions pkg/kv/kvpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -2085,6 +2085,14 @@ message SubsumeResponse {
(gogoproto.customname) = "MVCCStats"
];

// RangeIDLocalMVCCStats are the MVCC statistics for the replicated range
// ID-local keys. During a merge, these must be subtracted from MVCCStats
// since they won't be present in the merged range.
storage.enginepb.MVCCStats range_id_local_mvcc_stats = 8 [
(gogoproto.nullable) = false,
(gogoproto.customname) = "RangeIDLocalMVCCStats"
];

// LeaseAppliedIndex is the lease applied index of the last applied command
// at the time that the Subsume request executed. This is NOT intended to be
// the lease index of the SubsumeRequest itself. Instead, it is intended to
Expand Down
18 changes: 14 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan"
Expand Down Expand Up @@ -1272,17 +1273,26 @@ func mergeTrigger(

// The stats for the merged range are the sum of the LHS and RHS stats
// adjusted for range key merges (which is the inverse of the split
// adjustment). The RHS's replicated range ID stats are subtracted -- the only
// replicated range ID keys we copy from the RHS are the keys in the abort
// span, and we've already accounted for those stats above.
// adjustment).
ms.Add(merge.RightMVCCStats)
msRangeKeyDelta, err := computeSplitRangeKeyStatsDelta(batch, merge.LeftDesc, merge.RightDesc)
if err != nil {
return result.Result{}, err
}
ms.Subtract(msRangeKeyDelta)

{
// The RHS's replicated range ID stats are subtracted -- the only replicated
// range ID keys we copy from the RHS are the keys in the abort span, and
// we've already accounted for those stats above.
//
// NB: RangeIDLocalMVCCStats is introduced in 23.2 to mitigate a SysBytes race
// with lease requests (which ignore latches). For 23.1 compatibility, we fall
// back to computing it here when not set. We don't need a version gate since
// it's only used at evaluation time and doesn't affect below-Raft state.
if merge.RightRangeIDLocalMVCCStats != (enginepb.MVCCStats{}) {
ms.Subtract(merge.RightRangeIDLocalMVCCStats)
} else {
_ = clusterversion.V23_1 // remove this branch when 23.1 support is removed
ridPrefix := keys.MakeRangeIDReplicatedPrefix(merge.RightDesc.RangeID)
sysMS, err := storage.ComputeStats(batch, ridPrefix, ridPrefix.PrefixEnd(), 0 /* nowNanos */)
if err != nil {
Expand Down
15 changes: 15 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_subsume.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,21 @@ func Subsume(
reply.LeaseAppliedIndex = cArgs.EvalCtx.GetLeaseAppliedIndex()
reply.FreezeStart = cArgs.EvalCtx.Clock().NowAsClockTimestamp()

// We ship the range ID-local replicated stats as well, since these must be
// subtracted from MVCCStats for the merged range.
//
// NB: lease requests can race with this computation, since they ignore
// latches and write to the range ID-local keyspace. This can very rarely
// result in a minor SysBytes discrepancy when the GetMVCCStats() call above
// is not consistent with this readWriter snapshot. We accept this for now,
// rather than introducing additional synchronization complexity.
ridPrefix := keys.MakeRangeIDReplicatedPrefix(desc.RangeID)
reply.RangeIDLocalMVCCStats, err = storage.ComputeStats(
readWriter, ridPrefix, ridPrefix.PrefixEnd(), 0 /* nowNanos */)
if err != nil {
return result.Result{}, err
}

// Collect a read summary from the RHS leaseholder to ship to the LHS
// leaseholder. This is used to instruct the LHS on how to update its
// timestamp cache to ensure that no future writes are allowed to invalidate
Expand Down
13 changes: 7 additions & 6 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -764,12 +764,13 @@ func (r *Replica) AdminMerge(
Commit: true,
InternalCommitTrigger: &roachpb.InternalCommitTrigger{
MergeTrigger: &roachpb.MergeTrigger{
LeftDesc: updatedLeftDesc,
RightDesc: rightDesc,
RightMVCCStats: rhsSnapshotRes.MVCCStats,
FreezeStart: rhsSnapshotRes.FreezeStart,
RightClosedTimestamp: rhsSnapshotRes.ClosedTimestamp,
RightReadSummary: rhsSnapshotRes.ReadSummary,
LeftDesc: updatedLeftDesc,
RightDesc: rightDesc,
RightMVCCStats: rhsSnapshotRes.MVCCStats,
RightRangeIDLocalMVCCStats: rhsSnapshotRes.RangeIDLocalMVCCStats,
FreezeStart: rhsSnapshotRes.FreezeStart,
RightClosedTimestamp: rhsSnapshotRes.ClosedTimestamp,
RightReadSummary: rhsSnapshotRes.ReadSummary,
},
},
})
Expand Down
5 changes: 5 additions & 0 deletions pkg/roachpb/data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,11 @@ message MergeTrigger {
(gogoproto.nullable) = false
];

storage.enginepb.MVCCStats right_range_id_local_mvcc_stats = 9 [
(gogoproto.customname) = "RightRangeIDLocalMVCCStats",
(gogoproto.nullable) = false
];

// FreezeStart is a timestamp that is guaranteed to be greater than the
// timestamps at which any requests were serviced by the right-hand side range
// before it stopped responding to requests altogether (in anticipation of
Expand Down

0 comments on commit 3c3d2a5

Please sign in to comment.