From 7f87857fc26611b972d02ee266c6aa73eac99b5e Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sun, 1 May 2022 16:08:37 +0000 Subject: [PATCH] batcheval: handle MVCC range tombstones in `ClearRange` This patch makes `ClearRange` account for MVCC range tombstones when updating MVCC stats. Release note: None --- pkg/kv/kvserver/batcheval/cmd_clear_range.go | 80 ++++- .../batcheval/cmd_clear_range_test.go | 277 +++++++++++------- 2 files changed, 241 insertions(+), 116 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 9565a4007992..5cd098ef8b6e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -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 @@ -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 @@ -155,12 +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 when - // ClearRange is extended to clear them. - 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, @@ -172,7 +182,7 @@ 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", @@ -180,6 +190,58 @@ func computeStatsDelta( } } delta = computed + + // If we're not clearing the whole range, we need to adjust for any MVCC + // range tombstones that straddle the span bounds, since these will now get + // new bounds and possibly be split into two (but we make sure we don't peek + // outside the range bounds). + // + // Conveniently, due to the symmetry of the timestamps and the from/to and + // end/start keys of the removed and remaining range tombstone fragments, + // 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. + 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 { + if rangeStart, _ := iter.RangeBounds(); rangeStart.Compare(bound) < 0 { + for i, rk := range iter.RangeKeys() { + keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rk.Timestamp)) + if i == 0 { + delta.RangeKeyCount-- + keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(bound)) + } + delta.RangeKeyBytes -= keyBytes + delta.GCBytesAge -= keyBytes * (delta.LastUpdateNanos/1e9 - rk.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 diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index 97f90f03c61d..c1803dd86a3a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -17,6 +17,8 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" @@ -25,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -45,129 +46,191 @@ func (wb *wrappedBatch) ClearMVCCRange(start, end roachpb.Key) error { return wb.Batch.ClearMVCCRange(start, end) } -// TestCmdClearRangeBytesThreshold verifies that clear range resorts to -// clearing keys individually if under the bytes threshold and issues a -// clear range command to the batch otherwise. -func TestCmdClearRangeBytesThreshold(t *testing.T) { +// TestCmdClearRange verifies that ClearRange clears point and range keys in the +// given span, and that MVCC stats are updated correctly (both when clearing a +// complete range and just parts of it). It should clear keys using an iterator +// if under the bytes threshold, or using a Pebble range tombstone otherwise. +func TestCmdClearRange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - startKey := roachpb.Key("0000") + nowNanos := int64(100e9) + startKey := roachpb.Key("000") // NB: not 0000, different bound lengths for MVCC stats testing endKey := roachpb.Key("9999") - desc := roachpb.RangeDescriptor{ - RangeID: 99, - StartKey: roachpb.RKey(startKey), - EndKey: roachpb.RKey(endKey), - } valueStr := strings.Repeat("0123456789", 1024) var value roachpb.Value value.SetString(valueStr) // 10KiB + halfFull := ClearRangeBytesThreshold / (2 * len(valueStr)) overFull := ClearRangeBytesThreshold/len(valueStr) + 1 - tests := []struct { - keyCount int - estimatedStats bool - expClearIterCount int - expClearRangeCount int + testcases := map[string]struct { + keyCount int + estimatedStats bool + partialRange bool + expClearIter bool }{ - { - keyCount: 1, - expClearIterCount: 1, - expClearRangeCount: 0, + "single key": { + keyCount: 1, + expClearIter: true, + }, + "below threshold": { + keyCount: halfFull, + expClearIter: true, }, - // More than a single key, but not enough to use ClearRange. - { - keyCount: halfFull, - expClearIterCount: 1, - expClearRangeCount: 0, + "below threshold partial range": { + keyCount: halfFull, + partialRange: true, + expClearIter: true, }, - // With key sizes requiring additional space, this will overshoot. - { - keyCount: overFull, - expClearIterCount: 0, - expClearRangeCount: 1, + "above threshold": { + keyCount: overFull, + expClearIter: false, }, - // Estimated stats always use ClearRange. - { - keyCount: 1, - estimatedStats: true, - expClearIterCount: 0, - expClearRangeCount: 1, + "above threshold partial range": { + keyCount: overFull, + partialRange: true, + expClearIter: false, + }, + "estimated stats": { // must not use iterator, since we can't trust stats + keyCount: 1, + estimatedStats: true, + expClearIter: false, + }, + "estimated stats and partial range": { // stats get computed for partial ranges + keyCount: 1, + estimatedStats: true, + partialRange: true, + expClearIter: true, }, } - for _, test := range tests { - t.Run("", func(t *testing.T) { - ctx := context.Background() - eng := storage.NewDefaultInMemForTesting() - defer eng.Close() - - var stats enginepb.MVCCStats - for i := 0; i < test.keyCount; i++ { - key := roachpb.Key(fmt.Sprintf("%04d", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, hlc.Timestamp{WallTime: int64(i % 2)}, value, nil); err != nil { - t.Fatal(err) + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + testutils.RunTrueAndFalse(t, "spanningRangeTombstones", func(t *testing.T, spanningRangeTombstones bool) { + ctx := context.Background() + eng := storage.NewDefaultInMemForTesting() + defer eng.Close() + + // Set up range descriptor. If partialRange is true, we make the range + // wider than the cleared span, which disabled the MVCC stats fast path. + desc := roachpb.RangeDescriptor{ + RangeID: 99, + StartKey: roachpb.RKey(startKey), + EndKey: roachpb.RKey(endKey), + } + if tc.partialRange { + desc.StartKey = roachpb.RKey(keys.LocalMax) + desc.EndKey = roachpb.RKey(keys.MaxKey) + } + + // Write some range tombstones at the bottom of the keyspace, some of + // which straddle the clear span bounds. In particular, we need to + // ensure MVCC stats are updated correctly for range tombstones that + // get truncated by the ClearRange. + // + // If spanningRangeTombstone is true, we write very wide range + // tombstones that engulf the entire cleared span. Otherwise, we write + // additional range tombstones that span the start/end bounds as well as + // some in the middle -- these will fragment the very wide range + // tombstones, which is why we need to test both cases separately. + rangeTombstones := []storage.MVCCRangeKey{ + {StartKey: roachpb.Key("0"), EndKey: roachpb.Key("a"), Timestamp: hlc.Timestamp{WallTime: 1e9}}, + {StartKey: roachpb.Key("0"), EndKey: roachpb.Key("a"), Timestamp: hlc.Timestamp{WallTime: 2e9}}, + } + if !spanningRangeTombstones { + rangeTombstones = append(rangeTombstones, []storage.MVCCRangeKey{ + {StartKey: roachpb.Key("00"), EndKey: roachpb.Key("111"), Timestamp: hlc.Timestamp{WallTime: 3e9}}, + {StartKey: roachpb.Key("2"), EndKey: roachpb.Key("4"), Timestamp: hlc.Timestamp{WallTime: 3e9}}, + {StartKey: roachpb.Key("6"), EndKey: roachpb.Key("8"), Timestamp: hlc.Timestamp{WallTime: 3e9}}, + {StartKey: roachpb.Key("999"), EndKey: roachpb.Key("aa"), Timestamp: hlc.Timestamp{WallTime: 3e9}}, + }...) + } + for _, rk := range rangeTombstones { + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone( + ctx, eng, nil, rk.StartKey, rk.EndKey, rk.Timestamp, nil, nil, 0)) + } + + // Write some random point keys within the cleared span, above the range tombstones. + for i := 0; i < tc.keyCount; i++ { + key := roachpb.Key(fmt.Sprintf("%04d", i)) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, hlc.Timestamp{WallTime: int64(4+i%2) * 1e9}, value, nil)) + } + + // Calculate the range stats. + stats := computeStats(t, eng, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), nowNanos) + if tc.estimatedStats { + stats.ContainsEstimates++ + } + + // Set up the evaluation context. + cArgs := CommandArgs{ + EvalCtx: (&MockEvalCtx{ + ClusterSettings: cluster.MakeTestingClusterSettings(), + Desc: &desc, + Clock: hlc.NewClock(hlc.UnixNano, time.Nanosecond), + Stats: stats, + }).EvalContext(), + Header: roachpb.Header{ + RangeID: desc.RangeID, + Timestamp: hlc.Timestamp{WallTime: nowNanos}, + }, + Args: &roachpb.ClearRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: startKey, + EndKey: endKey, + }, + }, + Stats: &enginepb.MVCCStats{}, + } + + // Use a spanset batch to assert latching of all accesses. In + // particular, to test the additional seeks necessary to peek for + // adjacent range keys that we may truncate (for stats purposes) which + // should not cross the range bounds. + var latchSpans, lockSpans spanset.SpanSet + declareKeysClearRange(&desc, &cArgs.Header, cArgs.Args, &latchSpans, &lockSpans, 0) + batch := &wrappedBatch{Batch: spanset.NewBatchAt(eng.NewBatch(), &latchSpans, cArgs.Header.Timestamp)} + defer batch.Close() + + // Run the request. + result, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}) + require.NoError(t, err) + require.NotNil(t, result.Replicated.MVCCHistoryMutation) + require.Equal(t, result.Replicated.MVCCHistoryMutation.Spans, []roachpb.Span{{Key: startKey, EndKey: endKey}}) + + require.NoError(t, batch.Commit(true /* sync */)) + + // Verify that we see the correct counts for ClearMVCCIteratorRange and ClearMVCCRange. + require.Equal(t, tc.expClearIter, batch.clearIterCount == 1) + require.Equal(t, tc.expClearIter, batch.clearRangeCount == 0) + + // Ensure that the data is gone. + iter := eng.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: startKey, + UpperBound: endKey, + }) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: keys.LocalMax}) + ok, err := iter.Valid() + require.NoError(t, err) + require.False(t, ok, "expected empty span, found key %s", iter.UnsafeKey()) + + // Verify the stats delta by adding it to the original range stats and + // comparing with the computed range stats. If we're clearing the entire + // range then the new stats should be empty. + newStats := stats + newStats.ContainsEstimates, cArgs.Stats.ContainsEstimates = 0, 0 + newStats.SysBytes, cArgs.Stats.SysBytes = 0, 0 + newStats.SysCount, cArgs.Stats.SysCount = 0, 0 + newStats.AbortSpanBytes, cArgs.Stats.AbortSpanBytes = 0, 0 + newStats.Add(*cArgs.Stats) + require.Equal(t, newStats, computeStats(t, eng, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), nowNanos)) + if !tc.partialRange { + newStats.LastUpdateNanos = 0 + require.Empty(t, newStats) } - } - if test.estimatedStats { - stats.ContainsEstimates++ - } - - batch := &wrappedBatch{Batch: eng.NewBatch()} - defer batch.Close() - - var h roachpb.Header - h.RangeID = desc.RangeID - - cArgs := CommandArgs{Header: h} - cArgs.EvalCtx = (&MockEvalCtx{ - ClusterSettings: cluster.MakeTestingClusterSettings(), - Desc: &desc, - Clock: hlc.NewClock(hlc.UnixNano, time.Nanosecond), - Stats: stats, - }).EvalContext() - cArgs.Args = &roachpb.ClearRangeRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: startKey, - EndKey: endKey, - }, - } - cArgs.Stats = &enginepb.MVCCStats{} - - result, err := ClearRange(ctx, batch, cArgs, &roachpb.ClearRangeResponse{}) - require.NoError(t, err) - require.NotNil(t, result.Replicated.MVCCHistoryMutation) - require.Equal(t, result.Replicated.MVCCHistoryMutation.Spans, []roachpb.Span{{Key: startKey, EndKey: endKey}}) - - // Verify cArgs.Stats is equal to the stats we wrote, ignoring some values. - newStats := stats - newStats.ContainsEstimates, cArgs.Stats.ContainsEstimates = 0, 0 - newStats.SysBytes, cArgs.Stats.SysBytes = 0, 0 - newStats.SysCount, cArgs.Stats.SysCount = 0, 0 - newStats.AbortSpanBytes, cArgs.Stats.AbortSpanBytes = 0, 0 - newStats.Add(*cArgs.Stats) - newStats.AgeTo(0) // pin at LastUpdateNanos==0 - if !newStats.Equal(enginepb.MVCCStats{}) { - t.Errorf("expected stats on original writes to be negated on clear range: %+v vs %+v", stats, *cArgs.Stats) - } - - // Verify we see the correct counts for Clear and ClearRange. - if a, e := batch.clearIterCount, test.expClearIterCount; a != e { - t.Errorf("expected %d iter range clears; got %d", e, a) - } - if a, e := batch.clearRangeCount, test.expClearRangeCount; a != e { - t.Errorf("expected %d clear ranges; got %d", e, a) - } - - // Now ensure that the data is gone, whether it was a ClearRange or individual calls to clear. - if err := batch.Commit(true /* commit */); err != nil { - t.Fatal(err) - } - if err := eng.MVCCIterate(startKey, endKey, storage.MVCCKeyAndIntentsIterKind, func(kv storage.MVCCKeyValue) error { - return errors.New("expected no data in underlying engine") - }); err != nil { - t.Fatal(err) - } + }) }) } }