Skip to content

Commit

Permalink
Merge #78085
Browse files Browse the repository at this point in the history
78085: storage: add `MVCCStats` for range keys r=jbowens a=erikgrinaker

**storage: export some MVCC key encoding functions**

Release note: None

**roachpb: add `Key.Prevish()` to find a previous key**

This patch adds `Key.Prevish()`, which returns a previous key in
lexicographical sort order. This is needed to expand a latch span
leftwards to peek at any left-adjacent range keys.

It is impossible to find the exact immediate predecessor of a key,
because it can have an infinite number of `0xff` bytes at the end, so
this returns the nearest previous key right-padded with `0xff` up to the
given length. It is still possible for an infinite number of keys to
exist between `Key` and `Key.Prevish()`, as keys have unbounded length.

Release note: None

**storage: add `MVCCStats` for range keys**

This patch adds `MVCCStats` tracking for range keys. Four new fields are
added to `MVCCStats`:

* `RangeKeyCount`: the number of (fragmented) range keys, not counting
  historical versions.

* `RangeKeyBytes`: the logical encoded byte size of all range keys.
  The latest version contributes the encoded key bounds, and all
  versions contribute encoded timestamps. Unlike point keys, which for
  historical reasons use a fixed-size timestamp contribution, this uses
  the actual variable-length timestamp size.

* `RangeValCount`: the number of (fragmented) range key versions.

* `RangeValBytes`: the encoded size of all range key values across
  all versions. The same value can be stored in multiple range keys
  due to fragmentation, which will be counted separately. Even though
  all range keys are currently MVCC range tombstones with no value, the
  `MVCCValueHeader` contribution can be non-zero due to e.g. a local
  timestamp.

`ComputeStatsForRange()` has been extended to calculate the above
quantities, and additionally account for range tombstones themselves in
`GCBytesAge` along with their effect on point keys. All relevant call
sites have been updated to surface range keys for the MVCC iterators
passed to `ComputeStatsForRange()`.

Most MVCC operations have been updated to correctly account for MVCC
range tombstones, e.g. during point key writes and intent resolution. KV
APIs are not yet updated, this will be addressed later.

Range key stats are also adjusted during range splits and merges, which
will split and merge any range keys that straddle the split key. This
requires a single range key seek to the left and right of the split key
during these operations.

Touches #70412.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Jun 28, 2022
2 parents 5dc6228 + 8fe2362 commit 443c4d5
Show file tree
Hide file tree
Showing 47 changed files with 3,587 additions and 539 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1506,6 +1506,7 @@ func engineStats(t *testing.T, engine storage.Engine, nowNanos int64) *enginepb.
t.Helper()

iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
KeyTypes: storage.IterKeyTypePointsAndRanges,
LowerBound: keys.LocalMax,
UpperBound: keys.MaxKey,
})
Expand Down
8 changes: 7 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,8 +157,14 @@ func computeStatsDelta(

// 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 {
iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: to})
iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
KeyTypes: storage.IterKeyTypePointsAndRanges,
LowerBound: from,
UpperBound: to,
})
computed, err := iter.ComputeStats(from, to, delta.LastUpdateNanos)
iter.Close()
if err != nil {
Expand Down
46 changes: 44 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand All @@ -39,6 +40,22 @@ func declareKeysDeleteRange(
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}

// When writing range tombstones, we must look for adjacent range tombstones
// that we merge with or fragment, to update MVCC stats accordingly. But we
// make sure to stay within the range bounds.
if args.UseExperimentalRangeTombstone {
// 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.
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)

// We need to read the range descriptor to determine the bounds during eval.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.RangeDescriptorKey(rs.GetStartKey()),
})
}
}

// DeleteRange deletes the range of key/value pairs specified by
Expand All @@ -62,9 +79,14 @@ func DeleteRange(
return result.Result{}, errors.AssertionFailedf(
"ReturnKeys can't be used with range tombstones")
}

desc := cArgs.EvalCtx.Desc()
leftPeekBound, rightPeekBound := rangeTombstonePeekBounds(
args.Key, args.EndKey, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey())
maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV)
err := storage.ExperimentalMVCCDeleteRangeUsingTombstone(
ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, h.Timestamp, cArgs.Now, maxIntents)

err := storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents)
return result.Result{}, err
}

Expand Down Expand Up @@ -95,3 +117,23 @@ func DeleteRange(
// error is not consumed by the caller because the result will be discarded.
return result.FromAcquiredLocks(h.Txn, deleted...), err
}

// rangeTombstonePeekBounds returns the left and right bounds that
// ExperimentalMVCCDeleteRangeUsingTombstone can read in order to detect
// adjacent range tombstones to merge with or fragment. The bounds will be
// truncated to the Raft range bounds if given.
func rangeTombstonePeekBounds(
startKey, endKey, rangeStart, rangeEnd roachpb.Key,
) (roachpb.Key, roachpb.Key) {
leftPeekBound := startKey.Prevish(roachpb.PrevishKeyLength)
if len(rangeStart) > 0 && leftPeekBound.Compare(rangeStart) <= 0 {
leftPeekBound = rangeStart
}

rightPeekBound := endKey.Next()
if len(rangeEnd) > 0 && rightPeekBound.Compare(rangeEnd) >= 0 {
rightPeekBound = rangeEnd
}

return leftPeekBound.Clone(), rightPeekBound.Clone()
}
156 changes: 115 additions & 41 deletions pkg/kv/kvserver/batcheval/cmd_delete_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,13 +8,14 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package batcheval_test
package batcheval

import (
"context"
"testing"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"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"
Expand All @@ -35,6 +36,8 @@ func TestDeleteRangeTombstone(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

storage.DisableMetamorphicSimpleValueEncoding(t)

// Initial data for each test. x is point tombstone, [] is intent,
// o---o is range tombstone.
//
Expand All @@ -44,19 +47,26 @@ func TestDeleteRangeTombstone(t *testing.T) {
// 2 b2 d2 o-------o
// 1
// a b c d e f g h i
//
// We also write two range tombstones abutting the Raft range a-z at [Z-a)@100
// and [z-|)@100. Writing a range tombstone should not merge with these.
writeInitialData := func(t *testing.T, ctx context.Context, rw storage.ReadWriter) {
t.Helper()
var localTS hlc.ClockTimestamp
txn := roachpb.MakeTransaction("test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5}, 0, 0)
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2}, localTS, roachpb.MakeValueFromString("b2"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4}, localTS, roachpb.MakeValueFromString("c4"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2}, localTS, roachpb.MakeValueFromString("d2"), nil))
require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3}, localTS, nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5}, localTS, roachpb.MakeValueFromString("i5"), &txn))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3}, localTS, 0))

txn := roachpb.MakeTransaction("test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5e9}, 0, 0)
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("b2"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, localTS, roachpb.MakeValueFromString("c4"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("d2"), nil))
require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, localTS, roachpb.MakeValueFromString("i5"), &txn))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, 0))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0))
}

now := hlc.ClockTimestamp{Logical: 9}
rangeStart, rangeEnd := roachpb.Key("a"), roachpb.Key("z")

testcases := map[string]struct {
start string
Expand All @@ -68,54 +78,68 @@ func TestDeleteRangeTombstone(t *testing.T) {
expectErr interface{} // error type, substring, or true (any)
}{
"above points succeed": {
start: "a",
end: "f",
ts: 10,
expectErr: nil,
start: "a",
end: "f",
ts: 10e9,
},
"above range tombstone succeed": {
start: "f",
end: "h",
ts: 10,
ts: 10e9,
expectErr: nil,
},
"merging succeeds": {
start: "e",
end: "f",
ts: 3e9,
},
"adjacent to external LHS range key": {
start: "a",
end: "f",
ts: 100e9,
},
"adjacent to external RHS range key": {
start: "q",
end: "z",
ts: 100e9,
},
"transaction errors": {
start: "a",
end: "f",
ts: 10,
ts: 10e9,
txn: true,
expectErr: batcheval.ErrTransactionUnsupported,
expectErr: ErrTransactionUnsupported,
},
"inline errors": {
start: "a",
end: "f",
ts: 10,
ts: 10e9,
inline: true,
expectErr: "Inline can't be used with range tombstones",
},
"returnKeys errors": {
start: "a",
end: "f",
ts: 10,
ts: 10e9,
returnKeys: true,
expectErr: "ReturnKeys can't be used with range tombstones",
},
"intent errors with WriteIntentError": {
start: "i",
end: "j",
ts: 10,
ts: 10e9,
expectErr: &roachpb.WriteIntentError{},
},
"below point errors with WriteTooOldError": {
start: "a",
end: "d",
ts: 1,
ts: 1e9,
expectErr: &roachpb.WriteTooOldError{},
},
"below range tombstone errors with WriteTooOldError": {
start: "f",
end: "h",
ts: 1,
ts: 1e9,
expectErr: &roachpb.WriteTooOldError{},
},
}
Expand All @@ -134,32 +158,52 @@ func TestDeleteRangeTombstone(t *testing.T) {
Timestamp: hlc.Timestamp{WallTime: tc.ts},
}

var txn *roachpb.Transaction
// Prepare the request and environment.
evalCtx := &MockEvalCtx{
ClusterSettings: st,
Desc: &roachpb.RangeDescriptor{
StartKey: roachpb.RKey(rangeStart),
EndKey: roachpb.RKey(rangeEnd),
},
}

h := roachpb.Header{
Timestamp: rangeKey.Timestamp,
}
if tc.txn {
tx := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0)
txn = &tx
txn := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0)
h.Txn = &txn
}

req := &roachpb.DeleteRangeRequest{
RequestHeader: roachpb.RequestHeader{
Key: rangeKey.StartKey,
EndKey: rangeKey.EndKey,
},
UseExperimentalRangeTombstone: true,
Inline: tc.inline,
ReturnKeys: tc.returnKeys,
}

ms := computeStats(t, engine, rangeStart, rangeEnd, rangeKey.Timestamp.WallTime)

// Use a spanset batch to assert latching of all accesses. In particular,
// the additional seeks necessary to check for adjacent range keys that we
// may merge with (for stats purposes) which should not cross the range
// bounds.
var latchSpans, lockSpans spanset.SpanSet
declareKeysDeleteRange(evalCtx.Desc, &h, req, &latchSpans, &lockSpans, 0)
batch := spanset.NewBatchAt(engine.NewBatch(), &latchSpans, h.Timestamp)
defer batch.Close()

// Run the request.
var ms enginepb.MVCCStats
resp := &roachpb.DeleteRangeResponse{}
_, err := batcheval.DeleteRange(ctx, engine, batcheval.CommandArgs{
EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext(),
_, err := DeleteRange(ctx, batch, CommandArgs{
EvalCtx: evalCtx.EvalContext(),
Stats: &ms,
Now: now,
Header: roachpb.Header{
Timestamp: rangeKey.Timestamp,
Txn: txn,
},
Args: &roachpb.DeleteRangeRequest{
RequestHeader: roachpb.RequestHeader{
Key: rangeKey.StartKey,
EndKey: rangeKey.EndKey,
},
UseExperimentalRangeTombstone: true,
Inline: tc.inline,
ReturnKeys: tc.returnKeys,
},
Header: h,
Args: req,
}, resp)

// Check the error.
Expand All @@ -177,6 +221,7 @@ func TestDeleteRangeTombstone(t *testing.T) {
return
}
require.NoError(t, err)
require.NoError(t, batch.Commit(true))

// Check that the range tombstone was written successfully.
iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
Expand Down Expand Up @@ -215,7 +260,36 @@ func TestDeleteRangeTombstone(t *testing.T) {
require.True(t, value.IsTombstone())
require.Equal(t, now, value.LocalTimestamp)

// TODO(erikgrinaker): This should test MVCC stats when implemented.
// Check that range tombstone stats were updated correctly.
require.Equal(t, computeStats(t, engine, rangeStart, rangeEnd, rangeKey.Timestamp.WallTime), ms)
})
}
}

// computeStats computes MVCC stats for the given range.
//
// TODO(erikgrinaker): This, storage.computeStats(), and engineStats() should be
// moved into a testutils package, somehow avoiding import cycles with storage
// tests.
func computeStats(
t *testing.T, reader storage.Reader, from, to roachpb.Key, nowNanos int64,
) enginepb.MVCCStats {
t.Helper()

if len(from) == 0 {
from = keys.LocalMax
}
if len(to) == 0 {
to = keys.MaxKey
}

iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{
KeyTypes: storage.IterKeyTypePointsAndRanges,
LowerBound: from,
UpperBound: to,
})
defer iter.Close()
ms, err := storage.ComputeStatsForRange(iter, from, to, nowNanos)
require.NoError(t, err)
return ms
}
Loading

0 comments on commit 443c4d5

Please sign in to comment.