Skip to content

Commit

Permalink
batcheval: handle MVCC range tombstones in ClearRange
Browse files Browse the repository at this point in the history
This patch makes `ClearRange` account for MVCC range tombstones when
updating MVCC stats.

Release note: None
  • Loading branch information
erikgrinaker committed Jun 4, 2022
1 parent 55327c4 commit 50aac2d
Show file tree
Hide file tree
Showing 2 changed files with 248 additions and 115 deletions.
84 changes: 76 additions & 8 deletions pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,17 @@ func declareKeysClearRange(
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})

// We must peek beyond the span for MVCC range tombstones that straddle the
// span bounds, to update MVCC stats with their new bounds. But we make sure
// to stay within the range.
//
// NB: The range end key is not available, so this will pessimistically latch
// up to args.EndKey.Next(). If EndKey falls on the range end key, the span
// will be tightened during evaluation.
args := req.(*roachpb.ClearRangeRequest)
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)
}

// ClearRange wipes all MVCC versions of keys covered by the specified
Expand Down Expand Up @@ -144,8 +155,8 @@ func computeStatsDelta(

// We can avoid manually computing the stats delta if we're clearing
// the entire range.
fast := desc.StartKey.Equal(from) && desc.EndKey.Equal(to)
if fast {
entireRange := desc.StartKey.Equal(from) && desc.EndKey.Equal(to)
if entireRange {
// Note this it is safe to use the full range MVCC stats, as
// opposed to the usual method of computing only a localizied
// stats delta, because a full-range clear prevents any concurrent
Expand All @@ -155,11 +166,11 @@ func computeStatsDelta(
delta.SysCount, delta.SysBytes, delta.AbortSpanBytes = 0, 0, 0 // no change to system stats
}

// If we can't use the fast stats path, or race test is enabled,
// compute stats across the key span to be cleared.
//
// TODO(erikgrinaker): This must handle range key stats adjustments.
if !fast || util.RaceEnabled {
// If we can't use the fast stats path, or race test is enabled, compute stats
// across the key span to be cleared. In this case we must also look for MVCC
// range tombstones that straddle the span bounds, since we must adjust the
// stats for their new key bounds.
if !entireRange || util.RaceEnabled {
iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
KeyTypes: storage.IterKeyTypePointsAndRanges,
LowerBound: from,
Expand All @@ -171,14 +182,71 @@ func computeStatsDelta(
return enginepb.MVCCStats{}, err
}
// If we took the fast path but race is enabled, assert stats were correctly computed.
if fast {
if entireRange {
computed.ContainsEstimates = delta.ContainsEstimates // retained for tests under race
if !delta.Equal(computed) {
log.Fatalf(ctx, "fast-path MVCCStats computation gave wrong result: diff(fast, computed) = %s",
pretty.Diff(delta, computed))
}
}
delta = computed

// If we're not clearing the whole range, we need to adjust for any MVCC
// range tombstones that straddle the span bounds. These will now be
// truncated, or possibly split into two. We take care not to peek outside
// the range bounds.
//
// Conveniently, due to the symmetry of the range keys and their start/end
// bounds around the truncation point, this is equivalent to twice what was
// removed at each bound. This applies both in the truncation and
// split-in-two cases, again due to symmetry.
//
// TODO(erikgrinaker): Consolidate this logic with the corresponding logic
// during range splits/merges and MVCC range tombstone writes.
if !entireRange {
leftPeekBound, rightPeekBound := rangeTombstonePeekBounds(
from, to, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey())
iter = readWriter.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{
KeyTypes: storage.IterKeyTypeRangesOnly,
LowerBound: leftPeekBound,
UpperBound: rightPeekBound,
})
defer iter.Close()

addTruncatedRangeKeyStats := func(bound roachpb.Key) error {
iter.SeekGE(storage.MVCCKey{Key: bound})
if ok, err := iter.Valid(); err != nil {
return err
} else if ok && iter.RangeBounds().Key.Compare(bound) < 0 {
for i, rkv := range iter.RangeKeys() {
keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp))
valBytes := int64(len(rkv.Value))
if i == 0 {
delta.RangeKeyCount--
keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(bound))
}
delta.RangeKeyBytes -= keyBytes
delta.RangeValCount--
delta.RangeValBytes -= valBytes
delta.GCBytesAge -= (keyBytes + valBytes) *
(delta.LastUpdateNanos/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9)
}
}
return nil
}

if !leftPeekBound.Equal(from) {
if err := addTruncatedRangeKeyStats(from); err != nil {
return enginepb.MVCCStats{}, err
}
}

if !rightPeekBound.Equal(to) {
if err := addTruncatedRangeKeyStats(to); err != nil {
return enginepb.MVCCStats{}, err
}
}
}
}

return delta, nil
Expand Down
Loading

0 comments on commit 50aac2d

Please sign in to comment.