diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index bc1f9d96d251..4f04d14ea39e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1506,6 +1506,7 @@ func engineStats(t *testing.T, engine storage.Engine, nowNanos int64) *enginepb. t.Helper() iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, LowerBound: keys.LocalMax, UpperBound: keys.MaxKey, }) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 9e76dc065d04..5d66f6f63741 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -157,8 +157,14 @@ func computeStatsDelta( // If we can't use the fast stats path, or race test is enabled, // compute stats across the key span to be cleared. + // + // TODO(erikgrinaker): This must handle range key stats adjustments. if !fast || util.RaceEnabled { - iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: to}) + iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: from, + UpperBound: to, + }) computed, err := iter.ComputeStats(from, to, delta.LastUpdateNanos) iter.Close() if err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 51310294f327..b07432f9690f 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, cArgs.Now, maxIntents) + + err := storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats, + args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents) return result.Result{}, err } @@ -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 7fe61ebf0e5b..2865eaa9f877 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" @@ -35,6 +36,8 @@ func TestDeleteRangeTombstone(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.DisableMetamorphicSimpleValueEncoding(t) + // Initial data for each test. x is point tombstone, [] is intent, // o---o is range tombstone. // @@ -44,19 +47,26 @@ func TestDeleteRangeTombstone(t *testing.T) { // 2 b2 d2 o-------o // 1 // a b c d e f g h i + // + // We also write two range tombstones abutting the Raft range a-z at [Z-a)@100 + // and [z-|)@100. Writing a range tombstone should not merge with these. writeInitialData := func(t *testing.T, ctx context.Context, rw storage.ReadWriter) { t.Helper() var localTS hlc.ClockTimestamp - txn := roachpb.MakeTransaction("test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5}, 0, 0) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2}, localTS, roachpb.MakeValueFromString("b2"), nil)) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4}, localTS, roachpb.MakeValueFromString("c4"), nil)) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2}, localTS, roachpb.MakeValueFromString("d2"), nil)) - require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3}, localTS, nil)) - require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5}, localTS, roachpb.MakeValueFromString("i5"), &txn)) - require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3}, localTS, 0)) + + txn := roachpb.MakeTransaction("test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5e9}, 0, 0) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("b2"), nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4e9}, localTS, roachpb.MakeValueFromString("c4"), nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("d2"), nil)) + require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, localTS, roachpb.MakeValueFromString("i5"), &txn)) + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, 0)) + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0)) + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0)) } now := hlc.ClockTimestamp{Logical: 9} + rangeStart, rangeEnd := roachpb.Key("a"), roachpb.Key("z") testcases := map[string]struct { start string @@ -68,54 +78,68 @@ func TestDeleteRangeTombstone(t *testing.T) { expectErr interface{} // error type, substring, or true (any) }{ "above points succeed": { - start: "a", - end: "f", - ts: 10, - expectErr: nil, + start: "a", + end: "f", + ts: 10e9, }, "above range tombstone succeed": { start: "f", end: "h", - ts: 10, + ts: 10e9, expectErr: nil, }, + "merging succeeds": { + start: "e", + end: "f", + ts: 3e9, + }, + "adjacent to external LHS range key": { + start: "a", + end: "f", + ts: 100e9, + }, + "adjacent to external RHS range key": { + start: "q", + end: "z", + ts: 100e9, + }, "transaction errors": { start: "a", end: "f", - ts: 10, + ts: 10e9, txn: true, - expectErr: batcheval.ErrTransactionUnsupported, + expectErr: ErrTransactionUnsupported, }, "inline errors": { start: "a", end: "f", - ts: 10, + ts: 10e9, inline: true, expectErr: "Inline can't be used with range tombstones", }, "returnKeys errors": { start: "a", end: "f", - ts: 10, + ts: 10e9, returnKeys: true, expectErr: "ReturnKeys can't be used with range tombstones", }, "intent errors with WriteIntentError": { start: "i", end: "j", - ts: 10, + ts: 10e9, expectErr: &roachpb.WriteIntentError{}, }, "below point errors with WriteTooOldError": { start: "a", end: "d", - ts: 1, + ts: 1e9, expectErr: &roachpb.WriteTooOldError{}, }, "below range tombstone errors with WriteTooOldError": { start: "f", end: "h", - ts: 1, + ts: 1e9, expectErr: &roachpb.WriteTooOldError{}, }, } @@ -134,32 +158,52 @@ func TestDeleteRangeTombstone(t *testing.T) { Timestamp: hlc.Timestamp{WallTime: tc.ts}, } - var txn *roachpb.Transaction + // Prepare the request and environment. + evalCtx := &MockEvalCtx{ + ClusterSettings: st, + Desc: &roachpb.RangeDescriptor{ + StartKey: roachpb.RKey(rangeStart), + EndKey: roachpb.RKey(rangeEnd), + }, + } + + h := roachpb.Header{ + Timestamp: rangeKey.Timestamp, + } if tc.txn { - tx := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0) - txn = &tx + txn := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0) + h.Txn = &txn } + req := &roachpb.DeleteRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: rangeKey.StartKey, + EndKey: rangeKey.EndKey, + }, + UseExperimentalRangeTombstone: true, + Inline: tc.inline, + ReturnKeys: tc.returnKeys, + } + + ms := computeStats(t, engine, rangeStart, rangeEnd, rangeKey.Timestamp.WallTime) + + // Use a spanset batch to assert latching of all accesses. In particular, + // the additional seeks necessary to check for adjacent range keys that we + // may merge with (for stats purposes) which should not cross the range + // bounds. + var latchSpans, lockSpans spanset.SpanSet + declareKeysDeleteRange(evalCtx.Desc, &h, req, &latchSpans, &lockSpans, 0) + batch := spanset.NewBatchAt(engine.NewBatch(), &latchSpans, h.Timestamp) + defer batch.Close() + // Run the request. - var ms enginepb.MVCCStats resp := &roachpb.DeleteRangeResponse{} - _, err := batcheval.DeleteRange(ctx, engine, batcheval.CommandArgs{ - EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext(), + _, err := DeleteRange(ctx, batch, CommandArgs{ + EvalCtx: evalCtx.EvalContext(), Stats: &ms, Now: now, - Header: roachpb.Header{ - Timestamp: rangeKey.Timestamp, - Txn: txn, - }, - Args: &roachpb.DeleteRangeRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: rangeKey.StartKey, - EndKey: rangeKey.EndKey, - }, - UseExperimentalRangeTombstone: true, - Inline: tc.inline, - ReturnKeys: tc.returnKeys, - }, + Header: h, + Args: req, }, resp) // Check the error. @@ -177,6 +221,7 @@ func TestDeleteRangeTombstone(t *testing.T) { return } require.NoError(t, err) + require.NoError(t, batch.Commit(true)) // Check that the range tombstone was written successfully. iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ @@ -215,7 +260,36 @@ func TestDeleteRangeTombstone(t *testing.T) { require.True(t, value.IsTombstone()) require.Equal(t, now, value.LocalTimestamp) - // TODO(erikgrinaker): This should test MVCC stats when implemented. + // Check that range tombstone stats were updated correctly. + require.Equal(t, computeStats(t, engine, rangeStart, rangeEnd, rangeKey.Timestamp.WallTime), ms) }) } } + +// computeStats computes MVCC stats for the given range. +// +// TODO(erikgrinaker): This, storage.computeStats(), and engineStats() should be +// moved into a testutils package, somehow avoiding import cycles with storage +// tests. +func computeStats( + t *testing.T, reader storage.Reader, from, to roachpb.Key, nowNanos int64, +) enginepb.MVCCStats { + t.Helper() + + if len(from) == 0 { + from = keys.LocalMax + } + if len(to) == 0 { + to = keys.MaxKey + } + + iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: from, + UpperBound: to, + }) + defer iter.Close() + ms, err := storage.ComputeStatsForRange(iter, from, to, nowNanos) + require.NoError(t, err) + return ms +} diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 76f975c4b892..61e816d2fc30 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -169,6 +169,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, + }) } } } @@ -896,9 +909,17 @@ 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") + } + 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, @@ -1172,15 +1193,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 { @@ -1226,6 +1259,78 @@ 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 +// +// The same calculation can be used for merges, since Pebble will already have +// merged the range keys into one when appropriate. +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 iter.RangeBounds().Key.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, rkv := range iter.RangeKeys() { + keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) + valBytes := int64(len(rkv.Value)) + if i == 0 { + delta.RangeKeyCount++ + keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(splitKey)) + } + delta.RangeKeyBytes += keyBytes + delta.RangeValCount++ + delta.RangeValBytes += valBytes + delta.GCBytesAge += (keyBytes + valBytes) * (nowNanos/1e9 - rkv.RangeKey.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 132a55b3daeb..fb9554aba031 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -1183,3 +1183,112 @@ func TestCommitWaitBeforeIntentResolutionIfCommitTrigger(t *testing.T) { } }) } + +func TestComputeSplitRangeKeyStatsDelta(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + storage.DisableMetamorphicSimpleValueEncoding(t) + + emptyValue := func() storage.MVCCValue { + return storage.MVCCValue{} + } + + localTSValue := func(ts int) storage.MVCCValue { + var v storage.MVCCValue + v.MVCCValueHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: int64(ts)} + return v + } + + rangeKV := func(start, end string, ts int, value storage.MVCCValue) storage.MVCCRangeKeyValue { + valueRaw, err := storage.EncodeMVCCValue(value) + require.NoError(t, err) + return storage.MVCCRangeKeyValue{ + RangeKey: storage.MVCCRangeKey{ + StartKey: roachpb.Key(start), + EndKey: roachpb.Key(end), + Timestamp: hlc.Timestamp{WallTime: int64(ts)}, + }, + Value: valueRaw, + } + } + + 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 { + rangeKVs []storage.MVCCRangeKeyValue + 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.MVCCRangeKeyValue{rangeKV("a", "z", 1e9, emptyValue())}, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + RangeValCount: 1, + GCBytesAge: 117, + }}, + // a-z with local timestamp splits into a-l and l-z: simple +1 range key with value + "full value": {[]storage.MVCCRangeKeyValue{rangeKV("a", "z", 2e9, localTSValue(1))}, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + RangeValCount: 1, + RangeValBytes: 9, + GCBytesAge: 176, + }}, + // 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.MVCCRangeKeyValue{rangeKV("foo", "zzzz", 1e9, emptyValue())}, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 13, + RangeValCount: 1, + GCBytesAge: 117, + }}, + // Two abutting keys at different timestamps at the split point should not + // require a delta. + "no straddling, timestamp": {[]storage.MVCCRangeKeyValue{ + rangeKV("a", "l", 1e9, emptyValue()), + rangeKV("l", "z", 2e9, emptyValue()), + }, enginepb.MVCCStats{}}, + // Two abutting keys at different local timestamps (values) at the split + // point should not require a delta. + "no straddling, value": {[]storage.MVCCRangeKeyValue{ + rangeKV("a", "l", 2e9, localTSValue(1)), + rangeKV("l", "z", 2e9, localTSValue(2)), + }, enginepb.MVCCStats{}}, + // Multiple straddling keys. + "multiple": { + []storage.MVCCRangeKeyValue{ + rangeKV("a", "z", 2e9, localTSValue(1)), + rangeKV("k", "p", 3e9, localTSValue(2)), + rangeKV("foo", "m", 4e9, emptyValue()), + }, enginepb.MVCCStats{ + RangeKeyCount: 1, + RangeKeyBytes: 31, + RangeValCount: 3, + RangeValBytes: 18, + GCBytesAge: 348, + }}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + engine := storage.NewDefaultInMemForTesting() + defer engine.Close() + + for _, rkv := range tc.rangeKVs { + value, err := storage.DecodeMVCCValue(rkv.Value) + require.NoError(t, err) + require.NoError(t, engine.ExperimentalPutMVCCRangeKey(rkv.RangeKey, value)) + } + + 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 c7b7f1cf83cc..cdc3a06d320e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -117,7 +117,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..b8ab3301585b 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 + DeltaRangeKey // - 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) +// + DeltaRangeKey. // -// 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 +// + DeltaRangeKey. // // 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..89181b99f900 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: 4888917721712214316, }, 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 92158b2becd8..84fb1060b850 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -1308,15 +1308,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")) @@ -1327,30 +1325,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. @@ -1358,43 +1344,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 266b6c2e42c9..f3f2b878feae 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -745,12 +745,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) @@ -768,61 +769,54 @@ 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 = 0, 0 + pointMS.RangeValCount, pointMS.RangeValBytes = 0, 0 + pointMS.GCBytesAge = 0 expMS := enginepb.MVCCStats{ LiveBytes: msLeft.LiveBytes + msRight.LiveBytes, KeyBytes: msLeft.KeyBytes + msRight.KeyBytes, @@ -833,27 +827,35 @@ 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.RangeValCount+msRight.RangeValCount, ms.RangeValCount) + require.GreaterOrEqual(t, msLeft.RangeValBytes+msRight.RangeValBytes, ms.RangeValBytes) + 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 @@ -976,56 +978,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 a300b9dcdc4b..e3060da7efb8 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 @@ -144,38 +143,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 4d5868f21b83..651c944260de 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" ) @@ -485,24 +485,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 8e0d981503ea..09bfa7a587da 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2149,6 +2149,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 9390385c7afa..a6a0063110b4 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -11345,7 +11345,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/roachpb/data.go b/pkg/roachpb/data.go index 14c55ad19d91..1814d1ee63c0 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -50,6 +50,12 @@ const ( localPrefixByte = '\x01' // LocalMaxByte is the end of the local key range. LocalMaxByte = '\x02' + // PrevishKeyLength is a reasonable key length to use for Key.Prevish(), + // typically when peeking to the left of a known key. We want this to be as + // tight as possible, since it can e.g. be used for latch spans. However, the + // exact previous key has infinite length, so we assume that most keys are + // less than 1024 bytes, or have a fairly unique 1024-byte prefix. + PrevishKeyLength = 1024 ) var ( @@ -157,6 +163,21 @@ func (k Key) Next() Key { return Key(encoding.BytesNext(k)) } +// Prevish returns a previous key in lexicographic sort order. It is impossible +// in general to find the exact immediate predecessor key, because it has an +// infinite number of 0xff bytes at the end, so this returns the nearest +// previous key right-padded with 0xff up to length bytes. An infinite number of +// keys may exist between Key and Key.Prevish(), as keys have unbounded length. +// This also implies that k.Prevish().IsPrev(k) will often be false. +// +// PrevishKeyLength can be used as a reasonable length in most situations. +// +// The method may only take a shallow copy of the Key, so both the receiver and +// the return value should be treated as immutable after. +func (k Key) Prevish(length int) Key { + return Key(encoding.BytesPrevish(k, length)) +} + // IsPrev is a more efficient version of k.Next().Equal(m). func (k Key) IsPrev(m Key) bool { l := len(m) - 1 diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 2a3f51f10418..ab3a960d7f9f 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -12,6 +12,7 @@ package roachpb import ( "bytes" + "encoding/hex" "math" "math/rand" "reflect" @@ -221,6 +222,27 @@ func TestNextKey(t *testing.T) { } } +func TestPrevish(t *testing.T) { + const length = 4 + testcases := []struct { + key Key + expect Key + }{ + {nil, nil}, + {[]byte{}, []byte{}}, + {[]byte{0x00}, []byte{}}, + {[]byte{0x01, 0x00}, []byte{0x01}}, + {[]byte{0x01}, []byte{0x00, 0xff, 0xff, 0xff}}, + {[]byte{0x01, 0x01}, []byte{0x01, 0x00, 0xff, 0xff}}, + {[]byte{0xff, 0xff, 0xff, 0xff}, []byte{0xff, 0xff, 0xff, 0xfe}}, + } + for _, tc := range testcases { + t.Run(hex.EncodeToString(tc.key), func(t *testing.T) { + require.Equal(t, tc.expect, tc.key.Prevish(length)) + }) + } +} + func TestIsPrev(t *testing.T) { for i, tc := range []struct { k, m Key diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant b/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant index a35c18a41267..479fbe2371ee 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function_notenant @@ -19,7 +19,9 @@ 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: RangeValCount: RangeValBytes: 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 5fce60a01a30..85831e80cc84 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -148,7 +148,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 99f051072eb1..0f060ca619ba 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -837,7 +837,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() } @@ -1328,7 +1332,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 b7e7cdc3e1ad..392e310da380 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -252,11 +252,20 @@ 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. + // + // TODO(erikgrinaker): This should be replaced by ComputeStatsForRange + // instead, which should set up its own iterator with appropriate options. + // This isn't currently done in order to do spanset assertions on it, but this + // could be better solved by checking the iterator bounds in NewMVCCIterator + // and requiring callers to set them appropriately. 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 c22ca362c951..96aad88a2a07 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 + ms.RangeValBytes } // 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,10 @@ 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.RangeValCount += oms.RangeValCount + ms.RangeValBytes += oms.RangeValBytes ms.SysBytes += oms.SysBytes ms.SysCount += oms.SysCount ms.AbortSpanBytes += oms.AbortSpanBytes @@ -196,6 +202,10 @@ 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.RangeValCount -= oms.RangeValCount + ms.RangeValBytes -= oms.RangeValBytes 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 93bda348d0af..2d9cc1fbc4d1 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -163,9 +163,9 @@ 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). - // See the comment on MVCCStats. + // 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, and range_val_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 // principle be read by means of a Scan or Get in the far future, including @@ -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,31 @@ 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 may fragment into version stacks with the same + // start/end bounds, thus writing a single range key may cause range_key_count + // to increase by more than 1 due to fragmentation. Multiple range key + // versions with the same bounds count as a single range key, but are + // tracked under range_val_count. + // + // 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. Therefore, these + // do not contribute to live_count nor live_bytes. + optional sfixed64 range_key_count = 17 [(gogoproto.nullable) = false]; + // range_key_bytes is the encoded size of range keys. Only the latest + // version in a range key fragment stack contributes the key bounds, + // but each version contributes the version timestamp. + optional sfixed64 range_key_bytes = 18 [(gogoproto.nullable) = false]; + // range_val_count is the number of range key values tracked under + // range_val_bytes, i.e. the number of range key versions. + optional sfixed64 range_val_count = 19 [(gogoproto.nullable) = false]; + // range_val_bytes is the number of bytes stored in range keys. Even tough + // all range keys are currently MVCC range tombstones with no value, the + // MVCCValueHeader contribution can be non-zero. + optional sfixed64 range_val_bytes = 20 [(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 c10364435bc4..ec05dad30d65 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -191,6 +191,10 @@ 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 range_val_count = 19; + sint64 range_val_bytes = 20; sint64 sys_bytes = 12; sint64 sys_count = 13; sint64 abort_span_bytes = 15; @@ -220,6 +224,10 @@ 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 range_val_count = 19; + int64 range_val_bytes = 20; 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 ffa61a1b362a..deeeac6facec 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -167,7 +167,7 @@ func isAbortSpanKey(key roachpb.Key) bool { // updateStatsForInline updates stat counters for an inline value // (abort span entries for example). These are simpler as they don't -// involve intents or multiple versions. +// involve intents, multiple versions, or MVCC range tombstones. func updateStatsForInline( ms *enginepb.MVCCStats, key roachpb.Key, @@ -875,6 +875,7 @@ func MVCCGetAsTxn( // mvccGetMetadata returns or reconstructs the meta key for the given key. // A prefix scan using the iterator is performed, resulting in one of the // following successful outcomes: +// // 1) iterator finds nothing; returns (false, 0, 0, nil). // 2) iterator finds an explicit meta key; unmarshals and returns its size. // ok is set to true. @@ -884,37 +885,47 @@ func MVCCGetAsTxn( // that is the usual contribution of the meta key). The value size returned // will be zero, as there is no stored MVCCMetadata. // ok is set to true. -// The passed in MVCCMetadata must not be nil. Any MVCC range tombstones will be -// treated like point tombstones. +// 4) iterator finds an MVCC range tombstone above a value. In this case, +// metadata for a synthetic point tombstone is returned. +// +// The timestamp where the real point key last changed is also returned, if a +// real point key was found. This may differ from the metadata timestamp when a +// point key is covered by multiple MVCC range tombstones (in which case the +// point key disappeared at the _lowest_ range tombstone above it), or when a +// point tombstone is covered by a range tombstone (in which case the point key +// disappeared at the point tombstone). It is needed to correctly account for +// the GCBytesAge contribution of the key prefix, which is not affected by MVCC +// range tombstones, and would be incorrect if we used the synthetic point +// tombstone of the newest MVCC range tombstone instead. // -// If the supplied iterator is nil, no seek operation is performed. This is -// used by the Blind{Put,ConditionalPut} operations to avoid seeking when the -// metadata is known not to exist. +// The passed in MVCCMetadata must not be nil. If the supplied iterator is nil, +// no seek operation is performed. This is used by the Blind{Put,ConditionalPut} +// operations to avoid seeking when the metadata is known not to exist. func mvccGetMetadata( iter MVCCIterator, metaKey MVCCKey, meta *enginepb.MVCCMetadata, -) (ok bool, keyBytes, valBytes int64, err error) { +) (ok bool, keyBytes, valBytes int64, realKeyChanged hlc.Timestamp, err error) { if iter == nil { - return false, 0, 0, nil + return false, 0, 0, hlc.Timestamp{}, nil } iter.SeekGE(metaKey) if ok, err = iter.Valid(); !ok { - return false, 0, 0, err + return false, 0, 0, hlc.Timestamp{}, err } unsafeKey := iter.UnsafeKey() if !unsafeKey.Key.Equal(metaKey.Key) { - return false, 0, 0, nil + return false, 0, 0, hlc.Timestamp{}, nil } - hasPoint, hasRange := iter.HasPointAndRange() // Check for existing intent metadata. Intents will be emitted colocated with - // a covering range key when seeking to it, so we don't need to handle range - // keys here. + // a covering range key when seeking to it, and always located above range + // keys, so we don't need to check for range keys here. if hasPoint && !unsafeKey.IsValue() { if err := iter.ValueProto(meta); err != nil { - return false, 0, 0, err + return false, 0, 0, hlc.Timestamp{}, err } - return true, int64(unsafeKey.EncodedSize()), int64(len(iter.UnsafeValue())), nil + return true, int64(unsafeKey.EncodedSize()), int64(len(iter.UnsafeValue())), + meta.Timestamp.ToTimestamp(), nil } // Synthesize point key metadata. For values, the size of keys is always @@ -929,46 +940,56 @@ func mvccGetMetadata( iter.Next() if ok, err = iter.Valid(); err != nil { - return false, 0, 0, err + return false, 0, 0, hlc.Timestamp{}, err } else if ok { // NB: For !ok, hasPoint is already false. hasPoint, hasRange = iter.HasPointAndRange() unsafeKey = iter.UnsafeKey() } // If only a bare range tombstone was found at the seek key, synthesize - // point tombstone metadata for it. + // point tombstone metadata for it. realKeyChanged is empty since there + // was no real point key here. if !hasPoint || !unsafeKey.Key.Equal(metaKey.Key) { meta.Deleted = true meta.Timestamp = rkTimestamp.ToLegacyTimestamp() - return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil - } - } - - // We're now on a point key. Check if it's covered by an MVCC range tombstone, - // and synthesize point tombstone metadata for it in that case. - if hasRange { - if rkTS := iter.RangeKeys()[0].RangeKey.Timestamp; unsafeKey.Timestamp.LessEq(rkTS) { - meta.Deleted = true - meta.Timestamp = rkTS.ToLegacyTimestamp() - return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil + return true, int64(EncodedMVCCKeyPrefixLength(metaKey.Key)), 0, hlc.Timestamp{}, nil } } - // Synthesize metadata for a regular point key. + // We're now on a point key. Decode its value. var unsafeVal MVCCValue unsafeValRaw := iter.UnsafeValue() if unsafeVal, ok, err = tryDecodeSimpleMVCCValue(unsafeValRaw); !ok && err == nil { unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) } if err != nil { - return false, 0, 0, err + return false, 0, 0, hlc.Timestamp{}, err + } + + // Check if the point key is covered by an MVCC range tombstone, and + // synthesize point tombstone metadata for it in that case. realKeyChanged is + // set to the timestamp where the point key ceased to exist -- either the + // lowest range tombstone above the key (not the highest which is used for + // metadata), or the point version's timestamp if it was a tombstone. + if hasRange { + rangeKeys := iter.RangeKeys() + if rkv, ok := firstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + meta.Deleted = true + meta.Timestamp = rangeKeys[0].RangeKey.Timestamp.ToLegacyTimestamp() + keyLastSeen := rkv.RangeKey.Timestamp + if unsafeVal.IsTombstone() { + keyLastSeen = unsafeKey.Timestamp + } + return true, int64(EncodedMVCCKeyPrefixLength(metaKey.Key)), 0, keyLastSeen, nil + } } + // Synthesize metadata for a regular point key. meta.ValBytes = int64(len(unsafeValRaw)) meta.Deleted = unsafeVal.IsTombstone() meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() - return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil + return true, int64(EncodedMVCCKeyPrefixLength(metaKey.Key)), 0, unsafeKey.Timestamp, nil } // putBuffer holds pointer data needed by mvccPutInternal. Bundling @@ -1346,6 +1367,9 @@ func replayTransactionalWrite( // and vice versa. valueFn can delete by returning nil. Returning // []byte{} will write an empty value, not delete. // +// The given iter must surface range keys to correctly account for +// MVCC range tombstones in MVCC stats. +// // Note that, when writing transactionally, the txn's timestamps // dictate the timestamp of the operation, and the timestamp parameter // is redundant. Specifically, the intent is written at the txn's @@ -1395,7 +1419,8 @@ func mvccPutInternal( } metaKey := MakeMVCCMetadataKey(key) - ok, origMetaKeySize, origMetaValSize, err := mvccGetMetadata(iter, metaKey, &buf.meta) + ok, origMetaKeySize, origMetaValSize, origRealKeyChanged, err := + mvccGetMetadata(iter, metaKey, &buf.meta) if err != nil { return err } @@ -1406,8 +1431,9 @@ func mvccPutInternal( return errors.Errorf("%q: put is inline=%t, but existing value is inline=%t", metaKey, putIsInline, buf.meta.IsInline()) } - // Handle inline put. No IntentHistory is required for inline writes - // as they aren't allowed within transactions. + // Handle inline put. No IntentHistory is required for inline writes as they + // aren't allowed within transactions. MVCC range tombstones cannot exist + // across them either. if putIsInline { if txn != nil { return errors.Errorf("%q: inline writes not allowed within transactions", metaKey) @@ -1467,6 +1493,8 @@ func mvccPutInternal( meta = &buf.meta metaTimestamp := meta.Timestamp.ToTimestamp() + // Handle intents. MVCC range tombstones should not require any special + // handling, since they cannot be transactional. if meta.Txn != nil { // There is an uncommitted write intent. if txn == nil || meta.Txn.ID != txn.ID { @@ -1584,15 +1612,17 @@ func mvccPutInternal( // read its size because its GCBytesAge contribution may change as we // move the intent above it. A similar phenomenon occurs in // MVCCResolveWriteIntent. + // + // TODO(erikgrinaker): Consider using mvccGet() here instead, but + // needs benchmarking. prevKey := oldVersionKey.Next() iter.SeekGE(prevKey) valid, err := iter.Valid() if err != nil { return err } else if valid { - // TODO(erikgrinaker): We don't handle MVCC range tombstones in MVCC - // stats yet, so if we land on a bare range key just step onto the - // next point key (if any). + // If we land on a bare range key, step onto the next key. This may + // be a point key at the same key position, or a different key. if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint { iter.Next() if valid, err = iter.Valid(); err != nil { @@ -1606,13 +1636,20 @@ func mvccPutInternal( return errors.Errorf("expected an MVCC value key: %s", prevUnsafeKey) } - prevValRaw := iter.UnsafeValue() - prevVal, err := DecodeMVCCValue(prevValRaw) - if err != nil { - return err + // We must now be on a point key, but it may be covered by an + // existing MVCC range tombstone. If it isn't, account for it. + _, hasRange := iter.HasPointAndRange() + if !hasRange || iter.RangeKeys()[0].RangeKey.Timestamp.Less(prevUnsafeKey.Timestamp) { + prevValRaw := iter.UnsafeValue() + prevVal, err := DecodeMVCCValue(prevValRaw) + if err != nil { + return err + } + if prevVal.Value.IsPresent() { + prevIsValue = prevVal.Value.IsPresent() + prevValSize = int64(len(prevValRaw)) + } } - prevIsValue = prevVal.Value.IsPresent() - prevValSize = int64(len(prevValRaw)) } iter = nil // prevent accidental use below } @@ -1778,6 +1815,21 @@ func mvccPutInternal( // Update MVCC stats. if ms != nil { + // Adjust the stats metadata for MVCC range tombstones. The MVCC stats + // update only cares about changes to real point keys, but the above logic + // needs to care about MVCC range tombstones for conflict purposes. + // + // Specifically, if a real point key was covered by a range tombstone, we + // must set meta.Timestamp to the timestamp where the real point key was + // deleted (either by a point tombstone or the lowest range tombstone). If + // there was no real point key, meta must be nil. In all other cases, + // meta.Timestamp will already equal origRealKeyChanged. + if origRealKeyChanged.IsEmpty() { + meta = nil // no real point key was found + } + if meta != nil { + meta.Timestamp = origRealKeyChanged.ToLegacyTimestamp() + } ms.Add(updateStatsOnPut(key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, newMeta)) } @@ -2123,6 +2175,8 @@ func MVCCMerge( // // If the underlying iterator encounters an intent with a timestamp in the span // (startTime, endTime], or any inline meta, this method will return an error. +// +// TODO(erikgrinaker): This needs to handle MVCC range tombstones (stats too). func MVCCClearTimeRange( _ context.Context, rw ReadWriter, @@ -2392,11 +2446,14 @@ func MVCCDeleteRange( // 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 MVCC stats handling. func ExperimentalMVCCDeleteRangeUsingTombstone( ctx context.Context, rw ReadWriter, @@ -2404,6 +2461,7 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( startKey, endKey roachpb.Key, timestamp hlc.Timestamp, localTimestamp hlc.ClockTimestamp, + leftPeekBound, rightPeekBound roachpb.Key, maxIntents int64, ) error { // Validate the range key. We must do this first, to catch e.g. any bound violations. @@ -2412,6 +2470,17 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( return err } + // Encode the value. + var value MVCCValue + value.LocalTimestamp = localTimestamp + if !value.LocalTimestampNeeded(timestamp) || !rw.ShouldWriteLocalTimestamps(ctx) { + value.LocalTimestamp = hlc.ClockTimestamp{} + } + valueRaw, err := EncodeMVCCValue(value) + if err != nil { + return err + } + // Check for any overlapping intents, and return them to be resolved. if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { return err @@ -2419,22 +2488,31 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( return &roachpb.WriteIntentError{Intents: intents} } - // Check for any conflicts, i.e. newer values. We use a time-bounded scan, and - // ignore intents since we checked for those above already. + // 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 will probably need optimization, particularly when - // we're deleting an entire CRDB range. + // TODO(erikgrinaker): This introduces an O(n) read penalty. We should + // 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. + // However, in that case we'll still have to do a time-bounded scan to check + // for conflicts. iter := rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, - LowerBound: startKey, - UpperBound: endKey, - MinTimestampHint: timestamp, - MaxTimestampHint: hlc.MaxTimestamp, + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: startKey, + UpperBound: endKey, + RangeKeyMaskingBelow: timestamp, // lower point keys have already been accounted for }) defer iter.Close() - var prevRangeStart roachpb.Key - for iter.SeekGE(MVCCKey{Key: startKey}); ; iter.NextKey() { + iter.SeekGE(MVCCKey{Key: startKey}) + prevRangeEnd := startKey.Clone() + for { if ok, err := iter.Valid(); err != nil { return err } else if !ok { @@ -2442,7 +2520,9 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( } hasPoint, hasRange := iter.HasPointAndRange() + if hasPoint { + // Check for conflict with newer point key. key := iter.UnsafeKey() if timestamp.LessEq(key.Timestamp) { return roachpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key.Clone()) @@ -2450,24 +2530,212 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( 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. + vRaw := iter.UnsafeValue() + v, ok, err := tryDecodeSimpleMVCCValue(vRaw) + if !ok && err == nil { + v, err = decodeExtendedMVCCValue(vRaw) + } + if err != nil { + return err + } + + if ms != nil && !v.IsTombstone() { + ms.LiveCount-- + ms.LiveBytes -= int64(key.EncodedSize()) + int64(len(vRaw)) + } } + if hasRange { - if rangeStart := iter.RangeBounds().Key; !rangeStart.Equal(prevRangeStart) { - prevRangeStart = append(prevRangeStart[:0], rangeStart...) - if newest := iter.RangeKeys()[0].RangeKey; timestamp.LessEq(newest.Timestamp) { - return roachpb.NewWriteTooOldError(timestamp, newest.Timestamp.Next(), newest.StartKey) + // Check if we've encountered a new range key stack. + if rangeBounds := iter.RangeBounds(); !rangeBounds.EndKey.Equal(prevRangeEnd) { + newest := iter.RangeKeys()[0].RangeKey + + // Check for conflict with newer range key. + if timestamp.LessEq(newest.Timestamp) { + return roachpb.NewWriteTooOldError( + timestamp, newest.Timestamp.Next(), newest.StartKey.Clone()) + } + + 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 !rangeBounds.Key.Equal(prevRangeEnd) { + ms.RangeKeyCount++ + ms.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp) + + EncodedMVCCKeyPrefixLength(prevRangeEnd) + + EncodedMVCCKeyPrefixLength(rangeBounds.Key)) + ms.RangeValCount++ + ms.RangeValBytes += int64(len(valueRaw)) + } + // 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. It has no + // GCBytesAge contribution of its own because it's written at now. + ms.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp)) + ms.RangeValCount++ + ms.RangeValBytes += int64(len(valueRaw)) + ms.GCBytesAge -= (timestamp.WallTime/1e9 - newest.Timestamp.WallTime/1e9) * + int64(EncodedMVCCKeyPrefixLength(rangeBounds.Key)+ + EncodedMVCCKeyPrefixLength(rangeBounds.EndKey)) } + + prevRangeEnd = append(prevRangeEnd[:0], rangeBounds.EndKey...) } } + + // If we hit a bare range key, it's possible that there's a point key on the + // same key as its start key. NextKey() would skip over this, so we take a + // normal step to look for it. + if hasRange && !hasPoint { + iter.Next() + } else { + iter.NextKey() + } } - // Write the tombstone. - var value MVCCValue - value.LocalTimestamp = localTimestamp - if !value.LocalTimestampNeeded(timestamp) || !rw.ShouldWriteLocalTimestamps(ctx) { - value.LocalTimestamp = hlc.ClockTimestamp{} + // 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)) + ms.RangeValCount++ + ms.RangeValBytes += int64(len(valueRaw)) } + // Check if the range key will merge with or fragment any existing range keys + // at the bounds, and adjust stats accordingly. + // + // TODO(erikgrinaker): This code is a bit ugly. Also, similar logic will be + // needed elsewhere, e.g. in AddSSTable, ClearRange, RevertRange, MVCC garbage + // collection, CRDB range splits/merges, etc. This should eventually be + // cleaned up and consolidated, but it'll do for now. + // + // 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 []MVCCRangeKeyValue, splitKey roachpb.Key) { + for i, rkv := range rangeKeys { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) + valBytes := int64(len(rkv.Value)) + if i == 0 { + ms.RangeKeyCount++ + keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(splitKey)) + } + ms.RangeKeyBytes += keyBytes + ms.RangeValCount++ + ms.RangeValBytes += valBytes + ms.GCBytesAge += (keyBytes + valBytes) * (timestamp.WallTime/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + } + } + + // maybeMergeRangeKeys adjusts ms to merge two abutting range key stacks if + // they have the same timestamps and values. It assumes the lhs end key + // equals the rhs start key, and that they are in descending order. + maybeMergeRangeKeys := func(lhs, rhs []MVCCRangeKeyValue) { + if len(lhs) != len(rhs) || len(lhs) == 0 { + return + } + for i, l := range lhs { + if !l.RangeKey.Timestamp.Equal(rhs[i].RangeKey.Timestamp) { + return + } else if !bytes.Equal(l.Value, rhs[i].Value) { + return + } + } + mergeKey := rhs[0].RangeKey.StartKey + for i, rkv := range lhs { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) + valBytes := int64(len(rkv.Value)) + if i == 0 { + ms.RangeKeyCount-- + keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(mergeKey)) + } + ms.RangeKeyBytes -= keyBytes + ms.RangeValCount-- + ms.RangeValBytes -= valBytes + ms.GCBytesAge -= (keyBytes + valBytes) * + (timestamp.WallTime/1e9 - rkv.RangeKey.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 { + switch iter.RangeBounds().EndKey.Compare(startKey) { + case 1: // fragment + fragmentRangeKeys(iter.RangeKeys(), startKey) + case 0: // merge + lhs := iter.RangeKeys() + for i := range lhs { + lhs[i] = lhs[i].Clone() + } + rhs := []MVCCRangeKeyValue{{RangeKey: rangeKey, Value: valueRaw}} + 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 { + switch iter.RangeBounds().Key.Compare(endKey) { + case -1: // fragment + fragmentRangeKeys(iter.RangeKeys(), endKey) + case 0: // merge + lhs := []MVCCRangeKeyValue{{RangeKey: rangeKey, Value: valueRaw}} + rhs := iter.RangeKeys() + for i := range rhs { + rhs[i] = rhs[i].Clone() + } + 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, value) } @@ -2913,7 +3181,10 @@ func MVCCResolveWriteIntent( return false, errors.Errorf("can't resolve range intent as point intent") } - iterAndBuf := GetBufUsingIter(rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true})) + iterAndBuf := GetBufUsingIter(rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + })) iterAndBuf.iter.SeekIntentGE(intent.Key, intent.Txn.ID) ok, err := mvccResolveWriteIntent(ctx, rw, iterAndBuf.iter, ms, intent, iterAndBuf.buf) // Using defer would be more convenient, but it is measurably slower. @@ -2932,11 +3203,13 @@ func MVCCResolveWriteIntent( // to SeekGE. type iterForKeyVersions interface { Valid() (bool, error) + HasPointAndRange() (bool, bool) SeekGE(key MVCCKey) Next() UnsafeKey() MVCCKey UnsafeValue() []byte ValueProto(msg protoutil.Message) error + RangeKeys() []MVCCRangeKeyValue } // separatedIntentAndVersionIter is an implementation of iterForKeyVersions @@ -3004,6 +3277,18 @@ func (s *separatedIntentAndVersionIter) Valid() (bool, error) { return s.engineIterValid, s.engineIterErr } +func (s *separatedIntentAndVersionIter) HasPointAndRange() (bool, bool) { + hasPoint, hasRange := s.mvccIter.HasPointAndRange() + if !s.atMVCCIter { + hasPoint = s.engineIterValid + } + return hasPoint, hasRange +} + +func (s *separatedIntentAndVersionIter) RangeKeys() []MVCCRangeKeyValue { + return s.mvccIter.RangeKeys() +} + func (s *separatedIntentAndVersionIter) SeekGE(key MVCCKey) { if !key.IsValue() { panic(errors.AssertionFailedf("SeekGE only permitted for values")) @@ -3057,6 +3342,9 @@ func mvccGetIntent( if ok, err := iter.Valid(); !ok { return false, 0, 0, err } + if hasPoint, _ := iter.HasPointAndRange(); !hasPoint { + return false, 0, 0, nil + } unsafeKey := iter.UnsafeKey() if !unsafeKey.Key.Equal(metaKey.Key) { return false, 0, 0, nil @@ -3167,6 +3455,7 @@ func (h singleDelOptimizationHelper) onAbortIntent() bool { // mvccResolveWriteIntent is the core logic for resolving an intent. // REQUIRES: iter is already seeked to intent.Key. +// REQUIRES: iter surfaces range keys via IterKeyTypePointsAndRanges. // Returns whether an intent was found and resolved, false otherwise. func mvccResolveWriteIntent( ctx context.Context, @@ -3330,9 +3619,20 @@ func mvccResolveWriteIntent( // Rewrite the versioned value at the new timestamp. iter.SeekGE(oldKey) - if valid, err := iter.Valid(); err != nil { + valid, err := iter.Valid() + if err != nil { return false, err - } else if !valid || !iter.UnsafeKey().Equal(oldKey) { + } + if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint { + // If the seek lands on a bare range key, attempt to step to a point. + iter.Next() + if valid, err = iter.Valid(); err != nil { + return false, err + } else if valid { + valid, _ = iter.HasPointAndRange() + } + } + if !valid || !iter.UnsafeKey().Equal(oldKey) { return false, errors.Errorf("existing intent value missing: %s", oldKey) } oldValue, err := DecodeMVCCValue(iter.UnsafeValue()) @@ -3381,21 +3681,27 @@ func mvccResolveWriteIntent( // have to read that version's size. // // Look for the first real versioned key, i.e. the key just below - // the (old) meta's timestamp. + // the (old) meta's timestamp, and for any MVCC range tombstones. iter.Next() if valid, err := iter.Valid(); err != nil { return false, err - } else if valid && iter.UnsafeKey().Key.Equal(oldKey.Key) { - unsafeValRaw := iter.UnsafeValue() - prevVal, prevValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) - if !prevValOK && err == nil { - prevVal, err = decodeExtendedMVCCValue(unsafeValRaw) - } - if err != nil { - return false, err + } else if valid { + if hasPoint, hasRange := iter.HasPointAndRange(); hasPoint { + if unsafeKey := iter.UnsafeKey(); unsafeKey.Key.Equal(oldKey.Key) { + if !hasRange || iter.RangeKeys()[0].RangeKey.Timestamp.Less(unsafeKey.Timestamp) { + unsafeValRaw := iter.UnsafeValue() + prevVal, prevValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !prevValOK && err == nil { + prevVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return false, err + } + prevIsValue = prevVal.Value.IsPresent() + prevValSize = int64(len(iter.UnsafeValue())) + } + } } - prevIsValue = prevVal.Value.IsPresent() - prevValSize = int64(len(iter.UnsafeValue())) } } @@ -3460,20 +3766,27 @@ func mvccResolveWriteIntent( Key: intent.Key, }) - nextKey := latestKey.Next() ok = false var unsafeNextKey MVCCKey var unsafeNextValueRaw []byte - if nextKey.IsValue() { + if nextKey := latestKey.Next(); nextKey.IsValue() { // The latestKey was not the smallest possible timestamp {WallTime: 0, // Logical: 1}. Practically, this is the only case that will occur in // production. iter.SeekGE(nextKey) - ok, err = iter.Valid() - if err != nil { + if ok, err = iter.Valid(); err != nil { return false, err } - if ok && iter.UnsafeKey().Key.Equal(latestKey.Key) { + // If the seek lands on a bare range key, attempt to step to a point. + if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint { + iter.Next() + if ok, err = iter.Valid(); err != nil { + return false, err + } else if ok { + ok, _ = iter.HasPointAndRange() + } + } + if ok = ok && iter.UnsafeKey().Key.Equal(latestKey.Key); ok { unsafeNextKey = iter.UnsafeKey() if !unsafeNextKey.IsValue() { // Should never see an intent for this key since we seeked to a @@ -3481,8 +3794,15 @@ func mvccResolveWriteIntent( return false, errors.Errorf("expected an MVCC value key: %s", unsafeNextKey) } unsafeNextValueRaw = iter.UnsafeValue() - } else { - ok = false + // If a non-tombstone point key is covered by a range tombstone, then + // synthesize a point tombstone at the lowest range tombstone covering it. + // This is where the point key ceases to exist, contributing to GCBytesAge. + if len(unsafeNextValueRaw) > 0 { + if rk, found := firstRangeKeyAbove(iter.RangeKeys(), unsafeNextKey.Timestamp); found { + unsafeNextKey.Timestamp = rk.RangeKey.Timestamp + unsafeNextValueRaw = []byte{} + } + } } iter = nil // prevent accidental use below } @@ -3620,7 +3940,11 @@ func MVCCResolveWriteIntentRange( ltEnd, _ := keys.LockTableSingleKey(intent.EndKey, nil) engineIter := rw.NewEngineIterator(IterOptions{LowerBound: ltStart, UpperBound: ltEnd}) var mvccIter MVCCIterator - iterOpts := IterOptions{UpperBound: intent.EndKey} + iterOpts := IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: intent.Key, + UpperBound: intent.EndKey, + } if rw.ConsistentIterators() { // Production code should always have consistent iterators. mvccIter = rw.NewMVCCIterator(MVCCKeyIterKind, iterOpts) @@ -3704,6 +4028,8 @@ func MVCCResolveWriteIntentRange( // not a mix of the two. This is to accommodate the implementation below // that creates an iterator with bounds that span from the first to last // key (in sorted order). +// +// TODO(erikgrinaker): This must handle MVCC range tombstones. func MVCCGarbageCollect( ctx context.Context, rw ReadWriter, @@ -3744,7 +4070,7 @@ func MVCCGarbageCollect( meta := &enginepb.MVCCMetadata{} for _, gcKey := range keys { encKey := MakeMVCCMetadataKey(gcKey.Key) - ok, metaKeySize, metaValSize, err := mvccGetMetadata(iter, encKey, meta) + ok, metaKeySize, metaValSize, _, err := mvccGetMetadata(iter, encKey, meta) if err != nil { return err } @@ -4025,28 +4351,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(MVCCRangeKeyValue) 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 @@ -4056,22 +4397,63 @@ func ComputeStatsForRange( // of the point in time at which the current key begins to age. var accrueGCAgeNanos int64 mvccEndKey := MakeMVCCMetadataKey(end) + rangeKeys := []MVCCRangeKeyValue{} - 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().Key; !rangeStart.Equal(prevRangeStart) { + prevRangeStart = append(prevRangeStart[:0], rangeStart...) + rangeKeys = iter.RangeKeys() + + for i, rkv := range rangeKeys { + // Only the top-most fragment contributes the key and its bounds, but + // all versions contribute timestamps and values. + // + // 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. + keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) + valBytes := int64(len(rkv.Value)) + if i == 0 { + ms.RangeKeyCount++ + keyBytes += int64(EncodedMVCCKeyPrefixLength(rkv.RangeKey.StartKey) + + EncodedMVCCKeyPrefixLength(rkv.RangeKey.EndKey)) + } + ms.RangeKeyBytes += keyBytes + ms.RangeValCount++ + ms.RangeValBytes += valBytes + ms.GCBytesAge += (keyBytes + valBytes) * + (nowNanos/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + + if rangeKeyVisitor != nil { + if err := rangeKeyVisitor(rkv); 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 } } @@ -4098,6 +4480,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 rkv, ok := firstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + nextRangeTombstone = rkv.RangeKey.Timestamp + } + } + if implicitMeta { // No MVCCMetadata entry for this series of keys. var isTombstone bool @@ -4141,12 +4533,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 @@ -4166,11 +4562,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 @@ -4204,6 +4604,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 1847ca64cf46..c81642aab599 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -64,6 +64,7 @@ var sstIterVerify = util.ConstantWithMetamorphicTestBool("mvcc-histories-sst-ite // txn_ignore_seqs t= seqs=[-[,-...]] // // resolve_intent t= k= [status=] [clockWhilePending=[,]] +// resolve_intent_range t= k= end= [status=] // check_intent k= [none] // // cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] @@ -614,9 +615,9 @@ var commands = map[string]cmd{ "txn_step": {typTxnUpdate, cmdTxnStep}, "txn_update": {typTxnUpdate, cmdTxnUpdate}, - "resolve_intent": {typDataUpdate, cmdResolveIntent}, - // TODO(nvanbenschoten): test "resolve_intent_range". - "check_intent": {typReadOnly, cmdCheckIntent}, + "resolve_intent": {typDataUpdate, cmdResolveIntent}, + "resolve_intent_range": {typDataUpdate, cmdResolveIntentRange}, + "check_intent": {typReadOnly, cmdCheckIntent}, "clear": {typDataUpdate, cmdClear}, "clear_range": {typDataUpdate, cmdClearRange}, @@ -794,6 +795,17 @@ func cmdResolveIntent(e *evalCtx) error { return e.resolveIntent(e.tryWrapForIntentPrinting(e.engine), key, txn, status, clockWhilePending) } +func cmdResolveIntentRange(e *evalCtx) error { + txn := e.getTxn(mandatory) + start, end := e.getKeyRange() + status := e.getTxnStatus() + + intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: start, EndKey: end}) + intent.Status = status + _, _, err := MVCCResolveWriteIntentRange(e.ctx, e.tryWrapForIntentPrinting(e.engine), e.ms, intent, 0) + return err +} + func (e *evalCtx) resolveIntent( rw ReadWriter, key roachpb.Key, @@ -962,7 +974,7 @@ func cmdDeleteRangeTombstone(e *evalCtx) error { localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) return e.withWriter("del_range_ts", func(rw ReadWriter) error { - return ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, nil, key, endKey, ts, localTs, 0) + return ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, e.ms, key, endKey, ts, localTs, nil, nil, 0) }) } @@ -1525,6 +1537,7 @@ func formatStats(ms enginepb.MVCCStats, delta bool) string { // TODO(erikgrinaker): Consider just reordering the MVCCStats struct fields // instead, which determines the order of MVCCStats.String(). order := []string{"key_count", "key_bytes", "val_count", "val_bytes", + "range_key_count", "range_key_bytes", "range_val_count", "range_val_bytes", "live_count", "live_bytes", "gc_bytes_age", "intent_count", "intent_bytes", "separated_intent_count", "intent_age"} sort.SliceStable(fields, func(i, j int) bool { diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index d8a8666c536d..0d8903492c1e 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -1348,7 +1348,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 f128b69c77cd..8bf9b39abb9f 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" @@ -263,8 +264,8 @@ func encodeMVCCTimestampToBuf(buf []byte, ts hlc.Timestamp) { // encodedMVCCKeyLength returns the encoded length of the given MVCCKey. func encodedMVCCKeyLength(key MVCCKey) int { - // NB: We don't call into encodedMVCCKeyPrefixLength() or - // encodedMVCCTimestampSuffixLength() here because the additional function + // NB: We don't call into EncodedMVCCKeyPrefixLength() or + // EncodedMVCCTimestampSuffixLength() here because the additional function // call overhead is significant. keyLen := len(key.Key) + mvccEncodedTimeSentinelLen if !key.Timestamp.IsEmpty() { @@ -280,9 +281,9 @@ func encodedMVCCKeyLength(key MVCCKey) int { return keyLen } -// encodedMVCCKeyPrefixLength returns the encoded length of a roachpb.Key prefix +// EncodedMVCCKeyPrefixLength returns the encoded length of a roachpb.Key prefix // including the sentinel byte. -func encodedMVCCKeyPrefixLength(key roachpb.Key) int { +func EncodedMVCCKeyPrefixLength(key roachpb.Key) int { return len(key) + mvccEncodedTimeSentinelLen } @@ -300,10 +301,10 @@ func encodedMVCCTimestampLength(ts hlc.Timestamp) int { return tsLen } -// encodedMVCCTimestampSuffixLength returns the encoded length of the +// EncodedMVCCTimestampSuffixLength returns the encoded length of the // given MVCC timestamp, including the length suffix. It returns 0 // if the timestamp is empty. -func encodedMVCCTimestampSuffixLength(ts hlc.Timestamp) int { +func EncodedMVCCTimestampSuffixLength(ts hlc.Timestamp) int { // This is backwards, see comment in encodedMVCCTimestampLength() for why. return encodedMVCCKeyLength(MVCCKey{Timestamp: ts}) - mvccEncodedTimeSentinelLen } @@ -389,6 +390,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() @@ -423,3 +437,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 []MVCCRangeKeyValue, ts hlc.Timestamp) (MVCCRangeKeyValue, 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].RangeKey.Timestamp.Less(ts) + }); i > 0 { + return rangeKeys[i-1], true + } + return MVCCRangeKeyValue{}, false +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 51509ed3ba92..f99b1b5f1fa2 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -193,7 +193,7 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { require.Equal(t, expect, encoded) require.Equal(t, len(encoded), encodedMVCCKeyLength(mvccKey)) require.Equal(t, len(encoded), - encodedMVCCKeyPrefixLength(mvccKey.Key)+encodedMVCCTimestampSuffixLength(mvccKey.Timestamp)) + EncodedMVCCKeyPrefixLength(mvccKey.Key)+EncodedMVCCTimestampSuffixLength(mvccKey.Timestamp)) decoded, err := DecodeMVCCKey(encoded) require.NoError(t, err) @@ -203,7 +203,7 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { expectPrefix, err := hex.DecodeString(tc.encoded[:2*len(tc.key)+2]) require.NoError(t, err) require.Equal(t, expectPrefix, EncodeMVCCKeyPrefix(roachpb.Key(tc.key))) - require.Equal(t, len(expectPrefix), encodedMVCCKeyPrefixLength(roachpb.Key(tc.key))) + require.Equal(t, len(expectPrefix), EncodedMVCCKeyPrefixLength(roachpb.Key(tc.key))) // Test encode/decodeMVCCTimestampSuffix too, since we can trivially do so. expectTS, err := hex.DecodeString(tc.encoded[2*len(tc.key)+2:]) @@ -214,7 +214,7 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { encodedTS := EncodeMVCCTimestampSuffix(tc.ts) require.Equal(t, expectTS, encodedTS) - require.Equal(t, len(encodedTS), encodedMVCCTimestampSuffixLength(tc.ts)) + require.Equal(t, len(encodedTS), EncodedMVCCTimestampSuffixLength(tc.ts)) decodedTS, err := decodeMVCCTimestampSuffix(encodedTS) require.NoError(t, err) @@ -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)() + + rangeKVs := []MVCCRangeKeyValue{ + rangeKV("a", "f", 6, MVCCValue{}), + rangeKV("a", "f", 4, MVCCValue{}), + rangeKV("a", "f", 3, MVCCValue{}), + rangeKV("a", "f", 1, MVCCValue{}), + } + + 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) { + rkv, ok := firstRangeKeyAbove(rangeKVs, hlc.Timestamp{WallTime: tc.ts}) + if tc.expect == 0 { + require.False(t, ok) + require.Empty(t, rkv) + } else { + require.True(t, ok) + require.Equal(t, rangeKV("a", "f", int(tc.expect), MVCCValue{}), rkv) + } + }) + } +} + func pointKey(key string, ts int) MVCCKey { return MVCCKey{Key: roachpb.Key(key), Timestamp: wallTS(ts)} } diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 57c98d66e61f..b6d311a7970b 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -58,7 +58,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 { @@ -1624,6 +1628,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) @@ -1797,7 +1803,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 cca71031351f..9a75ef158b6d 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -31,7 +31,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" @@ -126,6 +125,10 @@ func TestMVCCStatsAddSubForward(t *testing.T) { ValCount: 1, IntentBytes: 1, IntentCount: 1, + RangeKeyCount: 1, + RangeKeyBytes: 1, + RangeValCount: 1, + RangeValBytes: 1, SeparatedIntentCount: 1, IntentAge: 1, GCBytesAge: 1, @@ -136,17 +139,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{ @@ -155,10 +148,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 @@ -166,7 +159,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{ @@ -176,31 +169,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. @@ -212,13 +201,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} @@ -229,13 +218,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 += 3 * 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). @@ -246,9 +235,9 @@ func TestMVCCStatsAddSubForward(t *testing.T) { neg.AgeTo(2e9) exp.LastUpdateNanos = 2e9 - exp.GCBytesAge = -3 + exp.GCBytesAge = -7 exp.IntentAge = -3 - cmp(neg, exp) + require.Equal(t, exp, neg) } func TestMVCCGetNotExist(t *testing.T) { @@ -909,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 } @@ -2199,13 +2191,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 afa737041672..a70ac7351556 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -96,6 +96,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, @@ -213,16 +217,18 @@ func (r *sstIterator) verifyValue() { } // 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.Span { - panic("not implemented") + return roachpb.Span{} } // RangeKeys implements SimpleMVCCIterator. func (r *sstIterator) RangeKeys() []MVCCRangeKeyValue { - panic("not implemented") + return []MVCCRangeKeyValue{} } diff --git a/pkg/storage/testdata/mvcc_histories/delete_range b/pkg/storage/testdata/mvcc_histories/delete_range index 4073e71af5ad..5687f627ea46 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range +++ b/pkg/storage/testdata/mvcc_histories/delete_range @@ -49,15 +49,17 @@ stats: key_count=8 key_bytes=152 val_count=10 val_bytes=64 live_count=6 live_byt ## The delete tombstone is placed alongside each of the previous values, at the newer timestamp. ## The deleted keys are returned. -run stat +run stats ok with t=A txn_begin ts=46 del_range k=b end=c returnKeys txn_remove ---- +>> del_range k=b end=c returnKeys t=A del_range: "b"-"c" -> deleted 2 key(s) del_range: returned "b" del_range: returned "b/123" +stats: key_bytes=+24 val_count=+2 val_bytes=+100 live_count=-2 live_bytes=-48 gc_bytes_age=+9288 intent_count=+2 intent_bytes=+24 separated_intent_count=+2 intent_age=+108 >> at end: data: "a"/45.000000000,0 -> / data: "a"/44.000000000,0 -> /BYTES/abc @@ -73,6 +75,7 @@ data: "c"/44.000000000,0 -> /BYTES/abc data: "c/123"/44.000000000,0 -> /BYTES/abc data: "d"/44.000000000,0 -> /BYTES/abc data: "d/123"/44.000000000,0 -> /BYTES/abc +stats: key_count=8 key_bytes=176 val_count=12 val_bytes=164 live_count=4 live_bytes=96 gc_bytes_age=13248 intent_count=2 intent_bytes=24 separated_intent_count=2 intent_age=108 # A limited non-txn that deletes a range of keys. diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts index 2c8a397dcfd3..ddc288deea55 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts @@ -9,7 +9,7 @@ # 2 x # 1 d1 e1 f1 g1 1 1 # a b c d e f g h i j k -run ok +run stats ok put k=d ts=1 v=d1 put k=e ts=1 v=e1 del k=e ts=2 @@ -23,8 +23,28 @@ with t=A ts=7 txn_begin put k=g v=7 ---- +>> put k=d ts=1 v=d1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=e ts=1 v=e1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=e ts=2 +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3234 +>> put k=f ts=1 v=f1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=1 v=g1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> increment k=i ts=1 inc: current value = 1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 +>> increment k=j ts=1 inc: current value = 1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 +>> del_range_ts k=a end=k ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-5 live_bytes=-103 gc_bytes_age=+11252 +>> del_range_ts k=c end=k ts=5 +stats: range_key_count=+1 range_key_bytes=+22 range_val_count=+2 gc_bytes_age=+2108 +>> put k=g v=7 t=A ts=7 +stats: key_bytes=+12 val_count=+1 val_bytes=+54 live_count=+1 live_bytes=+68 gc_bytes_age=-194 intent_count=+1 intent_bytes=+18 separated_intent_count=+1 intent_age=+93 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 rangekey: {a-c}/[3.000000000,0=/] @@ -38,11 +58,14 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 # Inline value or tombstone below range tombstone should error. -run error +run stats error put k=b ts=0 v=b0 ---- +>> put k=b ts=0 v=b0 +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -55,11 +78,14 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*withstack.withStack:) "b"/0,0: put is inline=true, but existing value is inline=false -run error +run stats error del k=b ts=0 ---- +>> del k=b ts=0 +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -72,12 +98,15 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*withstack.withStack:) "b"/0,0: put is inline=true, but existing value is inline=false # DeleteRange at ts=5 should error with WriteTooOldError. -run error +run stats error del_range k=a end=f ts=5 ---- +>> del_range k=a end=f ts=5 +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -90,17 +119,17 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 # Point key below range tombstones should error, but is written anyway at a -# higher timestamp. -# -# TODO(erikgrinaker): These should test stats too, once range tombstones are -# correctly accounted for: "Stats are updated correctly, even when there are -# existing point values and tombstones below the range tombstones". -run error +# higher timestamp. Stats are updated correctly, even when there are +# existing point values and tombstones below the range tombstones. +run stats error put k=c ts=3 v=c3 ---- +>> put k=c ts=3 v=c3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -114,11 +143,14 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=122 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16400 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 -run error +run stats error put k=d ts=3 v=d3 ---- +>> put k=d ts=3 v=d3 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -133,11 +165,14 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=134 val_count=10 val_bytes=108 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=3 live_bytes=110 gc_bytes_age=16206 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 -run error +run stats error put k=e ts=3 v=e3 ---- +>> put k=e ts=3 v=e3 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-196 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -153,12 +188,15 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 # CPuts expecting a value covered by a range tombstone should error. -run error +run stats error cput k=f ts=7 v=f7 cond=f1 ---- +>> cput k=f ts=7 v=f7 cond=f1 +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -174,14 +212,17 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: # A CPut replay of an intent expecting a value covered by a range tombstone # should error because of the range tombstone covering it. -run error +run stats error with t=A ts=7 cput k=g v=g7 cond=g1 ---- +>> cput k=g v=g7 cond=g1 t=A ts=7 +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -197,16 +238,19 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: # A CPut replacing an existing but ignored intent expecting a value covered # by a range tombstone should error because of the range tombstone covering it. -run error +run stats error with t=A ts=7 txn_step txn_ignore_seqs seqs=0-1 cput k=g v=g7 cond=g1 ---- +>> cput k=g v=g7 cond=g1 t=A ts=7 +stats: no change >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 isn=1 rangekey: {a-c}/[3.000000000,0=/] @@ -223,12 +267,15 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with failOnTombstones above a range tombstone should error. -run error +run stats error initput k=f ts=7 v=f7 failOnTombstones ---- +>> initput k=f ts=7 v=f7 failOnTombstones +stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -244,15 +291,16 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=146 val_count=11 val_bytes=115 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=131 gc_bytes_age=16010 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with a different value as an existing key should succeed when there's # a range tombstone covering the existing value. -# -# TODO(erikgrinaker): This should test stats too. -run ok +run stats ok initput k=f ts=7 v=f7 ---- +>> initput k=f ts=7 v=f7 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -269,12 +317,15 @@ data: "g"/7.000000000,0 -> /BYTES/7 data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=158 val_count=12 val_bytes=122 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=5 live_bytes=152 gc_bytes_age=15816 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 # An increment below a range tombstone should reset to 1 and write above it with -# a WriteTooOldError. -run error +# a WriteTooOldError. This should update stats correctly. +run stats error increment k=i ts=2 ---- +>> increment k=i ts=2 +stats: key_bytes=+12 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 gc_bytes_age=-194 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -292,13 +343,16 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/5.000000000,1 -> /INT/1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=170 val_count=13 val_bytes=128 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=6 live_bytes=172 gc_bytes_age=15622 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "i" at timestamp 2.000000000,0 too old; wrote at 5.000000000,1 # An increment above a range tombstone should reset to 1. -run ok +run stats ok increment k=j ts=7 ---- +>> increment k=j ts=7 inc: current value = 1 +stats: key_bytes=+12 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 gc_bytes_age=-194 >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] @@ -317,3 +371,4 @@ data: "i"/5.000000000,1 -> /INT/1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/7.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 +stats: key_count=7 key_bytes=182 val_count=14 val_bytes=134 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=7 live_bytes=192 gc_bytes_age=15428 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=93 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_merge_fragment b/pkg/storage/testdata/mvcc_histories/range_tombstone_merge_fragment new file mode 100644 index 000000000000..80984c156a86 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_merge_fragment @@ -0,0 +1,120 @@ +# Tests MVCC range tombstone merging and fragmentation. + +# Writing range tombstones next to others will merge them. First write +# two separate tombstones, then merge them in the middle, then extend +# left and right. +run stats trace ok +del_range_ts k=b end=c ts=1 +del_range_ts k=d end=e ts=1 +del_range_ts k=c end=d ts=1 +del_range_ts k=a end=b ts=1 +del_range_ts k=e end=f ts=1 +---- +>> del_range_ts k=b end=c ts=1 +rangekey: {b-c}/[1.000000000,0=/] +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=d end=e ts=1 +rangekey: {b-c}/[1.000000000,0=/] +rangekey: {d-e}/[1.000000000,0=/] +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=c end=d ts=1 +rangekey: {b-e}/[1.000000000,0=/] +stats: range_key_count=-1 range_key_bytes=-13 range_val_count=-1 gc_bytes_age=-1287 +>> del_range_ts k=a end=b ts=1 +rangekey: {a-e}/[1.000000000,0=/] +stats: no change +>> del_range_ts k=e end=f ts=1 +rangekey: {a-f}/[1.000000000,0=/] +stats: no change +stats: range_key_count=1 range_key_bytes=13 range_val_count=1 gc_bytes_age=1287 + +# Writing the same sequence above these should fragment and merge the existing +# ones too. We use different-length key bounds to test the stats contributions +# of those, and also a lower localTs to test the value handling. +run stats trace ok +del_range_ts k=bbb end=cc ts=2 localTs=1 +del_range_ts k=d end=eee ts=2 localTs=1 +del_range_ts k=cc end=d ts=2 localTs=1 +del_range_ts k=a end=bbb ts=2 localTs=1 +del_range_ts k=eee end=f ts=2 localTs=1 +---- +>> del_range_ts k=bbb end=cc ts=2 localTs=1 +rangekey: {a-bbb}/[1.000000000,0=/] +rangekey: {bbb-cc}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {cc-f}/[1.000000000,0=/] +stats: range_key_count=+2 range_key_bytes=+41 range_val_count=+3 range_val_bytes=+13 gc_bytes_age=+5317 +>> del_range_ts k=d end=eee ts=2 localTs=1 +rangekey: {a-bbb}/[1.000000000,0=/] +rangekey: {bbb-cc}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {cc-d}/[1.000000000,0=/] +rangekey: {d-eee}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {eee-f}/[1.000000000,0=/] +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 range_val_bytes=+13 gc_bytes_age=+5120 +>> del_range_ts k=cc end=d ts=2 localTs=1 +rangekey: {a-bbb}/[1.000000000,0=/] +rangekey: {bbb-eee}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {eee-f}/[1.000000000,0=/] +stats: range_key_count=-2 range_key_bytes=-37 range_val_count=-3 range_val_bytes=-13 gc_bytes_age=-4923 +>> del_range_ts k=a end=bbb ts=2 localTs=1 +rangekey: {a-eee}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {eee-f}/[1.000000000,0=/] +stats: range_key_count=-1 range_key_bytes=-17 range_val_count=-1 gc_bytes_age=-1681 +>> del_range_ts k=eee end=f ts=2 localTs=1 +rangekey: {a-f}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +stats: range_key_count=-1 range_key_bytes=-17 range_val_count=-1 gc_bytes_age=-1681 +stats: range_key_count=1 range_key_bytes=22 range_val_count=2 range_val_bytes=13 gc_bytes_age=3439 + +# Writing a key from the middle of the existing stack to outside the bounds +# should fragment appropriately. +run stats trace ok +del_range_ts k=d end=h ts=3 +---- +>> del_range_ts k=d end=h ts=3 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-h}/[3.000000000,0=/] +stats: range_key_count=+2 range_key_bytes=+44 range_val_count=+4 range_val_bytes=+13 gc_bytes_age=+5569 +stats: range_key_count=3 range_key_bytes=66 range_val_count=6 range_val_bytes=26 gc_bytes_age=9008 + +# Writing range tombstones at the same or higher local timestamp should +# still merge, since the values are normalized to empty. However, writing +# at a lower timestamp should not merge, since the values are different. +# Extending a range tombstone with the same local timestamp should +# merge with it, but different local timestamp should not merge. +run stats trace ok +del_range_ts k=h end=i ts=3 localTs=5 +del_range_ts k=i end=j ts=3 localTs=3 +del_range_ts k=j end=k ts=3 localTs=2 +del_range_ts k=k end=l ts=3 localTs=2 +del_range_ts k=l end=m ts=3 localTs=1 +---- +>> del_range_ts k=h end=i ts=3 localTs=5 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-i}/[3.000000000,0=/] +stats: no change +>> del_range_ts k=i end=j ts=3 localTs=3 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-j}/[3.000000000,0=/] +stats: no change +>> del_range_ts k=j end=k ts=3 localTs=2 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-j}/[3.000000000,0=/] +rangekey: {j-k}/[3.000000000,0={localTs=2.000000000,0}/] +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2522 +>> del_range_ts k=k end=l ts=3 localTs=2 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-j}/[3.000000000,0=/] +rangekey: {j-l}/[3.000000000,0={localTs=2.000000000,0}/] +stats: no change +>> del_range_ts k=l end=m ts=3 localTs=1 +rangekey: {a-d}/[2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {d-f}/[3.000000000,0=/ 2.000000000,0={localTs=1.000000000,0}/ 1.000000000,0=/] +rangekey: {f-j}/[3.000000000,0=/] +rangekey: {j-l}/[3.000000000,0={localTs=2.000000000,0}/] +rangekey: {l-m}/[3.000000000,0={localTs=1.000000000,0}/] +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2522 +stats: range_key_count=5 range_key_bytes=92 range_val_count=8 range_val_bytes=52 gc_bytes_age=14052 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats new file mode 100644 index 000000000000..f2ab5278deb4 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats @@ -0,0 +1,145 @@ +# Tests basic MVCC stats for put, del, and del_range_ts. +# +# Final state, where x is tombstone, o---o is range tombstone, [] is intent: +# +# 6 d6 e6 f6 +# 5 x x x +# 4 b4 x o-----------------------o Two range tombstones: the lowest is the one +# 3 x c3 o-----------------------o that matters for point key GCBytesAge. +# 2 x +# 1 b1 x e1 x h1 x +# 0 a0 +# a b c d e f g h i j + +run stats ok +put k=a ts=0 v=a0 +put k=b ts=1 v=b1 +del k=b ts=2 +del k=b ts=3 +put k=b ts=4 v=b4 +del k=c ts=1 +put k=c ts=3 v=c3 +del k=c ts=4 +put k=e ts=1 v=e1 +del k=f ts=1 +put k=h ts=1 v=h1 +del k=i ts=1 +del_range_ts k=d end=j ts=3 +del_range_ts k=d end=f ts=4 localTs=3 # Write as two to test split/merge. +del_range_ts k=f end=j ts=4 localTs=3 +put k=d ts=6 v=d6 +put k=e ts=6 v=e6 +put k=f ts=6 v=f6 +del k=g ts=5 +del k=h ts=5 +del k=i ts=5 +---- +>> put k=a ts=0 v=a0 +stats: key_count=+1 key_bytes=+2 val_count=+1 val_bytes=+21 live_count=+1 live_bytes=+23 +>> put k=b ts=1 v=b1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=b ts=2 +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3234 +>> del k=b ts=3 +stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1162 +>> put k=b ts=4 v=b4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=c ts=3 v=c3 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-198 +>> del k=c ts=4 +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> put k=e ts=1 v=e1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=h ts=1 v=h1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=i ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=d end=j ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=d end=f ts=4 localTs=3 # Write as two to test split/merge. +stats: range_key_count=+1 range_key_bytes=+22 range_val_count=+2 range_val_bytes=+13 gc_bytes_age=+3369 +>> del_range_ts k=f end=j ts=4 localTs=3 +stats: range_key_count=-1 range_key_bytes=-13 range_val_count=-1 gc_bytes_age=-1261 +>> put k=d ts=6 v=d6 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=e ts=6 v=e6 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=f ts=6 v=f6 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-198 +>> del k=g ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> del k=h ts=5 +stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1136 +>> del k=i ts=5 +stats: key_bytes=+12 val_count=+1 gc_bytes_age=+1132 +>> at end: +rangekey: {d-j}/[4.000000000,0={localTs=3.000000000,0}/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/a0 mergeTs= txnDidNotUpdateMeta=false +data: "b"/4.000000000,0 -> /BYTES/b4 +data: "b"/3.000000000,0 -> / +data: "b"/2.000000000,0 -> / +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/4.000000000,0 -> / +data: "c"/3.000000000,0 -> /BYTES/c3 +data: "c"/1.000000000,0 -> / +data: "d"/6.000000000,0 -> /BYTES/d6 +data: "e"/6.000000000,0 -> /BYTES/e6 +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/1.000000000,0 -> / +data: "g"/5.000000000,0 -> / +data: "h"/5.000000000,0 -> / +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/5.000000000,0 -> / +data: "i"/1.000000000,0 -> / +stats: key_count=9 key_bytes=222 val_count=18 val_bytes=77 range_key_count=1 range_key_bytes=22 range_val_count=2 range_val_bytes=13 live_count=5 live_bytes=107 gc_bytes_age=21979 + +# Finally, let's delete everything -- first in parts, then all again. We can't +# delete the inline value at "a" though. +run stats ok +del_range_ts k=c end=d ts=8 localTs=7 +del_range_ts k=e end=f ts=8 localTs=7 +del_range_ts k=d end=e ts=8 localTs=7 +del_range_ts k=b end=c ts=8 localTs=7 +del_range_ts k=f end=j ts=8 +del_range_ts k=b end=j ts=9 +---- +>> del_range_ts k=c end=d ts=8 localTs=7 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2392 +>> del_range_ts k=e end=f ts=8 localTs=7 +stats: range_key_count=+2 range_key_bytes=+53 range_val_count=+5 range_val_bytes=+39 live_count=-1 live_bytes=-21 gc_bytes_age=+10678 +>> del_range_ts k=d end=e ts=8 localTs=7 +stats: range_key_count=-1 range_key_bytes=-22 range_val_count=-2 range_val_bytes=-13 live_count=-1 live_bytes=-21 gc_bytes_age=-1437 +>> del_range_ts k=b end=c ts=8 localTs=7 +stats: live_count=-1 live_bytes=-21 gc_bytes_age=+1932 +>> del_range_ts k=f end=j ts=8 +stats: range_key_bytes=+9 range_val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+2744 +>> del_range_ts k=b end=j ts=9 +stats: range_key_bytes=+27 range_val_count=+3 gc_bytes_age=+2445 +>> at end: +rangekey: {b-d}/[9.000000000,0=/ 8.000000000,0={localTs=7.000000000,0}/] +rangekey: {d-f}/[9.000000000,0=/ 8.000000000,0={localTs=7.000000000,0}/ 4.000000000,0={localTs=3.000000000,0}/ 3.000000000,0=/] +rangekey: {f-j}/[9.000000000,0=/ 8.000000000,0=/ 4.000000000,0={localTs=3.000000000,0}/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/a0 mergeTs= txnDidNotUpdateMeta=false +data: "b"/4.000000000,0 -> /BYTES/b4 +data: "b"/3.000000000,0 -> / +data: "b"/2.000000000,0 -> / +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/4.000000000,0 -> / +data: "c"/3.000000000,0 -> /BYTES/c3 +data: "c"/1.000000000,0 -> / +data: "d"/6.000000000,0 -> /BYTES/d6 +data: "e"/6.000000000,0 -> /BYTES/e6 +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/1.000000000,0 -> / +data: "g"/5.000000000,0 -> / +data: "h"/5.000000000,0 -> / +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/5.000000000,0 -> / +data: "i"/1.000000000,0 -> / +stats: key_count=9 key_bytes=222 val_count=18 val_bytes=77 range_key_count=3 range_key_bytes=102 range_val_count=10 range_val_bytes=52 live_count=1 live_bytes=23 gc_bytes_age=40733 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort new file mode 100644 index 000000000000..518ee3a2ee8c --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort @@ -0,0 +1,250 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +with t=A status=ABORTED + resolve_intent k=a + resolve_intent k=b + resolve_intent k=c + resolve_intent k=d + resolve_intent k=e + resolve_intent k=f + resolve_intent k=g + resolve_intent k=h + resolve_intent k=i + resolve_intent k=j + resolve_intent k=k + resolve_intent k=l + resolve_intent k=m + resolve_intent k=n + resolve_intent k=o + resolve_intent k=p + resolve_intent k=q + resolve_intent k=r +---- +>> resolve_intent k=a t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=b t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_bytes=-48 gc_bytes_age=-1786 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=c t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+198 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=d t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5828 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=e t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-48 live_count=+1 live_bytes=+21 gc_bytes_age=-7614 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=f t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-48 gc_bytes_age=-5630 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=g t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+194 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=h t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+198 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=i t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+194 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=j t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-48 gc_bytes_age=-5630 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=k t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5828 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=l t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5828 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=m t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=n t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_bytes=-48 gc_bytes_age=-1786 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=o t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+190 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=p t=A status=ABORTED +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5828 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=q t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-48 live_count=+1 live_bytes=+21 gc_bytes_age=-7614 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=r t=A status=ABORTED +stats: key_bytes=-12 val_count=-1 val_bytes=-48 gc_bytes_age=-5638 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> at end: +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/1.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> / +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/1.000000000,0 -> / +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/1.000000000,0 -> / +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/5.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/5.000000000,0 -> / +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=42 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=4 live_bytes=84 gc_bytes_age=14399 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit new file mode 100644 index 000000000000..d1ac8745f512 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit @@ -0,0 +1,268 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +with t=A status=COMMITTED + resolve_intent k=a + resolve_intent k=b + resolve_intent k=c + resolve_intent k=d + resolve_intent k=e + resolve_intent k=f + resolve_intent k=g + resolve_intent k=h + resolve_intent k=i + resolve_intent k=j + resolve_intent k=k + resolve_intent k=l + resolve_intent k=m + resolve_intent k=n + resolve_intent k=o + resolve_intent k=p + resolve_intent k=q + resolve_intent k=r +---- +>> resolve_intent k=a t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=b t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=c t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=d t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=e t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=f t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=g t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=h t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=i t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=j t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=k t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=l t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=m t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=n t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=o t=A status=COMMITTED +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=p t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=q t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=r t=A status=COMMITTED +stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> at end: +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "a"/6.000000000,0 -> /BYTES/a6 +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +data: "d"/6.000000000,0 -> / +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +data: "k"/6.000000000,0 -> / +data: "l"/6.000000000,0 -> / +data: "m"/6.000000000,0 -> /BYTES/m6 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +data: "p"/6.000000000,0 -> / +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=105 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=189 gc_bytes_age=31827 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed new file mode 100644 index 000000000000..d927491cf8cb --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed @@ -0,0 +1,270 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +with t=A status=COMMITTED + txn_advance ts=7 + resolve_intent k=a + resolve_intent k=b + resolve_intent k=c + resolve_intent k=d + resolve_intent k=e + resolve_intent k=f + resolve_intent k=g + resolve_intent k=h + resolve_intent k=i + resolve_intent k=j + resolve_intent k=k + resolve_intent k=l + resolve_intent k=m + resolve_intent k=n + resolve_intent k=o + resolve_intent k=p + resolve_intent k=q + resolve_intent k=r +---- +>> resolve_intent k=a t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=b t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 gc_bytes_age=-19 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=c t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=d t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=e t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3336 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=f t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=g t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=h t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=i t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=j t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=k t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=l t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=m t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=n t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 gc_bytes_age=-19 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=o t=A status=COMMITTED +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=p t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=q t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3336 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> resolve_intent k=r t=A status=COMMITTED +stats: val_bytes=-35 gc_bytes_age=-3317 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/a6 +data: "b"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/c6 +data: "c"/1.000000000,0 -> / +data: "d"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "e"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "f"/1.000000000,0 -> / +data: "g"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/h6 +data: "h"/1.000000000,0 -> / +data: "i"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "j"/1.000000000,0 -> / +data: "k"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "l"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "m"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/m6 +data: "n"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/o6 +data: "o"/5.000000000,0 -> / +data: "p"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "q"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=339 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=306 gc_bytes_age=42506 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort new file mode 100644 index 000000000000..0bab1efcbfba --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_abort @@ -0,0 +1,198 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +resolve_intent_range t=A k=a end=z status=ABORTED +---- +>> resolve_intent_range t=A k=a end=z status=ABORTED +stats: key_count=-6 key_bytes=-228 val_count=-18 val_bytes=-927 live_count=-5 live_bytes=-537 gc_bytes_age=-58036 intent_count=-18 intent_bytes=-279 separated_intent_count=-18 intent_age=-1692 +>> at end: +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/1.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> / +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/1.000000000,0 -> / +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/1.000000000,0 -> / +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/5.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/5.000000000,0 -> / +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=42 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=4 live_bytes=84 gc_bytes_age=14399 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit new file mode 100644 index 000000000000..5bc127d55963 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_commit @@ -0,0 +1,216 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +resolve_intent_range t=A k=a end=z status=COMMITTED +---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED +stats: val_bytes=-864 live_bytes=-432 gc_bytes_age=-40608 intent_count=-18 intent_bytes=-279 separated_intent_count=-18 intent_age=-1692 +>> at end: +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "a"/6.000000000,0 -> /BYTES/a6 +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +data: "d"/6.000000000,0 -> / +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +data: "k"/6.000000000,0 -> / +data: "l"/6.000000000,0 -> / +data: "m"/6.000000000,0 -> /BYTES/m6 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +data: "p"/6.000000000,0 -> / +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=105 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=189 gc_bytes_age=31827 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed new file mode 100644 index 000000000000..454076c6037c --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_range_pushed @@ -0,0 +1,218 @@ +# Tests MVCC stats calculations when resolving intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +run stats ok +txn_advance t=A ts=7 +resolve_intent_range t=A k=a end=z status=COMMITTED +---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED +stats: val_bytes=-630 live_bytes=-315 gc_bytes_age=-29929 intent_count=-18 intent_bytes=-279 separated_intent_count=-18 intent_age=-1692 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/a6 +data: "b"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/c6 +data: "c"/1.000000000,0 -> / +data: "d"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "e"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "f"/1.000000000,0 -> / +data: "g"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "h"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/h6 +data: "h"/1.000000000,0 -> / +data: "i"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +data: "j"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "j"/1.000000000,0 -> / +data: "k"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "l"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "m"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/m6 +data: "n"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +data: "o"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/o6 +data: "o"/5.000000000,0 -> / +data: "p"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "q"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "q"/5.000000000,0 -> /BYTES/q5 +data: "r"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=339 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=306 gc_bytes_age=42506 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite new file mode 100644 index 000000000000..07f5a75714d9 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite @@ -0,0 +1,385 @@ +# Tests MVCC stats calculations when rewriting intents. Intermediate states are +# tested through stats traces. Initial state: +# +# (x is tombstone, o---o is range tombstone, [] is intent) +# +# 7 +# 6 [a6][b6][c6][x] [x] [x] [g6][h6][i6][x] [x] [x] [m6][n6][o6] x x x +# 5 n5 x q5 x +# 4 o-----------------------------------------------o +# 3 o-----------------------------------------------o +# 2 +# 1 b1 x e1 x h1 x k1 x +# a b c d e f g h i j k l m n o p q r s +# +# This uses two range tombstones, since the lowest is the one that matters for +# point key GCBytesAge. It also uses points below/above range tombstones, +# because iterators surface range keys separately from point keys, which can +# cause bugs if callers don't step onto the point key. +# +# TODO(erikgrinaker): This is probably better handled by randomized or +# generative testing, since the combinations are getting unwieldy. But it'll do +# for now. + +run stats ok +with ts=1 + put k=b v=b1 + del k=c + put k=e v=e1 + del k=f + put k=g v=g1 + del k=h + put k=i v=i1 + del k=j +del_range_ts k=g end=s ts=3 +del_range_ts k=g end=s ts=4 +with ts=5 + put k=n v=n5 + del k=o + put k=q v=q5 + del k=r +with t=A + txn_begin ts=6 + put k=a v=a6 + put k=b v=b6 + put k=c v=c6 + del k=d + del k=e + del k=f + put k=g v=g6 + put k=h v=h6 + put k=i v=i6 + del k=j + del k=k + del k=l + put k=m v=m6 + put k=n v=n6 + put k=o v=o6 + del k=p + del k=q + del k=r +---- +>> put k=b v=b1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=c ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=e v=e1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=f ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=g v=g1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> put k=i v=i1 ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=1 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1386 +>> del_range_ts k=g end=s ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> del_range_ts k=g end=s ts=4 +stats: range_key_bytes=+9 range_val_count=+1 gc_bytes_age=+860 +>> put k=n v=n5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=o ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=r ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1330 +>> put k=a v=a6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=b v=b6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=c v=c6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=d t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=e t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=f t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=g v=g6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=h v=h6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-198 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=i v=i6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-194 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=j t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5630 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=k t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=l t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> put k=m v=m6 t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=n v=n6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1786 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> put k=o v=o6 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-190 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+94 +>> del k=p t=A +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5828 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=q t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7614 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> del k=r t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5638 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=6.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "a"/6.000000000,0 -> /BYTES/a6 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "b"/6.000000000,0 -> /BYTES/b6 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/6.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "f"/6.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "g"/6.000000000,0 -> /BYTES/g6 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "h"/6.000000000,0 -> /BYTES/h6 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/6.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/6.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "l"/6.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "m"/6.000000000,0 -> /BYTES/m6 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "o"/6.000000000,0 -> /BYTES/o6 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/6.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/6.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "r"/6.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=72435 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1692 + +# Rewrite the same keys at a higher timestamp. +run stats ok +with t=A ts=7 + txn_advance + txn_restart + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 + del k=d + del k=e + del k=f + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 + del k=j + del k=k + del k=l + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 + del k=p + del k=q + del k=r +---- +>> put k=a v=a7 t=A ts=7 +stats: intent_age=-1 +>> put k=b v=b7 t=A ts=7 +stats: gc_bytes_age=-19 intent_age=-1 +>> put k=c v=c7 t=A ts=7 +stats: intent_age=-1 +>> del k=d t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> del k=e t=A ts=7 +stats: gc_bytes_age=-81 intent_age=-1 +>> del k=f t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> put k=g v=g7 t=A ts=7 +stats: intent_age=-1 +>> put k=h v=h7 t=A ts=7 +stats: intent_age=-1 +>> put k=i v=i7 t=A ts=7 +stats: intent_age=-1 +>> del k=j t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> del k=k t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> del k=l t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> put k=m v=m7 t=A ts=7 +stats: intent_age=-1 +>> put k=n v=n7 t=A ts=7 +stats: gc_bytes_age=-19 intent_age=-1 +>> put k=o v=o7 t=A ts=7 +stats: intent_age=-1 +>> del k=p t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> del k=q t=A ts=7 +stats: gc_bytes_age=-81 intent_age=-1 +>> del k=r t=A ts=7 +stats: gc_bytes_age=-62 intent_age=-1 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "d"/7.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "e"/7.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "f"/7.000000000,0 -> / +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "g"/7.000000000,0 -> /BYTES/g7 +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "h"/7.000000000,0 -> /BYTES/h7 +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "i"/7.000000000,0 -> /BYTES/i7 +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "j"/7.000000000,0 -> / +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "k"/7.000000000,0 -> / +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "l"/7.000000000,0 -> / +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "n"/7.000000000,0 -> /BYTES/n7 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "p"/7.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "q"/7.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "r"/7.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=71801 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1674 + +# Rewrite keys<->tombstones at a higher timestamp. +run stats ok +with t=A ts=8 + txn_advance + txn_restart + del k=a + del k=b + del k=c + put k=d v=d8 + put k=e v=e8 + put k=f v=f8 + del k=g + del k=h + del k=i + put k=j v=j8 + put k=k v=k8 + put k=l v=l8 +---- +>> del k=a t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 +>> del k=b t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5685 intent_bytes=-7 intent_age=-1 +>> del k=c t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 +>> put k=d v=d8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 +>> put k=e v=e8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5785 intent_bytes=+7 intent_age=-1 +>> put k=f v=f8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 +>> del k=g t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 +>> del k=h t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 +>> del k=i t=A ts=8 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5704 intent_bytes=-7 intent_age=-1 +>> put k=j v=j8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 +>> put k=k v=k8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 +>> put k=l v=l8 t=A ts=8 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5766 intent_bytes=+7 intent_age=-1 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=8.000000000,0 wto=false gul=0,0 +rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "a"/8.000000000,0 -> / +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "b"/8.000000000,0 -> / +data: "b"/1.000000000,0 -> /BYTES/b1 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "c"/8.000000000,0 -> / +data: "c"/1.000000000,0 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "d"/8.000000000,0 -> /BYTES/d8 +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "e"/8.000000000,0 -> /BYTES/e8 +data: "e"/1.000000000,0 -> /BYTES/e1 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "f"/8.000000000,0 -> /BYTES/f8 +data: "f"/1.000000000,0 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "g"/8.000000000,0 -> / +data: "g"/1.000000000,0 -> /BYTES/g1 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "h"/8.000000000,0 -> / +data: "h"/1.000000000,0 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "i"/8.000000000,0 -> / +data: "i"/1.000000000,0 -> /BYTES/i1 +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "j"/8.000000000,0 -> /BYTES/j8 +data: "j"/1.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "k"/8.000000000,0 -> /BYTES/k8 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "l"/8.000000000,0 -> /BYTES/l8 +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "n"/7.000000000,0 -> /BYTES/n7 +data: "n"/5.000000000,0 -> /BYTES/n5 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/5.000000000,0 -> / +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "p"/7.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "q"/7.000000000,0 -> / +data: "q"/5.000000000,0 -> /BYTES/q5 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "r"/7.000000000,0 -> / +data: "r"/5.000000000,0 -> / +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=969 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=71391 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1662 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes index 3064611f7171..1a5d13cc649b 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes @@ -15,6 +15,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 @@ -31,6 +32,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 @@ -49,6 +51,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 @@ -67,6 +70,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 @@ -85,6 +89,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 @@ -105,6 +110,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 @@ -123,6 +129,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 @@ -141,6 +148,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 @@ -161,6 +169,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 @@ -179,6 +188,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 @@ -197,6 +207,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 @@ -215,6 +226,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 @@ -233,6 +245,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 @@ -251,6 +264,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 @@ -271,6 +285,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 @@ -289,6 +304,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 @@ -307,6 +323,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 @@ -325,6 +342,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 @@ -343,6 +361,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 @@ -362,6 +381,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 @@ -384,98 +404,7 @@ rangekey: {b-d}/[4.000000000,0=/] rangekey: {k-p}/[4.000000000,0=/] data: "a"/4.000000000,0 -> / data: "a"/2.000000000,0 -> /BYTES/a2 -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 - -# 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: -rangekey: {a-b}/[10.000000000,0=/] -rangekey: {b-d}/[4.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 -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 - -run ok -del_range_ts k=p end=s ts=4 ----- ->> at end: -rangekey: {a-b}/[10.000000000,0=/] -rangekey: {b-d}/[4.000000000,0=/] -rangekey: {k-p}/[4.000000000,0=/] -rangekey: {p-s}/[4.000000000,0=/ 3.000000000,0=/] -rangekey: {s-x}/[4.000000000,0=/] -data: "a"/4.000000000,0 -> / -data: "a"/2.000000000,0 -> /BYTES/a2 -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 - -# Writing a local timestamp above the key timestamp should be rounded down, -# and merge with an adjacent range tombstone. -run ok -del_range_ts k=x end=y ts=4 localTs=5 ----- ->> at end: -rangekey: {a-b}/[10.000000000,0=/] -rangekey: {b-d}/[4.000000000,0=/] -rangekey: {k-p}/[4.000000000,0=/] -rangekey: {p-s}/[4.000000000,0=/ 3.000000000,0=/] -rangekey: {s-y}/[4.000000000,0=/] -data: "a"/4.000000000,0 -> / -data: "a"/2.000000000,0 -> /BYTES/a2 -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 - -# Writing range tombstones next to others with a different local timestamp -# (and thus value) should not merge. -run ok -del_range_ts k=y end=z ts=4 localTs=3 ----- ->> at end: -rangekey: {a-b}/[10.000000000,0=/] -rangekey: {b-d}/[4.000000000,0=/] -rangekey: {k-p}/[4.000000000,0=/] -rangekey: {p-s}/[4.000000000,0=/ 3.000000000,0=/] -rangekey: {s-y}/[4.000000000,0=/] -rangekey: {y-z}/[4.000000000,0={localTs=3.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 diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index 4a24429dc4dc..f3db0756247e 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -3215,3 +3215,29 @@ func BytesNext(b []byte) []byte { bn[len(bn)-1] = 0 return bn } + +// BytesPrevish returns a previous byte slice in lexicographical ordering. It is +// impossible in general to find the exact previous byte slice, because it has +// an infinite number of 0xff bytes at the end, so this returns the nearest +// previous slice right-padded with 0xff up to length bytes. It may reuse the +// given slice when possible. +func BytesPrevish(b []byte, length int) []byte { + bLen := len(b) + // An empty slice has no previous slice. + if bLen == 0 { + return b + } + // If the last byte is 0, just remove it. + if b[bLen-1] == 0 { + return b[:bLen-1] + } + // Otherwise, decrement the last byte and right-pad with 0xff. + if bLen > length { + length = bLen + } + buf := make([]byte, length) + copy(buf, b) + buf[bLen-1]-- + copy(buf[bLen:], bytes.Repeat([]byte{0xff}, length-bLen)) + return buf +}