Skip to content

Commit

Permalink
storage: add MVCCStats for range keys
Browse files Browse the repository at this point in the history
This patch adds `MVCCStats` tracking for range keys. Two 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.

`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()`.

Rudimentary range tombstone checks have been added during MVCC point
writes to correctly account for them in MVCC statistics. Any further
integration of point writes and range tombstones (in particular,
conflict handling) will be properly implemented 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.

Release note: None
  • Loading branch information
erikgrinaker committed Apr 3, 2022
1 parent f8ce1fa commit 1d48763
Show file tree
Hide file tree
Showing 32 changed files with 1,550 additions and 370 deletions.
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1477,12 +1477,11 @@ 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,
})
defer iter.Close()
// We don't care about nowNanos, because the SST can't contain intents or
// tombstones and all existing intents will be resolved.
stats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, keys.MaxKey, nowNanos)
require.NoError(t, err)
return &stats
Expand Down
9 changes: 8 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,15 @@ 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 when
// ClearRange is extended to clear them.
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
39 changes: 37 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,14 @@ func declareKeysDeleteRange(
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}

// When writing range tombstones, we must read the adjacent range tombstones
// in case we merge with or fragment them, to update MVCC stats accordingly.
// But we make sure to keep within the range bounds.
if args.UseExperimentalRangeTombstone {
l, r := rangeTombstoneReadBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)
}
}

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

desc := cArgs.EvalCtx.Desc()
leftBound, rightBound := rangeTombstoneReadBounds(
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, maxIntents)

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

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

// rangeTombstoneReadBounds returns the left and right bounds that
// ExperimentalMVCCDeleteRangeUsingTombstone can scan to in order to detect
// adjacent range tombstones to merge with or fragment. The bounds will be
// truncated to the Raft range bounds.
func rangeTombstoneReadBounds(
startKey, endKey, rangeStart, rangeEnd roachpb.Key,
) (roachpb.Key, roachpb.Key) {
const prevKeyLength = 8192

leftBound := startKey.Prevish(prevKeyLength)
if len(rangeStart) > 0 && leftBound.Compare(rangeStart) <= 0 {
leftBound = rangeStart
}

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

return leftBound, rightBound
}
154 changes: 113 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 Down Expand Up @@ -44,17 +45,24 @@ 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()
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}, roachpb.MakeValueFromString("b2"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4}, roachpb.MakeValueFromString("c4"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2}, roachpb.MakeValueFromString("d2"), nil))
require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3}, nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5}, roachpb.MakeValueFromString("i5"), &txn))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3}, 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}, roachpb.MakeValueFromString("b2"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, roachpb.MakeValueFromString("c4"), nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, roachpb.MakeValueFromString("d2"), nil))
require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, roachpb.MakeValueFromString("i5"), &txn))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, nil, nil, 0))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, nil, nil, 0))
require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, nil, nil, 0))
}

rangeStart, rangeEnd := roachpb.Key("a"), roachpb.Key("z")

testcases := map[string]struct {
start string
end string
Expand All @@ -65,54 +73,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 @@ -131,31 +153,51 @@ 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,
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 @@ -173,6 +215,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 @@ -201,7 +244,36 @@ func TestDeleteRangeTombstone(t *testing.T) {
}
require.Equal(t, rangeKey.EndKey, endSeen)

// 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 1d48763

Please sign in to comment.