From 905b1dfdb546ac861d11bf19e538f4feaae97ca8 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sun, 20 Feb 2022 14:15:07 +0000 Subject: [PATCH] storage: add `MVCCStats` for range keys 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 --- .../batcheval/cmd_add_sstable_test.go | 3 +- pkg/kv/kvserver/batcheval/cmd_clear_range.go | 9 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 46 ++- .../batcheval/cmd_delete_range_test.go | 154 +++++-- .../kvserver/batcheval/cmd_end_transaction.go | 110 ++++- .../batcheval/cmd_end_transaction_test.go | 81 ++++ pkg/kv/kvserver/batcheval/cmd_truncate_log.go | 6 +- .../kvserver/batcheval/split_stats_helper.go | 19 +- pkg/kv/kvserver/below_raft_protos_test.go | 2 +- pkg/kv/kvserver/client_merge_test.go | 59 +-- pkg/kv/kvserver/client_split_test.go | 157 ++++--- pkg/kv/kvserver/client_test.go | 47 ++- pkg/kv/kvserver/helpers_test.go | 50 ++- pkg/kv/kvserver/rditer/stats.go | 7 +- pkg/kv/kvserver/replica_consistency.go | 15 +- pkg/kv/kvserver/replica_raft.go | 1 + pkg/kv/kvserver/replica_test.go | 2 +- .../logic_test/builtin_function_notenant | 3 +- pkg/storage/BUILD.bazel | 1 - pkg/storage/bench_test.go | 12 +- pkg/storage/engine.go | 13 +- pkg/storage/enginepb/mvcc.go | 14 +- pkg/storage/enginepb/mvcc.proto | 22 +- pkg/storage/enginepb/mvcc3.proto | 4 + pkg/storage/mvcc.go | 382 +++++++++++++++--- pkg/storage/mvcc_history_test.go | 2 +- pkg/storage/mvcc_incremental_iterator_test.go | 6 +- pkg/storage/mvcc_key.go | 30 ++ pkg/storage/mvcc_key_test.go | 62 +++ pkg/storage/mvcc_stats_test.go | 116 +++++- pkg/storage/mvcc_test.go | 76 ++-- pkg/storage/sst_iterator.go | 12 +- .../testdata/mvcc_histories/range_key_iter | 6 +- .../mvcc_histories/range_tombstone_mutations | 219 +++++++++- pkg/storage/testdata/mvcc_histories/stats | 204 +++++++++- 35 files changed, 1568 insertions(+), 384 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 352c7b1807bd..807ecd5aeaa7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -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 diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 41755777c690..89667309f4e9 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -157,8 +157,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 { diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 8110ed488768..f15219f9a7d4 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -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" @@ -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 @@ -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, maxIntents) + + err := storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats, + args.Key, args.EndKey, h.Timestamp, leftPeekBound, rightPeekBound, maxIntents) return result.Result{}, err } @@ -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() +} diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index e11e28635f3f..50a25f904754 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -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" @@ -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 @@ -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{}, }, } @@ -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. @@ -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{ @@ -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 +} diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 62d919dc9305..febb95592819 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -170,6 +170,19 @@ func declareKeysEndTxn( latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ Key: keys.RangePriorReadSummaryKey(mt.LeftDesc.RangeID), }) + // Merges need to adjust MVCC stats for merged MVCC range tombstones + // that straddle the ranges, by peeking to the left and right of the RHS + // start key. Since Prevish() is imprecise, we must also ensure we don't + // go outside of the LHS bounds. + leftPeekBound := mt.RightDesc.StartKey.AsRawKey().Prevish(roachpb.PrevishKeyLength) + rightPeekBound := mt.RightDesc.StartKey.AsRawKey().Next() + if leftPeekBound.Compare(mt.LeftDesc.StartKey.AsRawKey()) < 0 { + leftPeekBound = mt.LeftDesc.StartKey.AsRawKey() + } + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: leftPeekBound, + EndKey: rightPeekBound, + }) } } } @@ -934,9 +947,18 @@ func splitTrigger( "unable to determine whether right hand side of split is empty") } + rangeKeyDeltaMS, err := computeSplitRangeKeyStatsDelta( + batch, split.LeftDesc, split.RightDesc, ts.WallTime) + if err != nil { + return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, + "unable to compute range key stats delta for RHS") + } + log.Event(ctx, "computed range key delta stats for right hand side range") + h := splitStatsHelperInput{ AbsPreSplitBothEstimated: rec.GetMVCCStats(), DeltaBatchEstimated: bothDeltaMS, + DeltaRangeKey: rangeKeyDeltaMS, AbsPostSplitLeftFn: makeScanStatsFn(ctx, batch, ts, &split.LeftDesc, "left hand side"), AbsPostSplitRightFn: makeScanStatsFn(ctx, batch, ts, &split.RightDesc, "right hand side"), ScanRightFirst: splitScansRightForStatsFirst || emptyRHS, @@ -1210,15 +1232,27 @@ func mergeTrigger( } } - // The stats for the merged range are the sum of the LHS and RHS stats, less - // the RHS's replicated range ID stats. 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. + // 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. ms.Add(merge.RightMVCCStats) + msRangeKeyDelta, err := computeSplitRangeKeyStatsDelta( + batch, merge.LeftDesc, merge.RightDesc, ts.WallTime) + if err != nil { + return result.Result{}, err + } + ms.Subtract(msRangeKeyDelta) + { ridPrefix := keys.MakeRangeIDReplicatedPrefix(merge.RightDesc.RangeID) // NB: Range-ID local keys have no versions and no intents. - iter := batch.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{UpperBound: ridPrefix.PrefixEnd()}) + iter := batch.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: ridPrefix, + UpperBound: ridPrefix.PrefixEnd(), + }) defer iter.Close() sysMS, err := iter.ComputeStats(ridPrefix, ridPrefix.PrefixEnd(), 0 /* nowNanos */) if err != nil { @@ -1264,6 +1298,72 @@ func changeReplicasTrigger( return pd } +// computeSplitRangeKeyStatsDelta computes the delta in MVCCStats caused by +// the splitting of range keys that straddle the range split point. The inverse +// applies during range merges. Consider a range key [a-foo)@1 split at cc: +// +// Before: [a-foo)@1 RangeKeyCount=1 RangeKeyBytes=15 +// LHS: [a-cc)@1 RangeKeyCount=1 RangeKeyBytes=14 +// RHS: [cc-foo)@1 RangeKeyCount=1 RangeKeyBytes=16 +// +// If the LHS is computed directly then the RHS is calculated as: +// +// RHS = Before - LHS = RangeKeyCount=0 RangeKeyBytes=1 +// +// This is clearly incorrect. This function determines the delta such that: +// +// RHS = Before - LHS + Delta = RangeKeyCount=1 RangeKeyBytes=16 +func computeSplitRangeKeyStatsDelta( + r storage.Reader, lhs, rhs roachpb.RangeDescriptor, nowNanos int64, +) (enginepb.MVCCStats, error) { + var delta enginepb.MVCCStats + delta.AgeTo(nowNanos) + + // NB: When called during a merge trigger (for the inverse adjustment), lhs + // will contain the descriptor for the full, merged range. We therefore have + // to use the rhs start key as the reference split point. We also have to make + // sure the bounds fall within the ranges, since Prevish is imprecise. + splitKey := rhs.StartKey.AsRawKey() + lowerBound := splitKey.Prevish(roachpb.PrevishKeyLength) + if lowerBound.Compare(lhs.StartKey.AsRawKey()) < 0 { + lowerBound = lhs.StartKey.AsRawKey() + } + upperBound := splitKey.Next() + + // Check for range keys that straddle the split point. + iter := r.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypeRangesOnly, + LowerBound: lowerBound, + UpperBound: upperBound, + }) + defer iter.Close() + + iter.SeekGE(storage.MVCCKey{Key: splitKey}) + if ok, err := iter.Valid(); err != nil { + return enginepb.MVCCStats{}, err + } else if !ok { + return delta, nil + } else if rangeStart, _ := iter.RangeBounds(); rangeStart.Equal(splitKey) { + return delta, nil + } + + // Calculate the RHS adjustment, which turns out to be equivalent to the stats + // contribution of the range key fragmentation. The naïve calculation would be + // rhs.EncodedSize() - (keyLen(rhs.EndKey) - keyLen(lhs.EndKey)) + // which simplifies to 2 * keyLen(rhs.StartKey) + tsLen(rhs.Timestamp). + for i, rk := range iter.RangeKeys() { + keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rk.Timestamp)) + if i == 0 { + delta.RangeKeyCount++ + keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(splitKey)) + } + delta.RangeKeyBytes += keyBytes + delta.GCBytesAge += keyBytes * (nowNanos/1e9 - rk.Timestamp.WallTime/1e9) + } + + return delta, nil +} + // txnAutoGC controls whether Transaction entries are automatically gc'ed upon // EndTxn if they only have local locks (which can be resolved synchronously // with EndTxn). Certain tests become simpler with this being turned off. diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 19cb34232cda..5aadfd0be780 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -1183,3 +1183,84 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { } }) } + +func TestComputeSplitRangeKeyStatsDelta(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + rangeKey := func(start, end string, ts int) storage.MVCCRangeKey { + return storage.MVCCRangeKey{ + StartKey: roachpb.Key(start), + EndKey: roachpb.Key(end), + Timestamp: hlc.Timestamp{WallTime: int64(ts)}, + } + } + + const nowNanos = 10e9 + lhsDesc := roachpb.RangeDescriptor{StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("l")} + rhsDesc := roachpb.RangeDescriptor{StartKey: roachpb.RKey("l"), EndKey: roachpb.RKey("z").PrefixEnd()} + + testcases := map[string]struct { + rangeKeys []storage.MVCCRangeKey + expect enginepb.MVCCStats + }{ + // Empty stats shouldn't do anything. + "empty": {nil, enginepb.MVCCStats{}}, + // a-z splits into a-l and l-z: simple +1 range key + "full": {[]storage.MVCCRangeKey{rangeKey("a", "z", 1e9)}, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + GCBytesAge: 117, + }}, + // foo-zz splits into foo-l and l-zzzz: contribution is same as for short + // keys, because we have to adjust for the change in LHS end key which ends + // up only depending on the split key, and that doesn't change. + "different key length": {[]storage.MVCCRangeKey{rangeKey("foo", "zzzz", 1e9)}, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + GCBytesAge: 117, + }}, + // Two abutting keys at different timestamps at the split point should not + // require a delta. + "no straddling": {[]storage.MVCCRangeKey{ + rangeKey("a", "l", 1e9), + rangeKey("l", "z", 2e9), + }, enginepb.MVCCStats{}}, + // Fragments at split point should be equivalent to a single key. + "fragments at split": {[]storage.MVCCRangeKey{ + rangeKey("a", "l", 1e9), + rangeKey("l", "z", 1e9), + }, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + GCBytesAge: 117, + }}, + // Multiple straddling keys. + "multiple": { + []storage.MVCCRangeKey{ + rangeKey("a", "z", 1e9), + rangeKey("k", "p", 2e9), + rangeKey("foo", "m", 3e9), + }, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 31, + GCBytesAge: 244, + }}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + engine := storage.NewDefaultInMemForTesting() + defer engine.Close() + + for _, rk := range tc.rangeKeys { + require.NoError(t, engine.ExperimentalPutMVCCRangeKey(rk)) + } + + tc.expect.LastUpdateNanos = nowNanos + + msDelta, err := computeSplitRangeKeyStatsDelta(engine, lhsDesc, rhsDesc, nowNanos) + require.NoError(t, err) + require.Equal(t, tc.expect, msDelta) + }) + } +} diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go index 633231a2cb5f..72d1aca50edb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -120,7 +120,11 @@ func TruncateLog( // Note that any sideloaded payloads that may be removed by this truncation // are not tracked in the raft log delta. The delta will be adjusted below // raft. - iter := readWriter.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{UpperBound: end}) + iter := readWriter.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: start, + UpperBound: end, + }) defer iter.Close() // We can pass zero as nowNanos because we're only interested in SysBytes. ms, err := iter.ComputeStats(start, end, 0 /* nowNanos */) diff --git a/pkg/kv/kvserver/batcheval/split_stats_helper.go b/pkg/kv/kvserver/batcheval/split_stats_helper.go index d238d0704428..29dbe2e65039 100644 --- a/pkg/kv/kvserver/batcheval/split_stats_helper.go +++ b/pkg/kv/kvserver/batcheval/split_stats_helper.go @@ -31,6 +31,9 @@ import "github.com/cockroachdb/cockroach/pkg/storage/enginepb" // practice, we obtain this by recomputing the stats using the corresponding // AbsPostSplit{Left,Right}Fn, and so we don't expect ContainsEstimates to be // set in them. The choice of which side to scan is controlled by ScanRightFirst. +// - DeltaRangeKey: the stats delta that must be added to the non-computed +// half's stats to account for the splitting of range keys straddling the split +// point. See computeSplitRangeKeyStatsDelta() for details. // // We are interested in computing from this the quantities // @@ -60,7 +63,7 @@ import "github.com/cockroachdb/cockroach/pkg/storage/enginepb" // The two unknown quantities can be expressed in terms of the known quantities // because // -// (1) AbsPreSplitBoth + DeltaBatch +// (1) AbsPreSplitBoth + DeltaBatch + DeltaRangeKeyRight // - CombinedErrorDelta = AbsPostSplitLeft + AbsPostSplitRight // // In words, this corresponds to "all bytes are accounted for": from the initial @@ -88,14 +91,16 @@ import "github.com/cockroachdb/cockroach/pkg/storage/enginepb" // // For AbsPostSplitRight(), there are two cases. First, due to the identity // -// CombinedErrorDelta = AbsPreSplitBothEstimated + DeltaBatchEstimated +// CombinedErrorDelta = AbsPreSplitBothEstimated + DeltaBatchEstimated // -(AbsPostSplitLeft + AbsPostSplitRight) +// + DeltaRangeKeyRight. // -// and the fact that the second line contains no estimates, we know that -// CombinedErrorDelta is zero if the first line contains no estimates. Using -// this, we can rearrange as +// and the fact that the second and third lines contain no estimates, we know +// that CombinedErrorDelta is zero if the first line contains no estimates. +// Using this, we can rearrange as // -// AbsPostSplitRight() = AbsPreSplitBoth + DeltaBatch - AbsPostSplitLeft. +// AbsPostSplitRight() = AbsPreSplitBoth + DeltaBatch - AbsPostSplitLeft +// + DeltaRangeKeyRight. // // where all quantities on the right are known. If CombinedErrorDelta is // nonzero, we effectively have one more unknown in our linear system and we @@ -116,6 +121,7 @@ type splitStatsScanFn func() (enginepb.MVCCStats, error) type splitStatsHelperInput struct { AbsPreSplitBothEstimated enginepb.MVCCStats DeltaBatchEstimated enginepb.MVCCStats + DeltaRangeKey enginepb.MVCCStats // AbsPostSplitLeftFn returns the stats for the left hand side of the // split. AbsPostSplitLeftFn splitStatsScanFn @@ -160,6 +166,7 @@ func makeSplitStatsHelper(input splitStatsHelperInput) (splitStatsHelper, error) ms := h.in.AbsPreSplitBothEstimated ms.Subtract(absPostSplitFirst) ms.Add(h.in.DeltaBatchEstimated) + ms.Add(h.in.DeltaRangeKey) if h.in.ScanRightFirst { h.absPostSplitLeft = &ms } else { diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index 02c5e8027863..07b55aafd9d2 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -78,7 +78,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ return enginepb.NewPopulatedRangeAppliedState(r, false) }, emptySum: 615555020845646359, - populatedSum: 12125419916111069931, + populatedSum: 10260703972438987883, }, reflect.TypeOf(&raftpb.HardState{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index d4a6b0d0e10a..634c2d808905 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -1310,15 +1310,13 @@ func TestStoreRangeMergeStats(t *testing.T) { base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, }) - defer tc.Stopper().Stop(context.Background()) + defer tc.Stopper().Stop(ctx) scratch := tc.ScratchRange(t) store := tc.GetFirstStoreFromServer(t, 0) // Split the range. lhsDesc, rhsDesc, err := createSplitRanges(ctx, scratch, store) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) // Write some values left and right of the proposed split key. kvserver.WriteRandomDataToRange(t, store, lhsDesc.RangeID, scratchKey("aaa")) @@ -1329,30 +1327,18 @@ func TestStoreRangeMergeStats(t *testing.T) { // tests whether the merge code properly accounts for merging abort span // records for the same transaction. txn1 := kv.NewTxn(ctx, store.DB(), 0 /* gatewayNodeID */) - if err := txn1.Put(ctx, scratchKey("a-txn1"), "val"); err != nil { - t.Fatal(err) - } + require.NoError(t, txn1.Put(ctx, scratchKey("a-txn1"), "val")) txn2 := kv.NewTxn(ctx, store.DB(), 0 /* gatewayNodeID */) - if err := txn2.Put(ctx, scratchKey("c-txn2"), "val"); err != nil { - t.Fatal(err) - } + require.NoError(t, txn2.Put(ctx, scratchKey("c-txn2"), "val")) txn3 := kv.NewTxn(ctx, store.DB(), 0 /* gatewayNodeID */) - if err := txn3.Put(ctx, scratchKey("a-txn3"), "val"); err != nil { - t.Fatal(err) - } - if err := txn3.Put(ctx, scratchKey("c-txn3"), "val"); err != nil { - t.Fatal(err) - } + require.NoError(t, txn3.Put(ctx, scratchKey("a-txn3"), "val")) + require.NoError(t, txn3.Put(ctx, scratchKey("c-txn3"), "val")) hiPriTxn := kv.NewTxn(ctx, store.DB(), 0 /* gatewayNodeID */) hiPriTxn.TestingSetPriority(enginepb.MaxTxnPriority) for _, key := range []string{"a-txn1", "c-txn2", "a-txn3", "c-txn3"} { - if err := hiPriTxn.Put(ctx, scratchKey(key), "val"); err != nil { - t.Fatal(err) - } - } - if err := hiPriTxn.Commit(ctx); err != nil { - t.Fatal(err) + require.NoError(t, hiPriTxn.Put(ctx, scratchKey(key), "val")) } + require.NoError(t, hiPriTxn.Commit(ctx)) // Leave txn1-txn3 open so that their abort span records exist during the // merge below. @@ -1360,43 +1346,30 @@ func TestStoreRangeMergeStats(t *testing.T) { snap := store.Engine().NewSnapshot() defer snap.Close() msA, err := stateloader.Make(lhsDesc.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) msB, err := stateloader.Make(rhsDesc.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) // Stats should agree with recomputation. - if err := verifyRecomputedStats(snap, lhsDesc, msA, tc.Servers[0].Clock().Now().WallTime); err != nil { - t.Fatalf("failed to verify range A's stats before split: %+v", err) - } - if err := verifyRecomputedStats(snap, rhsDesc, msB, tc.Servers[0].Clock().Now().WallTime); err != nil { - t.Fatalf("failed to verify range B's stats before split: %+v", err) - } + assertRecomputedStats(t, "range A before split", snap, lhsDesc, msA, store.Clock().PhysicalNow()) + assertRecomputedStats(t, "range B before split", snap, rhsDesc, msB, store.Clock().PhysicalNow()) // Merge the b range back into the a range. args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) - if _, err := kv.SendWrapped(ctx, store.TestSender(), args); err != nil { - t.Fatal(err) - } + _, pErr := kv.SendWrapped(ctx, store.TestSender(), args) + require.NoError(t, pErr.GoError()) replMerged := store.LookupReplica(lhsDesc.StartKey) // Get the range stats for the merged range and verify. snap = store.Engine().NewSnapshot() defer snap.Close() msMerged, err := stateloader.Make(replMerged.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) // Merged stats should agree with recomputation. nowNanos := tc.Servers[0].Clock().Now().WallTime msMerged.AgeTo(nowNanos) - if err := verifyRecomputedStats(snap, replMerged.Desc(), msMerged, nowNanos); err != nil { - t.Errorf("failed to verify range's stats after merge: %+v", err) - } + assertRecomputedStats(t, "merged range", snap, replMerged.Desc(), msMerged, nowNanos) } func TestStoreRangeMergeInFlightTxns(t *testing.T) { diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 693c5033f8aa..f1c52c9dd1b6 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -746,12 +746,13 @@ func TestStoreRangeSplitIdempotency(t *testing.T) { } } -// TestStoreRangeSplitStats starts by splitting the system keys from user-space -// keys and verifying that the user space side of the split (which is empty), -// has all zeros for stats. It then writes random data to the user space side, -// splits it halfway and verifies the two splits have stats exactly equaling -// the pre-split. -func TestStoreRangeSplitStats(t *testing.T) { +// TestStoreRangeSplitMergeStats starts by splitting the system keys from +// user-space keys and verifying that the user space side of the split (which is +// empty), has all zeros for stats. It then writes random data to the user space +// side, splits it halfway and verifies the two splits have appropriate stats. +// Finally, it merges the ranges back and asserts that the stats equal the +// original stats. +func TestStoreRangeSplitMergeStats(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -769,61 +770,52 @@ func TestStoreRangeSplitStats(t *testing.T) { store, err := s.Stores().GetStore(s.GetFirstStoreID()) require.NoError(t, err) - start := s.Clock().Now() - // Split the range after the last table data key. keyPrefix := keys.SystemSQLCodec.TablePrefix(bootstrap.TestingUserDescID(0)) args := adminSplitArgs(keyPrefix) - if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil { - t.Fatal(pErr) - } + _, pErr := kv.SendWrapped(ctx, store.TestSender(), args) + require.NoError(t, pErr.GoError()) + // Verify empty range has empty stats. repl := store.LookupReplica(roachpb.RKey(keyPrefix)) - // NOTE that this value is expected to change over time, depending on what - // we store in the sys-local keyspace. Update it accordingly for this test. - empty := enginepb.MVCCStats{LastUpdateNanos: start.WallTime} - if err := verifyRangeStats(store.Engine(), repl.RangeID, empty); err != nil { - t.Fatal(err) - } + assertRangeStats(t, "empty stats", store.Engine(), repl.RangeID, enginepb.MVCCStats{}) // Write random data. - midKey := kvserver.WriteRandomDataToRange(t, store, repl.RangeID, keyPrefix) + splitKey := kvserver.WriteRandomDataToRange(t, store, repl.RangeID, keyPrefix) + + start := s.Clock().Now() // Get the range stats now that we have data. snap := store.Engine().NewSnapshot() defer snap.Close() ms, err := stateloader.Make(repl.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } - if err := verifyRecomputedStats(snap, repl.Desc(), ms, start.WallTime); err != nil { - t.Fatalf("failed to verify range's stats before split: %+v", err) - } - if inMemMS := repl.GetMVCCStats(); inMemMS != ms { - t.Fatalf("in-memory and on-disk diverged:\n%+v\n!=\n%+v", inMemMS, ms) - } + require.NoError(t, err) + assertRecomputedStats(t, "before split", snap, repl.Desc(), ms, start.WallTime) + require.Equal(t, repl.GetMVCCStats(), ms, "in-memory and on-disk stats diverge") // Split the range at approximate halfway point. - args = adminSplitArgs(midKey) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ - RangeID: repl.RangeID, - }, args); pErr != nil { - t.Fatal(pErr) - } + _, pErr = kv.SendWrapped(ctx, store.TestSender(), adminSplitArgs(splitKey)) + require.NoError(t, pErr.GoError()) snap = store.Engine().NewSnapshot() defer snap.Close() msLeft, err := stateloader.Make(repl.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } - replRight := store.LookupReplica(midKey) + require.NoError(t, err) + replRight := store.LookupReplica(splitKey) msRight, err := stateloader.Make(replRight.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) + + // Stats should both have the new timestamp. + require.Less(t, start.WallTime, msLeft.LastUpdateNanos, "LHS stats have old timestamp") + require.Less(t, start.WallTime, msRight.LastUpdateNanos, "RHS stats have old timestamp") + + // We don't care about system data. + ms.SysBytes, ms.SysCount, ms.AbortSpanBytes = 0, 0, 0 - // The stats should be exactly equal when added. + // The point key stats should be exactly equal when added. + pointMS := ms + pointMS.LastUpdateNanos = 0 + pointMS.RangeKeyCount, pointMS.RangeKeyBytes, pointMS.GCBytesAge = 0, 0, 0 expMS := enginepb.MVCCStats{ LiveBytes: msLeft.LiveBytes + msRight.LiveBytes, KeyBytes: msLeft.KeyBytes + msRight.KeyBytes, @@ -834,27 +826,33 @@ func TestStoreRangeSplitStats(t *testing.T) { ValCount: msLeft.ValCount + msRight.ValCount, IntentCount: msLeft.IntentCount + msRight.IntentCount, } - ms.SysBytes, ms.SysCount, ms.AbortSpanBytes = 0, 0, 0 - ms.LastUpdateNanos = 0 - if expMS != ms { - t.Errorf("expected left plus right ranges to equal original, but\n %+v\n+\n %+v\n!=\n %+v", msLeft, msRight, ms) - } + require.Equal(t, expMS, pointMS, "left plus right point key stats does not match original") - // Stats should both have the new timestamp. - if lTs := msLeft.LastUpdateNanos; lTs < start.WallTime { - t.Errorf("expected left range stats to have new timestamp, want %d, got %d", start.WallTime, lTs) - } - if rTs := msRight.LastUpdateNanos; rTs < start.WallTime { - t.Errorf("expected right range stats to have new timestamp, want %d, got %d", start.WallTime, rTs) - } + // The range key stats should be equal or greater. + require.GreaterOrEqual(t, msLeft.RangeKeyCount+msRight.RangeKeyCount, ms.RangeKeyCount) + require.GreaterOrEqual(t, msLeft.RangeKeyBytes+msRight.RangeKeyBytes, ms.RangeKeyBytes) + require.GreaterOrEqual(t, msLeft.GCBytesAge+msRight.GCBytesAge, ms.GCBytesAge) // Stats should agree with recomputation. - if err := verifyRecomputedStats(snap, repl.Desc(), msLeft, s.Clock().PhysicalNow()); err != nil { - t.Fatalf("failed to verify left range's stats after split: %+v", err) - } - if err := verifyRecomputedStats(snap, replRight.Desc(), msRight, s.Clock().PhysicalNow()); err != nil { - t.Fatalf("failed to verify right range's stats after split: %+v", err) - } + assertRecomputedStats(t, "LHS after split", snap, repl.Desc(), msLeft, s.Clock().PhysicalNow()) + assertRecomputedStats(t, "RHS after split", snap, replRight.Desc(), msRight, s.Clock().PhysicalNow()) + + // Merge the ranges back together, and assert that the merged stats + // agree with the pre-split stats. + _, pErr = kv.SendWrapped(ctx, store.TestSender(), adminMergeArgs(repl.Desc().StartKey.AsRawKey())) + require.NoError(t, pErr.GoError()) + + repl = store.LookupReplica(roachpb.RKey(keyPrefix)) + snap = store.Engine().NewSnapshot() + defer snap.Close() + + msMerged, err := stateloader.Make(repl.RangeID).LoadMVCCStats(ctx, snap) + require.NoError(t, err) + assertRecomputedStats(t, "in-mem after merge", snap, repl.Desc(), msMerged, s.Clock().PhysicalNow()) + + msMerged.SysBytes, msMerged.SysCount, msMerged.AbortSpanBytes = 0, 0, 0 + ms.AgeTo(msMerged.LastUpdateNanos) + require.Equal(t, ms, msMerged, "post-merge stats differ from pre-split") } // RaftMessageHandlerInterceptor wraps a storage.RaftMessageHandler. It @@ -977,56 +975,41 @@ func TestStoreRangeSplitStatsWithMerges(t *testing.T) { // Split the range after the last table data key. keyPrefix := keys.SystemSQLCodec.TablePrefix(bootstrap.TestingUserDescID(0)) args := adminSplitArgs(keyPrefix) - if _, pErr := kv.SendWrapped(ctx, store.TestSender(), args); pErr != nil { - t.Fatal(pErr) - } + _, pErr := kv.SendWrapped(ctx, store.TestSender(), args) + require.NoError(t, pErr.GoError()) + // Verify empty range has empty stats. repl := store.LookupReplica(roachpb.RKey(keyPrefix)) // NOTE that this value is expected to change over time, depending on what // we store in the sys-local keyspace. Update it accordingly for this test. empty := enginepb.MVCCStats{LastUpdateNanos: start.WallTime} - if err := verifyRangeStats(store.Engine(), repl.RangeID, empty); err != nil { - t.Fatal(err) - } + assertRangeStats(t, "empty stats", store.Engine(), repl.RangeID, empty) // Write random TimeSeries data. midKey := writeRandomTimeSeriesDataToRange(t, store, repl.RangeID, keyPrefix) // Split the range at approximate halfway point. args = adminSplitArgs(midKey) - if _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ + _, pErr = kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ RangeID: repl.RangeID, - }, args); pErr != nil { - t.Fatal(pErr) - } + }, args) + require.NoError(t, pErr.GoError()) snap := store.Engine().NewSnapshot() defer snap.Close() msLeft, err := stateloader.Make(repl.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) replRight := store.LookupReplica(midKey) msRight, err := stateloader.Make(replRight.RangeID).LoadMVCCStats(ctx, snap) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) // Stats should both have the new timestamp. - if lTs := msLeft.LastUpdateNanos; lTs < start.WallTime { - t.Errorf("expected left range stats to have new timestamp, want %d, got %d", start.WallTime, lTs) - } - if rTs := msRight.LastUpdateNanos; rTs < start.WallTime { - t.Errorf("expected right range stats to have new timestamp, want %d, got %d", start.WallTime, rTs) - } + require.Less(t, start.WallTime, msLeft.LastUpdateNanos, "LHS stats have old timestamp") + require.Less(t, start.WallTime, msRight.LastUpdateNanos, "RHS stats have old timestamp") // Stats should agree with recomputation. - if err := verifyRecomputedStats(snap, repl.Desc(), msLeft, s.Clock().PhysicalNow()); err != nil { - t.Fatalf("failed to verify left range's stats after split: %+v", err) - } - if err := verifyRecomputedStats(snap, replRight.Desc(), msRight, s.Clock().PhysicalNow()); err != nil { - t.Fatalf("failed to verify right range's stats after split: %+v", err) - } + assertRecomputedStats(t, "LHS after split", snap, repl.Desc(), msLeft, s.Clock().PhysicalNow()) + assertRecomputedStats(t, "RHS after split", snap, replRight.Desc(), msRight, s.Clock().PhysicalNow()) } // fillRange writes keys with the given prefix and associated values diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index 54405929f758..53e4549a40f0 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -31,8 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" - "github.com/kr/pretty" + "github.com/stretchr/testify/require" ) // getArgs returns a GetRequest and GetResponse pair addressed to @@ -135,38 +134,38 @@ func adminTransferLeaseArgs(key roachpb.Key, target roachpb.StoreID) roachpb.Req } } -func verifyRangeStats( - reader storage.Reader, rangeID roachpb.RangeID, expMS enginepb.MVCCStats, -) error { - ms, err := stateloader.Make(rangeID).LoadMVCCStats(context.Background(), reader) - if err != nil { - return err - } +func assertRangeStats( + t *testing.T, name string, r storage.Reader, rangeID roachpb.RangeID, expMS enginepb.MVCCStats, +) { + t.Helper() + + ms, err := stateloader.Make(rangeID).LoadMVCCStats(context.Background(), r) + require.NoError(t, err) // When used with a real wall clock these will not be the same, since it // takes time to load stats. expMS.AgeTo(ms.LastUpdateNanos) // Clear system counts as these are expected to vary. ms.SysBytes, ms.SysCount, ms.AbortSpanBytes = 0, 0, 0 - if ms != expMS { - return errors.Errorf("expected and actual stats differ:\n%s", pretty.Diff(expMS, ms)) - } - return nil + require.Equal(t, expMS, ms, "%s: stats differ", name) } -func verifyRecomputedStats( - reader storage.Reader, d *roachpb.RangeDescriptor, expMS enginepb.MVCCStats, nowNanos int64, -) error { - ms, err := rditer.ComputeStatsForRange(d, reader, nowNanos) - if err != nil { - return err - } +func assertRecomputedStats( + t *testing.T, + name string, + r storage.Reader, + desc *roachpb.RangeDescriptor, + expMS enginepb.MVCCStats, + nowNanos int64, +) { + t.Helper() + + ms, err := rditer.ComputeStatsForRange(desc, r, nowNanos) + require.NoError(t, err) + // When used with a real wall clock these will not be the same, since it // takes time to load stats. expMS.AgeTo(ms.LastUpdateNanos) - if expMS != ms { - return fmt.Errorf("expected range's stats to agree with recomputation: got\n%+v\nrecomputed\n%+v", expMS, ms) - } - return nil + require.Equal(t, expMS, ms, "%s: recomputed stats diverge", name) } func waitForTombstone( diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 7e812a5593fd..eb8a50dedf56 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -18,7 +18,6 @@ package kvserver import ( "context" "fmt" - "math/rand" "testing" "time" @@ -45,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/v3" ) @@ -496,24 +496,40 @@ func (t *RaftTransport) GetCircuitBreaker( } func WriteRandomDataToRange( - t testing.TB, store *Store, rangeID roachpb.RangeID, keyPrefix []byte, -) (midpoint []byte) { - src := rand.New(rand.NewSource(0)) - for i := 0; i < 100; i++ { - key := append([]byte(nil), keyPrefix...) - key = append(key, randutil.RandBytes(src, int(src.Int31n(1<<7)))...) - val := randutil.RandBytes(src, int(src.Int31n(1<<8))) - pArgs := putArgs(key, val) - if _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{ - RangeID: rangeID, - }, &pArgs); pErr != nil { - t.Fatal(pErr) + t testing.TB, store *Store, rangeID roachpb.RangeID, keyPrefix roachpb.Key, +) (splitKey []byte) { + t.Helper() + + ctx := context.Background() + src, _ := randutil.NewTestRand() + for i := 0; i < 1000; i++ { + var req roachpb.Request + if src.Float64() < 0.05 { + // Write some occasional range tombstones. + startKey := append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4)))...) + var endKey roachpb.Key + for startKey.Compare(endKey) >= 0 { + endKey = append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4)))...) + } + req = &roachpb.DeleteRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: startKey, + EndKey: endKey, + }, + UseExperimentalRangeTombstone: true, + } + } else { + // Write regular point keys. + key := append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4)))...) + val := randutil.RandBytes(src, int(src.Int31n(1<<8))) + pArgs := putArgs(key, val) + req = &pArgs } + _, pErr := kv.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{RangeID: rangeID}, req) + require.NoError(t, pErr.GoError()) } - // Return approximate midway point ("Z" in string "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"). - midKey := append([]byte(nil), keyPrefix...) - midKey = append(midKey, []byte("Z")...) - return midKey + // Return a random non-empty split key. + return append(keyPrefix.Clone(), randutil.RandBytes(src, int(src.Int31n(1<<4))+1)...) } func WatchForDisappearingReplicas(t testing.TB, store *Store) { diff --git a/pkg/kv/kvserver/rditer/stats.go b/pkg/kv/kvserver/rditer/stats.go index 8cf5ce11c1b7..20ae426748bd 100644 --- a/pkg/kv/kvserver/rditer/stats.go +++ b/pkg/kv/kvserver/rditer/stats.go @@ -26,8 +26,11 @@ func ComputeStatsForRange( var err error for _, keyRange := range MakeReplicatedKeyRangesExceptLockTable(d) { func() { - iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, - storage.IterOptions{UpperBound: keyRange.End}) + iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: keyRange.Start, + UpperBound: keyRange.End, + }) defer iter.Close() var msDelta enginepb.MVCCStats diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 58fb8794839e..861696ab2edb 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -598,7 +598,8 @@ func (*Replica) sha512( var timestampBuf []byte hasher := sha512.New() - visitor := func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { + // TODO(erikgrinaker): add a range key visitor to hash range keys. + pointKeyVisitor := func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { // Rate Limit the scan through the range if err := limiter.WaitN(ctx, int64(len(unsafeKey.Key)+len(unsafeValue))); err != nil { return err @@ -653,11 +654,13 @@ func (*Replica) sha512( // we will probably not have any interleaved intents so we could stop // using MVCCKeyAndIntentsIterKind and consider all locks here. for _, span := range rditer.MakeReplicatedKeyRangesExceptLockTable(&desc) { - iter := snap.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, - storage.IterOptions{UpperBound: span.End}) - spanMS, err := storage.ComputeStatsForRange( - iter, span.Start, span.End, 0 /* nowNanos */, visitor, - ) + iter := snap.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: span.Start, + UpperBound: span.End, + }) + spanMS, err := storage.ComputeStatsForRangeWithVisitors( + iter, span.Start, span.End, 0 /* nowNanos */, pointKeyVisitor, nil /* rangeKeyVisitor */) iter.Close() if err != nil { return nil, err diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 8b8a1ddbf106..916171e81186 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2137,6 +2137,7 @@ func ComputeRaftLogSize( prefix := keys.RaftLogPrefix(rangeID) prefixEnd := prefix.PrefixEnd() iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, LowerBound: prefix, UpperBound: prefixEnd, }) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 47a2a6d067d8..cd2422221c6f 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -11495,7 +11495,7 @@ func TestRangeStatsRequest(t *testing.T) { defer stopper.Stop(ctx) tc.Start(ctx, t, stopper) - keyPrefix := roachpb.RKey("dummy-prefix") + keyPrefix := roachpb.Key("dummy-prefix") // Write some random data to the range and verify that a RangeStatsRequest // returns the same MVCC stats as the replica's in-memory state. diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant b/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant index cc31761c66a9..4bbf12461801 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant @@ -19,7 +19,8 @@ SELECT crdb_internal.check_consistency(true, '\x03', '\x02') query ITT SELECT range_id, status, regexp_replace(detail, '[0-9]+', '', 'g') FROM crdb_internal.check_consistency(true, '\x02', '\xffff') WHERE range_id = 1 ---- -1 RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: IntentAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: SeparatedIntentCount: SysBytes: SysCount: AbortSpanBytes:} +1 RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: IntentAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: SeparatedIntentCount: RangeKeyCount: RangeKeyBytes: SysBytes: SysCount: AbortSpanBytes:} + # Without explicit keys, scans all ranges (we don't test this too precisely to # avoid flaking the test when the range count changes, just want to know that diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index a92b50a04867..631f6767f05f 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -141,7 +141,6 @@ go_test( "//pkg/testutils/skip", "//pkg/testutils/zerofields", "//pkg/util", - "//pkg/util/caller", "//pkg/util/encoding", "//pkg/util/fileutil", "//pkg/util/hlc", diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index aebab38308bb..4787aa80265b 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -853,7 +853,11 @@ func runMVCCScan(ctx context.Context, b *testing.B, emk engineMaker, opts benchS // Pull all of the sstables into the RocksDB cache in order to make the // timings more stable. Otherwise, the first run will be penalized pulling // data into the cache while later runs will not. - iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: keys.LocalMax, + UpperBound: roachpb.KeyMax, + }) _, _ = iter.ComputeStats(keys.LocalMax, roachpb.KeyMax, 0) iter.Close() } @@ -1321,7 +1325,11 @@ func runMVCCComputeStats(ctx context.Context, b *testing.B, emk engineMaker, val var stats enginepb.MVCCStats var err error for i := 0; i < b.N; i++ { - iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: keys.LocalMax, + UpperBound: roachpb.KeyMax, + }) stats, err = iter.ComputeStats(keys.LocalMax, roachpb.KeyMax, 0) iter.Close() if err != nil { diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index f5454a7502f3..6c9ccd3eae64 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -200,11 +200,14 @@ type MVCCIterator interface { ValueProto(msg protoutil.Message) error // ComputeStats scans the underlying engine from start to end keys and // computes stats counters based on the values. This method is used after a - // range is split to recompute stats for each subrange. The start key is - // always adjusted to avoid counting local keys in the event stats are being - // recomputed for the first range (i.e. the one with start key == KeyMin). - // The nowNanos arg specifies the wall time in nanoseconds since the - // epoch and is used to compute the total age of all intents. + // range is split to recompute stats for each subrange. The nowNanos arg + // specifies the wall time in nanoseconds since the epoch and is used to + // compute the total age of intents and garbage. + // + // To properly account for intents and range keys, the iterator must be + // created with MVCCKeyAndIntentsIterKind and IterKeyTypePointsAndRanges, + // and the LowerBound and UpperBound must be set equal to start and end + // in order for range keys to be truncated to the bounds. ComputeStats(start, end roachpb.Key, nowNanos int64) (enginepb.MVCCStats, error) // FindSplitKey finds a key from the given span such that the left side of // the split is roughly targetSize bytes. The returned key will never be diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index 8e9c9a427633..de3ac5a56103 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -90,15 +90,17 @@ func (t TxnMeta) Short() redact.SafeString { } // Total returns the range size as the sum of the key and value -// bytes. This includes all non-live keys and all versioned values. +// bytes. This includes all non-live keys and all versioned values, +// both for point and range keys. func (ms MVCCStats) Total() int64 { - return ms.KeyBytes + ms.ValBytes + return ms.KeyBytes + ms.ValBytes + ms.RangeKeyBytes } // GCBytes is a convenience function which returns the number of gc bytes, -// that is the key and value bytes excluding the live bytes. +// that is the key and value bytes excluding the live bytes, both for +// point keys and range keys. func (ms MVCCStats) GCBytes() int64 { - return ms.KeyBytes + ms.ValBytes - ms.LiveBytes + return ms.Total() - ms.LiveBytes } // AvgIntentAge returns the average age of outstanding intents, @@ -169,6 +171,8 @@ func (ms *MVCCStats) Add(oms MVCCStats) { ms.ValCount += oms.ValCount ms.IntentCount += oms.IntentCount ms.SeparatedIntentCount += oms.SeparatedIntentCount + ms.RangeKeyCount += oms.RangeKeyCount + ms.RangeKeyBytes += oms.RangeKeyBytes ms.SysBytes += oms.SysBytes ms.SysCount += oms.SysCount ms.AbortSpanBytes += oms.AbortSpanBytes @@ -196,6 +200,8 @@ func (ms *MVCCStats) Subtract(oms MVCCStats) { ms.ValCount -= oms.ValCount ms.IntentCount -= oms.IntentCount ms.SeparatedIntentCount -= oms.SeparatedIntentCount + ms.RangeKeyCount -= oms.RangeKeyCount + ms.RangeKeyBytes -= oms.RangeKeyBytes ms.SysBytes -= oms.SysBytes ms.SysCount -= oms.SysCount ms.AbortSpanBytes -= oms.AbortSpanBytes diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index ffaf911e898e..b246e7f11550 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -163,8 +163,8 @@ message MVCCStats { // intent_age is the cumulative age of the tracked intents. // See the comment on MVCCStats. optional sfixed64 intent_age = 2 [(gogoproto.nullable) = false]; - // gc_bytes_age is the cumulative age of the non-live data (i.e. - // data included in key_bytes and val_bytes, but not live_bytes). + // gc_bytes_age is the cumulative age of the non-live data (i.e. data + // included in key_bytes, val_bytes, and range_key_bytes, but not live_bytes). // See the comment on MVCCStats. optional sfixed64 gc_bytes_age = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "GCBytesAge"]; // live_bytes is the number of bytes stored in keys and values which can in @@ -177,7 +177,7 @@ message MVCCStats { // live_count is the number of meta keys tracked under live_bytes. optional sfixed64 live_count = 5 [(gogoproto.nullable) = false]; // key_bytes is the number of bytes stored in all non-system - // keys, including live, meta, old, and deleted keys. + // point keys, including live, meta, old, and deleted keys. // Only meta keys really account for the "full" key; value // keys only for the timestamp suffix. optional sfixed64 key_bytes = 6 [(gogoproto.nullable) = false]; @@ -201,6 +201,22 @@ message MVCCStats { // intents, so mixed-version clusters with nodes preceding this knowledge // will always have a 0 value for this field. optional sfixed64 separated_intent_count = 16 [(gogoproto.nullable) = false]; + // range_key_count is the number of range keys tracked under range_key_bytes. + // Overlapping range keys get fragmented, such that writing a single range key + // may cause range_key_count to increase by more than 1 due to fragmentation. + // Multiple range key versions count as a single range key. + // + // Range keys that straddle range split boundaries will become two separate + // logical range keys (one in each range), and merge back to one range key + // when the ranges merge. + // + // NB: Currently, all range keys are MVCC range tombstones with no value. + // Therefore, these do not contribute to live_count nor live_bytes, and there + // is no range_val_bytes or range_val_count. + optional sfixed64 range_key_count = 17 [(gogoproto.nullable) = false]; + // range_key_bytes is the encoded size of logical range keys, disregarding + // fragmentation and overlap. + optional sfixed64 range_key_bytes = 18 [(gogoproto.nullable) = false]; // sys_bytes is the number of bytes stored in system-local kv-pairs. // This tracks the same quantity as (key_bytes + val_bytes), but diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index a528febf764f..2fc8927bb650 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -159,6 +159,8 @@ message MVCCStatsDelta { sint64 intent_bytes = 10; sint64 intent_count = 11; sint64 separated_intent_count = 16; + sint64 range_key_count = 17; + sint64 range_key_bytes = 18; sint64 sys_bytes = 12; sint64 sys_count = 13; sint64 abort_span_bytes = 15; @@ -188,6 +190,8 @@ message MVCCPersistentStats { int64 intent_bytes = 10; int64 intent_count = 11; int64 separated_intent_count = 16; + int64 range_key_count = 17; + int64 range_key_bytes = 18; int64 sys_bytes = 12; int64 sys_count = 13; int64 abort_span_bytes = 15; diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 7bb683d03188..d08670868eb1 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -847,39 +847,62 @@ func MVCCGetAsTxn( // iterator has already been seeked to metaKey, so a wasteful seek can be // avoided. func mvccGetMetadata( - iter MVCCIterator, metaKey MVCCKey, iterAlreadyPositioned bool, meta *enginepb.MVCCMetadata, + iter MVCCIterator, metaKey MVCCKey, meta *enginepb.MVCCMetadata, ) (ok bool, keyBytes, valBytes int64, err error) { if iter == nil { return false, 0, 0, nil } - if !iterAlreadyPositioned { - iter.SeekGE(metaKey) + iter.SeekGE(metaKey) + // If we land on a range key, move forward to look for a point key. + if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint { + iter.Next() } if ok, err := iter.Valid(); !ok { return false, 0, 0, err } + // TODO(erikgrinaker): For now, we only care about the stats effect range + // tombstones have on point keys, so if we don't find a point key we're done. + if hasPoint, _ := iter.HasPointAndRange(); !hasPoint { + return false, 0, 0, nil + } unsafeKey := iter.UnsafeKey() if !unsafeKey.Key.Equal(metaKey.Key) { return false, 0, 0, nil } + // Check for an existing intent. We don't need to check for range tombstones + // here, because any intents always exist above range tombstones, and range + // tombstones cannot be written across inline values. if !unsafeKey.IsValue() { if err := iter.ValueProto(meta); err != nil { return false, 0, 0, err } - return true, int64(unsafeKey.EncodedSize()), - int64(len(iter.UnsafeValue())), nil + return true, int64(unsafeKey.EncodedSize()), int64(len(iter.UnsafeValue())), nil } - meta.Reset() - // For values, the size of keys is always accounted for as - // MVCCVersionTimestampSize. The size of the metadata key is + // We set KeyBytes even if there is a range key here, because range keys are + // accounted for separately from point keys. The size of keys is always + // accounted for as MVCCVersionTimestampSize. The size of the metadata key is // accounted for separately. + meta.Reset() meta.KeyBytes = MVCCVersionTimestampSize meta.ValBytes = int64(len(iter.UnsafeValue())) meta.Deleted = meta.ValBytes == 0 meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() + + // TODO(erikgrinaker): For stats purposes only, if a live point key is covered + // by any number of range tombstones we return meta for the *earliest* range + // tombstone instead. The GCBytesAge contribution of the Pebble prefix (i.e. + // the roachpb.Key) begins there, and we need to correct for this when writing + // a new point key above it. However, this does not address other aspects such + // as conflict handling (which should return the latest range tombstone + // timestamp) -- this will be properly dealt with later. + if rk, ok := firstRangeKeyAbove(iter.RangeKeys(), unsafeKey.Timestamp); ok && !meta.Deleted { + meta.Deleted = true + meta.Timestamp = rk.Timestamp.ToLegacyTimestamp() + } + return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil } @@ -995,7 +1018,10 @@ func MVCCPut( var iter MVCCIterator blind := ms == nil && timestamp.IsEmpty() if !blind { - iter = rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true}) + iter = rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() } return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, txn, nil /* valueFn */) @@ -1037,7 +1063,10 @@ func MVCCDelete( timestamp hlc.Timestamp, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, nil /* valueFn */) @@ -1283,8 +1312,7 @@ func mvccPutInternal( } metaKey := MakeMVCCMetadataKey(key) - ok, origMetaKeySize, origMetaValSize, err := - mvccGetMetadata(iter, metaKey, false /* iterAlreadyPositioned */, &buf.meta) + ok, origMetaKeySize, origMetaValSize, err := mvccGetMetadata(iter, metaKey, &buf.meta) if err != nil { return err } @@ -1659,7 +1687,10 @@ func MVCCIncrement( txn *roachpb.Transaction, inc int64, ) (int64, error) { - iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() var int64Val int64 @@ -1731,7 +1762,10 @@ func MVCCConditionalPut( allowIfDoesNotExist CPutMissingBehavior, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccConditionalPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn) @@ -1809,7 +1843,10 @@ func MVCCInitPut( failOnTombstones bool, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, failOnTombstones, txn) } @@ -2207,17 +2244,21 @@ func MVCCDeleteRange( // This operation is non-transactional, but will check for existing intents and // return a WriteIntentError containing up to maxIntents intents. // +// The leftPeekBound and rightPeekBound parameters are used when looking for +// range tombstones that we'll merge or overlap with. These are provided to +// prevent the command from reading outside of the CRDB range bounds and latch +// bounds. nil means no bounds. +// // This method is EXPERIMENTAL: range keys are under active development, and // have severe limitations including being ignored by all KV and MVCC APIs and // only being stored in memory. -// -// TODO(erikgrinaker): This needs handling of stats. func ExperimentalMVCCDeleteRangeUsingTombstone( ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, startKey, endKey roachpb.Key, timestamp hlc.Timestamp, + leftPeekBound, rightPeekBound roachpb.Key, maxIntents int64, ) error { // Validate the range key. We must do this first, to catch e.g. any bound violations. @@ -2233,21 +2274,28 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( return &roachpb.WriteIntentError{Intents: intents} } - // Check for any conflicts, i.e. newer values. + // Forward the (empty) stats time to the deletion timestamp first, making the + // range tombstone's own GCBytesAge contributions 0 at this timestamp. + if ms != nil { + ms.Forward(timestamp.WallTime) + } + + // First, set up an iterator covering only the range key span itself, and scan + // it to find conflicts and update MVCC stats within it. // // TODO(erikgrinaker): This introduces an O(n) read penalty. We should - // consider either optimizing it or making the check optional somehow. + // optimize it, in particular by making this optional in cases where we're + // deleting an entire range and the stats can be computed without the scan. iter := rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, - LowerBound: startKey, - UpperBound: endKey, - MinTimestampHint: timestamp, - MaxTimestampHint: hlc.MaxTimestamp, + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: startKey, + UpperBound: endKey, + RangeKeyMaskingBelow: timestamp, }) defer iter.Close() iter.SeekGE(MVCCKey{Key: startKey}) - var prevRangeStart roachpb.Key + prevRangeEnd := startKey.Clone() for { if ok, err := iter.Valid(); err != nil { return err @@ -2256,28 +2304,178 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( } hasPoint, hasRange := iter.HasPointAndRange() + if hasPoint { - key := iter.UnsafeKey() + // Check for conflict with newer point key. + key, value := iter.UnsafeKey(), iter.UnsafeValue() if timestamp.LessEq(key.Timestamp) { return roachpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key.Clone()) } if key.Timestamp.IsEmpty() { return errors.Errorf("can't write range tombstone across inline key %s", key) } + + // Update stats for the covered point key, if it was a live key. + if ms != nil && len(value) > 0 { + ms.LiveCount-- + ms.LiveBytes -= int64(key.EncodedSize()) + int64(len(value)) + } } + if hasRange { - if rangeStart, _ := iter.RangeBounds(); !rangeStart.Equal(prevRangeStart) { + // Check if we've encountered a new range key stack. + if rangeStart, rangeEnd := iter.RangeBounds(); !rangeEnd.Equal(prevRangeEnd) { latest := iter.RangeKeys()[0] + + // Check for conflict with newer range key. if timestamp.LessEq(latest.Timestamp) { return roachpb.NewWriteTooOldError( timestamp, latest.Timestamp.Next(), latest.StartKey.Clone()) } - prevRangeStart = append(prevRangeStart[:0], rangeStart...) + + if ms != nil { + // If the encountered range key does not abut the previous range key, + // we'll write a new range key fragment in the gap between them. It + // has no GCBytesAge contribution because it's written at now. + if !rangeStart.Equal(prevRangeEnd) { + ms.RangeKeyCount++ + ms.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp) + + EncodedMVCCKeyPrefixLength(prevRangeEnd) + EncodedMVCCKeyPrefixLength(rangeStart)) + } + // This range key will create a new version in the current fragment + // stack. It will also move the GCBytesAge contribution of the key + // bounds up from the latest existing range key to this one. + ms.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp)) + ms.GCBytesAge -= (timestamp.WallTime/1e9 - latest.Timestamp.WallTime/1e9) * + int64(EncodedMVCCKeyPrefixLength(rangeStart)+EncodedMVCCKeyPrefixLength(rangeEnd)) + } + + prevRangeEnd = append(prevRangeEnd[:0], rangeEnd...) } } + iter.NextKey() } + // Once we've iterated across the range key span, fill in the final gap + // between the previous existing range key fragment and the end of the range + // key if any. If no existing fragments were found during iteration above, + // this will be the entire new range key. + if ms != nil && !prevRangeEnd.Equal(endKey) { + ms.RangeKeyCount++ + ms.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp) + + EncodedMVCCKeyPrefixLength(prevRangeEnd) + EncodedMVCCKeyPrefixLength(endKey)) + } + + // Check if the range key will merge with or fragment any existing range keys + // at the bounds. + // + // TODO(erikgrinaker): This could be merged into the scan above to avoid the + // additional seeks. But we do the simple and correct thing for now and leave + // optimizations for later. + if ms != nil { + // fragmentRangeKeys adjusts ms to fragment an existing range key stack + // at the given split point. + fragmentRangeKeys := func(rangeKeys []MVCCRangeKey, splitKey roachpb.Key) { + for i, rk := range rangeKeys { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(rk.Timestamp)) + if i == 0 { + ms.RangeKeyCount++ + keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(splitKey)) + } + ms.RangeKeyBytes += keyBytes + ms.GCBytesAge += keyBytes * (timestamp.WallTime/1e9 - rk.Timestamp.WallTime/1e9) + } + } + + // maybeMergeRangeKeys adjusts ms to merge two abutting range key + // stacks if they have the same timestamps. It assumes the lhs end key + // equals the rhs start key, and that they are in descending order. + maybeMergeRangeKeys := func(lhs, rhs []MVCCRangeKey) { + if len(lhs) != len(rhs) || len(lhs) == 0 { + return + } + for i, l := range lhs { + if !l.Timestamp.Equal(rhs[i].Timestamp) { + return + } + } + mergeKey := rhs[0].StartKey + for i, rk := range lhs { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(rk.Timestamp)) + if i == 0 { + ms.RangeKeyCount-- + keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(mergeKey)) + } + ms.RangeKeyBytes -= keyBytes + ms.GCBytesAge -= keyBytes * (timestamp.WallTime/1e9 - rk.Timestamp.WallTime/1e9) + } + } + + // Peek to the left. + if !leftPeekBound.Equal(startKey) { + iter := rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + KeyTypes: IterKeyTypeRangesOnly, + LowerBound: leftPeekBound, + UpperBound: startKey.Next(), + }) + defer iter.Close() + iter.SeekLT(MVCCKey{Key: startKey}) + if ok, err := iter.Valid(); err != nil { + return err + } else if ok { + _, rangeEnd := iter.RangeBounds() + switch rangeEnd.Compare(startKey) { + case 1: // fragment + fragmentRangeKeys(iter.RangeKeys(), startKey) + case 0: // merge + lhs := iter.RangeKeys() + rhs := []MVCCRangeKey{rangeKey} + iter.SeekGE(MVCCKey{Key: startKey}) + if ok, err := iter.Valid(); err != nil { + return err + } else if ok { + rhs = append(rhs, iter.RangeKeys()...) + } + maybeMergeRangeKeys(lhs, rhs) + } + } + } + + // Peek to the right. + if rightPeekBound == nil { + rightPeekBound = keys.MaxKey + } + if !rightPeekBound.Equal(endKey) { + iter := rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + KeyTypes: IterKeyTypeRangesOnly, + LowerBound: endKey.Prevish(roachpb.PrevishKeyLength), + UpperBound: rightPeekBound, + }) + defer iter.Close() + iter.SeekGE(MVCCKey{Key: endKey}) + if ok, err := iter.Valid(); err != nil { + return err + } else if ok { + rangeStart, _ := iter.RangeBounds() + switch rangeStart.Compare(endKey) { + case -1: // fragment + fragmentRangeKeys(iter.RangeKeys(), endKey) + case 0: // merge + lhs := []MVCCRangeKey{rangeKey} + rhs := iter.RangeKeys() + iter.SeekLT(MVCCKey{Key: endKey}) + if ok, err := iter.Valid(); err != nil { + return err + } else if ok { + lhs = append(lhs, iter.RangeKeys()...) + } + maybeMergeRangeKeys(lhs, rhs) + } + } + } + } + // Write the tombstone. return rw.ExperimentalPutMVCCRangeKey(rangeKey) } @@ -3567,8 +3765,7 @@ func MVCCGarbageCollect( meta := &enginepb.MVCCMetadata{} for _, gcKey := range keys { encKey := MakeMVCCMetadataKey(gcKey.Key) - ok, metaKeySize, metaValSize, err := - mvccGetMetadata(iter, encKey, false /* iterAlreadyPositioned */, meta) + ok, metaKeySize, metaValSize, err := mvccGetMetadata(iter, encKey, meta) if err != nil { return err } @@ -3842,28 +4039,43 @@ func willOverflow(a, b int64) bool { return math.MinInt64-b > a } -// ComputeStatsForRange scans the underlying engine from start to end keys and -// computes stats counters based on the values. This method is used after a -// range is split to recompute stats for each subrange. The nowNanos arg -// specifies the wall time in nanoseconds since the epoch and is used to compute -// the total age of all intents. +// ComputeStatsForRange scans the iterator from start to end keys and computes +// stats counters based on the values. This method is used after a range is +// split to recompute stats for each subrange. The nowNanos arg specifies the +// wall time in nanoseconds since the epoch and is used to compute the total age +// of all intents. +// +// To account for intents and range keys, the iterator must be created with +// MVCCKeyAndIntentsIterKind and IterKeyTypePointsAndRanges. To correctly +// account for range key truncation bounds, the iterator must have an +// appropriate UpperBound and LowerBound. // -// When optional callbacks are specified, they are invoked for each physical +// TODO(erikgrinaker): Consider removing the start,end parameters, forcing the +// caller to set appropriate bounds on the iterator instead. +func ComputeStatsForRange( + iter SimpleMVCCIterator, start, end roachpb.Key, nowNanos int64, +) (enginepb.MVCCStats, error) { + return ComputeStatsForRangeWithVisitors(iter, start, end, nowNanos, nil, nil) +} + +// ComputeStatsForRangeWithVisitors is like ComputeStatsForRange, but also +// takes a point and/or range key callback that is invoked for each physical // key-value pair (i.e. not for implicit meta records), and iteration is aborted -// on the first error returned from any of them. +// on the first error returned from either of them. // // Callbacks must copy any data they intend to hold on to. -func ComputeStatsForRange( +func ComputeStatsForRangeWithVisitors( iter SimpleMVCCIterator, start, end roachpb.Key, nowNanos int64, - callbacks ...func(MVCCKey, []byte) error, + pointKeyVisitor func(MVCCKey, []byte) error, + rangeKeyVisitor func(MVCCRangeKey) error, ) (enginepb.MVCCStats, error) { var ms enginepb.MVCCStats // Only some callers are providing an MVCCIterator. The others don't have // any intents. var meta enginepb.MVCCMetadata - var prevKey []byte + var prevKey, prevRangeStart []byte first := false // Values start accruing GCBytesAge at the timestamp at which they @@ -3873,22 +4085,60 @@ func ComputeStatsForRange( // of the point in time at which the current key begins to age. var accrueGCAgeNanos int64 mvccEndKey := MakeMVCCMetadataKey(end) + rangeKeys := []MVCCRangeKey{} - iter.SeekGE(MakeMVCCMetadataKey(start)) - for ; ; iter.Next() { - ok, err := iter.Valid() - if err != nil { + for iter.SeekGE(MakeMVCCMetadataKey(start)); ; iter.Next() { + if ok, err := iter.Valid(); err != nil { return ms, err - } - if !ok || !iter.UnsafeKey().Less(mvccEndKey) { + } else if !ok || !iter.UnsafeKey().Less(mvccEndKey) { break } + hasPoint, hasRange := iter.HasPointAndRange() + + if hasRange { + if rangeStart, _ := iter.RangeBounds(); !bytes.Equal(rangeStart, prevRangeStart) { + prevRangeStart = rangeStart.Clone() + rangeKeys = iter.RangeKeys() + + for i, rk := range rangeKeys { + // Only the top-most fragment contributes the key and its bounds, but + // all versions contribute timestamps. + // + // NB: Point keys always use 12 bytes for the key timestamp, even + // though it is actually variable-length, likely for historical + // reasons. But for range keys we may as well use the actual + // variable-length encoded size. + var keyBytes int64 + if i == 0 { + ms.RangeKeyCount++ + keyBytes += int64(EncodedMVCCKeyPrefixLength(rk.StartKey) + + EncodedMVCCKeyPrefixLength(rk.EndKey)) + } + keyBytes += int64(EncodedMVCCTimestampSuffixLength(rk.Timestamp)) + ms.RangeKeyBytes += keyBytes + ms.GCBytesAge += keyBytes * (nowNanos/1e9 - rk.Timestamp.WallTime/1e9) + + if rangeKeyVisitor != nil { + if err := rangeKeyVisitor(rk); err != nil { + return enginepb.MVCCStats{}, err + } + } + } + } + } else if len(rangeKeys) > 0 { + rangeKeys = rangeKeys[:0] + } + + if !hasPoint { + continue + } + unsafeKey := iter.UnsafeKey() unsafeValue := iter.UnsafeValue() - for _, f := range callbacks { - if err := f(unsafeKey, unsafeValue); err != nil { + if pointKeyVisitor != nil { + if err := pointKeyVisitor(unsafeKey, unsafeValue); err != nil { return enginepb.MVCCStats{}, err } } @@ -3915,6 +4165,16 @@ func ComputeStatsForRange( implicitMeta := isValue && !bytes.Equal(unsafeKey.Key, prevKey) prevKey = append(prevKey[:0], unsafeKey.Key...) + // Find the closest range tombstone above the point key. Range tombstones + // cannot exist above intents, and are undefined across inline values, so we + // only take them into account for versioned values. + var nextRangeTombstone hlc.Timestamp + if isValue { + if rk, ok := firstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + nextRangeTombstone = rk.Timestamp + } + } + if implicitMeta { // No MVCCMetadata entry for this series of keys. meta.Reset() @@ -3946,12 +4206,16 @@ func ComputeStatsForRange( ms.AbortSpanBytes += totalBytes } } else { - if !meta.Deleted { - ms.LiveBytes += totalBytes - ms.LiveCount++ - } else { + if meta.Deleted { // First value is deleted, so it's GC'able; add meta key & value bytes to age stat. ms.GCBytesAge += totalBytes * (nowNanos/1e9 - meta.Timestamp.WallTime/1e9) + } else if nextRangeTombstone.IsSet() { + // First value was deleted by a range tombstone, so it accumulates GC age from + // the range tombstone's timestamp. + ms.GCBytesAge += totalBytes * (nowNanos/1e9 - nextRangeTombstone.WallTime/1e9) + } else { + ms.LiveBytes += totalBytes + ms.LiveCount++ } ms.KeyBytes += metaKeySize ms.ValBytes += metaValSize @@ -3971,11 +4235,15 @@ func ComputeStatsForRange( } else { if first { first = false - if !meta.Deleted { - ms.LiveBytes += totalBytes - } else { + if meta.Deleted { // First value is deleted, so it's GC'able; add key & value bytes to age stat. ms.GCBytesAge += totalBytes * (nowNanos/1e9 - meta.Timestamp.WallTime/1e9) + } else if nextRangeTombstone.IsSet() { + // First value was deleted by a range tombstone; add key & value bytes to + // age stat from range tombstone onwards. + ms.GCBytesAge += totalBytes * (nowNanos/1e9 - nextRangeTombstone.WallTime/1e9) + } else { + ms.LiveBytes += totalBytes } if meta.Txn != nil { ms.IntentBytes += totalBytes @@ -3998,6 +4266,10 @@ func ComputeStatsForRange( if isTombstone { // The contribution of the tombstone picks up GCByteAge from its own timestamp on. ms.GCBytesAge += totalBytes * (nowNanos/1e9 - unsafeKey.Timestamp.WallTime/1e9) + } else if nextRangeTombstone.IsSet() && nextRangeTombstone.WallTime < accrueGCAgeNanos { + // The kv pair was deleted by a range tombstone below the next + // version, so it accumulates garbage from the range tombstone. + ms.GCBytesAge += totalBytes * (nowNanos/1e9 - nextRangeTombstone.WallTime/1e9) } else { // The kv pair is an overwritten value, so it became non-live when the closest more // recent value was written. diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 3c7bcef120eb..b0cf29329d91 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -751,7 +751,7 @@ func cmdDeleteRangeTombstone(e *evalCtx) error { ts := e.getTs(nil) return e.withWriter("del_range_ts", func(rw ReadWriter) error { - return ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, e.ms, key, endKey, ts, 0) + return ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, e.ms, key, endKey, ts, nil, nil, 0) }) } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 494dc2d30066..c96bce7bcdf0 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -1647,7 +1647,11 @@ func runIncrementalBenchmark( // Pull all of the sstables into the cache. This // probably defeats a lot of the benefits of the // time-based optimization. - iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: roachpb.LocalMax, + UpperBound: roachpb.KeyMax, + }) _, _ = iter.ComputeStats(keys.LocalMax, roachpb.KeyMax, 0) iter.Close() } diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index a30921e3861b..2452b2b4e79b 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -13,6 +13,7 @@ package storage import ( "encoding/binary" "fmt" + "sort" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -361,6 +362,19 @@ func (k MVCCRangeKey) Compare(o MVCCRangeKey) int { return k.EndKey.Compare(o.EndKey) } +// EncodedSize returns the encoded size of this range key. This does not +// accurately reflect the on-disk size of the key, due to Pebble range key +// stacking and fragmentation. +// +// NB: This calculation differs from MVCCKey in that MVCCKey.EncodedSize() +// incorrectly always uses 13 bytes for the timestamp while this method +// calculates the actual encoded size. +func (k MVCCRangeKey) EncodedSize() int { + return EncodedMVCCKeyPrefixLength(k.StartKey) + + EncodedMVCCKeyPrefixLength(k.EndKey) + + EncodedMVCCTimestampSuffixLength(k.Timestamp) +} + // String formats the range key. func (k MVCCRangeKey) String() string { s := roachpb.Span{Key: k.StartKey, EndKey: k.EndKey}.String() @@ -395,3 +409,19 @@ func (k MVCCRangeKey) Validate() (err error) { return nil } } + +// firstRangeKeyAbove does a binary search for the first range key at or above +// the given timestamp. It assumes the range keys are ordered in descending +// timestamp order, as returned by SimpleMVCCIterator.RangeKeys(). Returns false +// if no matching range key was found. +func firstRangeKeyAbove(rangeKeys []MVCCRangeKey, ts hlc.Timestamp) (MVCCRangeKey, bool) { + // This is kind of odd due to sort.Search() semantics: we do a binary search + // for the first range tombstone that's below the timestamp, then return the + // previous range tombstone if any. + if i := sort.Search(len(rangeKeys), func(i int) bool { + return rangeKeys[i].Timestamp.Less(ts) + }); i > 0 { + return rangeKeys[i-1], true + } + return MVCCRangeKey{}, false +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index d152eb464d29..6ecab69d8363 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -439,6 +439,31 @@ func TestMVCCRangeKeyCompare(t *testing.T) { } } +func TestMVCCRangeKeyEncodedSize(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + rk MVCCRangeKey + expect int + }{ + "empty": {MVCCRangeKey{}, 2}, // sentinel byte for start and end + "only start": {MVCCRangeKey{StartKey: roachpb.Key("foo")}, 5}, + "only end": {MVCCRangeKey{EndKey: roachpb.Key("foo")}, 5}, + "only walltime": {MVCCRangeKey{Timestamp: hlc.Timestamp{WallTime: 1}}, 11}, + "only logical": {MVCCRangeKey{Timestamp: hlc.Timestamp{Logical: 1}}, 15}, + "all": {MVCCRangeKey{ + StartKey: roachpb.Key("start"), + EndKey: roachpb.Key("end"), + Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1, Synthetic: true}, + }, 24}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + require.Equal(t, tc.expect, tc.rk.EncodedSize()) + }) + } +} + func TestMVCCRangeKeyValidate(t *testing.T) { defer leaktest.AfterTest(t)() @@ -473,6 +498,43 @@ func TestMVCCRangeKeyValidate(t *testing.T) { } } +func TestFirstRangeKeyAbove(t *testing.T) { + defer leaktest.AfterTest(t)() + + rangeKeys := []MVCCRangeKey{ + rangeKey("a", "f", 6), + rangeKey("a", "f", 4), + rangeKey("a", "f", 3), + rangeKey("a", "f", 1), + } + + testcases := []struct { + ts int64 + expect int64 + }{ + {0, 1}, + {1, 1}, + {2, 3}, + {3, 3}, + {4, 4}, + {5, 6}, + {6, 6}, + {7, 0}, + } + for _, tc := range testcases { + t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { + rk, ok := firstRangeKeyAbove(rangeKeys, hlc.Timestamp{WallTime: tc.ts}) + if tc.expect == 0 { + require.False(t, ok) + require.Empty(t, rk) + } else { + require.True(t, ok) + require.Equal(t, rangeKey("a", "f", int(tc.expect)), rk) + } + }) + } +} + func pointKey(key string, ts int) MVCCKey { return MVCCKey{Key: roachpb.Key(key), Timestamp: hlc.Timestamp{WallTime: int64(ts)}} } diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 5cc09743c86d..406d24f9cf01 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -57,7 +57,11 @@ func assertEqImpl( keyMin = keys.LocalMax keyMax = roachpb.KeyMax } - it := rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: keyMax}) + it := rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: keyMin, + UpperBound: keyMax, + }) defer it.Close() for _, mvccStatsTest := range mvccStatsTests { @@ -1362,6 +1366,108 @@ func TestMVCCStatsSysPutPut(t *testing.T) { } } +func TestMVCCStatsRangeKeysOnly(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + eng := NewDefaultInMemForTesting() + defer eng.Close() + + type stats = enginepb.MVCCStats + const now = 10 * 1e9 + + // Add range keys one by one and assert their stats contribution. + testcases := []struct { + rangeKey MVCCRangeKey + stats enginepb.MVCCStats + }{ + {rangeKey("a", "f", 8e9), stats{RangeKeyCount: 1, RangeKeyBytes: 13, GCBytesAge: 2 * 13}}, + // Historical version of [a-f)@8. + {rangeKey("a", "f", 1e9), stats{RangeKeyBytes: 9, GCBytesAge: 9 * 9}}, + // Fragment [a-f) into [a-e) and [e-f). + {rangeKey("e", "f", 2e9), stats{RangeKeyCount: 1, RangeKeyBytes: 13 + 2*9, GCBytesAge: 2*13 + 8*9 + 9*9}}, + // Fragment [a-e) in the middle into [a-bb), [bb-ccc), and [ccc-e). + {rangeKey("bb", "ccc", 5e9), stats{ + RangeKeyCount: 2, // creates bb-ccc and ccc-e + RangeKeyBytes: 1 + 16 + 15 + 3*9, // [a-e) becomes [a-bb), adds keys [bb-ccc)@8 and [ccc-e)@8, plus versions [bb-ccc)@5, [bb-ccc)@1, [ccc-e)@1 + GCBytesAge: 2*1 + 2*16 + 2*15 + 5*9 + 9*9 + 9*9, + }}, + // Extending the range keys below others should only add itself, but it will + // create new versions across existing fragments. + {rangeKey("a", "p", 3e9), stats{RangeKeyCount: 1, RangeKeyBytes: 13 + 4*9, GCBytesAge: 7 * (13 + 4*9)}}, + // Dropping a range key covering all existing fragments will create new + // versions of each fragment with no GCBytesAge contribution, but will also + // reduce the GCBytesAge contribution of the key bounds of the topmost keys + // since these are now moved up to the latest version at timestamp 10. The + // topmost fragments before this were: + // + // [a-bb)@8 [bb-ccc)@8 [ccc-e)@8 [e-f)@8 [f-p)@3 + {rangeKey("a", "p", 10e9), stats{RangeKeyBytes: 5 * 9, GCBytesAge: -2*(14-9) - 2*(16-9) - 2*(15-9) - 2*(13-9) - 7*(13-9)}}, + } + + for _, tc := range testcases { + // We don't use t.Run() because the test cases are dependant. + before := computeStats(t, eng, roachpb.Key("a"), roachpb.Key("z"), now) + require.NoError(t, eng.ExperimentalPutMVCCRangeKey(tc.rangeKey)) + after := computeStats(t, eng, roachpb.Key("a"), roachpb.Key("z"), now) + + delta := after + delta.Subtract(before) + delta.LastUpdateNanos = 0 + require.Equal(t, tc.stats, delta, "range key %s", tc.rangeKey) + } +} + +func TestMVCCStatsRangeKeysAndPointKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + eng := NewDefaultInMemForTesting() + defer eng.Close() + + const now = 10e9 + + rangeKeys := []MVCCRangeKey{ + rangeKey("a", "z", 8e9), // RangeKeyCount:1 RangeKeyBytes:13 GCBytesAge:26 + rangeKey("k", "z", 5e9), // RangeKeyCount:1 RangeKeyBytes:13+9 GCBytesAge:5*13+3*2 + } + for _, rk := range rangeKeys { + require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rk)) + } + + pointKVs := []MVCCKeyValue{ + pointKV("a", 9e9, "a9"), // LiveCount:1 LiveBytes:16 KeyCount:1 KeyBytes:14 ValCount:1 ValBytes:2 + pointKV("b", 5e9, ""), // KeyCount:1 KeyBytes:14 ValCount:1 GCBytesAge:70 + pointKV("c", 5e9, "c5"), // KeyCount:1 KeyBytes:14 ValCount:1 ValBytes:2 GCBytesAge:32 + pointKV("d", 7e9, ""), // KeyCount:1 KeyBytes:14 ValCount:1 GCBytesAge:42 + pointKV("d", 6e9, "d6"), // KeyBytes:12 ValCount:1 ValBytes:2 GCBytesAge:42 + pointKV("d", 5e9, "d5"), // KeyBytes:12 ValCount:1 ValBytes:2 GCBytesAge:56 + pointKV("e", 9e9, "e9"), // LiveCount:1 LiveBytes:16 KeyCount:1 KeyBytes:14 ValCount:1 ValBytes:2 + pointKV("e", 5e9, "e5"), // KeyBytes:12 ValCount:1 ValBytes:2 GCBytesAge:28 + pointKV("m", 6e9, "m6"), // KeyCount:1 KeyBytes:14 ValCount:1 ValBytes:2 GCBytesAge:32 + pointKV("m", 3e9, "m3"), // KeyBytes:12 ValCount:1 ValBytes:2 GCBytesAge:70 + pointKV("o", 9e9, "o9"), // LiveCount:1 LiveBytes:16 KeyCount:1 KeyBytes:14 ValCount:1 ValBytes:2 + pointKV("o", 3e9, ""), // KeyBytes:12 ValCount:1 GCBytesAge:84 + pointKV("o", 1e9, "o1"), // KeyBytes:12 ValCount:1 ValBytes:2 GCBytesAge:98 + } + for _, kv := range pointKVs { + require.NoError(t, eng.PutMVCC(kv.Key, kv.Value)) + } + + require.Equal(t, enginepb.MVCCStats{ + LiveCount: 3, + LiveBytes: 48, + KeyCount: 7, + KeyBytes: 170, + ValCount: 13, + ValBytes: 20, + RangeKeyCount: 2, + RangeKeyBytes: 35, + GCBytesAge: 651, + LastUpdateNanos: now, + }, computeStats(t, eng, nil, nil, now)) +} + var mvccStatsTests = []struct { name string fn func(MVCCIterator, roachpb.Key, roachpb.Key, int64) (enginepb.MVCCStats, error) @@ -1480,6 +1586,8 @@ func (s *randomTest) step(t *testing.T) { } } +// TODO(erikgrinaker): Add ExperimentalMVCCDeleteRangeUsingTombstone operations +// once they are fully integrated with other MVCC operations. func TestMVCCStatsRandomized(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1651,7 +1759,11 @@ func TestMVCCComputeStatsError(t *testing.T) { t.Fatal(err) } - iter := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: roachpb.LocalMax, + UpperBound: roachpb.KeyMax, + }) defer iter.Close() for _, mvccStatsTest := range mvccStatsTests { t.Run(mvccStatsTest.name, func(t *testing.T) { diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 72dfa9114e2d..a4d69bc908fb 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/zerofields" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -124,6 +123,8 @@ func TestMVCCStatsAddSubForward(t *testing.T) { ValCount: 1, IntentBytes: 1, IntentCount: 1, + RangeKeyCount: 1, + RangeKeyBytes: 1, SeparatedIntentCount: 1, IntentAge: 1, GCBytesAge: 1, @@ -134,17 +135,7 @@ func TestMVCCStatsAddSubForward(t *testing.T) { LastUpdateNanos: 1, AbortSpanBytes: 1, } - if err := zerofields.NoZeroField(&goldMS); err != nil { - t.Fatal(err) // prevent rot as fields are added - } - - cmp := func(act, exp enginepb.MVCCStats) { - t.Helper() - f, l, _ := caller.Lookup(1) - if !reflect.DeepEqual(act, exp) { - t.Fatalf("%s:%d: wanted %+v back, got %+v", f, l, exp, act) - } - } + require.NoError(t, zerofields.NoZeroField(&goldMS)) ms := goldMS zeroWithLU := enginepb.MVCCStats{ @@ -153,10 +144,10 @@ func TestMVCCStatsAddSubForward(t *testing.T) { } ms.Subtract(goldMS) - cmp(ms, zeroWithLU) + require.Equal(t, zeroWithLU, ms) ms.Add(goldMS) - cmp(ms, goldMS) + require.Equal(t, goldMS, ms) // Double-add double-sub guards against mistaking `+=` for `=`. ms = zeroWithLU @@ -164,7 +155,7 @@ func TestMVCCStatsAddSubForward(t *testing.T) { ms.Add(goldMS) ms.Subtract(goldMS) ms.Subtract(goldMS) - cmp(ms, zeroWithLU) + require.Equal(t, zeroWithLU, ms) // Run some checks for Forward. goldDelta := enginepb.MVCCStats{ @@ -174,31 +165,27 @@ func TestMVCCStatsAddSubForward(t *testing.T) { } delta := goldDelta - for i, ns := range []int64{1, 1e9 - 1001, 1e9 - 1000, 1e9 - 1, 1e9, 1e9 + 1, 2e9 - 1} { + for _, ns := range []int64{1, 1e9 - 1001, 1e9 - 1000, 1e9 - 1, 1e9, 1e9 + 1, 2e9 - 1} { oldDelta := delta delta.AgeTo(ns) - if delta.LastUpdateNanos < ns { - t.Fatalf("%d: expected LastUpdateNanos < %d, got %d", i, ns, delta.LastUpdateNanos) - } + require.GreaterOrEqual(t, delta.LastUpdateNanos, ns, "LastUpdateNanos") shouldAge := ns/1e9-oldDelta.LastUpdateNanos/1e9 > 0 didAge := delta.IntentAge != oldDelta.IntentAge && delta.GCBytesAge != oldDelta.GCBytesAge - if shouldAge != didAge { - t.Fatalf("%d: should age: %t, but had\n%+v\nand now\n%+v", i, shouldAge, oldDelta, delta) - } + require.Equal(t, shouldAge, didAge) } expDelta := goldDelta expDelta.LastUpdateNanos = 2e9 - 1 expDelta.GCBytesAge = 42 expDelta.IntentAge = 11 - cmp(delta, expDelta) + require.Equal(t, expDelta, delta) delta.AgeTo(2e9) expDelta.LastUpdateNanos = 2e9 expDelta.GCBytesAge += 42 expDelta.IntentAge += 11 - cmp(delta, expDelta) + require.Equal(t, expDelta, delta) { // Verify that AgeTo can go backwards in time. @@ -210,13 +197,13 @@ func TestMVCCStatsAddSubForward(t *testing.T) { expDelta.LastUpdateNanos = 2e9 - 1 expDelta.GCBytesAge -= 42 expDelta.IntentAge -= 11 - cmp(tmpDelta, expDelta) + require.Equal(t, expDelta, tmpDelta) } delta.AgeTo(3e9 - 1) delta.Forward(5) // should be noop expDelta.LastUpdateNanos = 3e9 - 1 - cmp(delta, expDelta) + require.Equal(t, expDelta, delta) // Check that Add calls Forward appropriately. mss := []enginepb.MVCCStats{goldMS, goldMS} @@ -227,13 +214,13 @@ func TestMVCCStatsAddSubForward(t *testing.T) { expMS := goldMS expMS.Add(goldMS) expMS.LastUpdateNanos = 10e9 + 1 - expMS.IntentAge += 9 // from aging 9 ticks from 2E9-1 to 10E9+1 - expMS.GCBytesAge += 9 // ditto + expMS.IntentAge += 9 // from aging 9 ticks from 2E9-1 to 10E9+1 + expMS.GCBytesAge += 2 * 9 // ditto for i := range mss[:1] { ms := mss[(1+i)%2] ms.Add(mss[i]) - cmp(ms, expMS) + require.Equal(t, expMS, ms) } // Finally, check Forward with negative counts (can happen). @@ -244,9 +231,9 @@ func TestMVCCStatsAddSubForward(t *testing.T) { neg.AgeTo(2e9) exp.LastUpdateNanos = 2e9 - exp.GCBytesAge = -3 + exp.GCBytesAge = -5 exp.IntentAge = -3 - cmp(neg, exp) + require.Equal(t, exp, neg) } func TestMVCCGetNotExist(t *testing.T) { @@ -911,7 +898,10 @@ func TestMVCCInvalidateIterator(t *testing.T) { switch which { case "get": iterOptions.Prefix = true - case "scan", "findSplitKey", "computeStats": + case "computeStats": + iterOptions.KeyTypes = IterKeyTypePointsAndRanges + iterOptions.UpperBound = roachpb.KeyMax + case "scan", "findSplitKey": iterOptions.UpperBound = roachpb.KeyMax } @@ -2302,13 +2292,23 @@ func computeStats( t *testing.T, reader Reader, from, to roachpb.Key, nowNanos int64, ) enginepb.MVCCStats { t.Helper() - iter := reader.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: to}) - defer iter.Close() - s, err := ComputeStatsForRange(iter, from, to, nowNanos) - if err != nil { - t.Fatalf("%+v", err) + + if len(from) == 0 { + from = keys.LocalMax + } + if len(to) == 0 { + to = keys.MaxKey } - return s + + iter := reader.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: from, + UpperBound: to, + }) + defer iter.Close() + ms, err := ComputeStatsForRange(iter, from, to, nowNanos) + require.NoError(t, err) + return ms } // TestMVCCClearTimeRangeOnRandomData sets up mostly random KVs and then picks diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index 7a2c510e11e3..8b03acec9580 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -56,6 +56,10 @@ func NewSSTIterator(file sstable.ReadableFile) (SimpleMVCCIterator, error) { // It's compatible with sstables written by `RocksDBSstFileWriter` and // Pebble's `sstable.Writer`, and assumes the keys use Cockroach's MVCC // format. +// +// TODO(erikgrinaker): When this gets support for iterating over range keys, all +// call sites that use ComputeStatsForRange() must be updated to enable range +// keys for the iterators. func NewMemSSTIterator(data []byte, verify bool) (SimpleMVCCIterator, error) { sst, err := sstable.NewReader(vfs.NewMemFile(data), sstable.ReaderOptions{ Comparer: EngineComparer, @@ -160,16 +164,18 @@ func (r *sstIterator) UnsafeValue() []byte { } // HasPointAndRange implements SimpleMVCCIterator. +// +// TODO(erikgrinaker): implement range key support. func (r *sstIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") + return true, false } // RangeBounds implements SimpleMVCCIterator. func (r *sstIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") + return nil, nil } // RangeKeys implements SimpleMVCCIterator. func (r *sstIterator) RangeKeys() []MVCCRangeKey { - panic("not implemented") + return nil } diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter b/pkg/storage/testdata/mvcc_histories/range_key_iter index 369d6b633e29..70bd88406711 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter @@ -16,18 +16,18 @@ run ok put_rangekey k=a end=k ts=1 put_rangekey k=l end=m ts=1 put_rangekey k=b end=d ts=3 -put_rangekey k=f end=h ts=3 -put_rangekey k=c end=g ts=5 put k=a ts=2 v=a2 del k=a ts=4 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 +put k=g ts=2 v=g2 +put_rangekey k=f end=h ts=3 put k=f ts=4 v=f4 put k=f ts=6 v=f6 -put k=g ts=2 v=g2 put k=g ts=4 v=g4 +put_rangekey k=c end=g ts=5 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations b/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations index 8320cb96d3b9..4fa6688415ea 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations @@ -13,6 +13,7 @@ run ok put k=a ts=2 v=a2 del k=a ts=4 +put k=b ts=3 v=b3 put k=d ts=4 v=d4 put k=f ts=4 v=f4 put k=g ts=2 v=g2 @@ -29,6 +30,7 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0, rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -47,6 +49,7 @@ del_range_ts k=z end=x ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -65,6 +68,7 @@ del_range_ts k=x end=z ts=0 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -83,6 +87,7 @@ del_range_ts k=x end=x ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -103,6 +108,7 @@ del_range_ts k=a end=b ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -121,6 +127,7 @@ del_range_ts k=a end=b ts=4 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -139,6 +146,7 @@ del_range_ts k=e end=g ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -159,6 +167,7 @@ del_range_ts k=k end=p ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -177,6 +186,7 @@ del_range_ts k=k end=p ts=4 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -195,6 +205,7 @@ del_range_ts k=j end=m ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -213,6 +224,7 @@ del_range_ts k=o end=q ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -231,6 +243,7 @@ del_range_ts k=j end=q ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -249,6 +262,7 @@ del_range_ts k=k end=n ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -269,6 +283,7 @@ del_range_ts k=d end=e ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -287,6 +302,7 @@ del_range_ts k=d end=e ts=5 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -305,6 +321,7 @@ del_range_ts k=i end=j ts=5 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -323,6 +340,7 @@ del_range_ts k=i end=j ts=7 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -341,6 +359,7 @@ del_range_ts k=i end=j ts=10 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -360,6 +379,7 @@ del_range_ts k=h end=i ts=3 rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -371,17 +391,37 @@ meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 data: "i"/7.000000000,0 -> /BYTES/i7 error: (*withstack.withStack:) can't write range tombstone across inline key "h"/0,0 -# Writing next to or above point keys and tombstones should work. -run ok +# Writing next to or above point keys and tombstones should work, and should +# update MVCC stats accordingly. +run stats trace del_range_ts k=a end=b ts=10 del_range_ts k=b end=d ts=4 +del_range_ts k=f end=h ts=10 ---- ->> at end: +>> del_range_ts k=a end=b ts=10 +stats: range_key_count:+1 range_key_bytes:+13 +rangekey: {a-b}/[10.000000000,0] +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +>> del_range_ts k=b end=d ts=4 +stats: live_bytes:-21 live_count:-1 range_key_count:+1 range_key_bytes:+13 rangekey: {a-b}/[10.000000000,0] rangekey: {b-d}/[4.000000000,0] rangekey: {k-p}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -391,21 +431,15 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 - -# Writing range tombstones next to other range tombstones will merge them, but -# only at same timestamp. -run ok -del_range_ts k=s end=x ts=4 -del_range_ts k=p end=s ts=3 ----- ->> at end: +>> del_range_ts k=f end=h ts=10 +stats: live_bytes:-21 live_count:-1 range_key_count:+1 range_key_bytes:+13 rangekey: {a-b}/[10.000000000,0] rangekey: {b-d}/[4.000000000,0] +rangekey: {f-h}/[10.000000000,0] rangekey: {k-p}/[4.000000000,0] -rangekey: {p-s}/[3.000000000,0] -rangekey: {s-x}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -415,18 +449,145 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 +stats: { + contains_estimates:0 + last_update_nanos:1000000000000 + intent_age:1986 + gc_bytes_age:177945 + live_bytes:165 + live_count:3 + key_bytes:122 + key_count:7 + val_bytes:170 + val_count:10 + intent_bytes:38 + intent_count:2 + separated_intent_count:2 + range_key_count:4 + range_key_bytes:52 + sys_bytes:0 + sys_count:0 + abort_span_bytes:0 +} -run ok +# Writing a range tombstone above another should not affect stats, except for +# its own contribution. +run stats trace +del_range_ts k=a end=d ts=11 +---- +>> del_range_ts k=a end=d ts=11 +stats: gc_bytes_age:-32 range_key_bytes:+18 +rangekey: {a-b}/[11.000000000,0 10.000000000,0] +rangekey: {b-d}/[11.000000000,0 4.000000000,0] +rangekey: {f-h}/[10.000000000,0] +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +stats: { + contains_estimates:0 + last_update_nanos:1000000000000 + intent_age:1986 + gc_bytes_age:195715 + live_bytes:165 + live_count:3 + key_bytes:122 + key_count:7 + val_bytes:170 + val_count:10 + intent_bytes:38 + intent_count:2 + separated_intent_count:2 + range_key_count:4 + range_key_bytes:70 + sys_bytes:0 + sys_count:0 + abort_span_bytes:0 +} + +# Writing range tombstones next to other range tombstones will merge them, and +# stats take this into account. +run stats trace +del_range_ts k=www end=xxx ts=4 +del_range_ts k=xxx end=z ts=4 +del_range_ts k=s end=www ts=4 del_range_ts k=p end=s ts=4 ---- ->> at end: -rangekey: {a-b}/[10.000000000,0] -rangekey: {b-d}/[4.000000000,0] +>> del_range_ts k=www end=xxx ts=4 +stats: range_key_count:+1 range_key_bytes:+17 +rangekey: {a-b}/[11.000000000,0 10.000000000,0] +rangekey: {b-d}/[11.000000000,0 4.000000000,0] +rangekey: {f-h}/[10.000000000,0] +rangekey: {k-p}/[4.000000000,0] +rangekey: {www-xxx}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +>> del_range_ts k=xxx end=z ts=4 +stats: range_key_bytes:-2 +rangekey: {a-b}/[11.000000000,0 10.000000000,0] +rangekey: {b-d}/[11.000000000,0 4.000000000,0] +rangekey: {f-h}/[10.000000000,0] +rangekey: {k-p}/[4.000000000,0] +rangekey: {www-z}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +>> del_range_ts k=s end=www ts=4 +stats: range_key_bytes:-2 +rangekey: {a-b}/[11.000000000,0 10.000000000,0] +rangekey: {b-d}/[11.000000000,0 4.000000000,0] +rangekey: {f-h}/[10.000000000,0] rangekey: {k-p}/[4.000000000,0] -rangekey: {p-s}/[4.000000000,0 3.000000000,0] -rangekey: {s-x}/[4.000000000,0] +rangekey: {s-z}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +>> del_range_ts k=p end=s ts=4 +stats: range_key_count:-1 range_key_bytes:-13 +rangekey: {a-b}/[11.000000000,0 10.000000000,0] +rangekey: {b-d}/[11.000000000,0 4.000000000,0] +rangekey: {f-h}/[10.000000000,0] +rangekey: {k-z}/[4.000000000,0] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 @@ -436,3 +597,23 @@ data: "g"/2.000000000,0 -> /BYTES/g2 meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "i"/7.000000000,0 -> /BYTES/i7 +stats: { + contains_estimates:0 + last_update_nanos:1000000000000 + intent_age:1986 + gc_bytes_age:195715 + live_bytes:165 + live_count:3 + key_bytes:122 + key_count:7 + val_bytes:170 + val_count:10 + intent_bytes:38 + intent_count:2 + separated_intent_count:2 + range_key_count:4 + range_key_bytes:70 + sys_bytes:0 + sys_count:0 + abort_span_bytes:0 +} diff --git a/pkg/storage/testdata/mvcc_histories/stats b/pkg/storage/testdata/mvcc_histories/stats index d1b8c5701bb7..126c88ff4969 100644 --- a/pkg/storage/testdata/mvcc_histories/stats +++ b/pkg/storage/testdata/mvcc_histories/stats @@ -1,16 +1,40 @@ # Tests MVCC stats calculations. run stats trace +# Tombstone above point key. put k=a ts=2 v=a2 del k=a ts=4 +# Lone tombstone. del k=b ts=4 +# Lone point key (will be below range tombstone). +put k=c ts=4 v=c4 +# Point keys above each others. put k=d ts=2 v=d2 put k=d ts=4 v=d4 +# Lone point key. +put k=e ts=2 v=e2 +# Lone inline key. put k=i ts=0 v=inline +# Intents at d (existing key) and j (new key). with t=A txn_begin ts=7 put k=d v=d7 put k=j v=j7 +# Range tombstone covering b (tombstone) and c (point key), but not touching +# points and intent at d. +del_range_ts k=b end=d ts=5 +# Another range tombstone across b and c. The GCBytesAge contribution of a live +# point key's prefix (roachpb.Key) begins at the lowest range tombstone, which +# must be corrected for when we write a new point key above it. +del_range_ts k=b end=d ts=6 +# Point keys above range tombstones above existing point tombstone and key. +put k=b ts=7 v=b7 +put k=c ts=7 v=c7 +# Point above tombstones above point (mirroring c with range tombstones). +put k=f ts=2 v=f2 +del k=f ts=5 +del k=f ts=6 +put k=f ts=7 v=f7 ---- >> put k=a ts=2 v=a2 stats: live_bytes:+21 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+7 val_count:+1 @@ -24,26 +48,45 @@ stats: key_bytes:+14 key_count:+1 val_count:+1 data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +>> put k=c ts=4 v=c4 +stats: live_bytes:+21 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+7 val_count:+1 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 >> put k=d ts=2 v=d2 stats: live_bytes:+21 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+7 val_count:+1 data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 data: "d"/2.000000000,0 -> /BYTES/d2 >> put k=d ts=4 v=d4 stats: key_bytes:+12 val_bytes:+7 val_count:+1 data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +>> put k=e ts=2 v=e2 +stats: live_bytes:+21 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+7 val_count:+1 +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 data: "d"/4.000000000,0 -> /BYTES/d4 data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 >> put k=i ts=0 v=inline stats: live_bytes:+27 live_count:+1 key_bytes:+2 key_count:+1 val_bytes:+25 val_count:+1 data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 data: "d"/4.000000000,0 -> /BYTES/d4 data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false >> txn_begin ts=7 t=A stats: @@ -54,10 +97,12 @@ stats: live_bytes:+48 key_bytes:+12 val_bytes:+55 val_count:+1 intent_bytes:+19 data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false >> put k=j v=j7 t=A called PutIntent("j", _, 00000000-0000-0000-0000-000000000001) @@ -65,10 +110,153 @@ stats: live_bytes:+69 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+55 val data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> del_range_ts k=b end=d ts=5 +stats: live_bytes:-21 live_count:-1 range_key_count:+1 range_key_bytes:+13 +rangekey: {b-d}/[5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> del_range_ts k=b end=d ts=6 +stats: gc_bytes_age:-4 range_key_bytes:+9 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> put k=b ts=7 v=b7 +stats: gc_bytes_age:-6 live_bytes:+21 live_count:+1 key_bytes:+12 val_bytes:+7 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> put k=c ts=7 v=c7 +stats: gc_bytes_age:-4 live_bytes:+21 live_count:+1 key_bytes:+12 val_bytes:+7 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> put k=f ts=2 v=f2 +stats: live_bytes:+21 live_count:+1 key_bytes:+14 key_count:+1 val_bytes:+7 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +data: "f"/2.000000000,0 -> /BYTES/f2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> del k=f ts=5 +stats: live_bytes:-21 live_count:-1 key_bytes:+12 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +data: "f"/5.000000000,0 -> / +data: "f"/2.000000000,0 -> /BYTES/f2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> del k=f ts=6 +stats: gc_bytes_age:-2 key_bytes:+12 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/4.000000000,0 -> /BYTES/c4 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +data: "f"/6.000000000,0 -> / +data: "f"/5.000000000,0 -> / +data: "f"/2.000000000,0 -> /BYTES/f2 +meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> /BYTES/j7 +>> put k=f ts=7 v=f7 +stats: gc_bytes_age:-2 live_bytes:+21 live_count:+1 key_bytes:+12 val_bytes:+7 val_count:+1 +rangekey: {b-d}/[6.000000000,0 5.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/4.000000000,0 -> / +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/4.000000000,0 -> /BYTES/c4 meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/4.000000000,0 -> /BYTES/d4 data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/2.000000000,0 -> /BYTES/e2 +data: "f"/7.000000000,0 -> /BYTES/f7 +data: "f"/6.000000000,0 -> / +data: "f"/5.000000000,0 -> / +data: "f"/2.000000000,0 -> /BYTES/f2 meta: "i"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "j"/7.000000000,0 -> /BYTES/j7 @@ -76,16 +264,18 @@ stats: { contains_estimates:0 last_update_nanos:1000000000000 intent_age:1986 - gc_bytes_age:84603 - live_bytes:165 - live_count:3 - key_bytes:94 - key_count:5 - val_bytes:156 - val_count:8 + gc_bytes_age:166166 + live_bytes:249 + live_count:7 + key_bytes:196 + key_count:8 + val_bytes:198 + val_count:16 intent_bytes:38 intent_count:2 separated_intent_count:2 + range_key_count:1 + range_key_bytes:22 sys_bytes:0 sys_count:0 abort_span_bytes:0