From 0eafb65e9079a0f489d13f9998f1f290d1c37fab Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Fri, 30 Sep 2022 16:38:17 -0400 Subject: [PATCH 1/5] kvserver: skip TestLeasePreferencesDuringOutage See #88769. Release note: None --- pkg/kv/kvserver/client_lease_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 786e384a49e2..c64b1a3e9717 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -780,6 +780,7 @@ func gossipLiveness(t *testing.T, tc *testcluster.TestCluster) { // lease in a single cycle of the replicate_queue. func TestLeasePreferencesDuringOutage(t *testing.T) { defer leaktest.AfterTest(t)() + skip.WithIssue(t, 88769, "flaky test") defer log.Scope(t).Close(t) stickyRegistry := server.NewStickyInMemEnginesRegistry() From bea801919c243360b9078c8993d51be9c4f4a232 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 30 Sep 2022 10:27:27 +0000 Subject: [PATCH 2/5] storage: move `TestMVCCHistories` to `storage_test` This patch moves `TestMVCCHistories` into a separate `storage_test` package. This will allow it to make use of the `spanset` package for spanset assertions in a future patch. This required exporting a handful of private `storage` package members. There are no functional changes. Release note: None --- .../mvcc_history_metamorphic_iterator_test.go | 85 ++--- pkg/storage/mvcc_history_test.go | 326 +++++++++--------- pkg/storage/mvcc_incremental_iterator.go | 6 + pkg/storage/pebble.go | 6 +- pkg/storage/pebble_mvcc_scanner.go | 4 +- pkg/storage/point_synthesizing_iter.go | 92 ++--- 6 files changed, 264 insertions(+), 255 deletions(-) diff --git a/pkg/storage/mvcc_history_metamorphic_iterator_test.go b/pkg/storage/mvcc_history_metamorphic_iterator_test.go index a3e6a194dfd9..f5d99eaba741 100644 --- a/pkg/storage/mvcc_history_metamorphic_iterator_test.go +++ b/pkg/storage/mvcc_history_metamorphic_iterator_test.go @@ -9,7 +9,7 @@ // licenses/APL.txt. // -package storage +package storage_test import ( "bytes" @@ -19,6 +19,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" @@ -32,7 +33,7 @@ type metamorphicIterator struct { seed int64 t *testing.T r *rand.Rand - it SimpleMVCCIterator + it storage.SimpleMVCCIterator // isForward is true if the wrapped iterator is in forward mode at the // beginning of moveAround. We then need to leave the iterator in forward mode // because the caller might subsequently invoke NextKey which is illegal on an @@ -47,12 +48,14 @@ type metamorphicIterator struct { // - a metamorphicMVCCIterator, if `it` is an MVCCIterator // - a metamorphicMVCCIncrementalIterator, if `it` is an MVCCIncrementalIterator // a metamorphicIterator otherwise. -func newMetamorphicIterator(t *testing.T, seed int64, it SimpleMVCCIterator) SimpleMVCCIterator { +func newMetamorphicIterator( + t *testing.T, seed int64, it storage.SimpleMVCCIterator, +) storage.SimpleMVCCIterator { iter := &metamorphicIterator{t: t, seed: seed, r: rand.New(rand.NewSource(seed)), it: it} - if _, isMVCC := it.(MVCCIterator); isMVCC { + if _, isMVCC := it.(storage.MVCCIterator); isMVCC { return &metamorphicMVCCIterator{metamorphicIterator: iter} } - if _, isIncremental := it.(*MVCCIncrementalIterator); isIncremental { + if _, isIncremental := it.(*storage.MVCCIncrementalIterator); isIncremental { return &metamorphicMVCCIncrementalIterator{metamorphicIterator: iter} } return iter @@ -100,8 +103,8 @@ func (m *metamorphicIterator) moveAround() { } cur := m.it.UnsafeKey().Clone() - mvccIt, _ := m.it.(MVCCIterator) - iit, _ := m.it.(*MVCCIncrementalIterator) + mvccIt, _ := m.it.(storage.MVCCIterator) + iit, _ := m.it.(*storage.MVCCIncrementalIterator) var resetActions []action actions := []action{ @@ -124,7 +127,7 @@ func (m *metamorphicIterator) moveAround() { }, { "SeekGE(Max)", - func() { m.it.SeekGE(MVCCKeyMax) }, + func() { m.it.SeekGE(storage.MVCCKeyMax) }, }, } @@ -149,7 +152,7 @@ func (m *metamorphicIterator) moveAround() { }, }, action{ "SeekLT(Max)", - func() { mvccIt.SeekLT(MVCCKeyMax) }, + func() { mvccIt.SeekLT(storage.MVCCKeyMax) }, }) // Can only leave iterator in reverse mode if it's in reverse // initially, otherwise caller wouldn't be allowed to invoke NextKey @@ -170,7 +173,7 @@ func (m *metamorphicIterator) moveAround() { hasPoint, _ := m.it.HasPointAndRange() rangeKeys := m.it.RangeKeys().Clone() - var rangeKeysIgnoringTime MVCCRangeKeyStack + var rangeKeysIgnoringTime storage.MVCCRangeKeyStack if iit != nil { rangeKeysIgnoringTime = iit.RangeKeysIgnoringTime() } @@ -189,9 +192,9 @@ func (m *metamorphicIterator) moveAround() { choice := actions[m.r.Intn(len(actions))] printfln("action: %s", choice) - // NB: if this is an incr iter that ignores time, we can't expect SeekGE(cur) to + // NB: if this is an incr iter it may be ignoring time, so we can't expect SeekGE(cur) to // be able to retrieve the current key, as SeekGE always respects the time bound. - if iit == nil || !iit.ignoringTime { + if iit == nil || !iit.IgnoringTime() { resetActions = append(resetActions, action{ "SeekGE(cur)", func() { @@ -209,10 +212,10 @@ func (m *metamorphicIterator) moveAround() { if bytes.Compare(cur.Key, roachpb.LocalMax) >= 0 { // Make sure we don't put a global-only iter into local keyspace. printfln("seeking to LocalMax") - m.it.SeekGE(MakeMVCCMetadataKey(roachpb.LocalMax)) + m.it.SeekGE(storage.MakeMVCCMetadataKey(roachpb.LocalMax)) } else { printfln("seeking to KeyMin") - m.it.SeekGE(NilKey) + m.it.SeekGE(storage.NilKey) } for { valid, err := m.it.Valid() @@ -239,7 +242,7 @@ func (m *metamorphicIterator) moveAround() { resetActions = append(resetActions, action{ "SeekLT(max) && RevIterate", func() { - mvccIt.SeekLT(MVCCKeyMax) // NB: incompatible with IsPrefix, so we excluded that above + mvccIt.SeekLT(storage.MVCCKeyMax) // NB: incompatible with IsPrefix, so we excluded that above for { valid, err := m.it.Valid() require.Nil(m.t, err) @@ -260,7 +263,7 @@ func (m *metamorphicIterator) moveAround() { resetAction.do() { hasPoint2, _ := m.it.HasPointAndRange() // circumvent hated shadowing lint - var rangeKeysIgnoringTime2 MVCCRangeKeyStack + var rangeKeysIgnoringTime2 storage.MVCCRangeKeyStack if iit != nil { rangeKeysIgnoringTime2 = iit.RangeKeysIgnoringTime() } @@ -287,7 +290,7 @@ func (m *metamorphicIterator) Close() { m.it.Close() } -func (m *metamorphicIterator) SeekGE(key MVCCKey) { +func (m *metamorphicIterator) SeekGE(key storage.MVCCKey) { m.isForward = true m.it.SeekGE(key) m.moveAround() @@ -309,7 +312,7 @@ func (m *metamorphicIterator) NextKey() { m.moveAround() } -func (m *metamorphicIterator) UnsafeKey() MVCCKey { +func (m *metamorphicIterator) UnsafeKey() storage.MVCCKey { return m.it.UnsafeKey() } @@ -325,7 +328,7 @@ func (m *metamorphicIterator) RangeBounds() roachpb.Span { return m.it.RangeBounds() } -func (m *metamorphicIterator) RangeKeys() MVCCRangeKeyStack { +func (m *metamorphicIterator) RangeKeys() storage.MVCCRangeKeyStack { return m.it.RangeKeys() } @@ -340,56 +343,56 @@ type metamorphicMVCCIterator struct { *metamorphicIterator } -var _ MVCCIterator = (*metamorphicMVCCIterator)(nil) +var _ storage.MVCCIterator = (*metamorphicMVCCIterator)(nil) -func (m *metamorphicMVCCIterator) SeekLT(key MVCCKey) { - m.it.(MVCCIterator).SeekLT(key) +func (m *metamorphicMVCCIterator) SeekLT(key storage.MVCCKey) { + m.it.(storage.MVCCIterator).SeekLT(key) m.moveAround() } func (m *metamorphicMVCCIterator) Prev() { - m.it.(MVCCIterator).Prev() + m.it.(storage.MVCCIterator).Prev() m.moveAround() } func (m *metamorphicMVCCIterator) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) { - m.it.(MVCCIterator).SeekIntentGE(key, txnUUID) + m.it.(storage.MVCCIterator).SeekIntentGE(key, txnUUID) m.moveAround() } -func (m *metamorphicMVCCIterator) Key() MVCCKey { - return m.it.(MVCCIterator).Key() +func (m *metamorphicMVCCIterator) Key() storage.MVCCKey { + return m.it.(storage.MVCCIterator).Key() } func (m *metamorphicMVCCIterator) UnsafeRawKey() []byte { - return m.it.(MVCCIterator).UnsafeRawKey() + return m.it.(storage.MVCCIterator).UnsafeRawKey() } func (m *metamorphicMVCCIterator) UnsafeRawMVCCKey() []byte { - return m.it.(MVCCIterator).UnsafeRawMVCCKey() + return m.it.(storage.MVCCIterator).UnsafeRawMVCCKey() } func (m *metamorphicMVCCIterator) Value() []byte { - return m.it.(MVCCIterator).Value() + return m.it.(storage.MVCCIterator).Value() } func (m *metamorphicMVCCIterator) ValueProto(msg protoutil.Message) error { - return m.it.(MVCCIterator).ValueProto(msg) + return m.it.(storage.MVCCIterator).ValueProto(msg) } func (m *metamorphicMVCCIterator) FindSplitKey( start, end, minSplitKey roachpb.Key, targetSize int64, -) (MVCCKey, error) { - return m.it.(MVCCIterator).FindSplitKey(start, end, minSplitKey, targetSize) +) (storage.MVCCKey, error) { + return m.it.(storage.MVCCIterator).FindSplitKey(start, end, minSplitKey, targetSize) } -func (m *metamorphicMVCCIterator) Stats() IteratorStats { +func (m *metamorphicMVCCIterator) Stats() storage.IteratorStats { // TODO(tbg): these will be wrong since we do extra movement. - return m.it.(MVCCIterator).Stats() + return m.it.(storage.MVCCIterator).Stats() } func (m *metamorphicMVCCIterator) IsPrefix() bool { - return m.it.(MVCCIterator).IsPrefix() + return m.it.(storage.MVCCIterator).IsPrefix() } type metamorphicMVCCIncrementalIterator struct { @@ -398,29 +401,29 @@ type metamorphicMVCCIncrementalIterator struct { var _ mvccIncrementalIteratorI = (*metamorphicMVCCIncrementalIterator)(nil) -func (m *metamorphicMVCCIncrementalIterator) RangeKeysIgnoringTime() MVCCRangeKeyStack { - return m.it.(*MVCCIncrementalIterator).RangeKeysIgnoringTime() +func (m *metamorphicMVCCIncrementalIterator) RangeKeysIgnoringTime() storage.MVCCRangeKeyStack { + return m.it.(*storage.MVCCIncrementalIterator).RangeKeysIgnoringTime() } func (m *metamorphicMVCCIncrementalIterator) RangeKeyChangedIgnoringTime() bool { if m.seed != 0 { return m.rangeKeyChangedIgnoringTime } - return m.it.(*MVCCIncrementalIterator).RangeKeyChangedIgnoringTime() + return m.it.(*storage.MVCCIncrementalIterator).RangeKeyChangedIgnoringTime() } func (m *metamorphicMVCCIncrementalIterator) NextIgnoringTime() { - m.it.(*MVCCIncrementalIterator).NextIgnoringTime() + m.it.(*storage.MVCCIncrementalIterator).NextIgnoringTime() m.isForward = true m.moveAround() } func (m *metamorphicMVCCIncrementalIterator) NextKeyIgnoringTime() { - m.it.(*MVCCIncrementalIterator).NextKeyIgnoringTime() + m.it.(*storage.MVCCIncrementalIterator).NextKeyIgnoringTime() m.isForward = true m.moveAround() } func (m *metamorphicMVCCIncrementalIterator) TryGetIntentError() error { - return m.it.(*MVCCIncrementalIterator).TryGetIntentError() + return m.it.(*storage.MVCCIncrementalIterator).TryGetIntentError() } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 08d015121637..5c8f74776a10 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package storage +package storage_test import ( "context" @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -150,7 +151,7 @@ var ( func TestMVCCHistories(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - DisableMetamorphicSimpleValueEncoding(t) + storage.DisableMetamorphicSimpleValueEncoding(t) ctx := context.Background() @@ -161,42 +162,37 @@ func TestMVCCHistories(t *testing.T) { const statsTS = 100e9 datadriven.Walk(t, testutils.TestDataPath(t, "mvcc_histories"), func(t *testing.T, path string) { - disableSeparateEngineBlocks := strings.Contains(path, "_disable_separate_engine_blocks") - - engineOpts := []ConfigOption{CacheSize(1 << 20 /* 1 MiB */), ForTesting} - // If enabled by metamorphic parameter, use very small blocks to provoke TBI - // optimization. We'll also flush after each command. - if separateEngineBlocks && !disableSeparateEngineBlocks { - engineOpts = append(engineOpts, BlockSize(1), func(cfg *engineConfig) error { - cfg.Opts.DisableAutomaticCompactions = true - return nil - }) - } - - // We start from a clean slate in every test file. - engine, err := Open(ctx, InMemory(), engineOpts...) - if err != nil { - t.Fatal(err) - } - defer engine.Close() + st := cluster.MakeTestingClusterSettings() if strings.Contains(path, "_norace") { skip.UnderRace(t) } if strings.Contains(path, "_disable_local_timestamps") { - localTimestampsEnabled.Override(ctx, &engine.settings.SV, false) + storage.LocalTimestampsEnabled.Override(ctx, &st.SV, false) } + disableSeparateEngineBlocks := strings.Contains(path, "_disable_separate_engine_blocks") + + // We start from a clean slate in every test file. + engine, err := storage.Open(ctx, storage.InMemory(), + storage.CacheSize(1<<20 /* 1 MiB */), + storage.Settings(st), + storage.If(separateEngineBlocks && !disableSeparateEngineBlocks, storage.BlockSize(1)), + ) + require.NoError(t, err) + defer engine.Close() + reportDataEntries := func(buf *redact.StringBuilder) error { var hasData bool - err = engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, IterKeyTypeRangesOnly, - func(_ MVCCKeyValue, rangeKeys MVCCRangeKeyStack) error { + err = engine.MVCCIterate( + span.Key, span.EndKey, storage.MVCCKeyAndIntentsIterKind, storage.IterKeyTypeRangesOnly, + func(_ storage.MVCCKeyValue, rangeKeys storage.MVCCRangeKeyStack) error { hasData = true buf.Printf("rangekey: %s/[", rangeKeys.Bounds) for i, version := range rangeKeys.Versions { - val, err := DecodeMVCCValue(version.Value) + val, err := storage.DecodeMVCCValue(version.Value) require.NoError(t, err) if i > 0 { buf.Printf(" ") @@ -210,8 +206,9 @@ func TestMVCCHistories(t *testing.T) { return err } - err = engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, IterKeyTypePointsOnly, - func(r MVCCKeyValue, _ MVCCRangeKeyStack) error { + err = engine.MVCCIterate( + span.Key, span.EndKey, storage.MVCCKeyAndIntentsIterKind, storage.IterKeyTypePointsOnly, + func(r storage.MVCCKeyValue, _ storage.MVCCRangeKeyStack) error { hasData = true if r.Key.Timestamp.IsEmpty() { // Meta is at timestamp zero. @@ -222,7 +219,7 @@ func TestMVCCHistories(t *testing.T) { buf.Printf("meta: %v -> %+v\n", r.Key, &meta) } } else { - val, err := DecodeMVCCValue(r.Value) + val, err := storage.DecodeMVCCValue(r.Value) if err != nil { buf.Printf("data: %v -> error decoding value %v: %v\n", r.Key, r.Value, err) } else { @@ -241,7 +238,7 @@ func TestMVCCHistories(t *testing.T) { // SST iterator in order to accurately represent the raw SST data. reportSSTEntries := func(buf *redact.StringBuilder, name string, sst []byte) error { r, err := sstable.NewMemReader(sst, sstable.ReaderOptions{ - Comparer: EngineComparer, + Comparer: storage.EngineComparer, }) if err != nil { return err @@ -258,11 +255,11 @@ func TestMVCCHistories(t *testing.T) { if err := iter.Error(); err != nil { return err } - key, err := DecodeMVCCKey(k.UserKey) + key, err := storage.DecodeMVCCKey(k.UserKey) if err != nil { return err } - value, err := DecodeMVCCValue(v) + value, err := storage.DecodeMVCCValue(v) if err != nil { return err } @@ -278,11 +275,11 @@ func TestMVCCHistories(t *testing.T) { if err := rdIter.Error(); err != nil { return err } - start, err := DecodeMVCCKey(s.Start) + start, err := storage.DecodeMVCCKey(s.Start) if err != nil { return err } - end, err := DecodeMVCCKey(s.End) + end, err := storage.DecodeMVCCKey(s.End) if err != nil { return err } @@ -302,11 +299,11 @@ func TestMVCCHistories(t *testing.T) { if err := rkIter.Error(); err != nil { return err } - start, err := DecodeMVCCKey(s.Start) + start, err := storage.DecodeMVCCKey(s.Start) if err != nil { return err } - end, err := DecodeMVCCKey(s.End) + end, err := storage.DecodeMVCCKey(s.End) if err != nil { return err } @@ -314,14 +311,14 @@ func TestMVCCHistories(t *testing.T) { buf.Printf("%s: %s", strings.ToLower(k.Kind().String()), roachpb.Span{Key: start.Key, EndKey: end.Key}) if k.Suffix != nil { - ts, err := DecodeMVCCTimestampSuffix(k.Suffix) + ts, err := storage.DecodeMVCCTimestampSuffix(k.Suffix) if err != nil { return err } buf.Printf("/%s", ts) } if k.Kind() == pebble.InternalKeyKindRangeKeySet { - value, err := DecodeMVCCValue(k.Value) + value, err := storage.DecodeMVCCValue(k.Value) if err != nil { return err } @@ -539,7 +536,7 @@ func TestMVCCHistories(t *testing.T) { // that we can compare the deltas. var msEngineBefore enginepb.MVCCStats if stats { - msEngineBefore, err = ComputeStats(e.engine, span.Key, span.EndKey, statsTS) + msEngineBefore, err = storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) } msEvalBefore := *e.ms @@ -561,7 +558,7 @@ func TestMVCCHistories(t *testing.T) { if stats && cmd.typ == typDataUpdate { // If stats are enabled, emit evaluated stats returned by the // command, and compare them with the real computed stats diff. - msEngineDiff, err := ComputeStats(e.engine, span.Key, span.EndKey, statsTS) + msEngineDiff, err := storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) msEngineDiff.Subtract(msEngineBefore) @@ -605,7 +602,7 @@ func TestMVCCHistories(t *testing.T) { // Calculate and output final stats if requested and the data changed. if stats && dataChange { - ms, err := ComputeStats(e.engine, span.Key, span.EndKey, statsTS) + ms, err := storage.ComputeStats(e.engine, span.Key, span.EndKey, statsTS) require.NoError(t, err) buf.Printf("stats: %s\n", formatStats(ms, false)) } @@ -833,7 +830,7 @@ func cmdTxnUpdate(e *evalCtx) error { } type intentPrintingReadWriter struct { - ReadWriter + storage.ReadWriter buf *redact.StringBuilder } @@ -853,7 +850,7 @@ func (rw intentPrintingReadWriter) ClearIntent( return rw.ReadWriter.ClearIntent(key, txnDidNotUpdateMeta, txnUUID) } -func (e *evalCtx) tryWrapForIntentPrinting(rw ReadWriter) ReadWriter { +func (e *evalCtx) tryWrapForIntentPrinting(rw storage.ReadWriter) storage.ReadWriter { if e.results.traceIntentWrites { return intentPrintingReadWriter{ReadWriter: rw, buf: e.results.buf} } @@ -865,7 +862,7 @@ func cmdResolveIntent(e *evalCtx) error { key := e.getKey() status := e.getTxnStatus() clockWhilePending := hlc.ClockTimestamp(e.getTsWithName("clockWhilePending")) - return e.withWriter("resolve_intent", func(rw ReadWriter) error { + return e.withWriter("resolve_intent", func(rw storage.ReadWriter) error { return e.resolveIntent(rw, key, txn, status, clockWhilePending) }) } @@ -878,14 +875,14 @@ func cmdResolveIntentRange(e *evalCtx) error { intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: start, EndKey: end}) intent.Status = status - return e.withWriter("resolve_intent_range", func(rw ReadWriter) error { - _, _, err := MVCCResolveWriteIntentRange(e.ctx, rw, e.ms, intent, 0) + return e.withWriter("resolve_intent_range", func(rw storage.ReadWriter) error { + _, _, err := storage.MVCCResolveWriteIntentRange(e.ctx, rw, e.ms, intent, 0) return err }) } func (e *evalCtx) resolveIntent( - rw ReadWriter, + rw storage.ReadWriter, key roachpb.Key, txn *roachpb.Transaction, resolveStatus roachpb.TransactionStatus, @@ -894,7 +891,7 @@ func (e *evalCtx) resolveIntent( intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}) intent.Status = resolveStatus intent.ClockWhilePending = roachpb.ObservedTimestamp{Timestamp: clockWhilePending} - _, err := MVCCResolveWriteIntent(e.ctx, rw, e.ms, intent) + _, err := storage.MVCCResolveWriteIntent(e.ctx, rw, e.ms, intent) return err } @@ -905,11 +902,11 @@ func cmdCheckIntent(e *evalCtx) error { wantIntent = false } - return e.withReader(func(r Reader) error { + return e.withReader(func(r storage.Reader) error { var meta enginepb.MVCCMetadata - iter := r.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true}) + iter := r.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{Prefix: true}) defer iter.Close() - iter.SeekGE(MVCCKey{Key: key}) + iter.SeekGE(storage.MVCCKey{Key: key}) ok, err := iter.Valid() if err != nil { return err @@ -943,14 +940,14 @@ func cmdAddLock(e *evalCtx) error { func cmdClear(e *evalCtx) error { key := e.getKey() ts := e.getTs(nil) - return e.withWriter("clear", func(rw ReadWriter) error { - return rw.ClearMVCC(MVCCKey{Key: key, Timestamp: ts}) + return e.withWriter("clear", func(rw storage.ReadWriter) error { + return rw.ClearMVCC(storage.MVCCKey{Key: key, Timestamp: ts}) }) } func cmdClearRange(e *evalCtx) error { key, endKey := e.getKeyRange() - return e.withWriter("clear_range", func(rw ReadWriter) error { + return e.withWriter("clear_range", func(rw storage.ReadWriter) error { // NB: We can't test ClearRawRange or ClearRangeUsingHeuristic here, because // it does not handle separated intents. if clearRangeUsingIter { @@ -963,8 +960,8 @@ func cmdClearRange(e *evalCtx) error { func cmdClearRangeKey(e *evalCtx) error { key, endKey := e.getKeyRange() ts := e.getTs(nil) - return e.withWriter("clear_rangekey", func(rw ReadWriter) error { - return rw.ClearMVCCRangeKey(MVCCRangeKey{StartKey: key, EndKey: endKey, Timestamp: ts}) + return e.withWriter("clear_rangekey", func(rw storage.ReadWriter) error { + return rw.ClearMVCCRangeKey(storage.MVCCRangeKey{StartKey: key, EndKey: endKey, Timestamp: ts}) }) } @@ -987,7 +984,7 @@ func cmdClearTimeRange(e *evalCtx) error { batch := e.engine.NewBatch() defer batch.Close() - resume, err := MVCCClearTimeRange(e.ctx, batch, e.ms, key, endKey, targetTs, ts, + resume, err := storage.MVCCClearTimeRange(e.ctx, batch, e.ms, key, endKey, targetTs, ts, nil, nil, clearRangeThreshold, int64(maxBatchSize), int64(maxBatchByteSize)) if err != nil { return err @@ -1004,10 +1001,10 @@ func cmdClearTimeRange(e *evalCtx) error { func cmdGCClearRange(e *evalCtx) error { key, endKey := e.getKeyRange() gcTs := e.getTs(nil) - return e.withWriter("gc_clear_range", func(rw ReadWriter) error { - cms, err := ComputeStats(rw, key, endKey, 100e9) + return e.withWriter("gc_clear_range", func(rw storage.ReadWriter) error { + cms, err := storage.ComputeStats(rw, key, endKey, 100e9) require.NoError(e.t, err, "failed to compute range stats") - return MVCCGarbageCollectWholeRange(e.ctx, rw, e.ms, key, endKey, gcTs, cms) + return storage.MVCCGarbageCollectWholeRange(e.ctx, rw, e.ms, key, endKey, gcTs, cms) }) } @@ -1024,14 +1021,14 @@ func cmdCPut(e *evalCtx) error { rexpVal := e.getValInternal("cond") expVal = rexpVal.TagAndDataBytes() } - behavior := CPutFailIfMissing + behavior := storage.CPutFailIfMissing if e.hasArg("allow_missing") { - behavior = CPutAllowIfMissing + behavior = storage.CPutAllowIfMissing } resolve, resolveStatus := e.getResolve() - return e.withWriter("cput", func(rw ReadWriter) error { - if err := MVCCConditionalPut(e.ctx, rw, e.ms, key, ts, localTs, val, expVal, behavior, txn); err != nil { + return e.withWriter("cput", func(rw storage.ReadWriter) error { + if err := storage.MVCCConditionalPut(e.ctx, rw, e.ms, key, ts, localTs, val, expVal, behavior, txn); err != nil { return err } if resolve { @@ -1051,8 +1048,8 @@ func cmdInitPut(e *evalCtx) error { failOnTombstones := e.hasArg("failOnTombstones") resolve, resolveStatus := e.getResolve() - return e.withWriter("initput", func(rw ReadWriter) error { - if err := MVCCInitPut(e.ctx, rw, e.ms, key, ts, localTs, val, failOnTombstones, txn); err != nil { + return e.withWriter("initput", func(rw storage.ReadWriter) error { + if err := storage.MVCCInitPut(e.ctx, rw, e.ms, key, ts, localTs, val, failOnTombstones, txn); err != nil { return err } if resolve { @@ -1068,8 +1065,8 @@ func cmdDelete(e *evalCtx) error { ts := e.getTs(txn) localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) resolve, resolveStatus := e.getResolve() - return e.withWriter("del", func(rw ReadWriter) error { - deletedKey, err := MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn) + return e.withWriter("del", func(rw storage.ReadWriter) error { + deletedKey, err := storage.MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn) if err != nil { return err } @@ -1093,8 +1090,8 @@ func cmdDeleteRange(e *evalCtx) error { } resolve, resolveStatus := e.getResolve() - return e.withWriter("del_range", func(rw ReadWriter) error { - deleted, resumeSpan, num, err := MVCCDeleteRange( + return e.withWriter("del_range", func(rw storage.ReadWriter) error { + deleted, resumeSpan, num, err := storage.MVCCDeleteRange( e.ctx, rw, e.ms, key, endKey, int64(max), ts, localTs, txn, returnKeys) if err != nil { return err @@ -1126,7 +1123,7 @@ func cmdDeleteRangeTombstone(e *evalCtx) error { // before the start key -- don't attempt to compute covered stats for these // to avoid iterator panics. if key.Compare(endKey) < 0 && key.Compare(keys.LocalMax) >= 0 { - ms, err := ComputeStats(e.engine, key, endKey, ts.WallTime) + ms, err := storage.ComputeStats(e.engine, key, endKey, ts.WallTime) if err != nil { return err } @@ -1134,8 +1131,9 @@ func cmdDeleteRangeTombstone(e *evalCtx) error { } } - return e.withWriter("del_range_ts", func(rw ReadWriter) error { - return MVCCDeleteRangeUsingTombstone(e.ctx, rw, e.ms, key, endKey, ts, localTs, nil, nil, idempotent, 0, msCovered) + return e.withWriter("del_range_ts", func(rw storage.ReadWriter) error { + return storage.MVCCDeleteRangeUsingTombstone( + e.ctx, rw, e.ms, key, endKey, ts, localTs, nil, nil, idempotent, 0, msCovered) }) } @@ -1160,8 +1158,8 @@ func cmdDeleteRangePredicate(e *evalCtx) error { if e.hasArg("rangeThreshold") { e.scanArg("rangeThreshold", &rangeThreshold) } - return e.withWriter("del_range_pred", func(rw ReadWriter) error { - resumeSpan, err := MVCCPredicateDeleteRange(e.ctx, rw, e.ms, key, endKey, ts, + return e.withWriter("del_range_pred", func(rw storage.ReadWriter) error { + resumeSpan, err := storage.MVCCPredicateDeleteRange(e.ctx, rw, e.ms, key, endKey, ts, localTs, nil, nil, predicates, int64(max), int64(maxBytes), int64(rangeThreshold), 0) if resumeSpan != nil { @@ -1177,7 +1175,7 @@ func cmdGet(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() ts := e.getTs(txn) - opts := MVCCGetOptions{Txn: txn} + opts := storage.MVCCGetOptions{Txn: txn} if e.hasArg("inconsistent") { opts.Inconsistent = true opts.Txn = nil @@ -1203,8 +1201,8 @@ func cmdGet(e *evalCtx) error { opts.Uncertainty.GlobalLimit = txn.GlobalUncertaintyLimit } - return e.withReader(func(r Reader) error { - val, intent, err := MVCCGet(e.ctx, r, key, ts, opts) + return e.withReader(func(r storage.Reader) error { + val, intent, err := storage.MVCCGet(e.ctx, r, key, ts, opts) // NB: the error is returned below. This ensures the test can // ascertain no result is populated in the intent when an error // occurs. @@ -1235,8 +1233,8 @@ func cmdIncrement(e *evalCtx) error { resolve, resolveStatus := e.getResolve() - return e.withWriter("increment", func(rw ReadWriter) error { - curVal, err := MVCCIncrement(e.ctx, rw, e.ms, key, ts, localTs, txn, inc) + return e.withWriter("increment", func(rw storage.ReadWriter) error { + curVal, err := storage.MVCCIncrement(e.ctx, rw, e.ms, key, ts, localTs, txn, inc) if err != nil { return err } @@ -1252,8 +1250,8 @@ func cmdMerge(e *evalCtx) error { key := e.getKey() val := e.getVal() ts := e.getTs(nil) - return e.withWriter("merge", func(rw ReadWriter) error { - return MVCCMerge(e.ctx, rw, e.ms, key, ts, val) + return e.withWriter("merge", func(rw storage.ReadWriter) error { + return storage.MVCCMerge(e.ctx, rw, e.ms, key, ts, val) }) } @@ -1267,8 +1265,8 @@ func cmdPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() - return e.withWriter("put", func(rw ReadWriter) error { - if err := MVCCPut(e.ctx, rw, e.ms, key, ts, localTs, val, txn); err != nil { + return e.withWriter("put", func(rw storage.ReadWriter) error { + if err := storage.MVCCPut(e.ctx, rw, e.ms, key, ts, localTs, val, txn); err != nil { return err } if resolve { @@ -1279,9 +1277,9 @@ func cmdPut(e *evalCtx) error { } func cmdIsSpanEmpty(e *evalCtx) error { - return e.withReader(func(r Reader) error { + return e.withReader(func(r storage.Reader) error { key, endKey := e.getKeyRange() - isEmpty, err := MVCCIsSpanEmpty(e.ctx, r, MVCCIsSpanEmptyOptions{ + isEmpty, err := storage.MVCCIsSpanEmpty(e.ctx, r, storage.MVCCIsSpanEmptyOptions{ StartKey: key, EndKey: endKey, StartTS: e.getTsWithName("startTs"), @@ -1297,8 +1295,8 @@ func cmdIsSpanEmpty(e *evalCtx) error { func cmdExport(e *evalCtx) error { key, endKey := e.getKeyRange() - opts := MVCCExportOptions{ - StartKey: MVCCKey{Key: key, Timestamp: e.getTsWithName("kTs")}, + opts := storage.MVCCExportOptions{ + StartKey: storage.MVCCKey{Key: key, Timestamp: e.getTsWithName("kTs")}, EndKey: endKey, StartTS: e.getTsWithName("startTs"), EndTS: e.getTs(nil), @@ -1318,8 +1316,8 @@ func cmdExport(e *evalCtx) error { r := e.newReader() defer r.Close() - sstFile := &MemFile{} - summary, resume, err := MVCCExportToSST(e.ctx, e.st, r, opts, sstFile) + sstFile := &storage.MemFile{} + summary, resume, err := storage.MVCCExportToSST(e.ctx, e.st, r, opts, sstFile) if err != nil { return err } @@ -1330,8 +1328,8 @@ func cmdExport(e *evalCtx) error { } e.results.buf.Printf("\n") - iter, err := NewMemSSTIterator(sstFile.Bytes(), false /* verify */, IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, + iter, err := storage.NewMemSSTIterator(sstFile.Bytes(), false /* verify */, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, UpperBound: keys.MaxKey, }) if err != nil { @@ -1340,7 +1338,7 @@ func cmdExport(e *evalCtx) error { defer iter.Close() var rangeStart roachpb.Key - for iter.SeekGE(NilKey); ; iter.Next() { + for iter.SeekGE(storage.NilKey); ; iter.Next() { if ok, err := iter.Valid(); err != nil { return err } else if !ok { @@ -1352,7 +1350,7 @@ func cmdExport(e *evalCtx) error { rangeStart = append(rangeStart[:0], rangeBounds.Key...) e.results.buf.Printf("export: %s/[", rangeBounds) for i, version := range iter.RangeKeys().Versions { - val, err := DecodeMVCCValue(version.Value) + val, err := storage.DecodeMVCCValue(version.Value) if err != nil { return err } @@ -1367,7 +1365,7 @@ func cmdExport(e *evalCtx) error { if hasPoint { key := iter.UnsafeKey() value := iter.UnsafeValue() - mvccValue, err := DecodeMVCCValue(value) + mvccValue, err := storage.DecodeMVCCValue(value) if err != nil { return err } @@ -1382,7 +1380,7 @@ func cmdScan(e *evalCtx) error { txn := e.getTxn(optional) key, endKey := e.getKeyRange() ts := e.getTs(txn) - opts := MVCCScanOptions{Txn: txn} + opts := storage.MVCCScanOptions{Txn: txn} if e.hasArg("inconsistent") { opts.Inconsistent = true opts.Txn = nil @@ -1426,8 +1424,8 @@ func cmdScan(e *evalCtx) error { if e.hasArg("wholeRows") { opts.WholeRowsOfSize = 10 // arbitrary, must be greater than largest column family in tests } - return e.withReader(func(r Reader) error { - res, err := MVCCScan(e.ctx, r, key, endKey, ts, opts) + return e.withReader(func(r storage.Reader) error { + res, err := storage.MVCCScan(e.ctx, r, key, endKey, ts, opts) // NB: the error is returned below. This ensures the test can // ascertain no result is populated in the intents when an error // occurs. @@ -1451,19 +1449,19 @@ func cmdScan(e *evalCtx) error { } func cmdPutRangeKey(e *evalCtx) error { - var rangeKey MVCCRangeKey + var rangeKey storage.MVCCRangeKey rangeKey.StartKey, rangeKey.EndKey = e.getKeyRange() rangeKey.Timestamp = e.getTs(nil) - var value MVCCValue + var value storage.MVCCValue value.MVCCValueHeader.LocalTimestamp = hlc.ClockTimestamp(e.getTsWithName("localTs")) - return e.withWriter("put_rangekey", func(rw ReadWriter) error { + return e.withWriter("put_rangekey", func(rw storage.ReadWriter) error { return rw.PutMVCCRangeKey(rangeKey, value) }) } func cmdIterNew(e *evalCtx) error { - var opts IterOptions + var opts storage.IterOptions opts.Prefix = e.hasArg("prefix") if e.hasArg("k") { opts.LowerBound, opts.UpperBound = e.getKeyRange() @@ -1471,15 +1469,15 @@ func cmdIterNew(e *evalCtx) error { if len(opts.UpperBound) == 0 { opts.UpperBound = keys.MaxKey } - kind := MVCCKeyAndIntentsIterKind + kind := storage.MVCCKeyAndIntentsIterKind if e.hasArg("kind") { var arg string e.scanArg("kind", &arg) switch arg { case "keys": - kind = MVCCKeyIterKind + kind = storage.MVCCKeyIterKind case "keysAndIntents": - kind = MVCCKeyAndIntentsIterKind + kind = storage.MVCCKeyAndIntentsIterKind default: return errors.Errorf("unknown iterator kind %s", arg) } @@ -1489,11 +1487,11 @@ func cmdIterNew(e *evalCtx) error { e.scanArg("types", &arg) switch arg { case "pointsOnly": - opts.KeyTypes = IterKeyTypePointsOnly + opts.KeyTypes = storage.IterKeyTypePointsOnly case "pointsAndRanges": - opts.KeyTypes = IterKeyTypePointsAndRanges + opts.KeyTypes = storage.IterKeyTypePointsAndRanges case "rangesOnly": - opts.KeyTypes = IterKeyTypeRangesOnly + opts.KeyTypes = storage.IterKeyTypeRangesOnly default: return errors.Errorf("unknown key type %s", arg) } @@ -1509,9 +1507,9 @@ func cmdIterNew(e *evalCtx) error { r := e.newReader() iter := r.NewMVCCIterator(kind, opts) if e.hasArg("pointSynthesis") { - iter = newPointSynthesizingIter(iter) + iter = storage.NewPointSynthesizingIter(iter) } - iter = newMetamorphicIterator(e.t, e.metamorphicIterSeed(), iter).(MVCCIterator) + iter = newMetamorphicIterator(e.t, e.metamorphicIterSeed(), iter).(storage.MVCCIterator) if opts.Prefix != iter.IsPrefix() { return errors.Errorf("prefix iterator returned IsPrefix=false") } @@ -1522,7 +1520,7 @@ func cmdIterNew(e *evalCtx) error { } func cmdIterNewIncremental(e *evalCtx) error { - var opts MVCCIncrementalIterOptions + var opts storage.MVCCIncrementalIterOptions if e.hasArg("k") { opts.StartKey, opts.EndKey = e.getKeyRange() } @@ -1541,11 +1539,11 @@ func cmdIterNewIncremental(e *evalCtx) error { e.scanArg("types", &arg) switch arg { case "pointsOnly": - opts.KeyTypes = IterKeyTypePointsOnly + opts.KeyTypes = storage.IterKeyTypePointsOnly case "pointsAndRanges": - opts.KeyTypes = IterKeyTypePointsAndRanges + opts.KeyTypes = storage.IterKeyTypePointsAndRanges case "rangesOnly": - opts.KeyTypes = IterKeyTypeRangesOnly + opts.KeyTypes = storage.IterKeyTypeRangesOnly default: return errors.Errorf("unknown key type %s", arg) } @@ -1559,11 +1557,11 @@ func cmdIterNewIncremental(e *evalCtx) error { e.scanArg("intents", &arg) switch arg { case "error": - opts.IntentPolicy = MVCCIncrementalIterIntentPolicyError + opts.IntentPolicy = storage.MVCCIncrementalIterIntentPolicyError case "emit": - opts.IntentPolicy = MVCCIncrementalIterIntentPolicyEmit + opts.IntentPolicy = storage.MVCCIncrementalIterIntentPolicyEmit case "aggregate": - opts.IntentPolicy = MVCCIncrementalIterIntentPolicyAggregate + opts.IntentPolicy = storage.MVCCIncrementalIterIntentPolicyAggregate default: return errors.Errorf("unknown intent policy %s", arg) } @@ -1574,10 +1572,10 @@ func cmdIterNewIncremental(e *evalCtx) error { } r := e.newReader() - it := SimpleMVCCIterator(NewMVCCIncrementalIterator(r, opts)) + it := storage.SimpleMVCCIterator(storage.NewMVCCIncrementalIterator(r, opts)) // Can't metamorphically move the iterator around since when intents get aggregated // or emitted we can't undo that later at the level of the metamorphic iterator. - if opts.IntentPolicy == MVCCIncrementalIterIntentPolicyError { + if opts.IntentPolicy == storage.MVCCIncrementalIterIntentPolicyError { it = newMetamorphicIterator(e.t, e.metamorphicIterSeed(), it) } e.iter = &iterWithCloser{it, r.Close} @@ -1593,8 +1591,8 @@ func cmdIterNewReadAsOf(e *evalCtx) error { if e.hasArg("asOfTs") { asOf = e.getTsWithName("asOfTs") } - opts := IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, + opts := storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, RangeKeyMaskingBelow: asOf} if e.hasArg("k") { opts.LowerBound, opts.UpperBound = e.getKeyRange() @@ -1603,9 +1601,9 @@ func cmdIterNewReadAsOf(e *evalCtx) error { opts.UpperBound = keys.MaxKey } r := e.newReader() - innerIter := newMetamorphicIterator(e.t, e.metamorphicIterSeed(), r.NewMVCCIterator(MVCCKeyIterKind, opts)) + innerIter := newMetamorphicIterator(e.t, e.metamorphicIterSeed(), r.NewMVCCIterator(storage.MVCCKeyIterKind, opts)) iter := &iterWithCloser{innerIter, r.Close} - e.iter = NewReadAsOfIterator(iter, asOf) + e.iter = storage.NewReadAsOfIterator(iter, asOf) e.iterRangeKeys.Clear() return nil } @@ -1613,7 +1611,7 @@ func cmdIterNewReadAsOf(e *evalCtx) error { func cmdIterSeekGE(e *evalCtx) error { key := e.getKey() ts := e.getTs(nil) - e.iter.SeekGE(MVCCKey{Key: key, Timestamp: ts}) + e.iter.SeekGE(storage.MVCCKey{Key: key, Timestamp: ts}) printIter(e) return nil } @@ -1631,7 +1629,7 @@ func cmdIterSeekIntentGE(e *evalCtx) error { func cmdIterSeekLT(e *evalCtx) error { key := e.getKey() ts := e.getTs(nil) - e.mvccIter().SeekLT(MVCCKey{Key: key, Timestamp: ts}) + e.mvccIter().SeekLT(storage.MVCCKey{Key: key, Timestamp: ts}) printIter(e) return nil } @@ -1676,10 +1674,10 @@ func cmdIterScan(e *evalCtx) error { // we're fudging e.rangeKeys. if e.iter.RangeKeyChanged() { if e.iterRangeKeys.IsEmpty() { - e.iterRangeKeys = MVCCRangeKeyStack{ + e.iterRangeKeys = storage.MVCCRangeKeyStack{ // NB: Clone MinKey/MaxKey, since we write into these byte slices later. Bounds: roachpb.Span{Key: keys.MinKey.Next().Clone(), EndKey: keys.MaxKey.Clone()}, - Versions: MVCCRangeKeyVersions{{Timestamp: hlc.MinTimestamp}}, + Versions: storage.MVCCRangeKeyVersions{{Timestamp: hlc.MinTimestamp}}, } } else { e.iterRangeKeys.Clear() @@ -1708,14 +1706,14 @@ func cmdSSTPut(e *evalCtx) error { if e.hasArg("v") { val = e.getVal() } - return e.sst().PutMVCC(MVCCKey{Key: key, Timestamp: ts}, MVCCValue{Value: val}) + return e.sst().PutMVCC(storage.MVCCKey{Key: key, Timestamp: ts}, storage.MVCCValue{Value: val}) } func cmdSSTPutRangeKey(e *evalCtx) error { - var rangeKey MVCCRangeKey + var rangeKey storage.MVCCRangeKey rangeKey.StartKey, rangeKey.EndKey = e.getKeyRange() rangeKey.Timestamp = e.getTs(nil) - var value MVCCValue + var value storage.MVCCValue value.MVCCValueHeader.LocalTimestamp = hlc.ClockTimestamp(e.getTsWithName("localTs")) return e.sst().PutMVCCRangeKey(rangeKey, value) @@ -1727,7 +1725,7 @@ func cmdSSTClearRange(e *evalCtx) error { } func cmdSSTClearRangeKey(e *evalCtx) error { - var rangeKey MVCCRangeKey + var rangeKey storage.MVCCRangeKey rangeKey.StartKey, rangeKey.EndKey = e.getKeyRange() rangeKey.Timestamp = e.getTs(nil) @@ -1756,8 +1754,8 @@ func cmdSSTIterNew(e *evalCtx) error { for i, sst := range e.ssts { ssts[len(ssts)-i-1] = sst } - iter, err := NewMultiMemSSTIterator(ssts, sstIterVerify, IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, + iter, err := storage.NewMultiMemSSTIterator(ssts, sstIterVerify, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, UpperBound: keys.MaxKey, }) if err != nil { @@ -1770,7 +1768,7 @@ func cmdSSTIterNew(e *evalCtx) error { func cmdReplacePointTombstonesWithRangeTombstones(e *evalCtx) error { start, end := e.getKeyRange() - return ReplacePointTombstonesWithRangeTombstones(e.ctx, e.engine, e.ms, start, end) + return storage.ReplacePointTombstonesWithRangeTombstones(e.ctx, e.engine, e.ms, start, end) } func printIter(e *evalCtx) { @@ -1803,7 +1801,7 @@ func printIter(e *evalCtx) { } e.results.buf.Printf(" %s=%+v", e.iter.UnsafeKey(), &meta) } else { - value, err := DecodeMVCCValue(e.iter.UnsafeValue()) + value, err := storage.DecodeMVCCValue(e.iter.UnsafeValue()) if err != nil { e.Fatalf("%v", err) } @@ -1815,7 +1813,7 @@ func printIter(e *evalCtx) { rangeKeys := e.iter.RangeKeys() e.results.buf.Printf(" %s/[", rangeKeys.Bounds) for i, version := range rangeKeys.Versions { - value, err := DecodeMVCCValue(version.Value) + value, err := storage.DecodeMVCCValue(version.Value) if err != nil { e.Fatalf("%v", err) } @@ -1827,14 +1825,14 @@ func printIter(e *evalCtx) { e.results.buf.Printf("]") } - var rangeKeysIgnoringTime MVCCRangeKeyStack + var rangeKeysIgnoringTime storage.MVCCRangeKeyStack if maybeIIT != nil { rangeKeysIgnoringTime = maybeIIT.RangeKeysIgnoringTime() } if ignoringTime && !rangeKeysIgnoringTime.IsEmpty() && !rangeKeysIgnoringTime.Equal(e.iter.RangeKeys()) { e.results.buf.Printf(" (+%s/[", rangeKeysIgnoringTime.Bounds) for i, version := range rangeKeysIgnoringTime.Versions { - value, err := DecodeMVCCValue(version.Value) + value, err := storage.DecodeMVCCValue(version.Value) if err != nil { e.Fatalf("%v", err) } @@ -1941,22 +1939,22 @@ type evalCtx struct { } ctx context.Context st *cluster.Settings - engine Engine + engine storage.Engine noMetamorphicIter bool // never instantiate metamorphicIterator - iter SimpleMVCCIterator - iterRangeKeys MVCCRangeKeyStack + iter storage.SimpleMVCCIterator + iterRangeKeys storage.MVCCRangeKeyStack t *testing.T td *datadriven.TestData txns map[string]*roachpb.Transaction txnCounter uint128.Uint128 locks map[string]*roachpb.Transaction ms *enginepb.MVCCStats - sstWriter *SSTWriter - sstFile *MemFile + sstWriter *storage.SSTWriter + sstFile *storage.MemFile ssts [][]byte } -func newEvalCtx(ctx context.Context, engine Engine) *evalCtx { +func newEvalCtx(ctx context.Context, engine storage.Engine) *evalCtx { return &evalCtx{ ctx: ctx, st: cluster.MakeTestingClusterSettings(), @@ -2080,12 +2078,12 @@ func (e *evalCtx) getTxn(opt optArg) *roachpb.Transaction { // newReader returns a new (metamorphic) reader for use by a single command. The // caller must call Close on the reader when done. -func (e *evalCtx) newReader() Reader { +func (e *evalCtx) newReader() storage.Reader { switch mvccHistoriesReader { case "engine": return noopCloseReader{e.engine} case "readonly": - return e.engine.NewReadOnly(StandardDurability) + return e.engine.NewReadOnly(storage.StandardDurability) case "batch": return e.engine.NewBatch() case "snapshot": @@ -2097,7 +2095,7 @@ func (e *evalCtx) newReader() Reader { } // withReader calls the given closure with a new reader, closing it when done. -func (e *evalCtx) withReader(fn func(Reader) error) error { +func (e *evalCtx) withReader(fn func(storage.Reader) error) error { r := e.newReader() defer r.Close() return fn(r) @@ -2106,10 +2104,10 @@ func (e *evalCtx) withReader(fn func(Reader) error) error { // withWriter calls the given closure with a writer. The writer is // metamorphically chosen to be a batch, which will be committed and closed when // done. -func (e *evalCtx) withWriter(cmd string, fn func(_ ReadWriter) error) error { - var rw ReadWriter +func (e *evalCtx) withWriter(cmd string, fn func(_ storage.ReadWriter) error) error { + var rw storage.ReadWriter rw = e.engine - var batch Batch + var batch storage.Batch if e.hasArg("batched") || mvccHistoriesUseBatch { batch = e.engine.NewBatch() defer batch.Close() @@ -2193,10 +2191,10 @@ func (e *evalCtx) newTxn( return txn, nil } -func (e *evalCtx) sst() *SSTWriter { +func (e *evalCtx) sst() *storage.SSTWriter { if e.sstWriter == nil { - e.sstFile = &MemFile{} - w := MakeIngestionSSTWriter(e.ctx, e.st, e.sstFile) + e.sstFile = &storage.MemFile{} + w := storage.MakeIngestionSSTWriter(e.ctx, e.st, e.sstFile) e.sstWriter = &w } return e.sstWriter @@ -2232,7 +2230,9 @@ func (e *evalCtx) lookupTxn(txnName string) (*roachpb.Transaction, error) { return txn, nil } -func (e *evalCtx) newLockTableView(txn *roachpb.Transaction, ts hlc.Timestamp) LockTableView { +func (e *evalCtx) newLockTableView( + txn *roachpb.Transaction, ts hlc.Timestamp, +) storage.LockTableView { return &mockLockTableView{locks: e.locks, txn: txn, ts: ts} } @@ -2259,7 +2259,7 @@ func (lt *mockLockTableView) IsKeyLockedByConflictingTxn( return true, &holder.TxnMeta } -func (e *evalCtx) visitWrappedIters(fn func(it SimpleMVCCIterator) (done bool)) { +func (e *evalCtx) visitWrappedIters(fn func(it storage.SimpleMVCCIterator) (done bool)) { iter := e.iter if iter == nil { return @@ -2288,10 +2288,10 @@ func (e *evalCtx) visitWrappedIters(fn func(it SimpleMVCCIterator) (done bool)) } } -func (e *evalCtx) mvccIter() MVCCIterator { - var iter MVCCIterator - e.visitWrappedIters(func(it SimpleMVCCIterator) (done bool) { - iter, done = it.(MVCCIterator) +func (e *evalCtx) mvccIter() storage.MVCCIterator { + var iter storage.MVCCIterator + e.visitWrappedIters(func(it storage.SimpleMVCCIterator) (done bool) { + iter, done = it.(storage.MVCCIterator) return done }) require.NotNil(e.t, iter, "need an MVCC iterator") @@ -2305,15 +2305,15 @@ func (e *evalCtx) mvccIncrementalIter() mvccIncrementalIteratorI { } type mvccIncrementalIteratorI interface { - SimpleMVCCIterator - RangeKeysIgnoringTime() MVCCRangeKeyStack + storage.SimpleMVCCIterator + RangeKeysIgnoringTime() storage.MVCCRangeKeyStack RangeKeyChangedIgnoringTime() bool NextIgnoringTime() NextKeyIgnoringTime() TryGetIntentError() error } -var _ mvccIncrementalIteratorI = (*MVCCIncrementalIterator)(nil) +var _ mvccIncrementalIteratorI = (*storage.MVCCIncrementalIterator)(nil) // tryMVCCIncrementalIter unwraps an MVCCIncrementalIterator, if there is one. // This does not return the verbatim *MVCCIncrementalIterator but an interface, @@ -2321,7 +2321,7 @@ var _ mvccIncrementalIteratorI = (*MVCCIncrementalIterator)(nil) // movement and thus needs to mask RangeKeyChanged{,IgnoringTime}. func (e *evalCtx) tryMVCCIncrementalIter() mvccIncrementalIteratorI { var iter mvccIncrementalIteratorI - e.visitWrappedIters(func(it SimpleMVCCIterator) (done bool) { + e.visitWrappedIters(func(it storage.SimpleMVCCIterator) (done bool) { iter, done = it.(mvccIncrementalIteratorI) return done }) @@ -2390,7 +2390,7 @@ func toKey(s string) roachpb.Key { // iterWithCloser will call the given closer when the iterator // is closed. type iterWithCloser struct { - SimpleMVCCIterator + storage.SimpleMVCCIterator closer func() } @@ -2403,7 +2403,7 @@ func (i *iterWithCloser) Close() { // noopCloseReader overrides Reader.Close() with a noop. type noopCloseReader struct { - Reader + storage.Reader } func (noopCloseReader) Close() {} diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 6bc54c9e8925..38a97f0f75f3 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -731,6 +731,12 @@ func (i *MVCCIncrementalIterator) NextKeyIgnoringTime() { i.updateIgnoreTime() } +// IgnoringTime returns true if the previous positioning operation ignored time +// bounds. +func (i *MVCCIncrementalIterator) IgnoringTime() bool { + return i.ignoringTime +} + // NumCollectedIntents returns number of intents encountered during iteration. // This is only the case when intent aggregation is enabled, otherwise it is // always 0. diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 7211fbca8ce8..4b5914e50194 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1345,7 +1345,7 @@ func (p *Pebble) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalOpDetails // No-op. Logical logging disabled. } -// localTimestampsEnabled controls whether local timestamps are written in MVCC +// LocalTimestampsEnabled controls whether local timestamps are written in MVCC // values. A true setting is also gated on clusterversion.LocalTimestamps. After // all nodes in a cluster are at or beyond clusterversion.LocalTimestamps, // different nodes will see the version state transition at different times. @@ -1356,7 +1356,7 @@ func (p *Pebble) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalOpDetails // TODO(nvanbenschoten): remove this cluster setting and its associated plumbing // when removing the cluster version, once we're confident in the efficacy and // stability of local timestamps. -var localTimestampsEnabled = settings.RegisterBoolSetting( +var LocalTimestampsEnabled = settings.RegisterBoolSetting( settings.SystemOnly, "storage.transaction.local_timestamps.enabled", "if enabled, MVCC keys will be written with local timestamps", @@ -1364,7 +1364,7 @@ var localTimestampsEnabled = settings.RegisterBoolSetting( ) func shouldWriteLocalTimestamps(ctx context.Context, settings *cluster.Settings) bool { - if !localTimestampsEnabled.Get(&settings.SV) { + if !LocalTimestampsEnabled.Get(&settings.SV) { // Not enabled. return false } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 61d7bd750b6a..0c55a165f189 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -312,7 +312,7 @@ type pebbleMVCCScanner struct { // pointIter is a point synthesizing iterator that wraps and replaces parent // when an MVCC range tombstone is encountered. A separate reference to it is // kept in order to release it back to its pool when the scanner is done. - pointIter *pointSynthesizingIter + pointIter *PointSynthesizingIter // memAccount is used to account for the size of the scan results. memAccount *mon.BoundAccount // lockTable is used to determine whether keys are locked in the in-memory @@ -1213,7 +1213,7 @@ func (p *pebbleMVCCScanner) enablePointSynthesis() { p.parent.UnsafeKey())) } } - p.pointIter = newPointSynthesizingIterAtParent(p.parent) + p.pointIter = NewPointSynthesizingIterAtParent(p.parent) p.parent = p.pointIter if util.RaceEnabled { if ok, _ := p.parent.Valid(); !ok { diff --git a/pkg/storage/point_synthesizing_iter.go b/pkg/storage/point_synthesizing_iter.go index 3377610a7068..4e625a8e3aa5 100644 --- a/pkg/storage/point_synthesizing_iter.go +++ b/pkg/storage/point_synthesizing_iter.go @@ -25,11 +25,11 @@ import ( // pointSynthesizingIterPool reuses pointSynthesizingIters to avoid allocations. var pointSynthesizingIterPool = sync.Pool{ New: func() interface{} { - return &pointSynthesizingIter{} + return &PointSynthesizingIter{} }, } -// pointSynthesizingIter wraps an MVCCIterator, and synthesizes MVCC point keys +// PointSynthesizingIter wraps an MVCCIterator, and synthesizes MVCC point keys // for MVCC range keys above existing point keys (not below), and at the start // of range keys (truncated to iterator bounds). It does not emit MVCC range // keys at all, since these would appear to conflict with the synthesized point @@ -42,7 +42,7 @@ var pointSynthesizingIterPool = sync.Pool{ // rangeKeysEnd specifies where to end point synthesis at the current position, // i.e. the first range key below an existing point key. // -// The relative positioning of pointSynthesizingIter and the underlying iterator +// The relative positioning of PointSynthesizingIter and the underlying iterator // is as follows in the forward direction: // // - atPoint=true: rangeKeysIdx points to a range key following the point key, @@ -60,7 +60,7 @@ var pointSynthesizingIterPool = sync.Pool{ // in reverse. // // See also assertInvariants() which asserts positioning invariants. -type pointSynthesizingIter struct { +type PointSynthesizingIter struct { // iter is the underlying MVCC iterator. iter MVCCIterator @@ -124,13 +124,13 @@ type pointSynthesizingIter struct { rangeKeyChanged bool } -var _ MVCCIterator = (*pointSynthesizingIter)(nil) +var _ MVCCIterator = (*PointSynthesizingIter)(nil) -// newPointSynthesizingIter creates a new pointSynthesizingIter, or gets one +// NewPointSynthesizingIter creates a new pointSynthesizingIter, or gets one // from the pool. -func newPointSynthesizingIter(parent MVCCIterator) *pointSynthesizingIter { - iter := pointSynthesizingIterPool.Get().(*pointSynthesizingIter) - *iter = pointSynthesizingIter{ +func NewPointSynthesizingIter(parent MVCCIterator) *PointSynthesizingIter { + iter := pointSynthesizingIterPool.Get().(*PointSynthesizingIter) + *iter = PointSynthesizingIter{ iter: parent, prefix: parent.IsPrefix(), // Reuse pooled byte slices. @@ -141,10 +141,10 @@ func newPointSynthesizingIter(parent MVCCIterator) *pointSynthesizingIter { return iter } -// newPointSynthesizingIterAtParent creates a new pointSynthesizingIter and +// NewPointSynthesizingIterAtParent creates a new pointSynthesizingIter and // loads the position from the parent iterator. -func newPointSynthesizingIterAtParent(parent MVCCIterator) *pointSynthesizingIter { - iter := newPointSynthesizingIter(parent) +func NewPointSynthesizingIterAtParent(parent MVCCIterator) *PointSynthesizingIter { + iter := NewPointSynthesizingIter(parent) iter.rangeKeyChanged = true // force range key detection if ok, err := iter.updateIter(); ok && err == nil { iter.updateSeekGEPosition(parent.UnsafeKey()) @@ -156,14 +156,14 @@ func newPointSynthesizingIterAtParent(parent MVCCIterator) *pointSynthesizingIte // // Close will also close the underlying iterator. Use release() to release it // back to the pool without closing the parent iterator. -func (i *pointSynthesizingIter) Close() { +func (i *PointSynthesizingIter) Close() { i.iter.Close() i.release() } // release releases the iterator back into the pool. -func (i *pointSynthesizingIter) release() { - *i = pointSynthesizingIter{ +func (i *PointSynthesizingIter) release() { + *i = PointSynthesizingIter{ // Reuse slices. rangeKeysBuf: i.rangeKeysBuf[:0], rangeKeysPos: i.rangeKeysPos[:0], @@ -174,14 +174,14 @@ func (i *pointSynthesizingIter) release() { // iterNext is a convenience function that calls iter.Next() // and returns the value of updateIter(). -func (i *pointSynthesizingIter) iterNext() (bool, error) { +func (i *PointSynthesizingIter) iterNext() (bool, error) { i.iter.Next() return i.updateIter() } // iterPrev is a convenience function that calls iter.Prev() // and returns the value of updateIter(). -func (i *pointSynthesizingIter) iterPrev() (bool, error) { +func (i *PointSynthesizingIter) iterPrev() (bool, error) { i.iter.Prev() return i.updateIter() } @@ -189,7 +189,7 @@ func (i *pointSynthesizingIter) iterPrev() (bool, error) { // updateIter memoizes the iterator fields from the underlying iterator, and // also keeps track of rangeKeyChanged. It must be called after every iterator // positioning operation, and returns the iterator validity/error. -func (i *pointSynthesizingIter) updateIter() (bool, error) { +func (i *PointSynthesizingIter) updateIter() (bool, error) { if i.iterValid, i.iterErr = i.iter.Valid(); i.iterValid { i.iterHasPoint, i.iterHasRange = i.iter.HasPointAndRange() i.iterKey = i.iter.UnsafeKey() @@ -208,7 +208,7 @@ func (i *pointSynthesizingIter) updateIter() (bool, error) { // iterator state. It must be called very time the pointSynthesizingIter moves // to a new key position, i.e. after exhausting all point/range keys at the // current position. rangeKeysIdx and rangeKeysEnd are reset. -func (i *pointSynthesizingIter) updateRangeKeys() { +func (i *PointSynthesizingIter) updateRangeKeys() { if !i.iterHasRange { i.clearRangeKeys() return @@ -265,7 +265,7 @@ func (i *pointSynthesizingIter) updateRangeKeys() { // extendRangeKeysEnd extends i.rangeKeysEnd below the current point key's // timestamp in the underlying iterator. It never reduces i.rangeKeysEnd. -func (i *pointSynthesizingIter) extendRangeKeysEnd() { +func (i *PointSynthesizingIter) extendRangeKeysEnd() { if i.iterHasPoint && i.atRangeKeysPos && !i.iterKey.Timestamp.IsEmpty() { if l := len(i.rangeKeys); i.rangeKeysEnd < l { i.rangeKeysEnd = sort.Search(l-i.rangeKeysEnd, func(idx int) bool { @@ -278,7 +278,7 @@ func (i *pointSynthesizingIter) extendRangeKeysEnd() { // updateAtPoint updates i.atPoint according to whether the synthesizing // iterator is positioned on the real point key in the underlying iterator. // Requires i.rangeKeys to have been positioned first. -func (i *pointSynthesizingIter) updateAtPoint() { +func (i *PointSynthesizingIter) updateAtPoint() { if !i.iterHasPoint { i.atPoint = false } else if len(i.rangeKeys) == 0 { @@ -297,7 +297,7 @@ func (i *pointSynthesizingIter) updateAtPoint() { // updatePosition updates the synthesizing iterator for the position of the // underlying iterator. This may step the underlying iterator to position it // correctly relative to bare range keys. -func (i *pointSynthesizingIter) updatePosition() { +func (i *PointSynthesizingIter) updatePosition() { if !i.iterHasRange { // Fast path: no range keys, so just clear range keys and bail out. i.atPoint = i.iterHasPoint @@ -333,7 +333,7 @@ func (i *pointSynthesizingIter) updatePosition() { // clearRangeKeys resets the iterator by clearing out all range key state. // gcassert:inline -func (i *pointSynthesizingIter) clearRangeKeys() { +func (i *PointSynthesizingIter) clearRangeKeys() { if len(i.rangeKeys) != 0 { i.rangeKeys = i.rangeKeys[:0] i.rangeKeysPos = i.rangeKeysPos[:0] @@ -349,7 +349,7 @@ func (i *pointSynthesizingIter) clearRangeKeys() { } // SeekGE implements MVCCIterator. -func (i *pointSynthesizingIter) SeekGE(seekKey MVCCKey) { +func (i *PointSynthesizingIter) SeekGE(seekKey MVCCKey) { i.reverse = false i.iter.SeekGE(seekKey) if ok, _ := i.updateIter(); !ok { @@ -360,7 +360,7 @@ func (i *pointSynthesizingIter) SeekGE(seekKey MVCCKey) { } // SeekIntentGE implements MVCCIterator. -func (i *pointSynthesizingIter) SeekIntentGE(seekKey roachpb.Key, txnUUID uuid.UUID) { +func (i *PointSynthesizingIter) SeekIntentGE(seekKey roachpb.Key, txnUUID uuid.UUID) { i.reverse = false i.iter.SeekIntentGE(seekKey, txnUUID) if ok, _ := i.updateIter(); !ok { @@ -372,7 +372,7 @@ func (i *pointSynthesizingIter) SeekIntentGE(seekKey roachpb.Key, txnUUID uuid.U // updateSeekGEPosition updates the iterator state following a SeekGE call, or // to load the parent iterator's position in newPointSynthesizingIterAtParent. -func (i *pointSynthesizingIter) updateSeekGEPosition(seekKey MVCCKey) { +func (i *PointSynthesizingIter) updateSeekGEPosition(seekKey MVCCKey) { // Fast path: no range key, so just reset the iterator and bail out. if !i.iterHasRange { @@ -420,7 +420,7 @@ func (i *pointSynthesizingIter) updateSeekGEPosition(seekKey MVCCKey) { } // Next implements MVCCIterator. -func (i *pointSynthesizingIter) Next() { +func (i *PointSynthesizingIter) Next() { // When changing direction, flip the relative positioning with iter. if i.reverse { i.reverse = false @@ -456,7 +456,7 @@ func (i *pointSynthesizingIter) Next() { } // NextKey implements MVCCIterator. -func (i *pointSynthesizingIter) NextKey() { +func (i *PointSynthesizingIter) NextKey() { // When changing direction, flip the relative positioning with iter. // // NB: This isn't really supported by the MVCCIterator interface, but we have @@ -481,7 +481,7 @@ func (i *pointSynthesizingIter) NextKey() { } // SeekLT implements MVCCIterator. -func (i *pointSynthesizingIter) SeekLT(seekKey MVCCKey) { +func (i *PointSynthesizingIter) SeekLT(seekKey MVCCKey) { i.reverse = true i.iter.SeekLT(seekKey) if ok, _ := i.updateIter(); !ok { @@ -539,7 +539,7 @@ func (i *pointSynthesizingIter) SeekLT(seekKey MVCCKey) { } // Prev implements MVCCIterator. -func (i *pointSynthesizingIter) Prev() { +func (i *PointSynthesizingIter) Prev() { // When changing direction, flip the relative positioning with iter. if !i.reverse { i.reverse = true @@ -575,7 +575,7 @@ func (i *pointSynthesizingIter) Prev() { } // Valid implements MVCCIterator. -func (i *pointSynthesizingIter) Valid() (bool, error) { +func (i *PointSynthesizingIter) Valid() (bool, error) { valid := i.iterValid || // On synthetic point key. (i.iterErr == nil && !i.atPoint && i.rangeKeysIdx >= 0 && i.rangeKeysIdx < i.rangeKeysEnd) @@ -590,12 +590,12 @@ func (i *pointSynthesizingIter) Valid() (bool, error) { } // Key implements MVCCIterator. -func (i *pointSynthesizingIter) Key() MVCCKey { +func (i *PointSynthesizingIter) Key() MVCCKey { return i.UnsafeKey().Clone() } // UnsafeKey implements MVCCIterator. -func (i *pointSynthesizingIter) UnsafeKey() MVCCKey { +func (i *PointSynthesizingIter) UnsafeKey() MVCCKey { if i.atPoint { return i.iterKey } @@ -609,7 +609,7 @@ func (i *pointSynthesizingIter) UnsafeKey() MVCCKey { } // UnsafeRawKey implements MVCCIterator. -func (i *pointSynthesizingIter) UnsafeRawKey() []byte { +func (i *PointSynthesizingIter) UnsafeRawKey() []byte { if i.atPoint { return i.iter.UnsafeRawKey() } @@ -617,7 +617,7 @@ func (i *pointSynthesizingIter) UnsafeRawKey() []byte { } // UnsafeRawMVCCKey implements MVCCIterator. -func (i *pointSynthesizingIter) UnsafeRawMVCCKey() []byte { +func (i *PointSynthesizingIter) UnsafeRawMVCCKey() []byte { if i.atPoint { return i.iter.UnsafeRawMVCCKey() } @@ -625,7 +625,7 @@ func (i *pointSynthesizingIter) UnsafeRawMVCCKey() []byte { } // Value implements MVCCIterator. -func (i *pointSynthesizingIter) Value() []byte { +func (i *PointSynthesizingIter) Value() []byte { if v := i.UnsafeValue(); v != nil { return append([]byte{}, v...) } @@ -633,7 +633,7 @@ func (i *pointSynthesizingIter) Value() []byte { } // UnsafeValue implements MVCCIterator. -func (i *pointSynthesizingIter) UnsafeValue() []byte { +func (i *PointSynthesizingIter) UnsafeValue() []byte { if i.atPoint { return i.iter.UnsafeValue() } @@ -644,49 +644,49 @@ func (i *pointSynthesizingIter) UnsafeValue() []byte { } // ValueProto implements MVCCIterator. -func (i *pointSynthesizingIter) ValueProto(msg protoutil.Message) error { +func (i *PointSynthesizingIter) ValueProto(msg protoutil.Message) error { return protoutil.Unmarshal(i.UnsafeValue(), msg) } // HasPointAndRange implements MVCCIterator. -func (i *pointSynthesizingIter) HasPointAndRange() (bool, bool) { +func (i *PointSynthesizingIter) HasPointAndRange() (bool, bool) { return true, false } // RangeBounds implements MVCCIterator. -func (i *pointSynthesizingIter) RangeBounds() roachpb.Span { +func (i *PointSynthesizingIter) RangeBounds() roachpb.Span { return roachpb.Span{} } // RangeKeys implements MVCCIterator. -func (i *pointSynthesizingIter) RangeKeys() MVCCRangeKeyStack { +func (i *PointSynthesizingIter) RangeKeys() MVCCRangeKeyStack { return MVCCRangeKeyStack{} } // RangeKeyChanged implements MVCCIterator. -func (i *pointSynthesizingIter) RangeKeyChanged() bool { +func (i *PointSynthesizingIter) RangeKeyChanged() bool { return false } // FindSplitKey implements MVCCIterator. -func (i *pointSynthesizingIter) FindSplitKey( +func (i *PointSynthesizingIter) FindSplitKey( start, end, minSplitKey roachpb.Key, targetSize int64, ) (MVCCKey, error) { return i.iter.FindSplitKey(start, end, minSplitKey, targetSize) } // Stats implements MVCCIterator. -func (i *pointSynthesizingIter) Stats() IteratorStats { +func (i *PointSynthesizingIter) Stats() IteratorStats { return i.iter.Stats() } // IsPrefix implements the MVCCIterator interface. -func (i *pointSynthesizingIter) IsPrefix() bool { +func (i *PointSynthesizingIter) IsPrefix() bool { return i.prefix } // assertInvariants asserts iterator invariants. The iterator must be valid. -func (i *pointSynthesizingIter) assertInvariants() error { +func (i *PointSynthesizingIter) assertInvariants() error { // Check general MVCCIterator API invariants. if err := assertMVCCIteratorInvariants(i); err != nil { return err From 040fd0bf1b1c5cf4cd7ee59fc6c58a736c85d5ab Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 13:52:16 +0000 Subject: [PATCH 3/5] storage: add `TestMVCCHistories` metamorphic param for peek bounds This patch adds a metamorphic test parameter that enables peek bounds for MVCC range key-related commands, and asserts that command accesses respect the bounds. These peek bounds will always result in identical, correct MVCC stats. Release note: None --- pkg/storage/BUILD.bazel | 1 + pkg/storage/mvcc_history_test.go | 58 ++++++++++++++++++++++++++++---- 2 files changed, 53 insertions(+), 6 deletions(-) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 76cc03f108dc..25396a6e9525 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -143,6 +143,7 @@ go_test( "//pkg/keys", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", + "//pkg/kv/kvserver/spanset", "//pkg/kv/kvserver/uncertainty", "//pkg/roachpb", "//pkg/settings/cluster", diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 5c8f74776a10..f9b2a81cabcc 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -56,6 +57,8 @@ var ( mvccHistoriesReader = util.ConstantWithMetamorphicTestChoice("mvcc-histories-reader", "engine", "readonly", "batch", "snapshot").(string) mvccHistoriesUseBatch = util.ConstantWithMetamorphicTestBool("mvcc-histories-use-batch", false) + mvccHistoriesPeekBounds = util.ConstantWithMetamorphicTestChoice("mvcc-histories-peek-bounds", + "none", "left", "right", "both").(string) sstIterVerify = util.ConstantWithMetamorphicTestBool("mvcc-histories-sst-iter-verify", false) metamorphicIteratorSeed = util.ConstantWithMetamorphicTestRange("mvcc-metamorphic-iterator-seed", 0, 0, 100000) // 0 = disabled separateEngineBlocks = util.ConstantWithMetamorphicTestBool("mvcc-histories-separate-engine-blocks", false) @@ -984,8 +987,9 @@ func cmdClearTimeRange(e *evalCtx) error { batch := e.engine.NewBatch() defer batch.Close() - resume, err := storage.MVCCClearTimeRange(e.ctx, batch, e.ms, key, endKey, targetTs, ts, - nil, nil, clearRangeThreshold, int64(maxBatchSize), int64(maxBatchByteSize)) + rw, leftPeekBound, rightPeekBound := e.metamorphicPeekBounds(batch, key, endKey) + resume, err := storage.MVCCClearTimeRange(e.ctx, rw, e.ms, key, endKey, targetTs, ts, + leftPeekBound, rightPeekBound, clearRangeThreshold, int64(maxBatchSize), int64(maxBatchByteSize)) if err != nil { return err } @@ -1132,8 +1136,9 @@ func cmdDeleteRangeTombstone(e *evalCtx) error { } return e.withWriter("del_range_ts", func(rw storage.ReadWriter) error { - return storage.MVCCDeleteRangeUsingTombstone( - e.ctx, rw, e.ms, key, endKey, ts, localTs, nil, nil, idempotent, 0, msCovered) + rw, leftPeekBound, rightPeekBound := e.metamorphicPeekBounds(rw, key, endKey) + return storage.MVCCDeleteRangeUsingTombstone(e.ctx, rw, e.ms, key, endKey, ts, localTs, + leftPeekBound, rightPeekBound, idempotent, 0, msCovered) }) } @@ -1159,8 +1164,9 @@ func cmdDeleteRangePredicate(e *evalCtx) error { e.scanArg("rangeThreshold", &rangeThreshold) } return e.withWriter("del_range_pred", func(rw storage.ReadWriter) error { - resumeSpan, err := storage.MVCCPredicateDeleteRange(e.ctx, rw, e.ms, key, endKey, ts, - localTs, nil, nil, predicates, int64(max), int64(maxBytes), int64(rangeThreshold), 0) + rw, leftPeekBound, rightPeekBound := e.metamorphicPeekBounds(rw, key, endKey) + resumeSpan, err := storage.MVCCPredicateDeleteRange(e.ctx, rw, e.ms, key, endKey, ts, localTs, + leftPeekBound, rightPeekBound, predicates, int64(max), int64(maxBytes), int64(rangeThreshold), 0) if resumeSpan != nil { e.results.buf.Printf("del_range_pred: resume span [%s,%s)\n", resumeSpan.Key, @@ -2343,6 +2349,46 @@ func (e *evalCtx) iterErr() error { return nil } +// metamorphicPeekBounds generates MVCC range key peek bounds for a command +// based on its keyspan, and enables spanset assertions for the ReadWriter. +func (e *evalCtx) metamorphicPeekBounds( + rw storage.ReadWriter, start, end roachpb.Key, +) (storage.ReadWriter, roachpb.Key, roachpb.Key) { + leftPeekBound, rightPeekBound := start.Prevish(8), end.Next() + if end == nil { + rightPeekBound = nil + } + + switch mvccHistoriesPeekBounds { + case "none": + leftPeekBound, rightPeekBound = nil, nil + case "left": + rightPeekBound = nil + case "right": + leftPeekBound = nil + case "both": + default: + e.t.Fatalf("invalid peek bound kind %q", mvccHistoriesPeekBounds) + return nil, nil, nil + } + + if leftPeekBound != nil || rightPeekBound != nil { + ss := &spanset.SpanSet{} + ss.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: start, EndKey: end}) + peekSpan := roachpb.Span{Key: leftPeekBound, EndKey: rightPeekBound} + if peekSpan.Key == nil { + peekSpan.Key = keys.LocalMax + } + if peekSpan.EndKey == nil { + peekSpan.EndKey = keys.MaxKey + } + ss.AddNonMVCC(spanset.SpanReadOnly, peekSpan) + rw = spanset.NewReadWriterAt(rw, ss, hlc.Timestamp{}) + } + + return rw, leftPeekBound, rightPeekBound +} + func toKey(s string) roachpb.Key { if len(s) == 0 { return roachpb.Key(s) From ded67670ce69c180f65237e3e6960cc9594a42a3 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 23 Sep 2022 12:24:40 +0000 Subject: [PATCH 4/5] storage: use `del_range_ts` in `TestMVCCHistories` Many of the early `TestMVCCHistories` tests for MVCC range tombstones used the direct engine method `put_rangekey` rather than the MVCC range tombstone write `del_range_ts` (which does conflict checks, MVCC stats adjustments, etc), because the latter did not exist yet. This patch migrates most tests to `del_range_ts`, and also renames some of the test files to refer to range tombstone rather than range key. Stats assertions have also been enabled for some tests. There are no significant changes to the tests themselves. Release note: None --- .../testdata/mvcc_histories/clear_time_range | 40 +++---- .../mvcc_histories/clear_time_range_limits | 60 +++++------ pkg/storage/testdata/mvcc_histories/export | 10 +- ...ge_key_iter_read_as_of => iter_read_as_of} | 102 ++++++++++++------ .../testdata/mvcc_histories/range_key_clear | 2 +- ...thout_provisional_norace_nometamorphiciter | 2 + .../testdata/mvcc_histories/range_key_put | 2 +- .../range_tombstone_gets_complex | 67 ++++++++++-- .../{range_key_iter => range_tombstone_iter} | 63 +++++++++-- ...ental => range_tombstone_iter_incremental} | 63 +++++++++-- ...ge_tombstone_iter_nextkey_null_regression} | 8 +- ...s => range_tombstone_iter_point_synthesis} | 40 +++---- .../range_tombstone_scans_complex | 67 ++++++++++-- 13 files changed, 379 insertions(+), 147 deletions(-) rename pkg/storage/testdata/mvcc_histories/{range_key_iter_read_as_of => iter_read_as_of} (57%) rename pkg/storage/testdata/mvcc_histories/{range_key_iter => range_tombstone_iter} (95%) rename pkg/storage/testdata/mvcc_histories/{range_key_iter_incremental => range_tombstone_iter_incremental} (96%) rename pkg/storage/testdata/mvcc_histories/{range_key_iter_nextkey_null_regression => range_tombstone_iter_nextkey_null_regression} (84%) rename pkg/storage/testdata/mvcc_histories/{range_key_point_synthesis => range_tombstone_iter_point_synthesis} (97%) diff --git a/pkg/storage/testdata/mvcc_histories/clear_time_range b/pkg/storage/testdata/mvcc_histories/clear_time_range index afdd3a4a586a..044d0437b6e1 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_time_range +++ b/pkg/storage/testdata/mvcc_histories/clear_time_range @@ -14,25 +14,25 @@ # Clear the entire span. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -74,25 +74,25 @@ stats: # Clear individual timestamps 7-1. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -262,25 +262,25 @@ stats: # Clear individual keys a-n for times 3-6. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -830,25 +830,25 @@ stats: # Clear a few arbitrary keys and arbitrary timestamps. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false diff --git a/pkg/storage/testdata/mvcc_histories/clear_time_range_limits b/pkg/storage/testdata/mvcc_histories/clear_time_range_limits index 2e2671f5eaa2..cc4ca0b9b77e 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_time_range_limits +++ b/pkg/storage/testdata/mvcc_histories/clear_time_range_limits @@ -23,25 +23,25 @@ # Clear the entire span using clearRangeThreshold=1, clearRangeThreshold=2, and # clearRangeThreshold=1000. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -82,25 +82,25 @@ stats: key_count=-8 key_bytes=-184 val_count=-14 val_bytes=-63 range_key_count=- stats: run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -141,25 +141,25 @@ stats: key_count=-8 key_bytes=-184 val_count=-14 val_bytes=-63 range_key_count=- stats: run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -201,25 +201,25 @@ stats: # Clear the entire span using clearRangeThreshold=1000 and maxBatchSize=3. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -339,25 +339,25 @@ stats: # Clear timestamps 4-6 using clearRangeThreshold=1000 and maxBatchSize=3, # and then the remainder using maxBatchByteSize=1. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false @@ -547,25 +547,25 @@ stats: # Clear the entire span using clearRangeThreshold=1000 and maxBatchByteSize=1. run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 put k=d ts=2 v=d2 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 del k=f ts=3 put k=g ts=2 v=g2 -put_rangekey k=g end=i ts=3 +del_range_ts k=g end=i ts=3 del k=g ts=4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 put k=f ts=6 v=f6 put k=i ts=3 v=i3 del k=i ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- del: "a": found key true del: "b": found key false diff --git a/pkg/storage/testdata/mvcc_histories/export b/pkg/storage/testdata/mvcc_histories/export index 27df7b08cb16..398efe94ac11 100644 --- a/pkg/storage/testdata/mvcc_histories/export +++ b/pkg/storage/testdata/mvcc_histories/export @@ -14,24 +14,24 @@ # a b c d e f g h i j k l m n o run ok -put_rangekey k=a end=k ts=1 +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 localTs=2 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 +del_range_ts k=f end=h ts=3 put k=f ts=4 v=f4 put k=g ts=4 v=g4 -put_rangekey k=c end=g ts=5 localTs=4 +del_range_ts k=c end=g ts=5 localTs=4 put k=f ts=6 v=f6 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 localTs=4 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 with t=A txn_begin ts=7 put k=a v=a7 diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of b/pkg/storage/testdata/mvcc_histories/iter_read_as_of similarity index 57% rename from pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of rename to pkg/storage/testdata/mvcc_histories/iter_read_as_of index b669ff8a8980..5d8f5179a06e 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_read_as_of +++ b/pkg/storage/testdata/mvcc_histories/iter_read_as_of @@ -1,7 +1,6 @@ -# Tests range key handling in ReadAsOfIterator. Note that the iterator assumes it will not see an -# intent. +# Tests ReadAsOfIterator. # -# Sets up the following dataset, where x is tombstone, o-o is range tombstone +# Sets up the following dataset, where x is tombstone, o-o is range tombstone: # # 6 f6 # 5 o---------------o k5 @@ -11,29 +10,65 @@ # 1 o---------------------------------------o # a b c d e f g h i j k l m n o # -run ok -put_rangekey k=a end=k ts=1 +run stats ok +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 +del_range_ts k=f end=h ts=3 put k=f ts=4 v=f4 -put_rangekey k=c end=g ts=5 +del_range_ts k=c end=g ts=5 put k=f ts=6 v=f6 put k=g ts=4 v=g4 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 ---- +>> del_range_ts k=a end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del k=b ts=4 del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=4 v=d4 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 live_count=-2 live_bytes=-42 gc_bytes_age=+7513 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=g ts=5 +stats: range_key_count=+2 range_key_bytes=+71 range_val_count=+7 live_count=-3 live_bytes=-63 gc_bytes_age=+12822 +>> 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=-190 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=4 del: "h": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> put k=k ts=5 v=k5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=m end=n ts=3 localTs=2 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2522 >> at end: rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] @@ -56,8 +91,29 @@ data: "g"/2.000000000,0 -> /BYTES/g2 data: "h"/4.000000000,0 -> / data: "h"/3.000000000,0 -> /BYTES/h3 data: "k"/5.000000000,0 -> /BYTES/k5 +stats: key_count=8 key_bytes=172 val_count=13 val_bytes=70 range_key_count=8 range_key_bytes=167 range_val_count=15 range_val_bytes=13 live_count=3 live_bytes=63 gc_bytes_age=34685 + +# Scan with no asOf timestamp. +run ok +iter_new_read_as_of +iter_seek_ge k=a +iter_scan +---- +iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 +iter_scan: "f"/6.000000000,0=/BYTES/f6 +iter_scan: "g"/4.000000000,0=/BYTES/g4 +iter_scan: "k"/5.000000000,0=/BYTES/k5 +iter_scan: . + +# Run scans as of every timestamp. +run ok +iter_new_read_as_of asOfTs=1 +iter_seek_ge k=a +iter_scan +---- +iter_seek_ge: . +iter_scan: . -# test range keys are ignored if above asOf, even with multiple range keys run ok iter_new_read_as_of asOfTs=2 iter_seek_ge k=a @@ -69,7 +125,6 @@ iter_scan: "f"/2.000000000,0=/BYTES/f2 iter_scan: "g"/2.000000000,0=/BYTES/g2 iter_scan: . -# test range key at or below asOf properly shadows keys run ok iter_new_read_as_of asOfTs=3 iter_seek_ge k=a @@ -81,7 +136,6 @@ iter_scan: "e"/3.000000000,0=/BYTES/e3 iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: . -# iterate over a few point tombstones at the asOf time run ok iter_new_read_as_of asOfTs=4 iter_seek_ge k=a @@ -94,7 +148,6 @@ iter_scan: "f"/4.000000000,0=/BYTES/f4 iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: . -# iterate over ts 5-7 because the test is cheap run ok iter_new_read_as_of asOfTs=5 iter_seek_ge k=a @@ -105,7 +158,6 @@ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: . -# iterate over ts 5-7 because the test is cheap run ok iter_new_read_as_of asOfTs=6 iter_seek_ge k=a @@ -117,7 +169,6 @@ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: . -# iterate over ts 5-7 for completeness run ok iter_new_read_as_of asOfTs=7 iter_seek_ge k=a @@ -129,41 +180,28 @@ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "k"/5.000000000,0=/BYTES/k5 iter_scan: . - -# test range key handling when asOf is empty -run ok -iter_new_read_as_of -iter_seek_ge k=a -iter_scan ----- -iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 -iter_scan: "f"/6.000000000,0=/BYTES/f6 -iter_scan: "g"/4.000000000,0=/BYTES/g4 -iter_scan: "k"/5.000000000,0=/BYTES/k5 -iter_scan: . - -# seek to a point key shadowed by a range key +# Seek to a point key shadowed by a range tombstone. run ok iter_new_read_as_of asOfTs=5 iter_seek_ge k=d ---- iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 -# seek to the start of a range key +# Seek to the start of a range tombstone. run ok iter_new_read_as_of asOfTs=5 iter_seek_ge k=c ---- iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 -# seek to the same point key, with AsOf empty +# Seek to the same point key, without asOf. run ok iter_new_read_as_of iter_seek_ge k=d ---- iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 -# attempt seek to the same point key, but ignore the range key because its above AsOf +# Attempt seek to the same point key, ignoring the above range tombstone. run ok iter_new_read_as_of asOfTs=4 iter_seek_ge k=d diff --git a/pkg/storage/testdata/mvcc_histories/range_key_clear b/pkg/storage/testdata/mvcc_histories/range_key_clear index a9bde12c2a11..0801feb2f96e 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_clear +++ b/pkg/storage/testdata/mvcc_histories/range_key_clear @@ -1,4 +1,4 @@ -# Tests MVCC range key clearing. +# Tests MVCC range key clearing, using the engine directly. # # Sets up following dataset, where x is tombstone, o-o is range tombstone, [] is intent. # diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_intent_without_provisional_norace_nometamorphiciter b/pkg/storage/testdata/mvcc_histories/range_key_iter_intent_without_provisional_norace_nometamorphiciter index 48fdb1cdb8ed..bdf29a1c546f 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_intent_without_provisional_norace_nometamorphiciter +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter_intent_without_provisional_norace_nometamorphiciter @@ -2,6 +2,8 @@ # values. The initial setup will fail when it's scanning the data afterwards, # but the data is still written so that's fine. # +# The test uses put_rangekey directly on the engine to omit conflict checks. +# # This test does not run under race, because the error behavior differs due # to additional (costly) assertions. We want to test that we are reasonably # resistant to invariant violations even when not under race. diff --git a/pkg/storage/testdata/mvcc_histories/range_key_put b/pkg/storage/testdata/mvcc_histories/range_key_put index 622d175786db..e53b925397ca 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_put +++ b/pkg/storage/testdata/mvcc_histories/range_key_put @@ -1,4 +1,4 @@ -# Test basic MVCC range key mutations. +# Test basic MVCC range key mutations, using the engine directly. run trace # Write three range keys that extend each other on both sides. diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets_complex b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets_complex index 274f6e6e9608..00b817931319 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets_complex +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets_complex @@ -12,12 +12,12 @@ # 1 o-------------------o o-----------o # a b c d e f g h i j k l m n o p # -run ok -put_rangekey k=a end=f ts=1 -put_rangekey k=h end=k ts=1 -put_rangekey k=b end=d ts=3 -put_rangekey k=n end=o ts=3 -put_rangekey k=l end=o ts=5 +run stats ok +del_range_ts k=a end=f ts=1 +del_range_ts k=h end=k ts=1 +del_range_ts k=b end=d ts=3 +del_range_ts k=n end=o ts=3 +del_range_ts k=l end=o ts=5 put k=a ts=2 v=a2 del k=a ts=4 del k=b ts=4 @@ -26,11 +26,11 @@ put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 localTs=4 +del_range_ts k=f end=h ts=3 localTs=4 put k=f ts=4 v=f4 -put k=f ts=6 v=f6 put k=g ts=4 v=g4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 +put k=f ts=6 v=f6 put k=h ts=3 v=h3 put k=k ts=5 v=k5 with t=A @@ -39,15 +39,61 @@ with t=A put k=f v=f7 put k=j v=j7 ---- +>> del_range_ts k=a end=f ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=h end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del_range_ts k=n end=o ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1261 +>> del_range_ts k=l end=o ts=5 +stats: range_key_count=+1 range_key_bytes=+22 range_val_count=+2 gc_bytes_age=+2082 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del k=b ts=4 del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=2 v=d2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=d ts=4 v=d4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 gc_bytes_age=+1824 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 localTs=4 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=h ts=5 +stats: range_key_count=+1 range_key_bytes=+49 range_val_count=+5 live_count=-4 live_bytes=-84 gc_bytes_age=+12665 +>> 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=-190 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=k ts=5 v=k5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=d v=d7 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=+93 +>> put k=f v=f7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=j v=j7 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=+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-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] -rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0={localTs=4.000000000,0}/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] rangekey: {h-k}/[1.000000000,0=/] rangekey: {l-n}/[5.000000000,0=/] rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] @@ -70,6 +116,7 @@ data: "h"/3.000000000,0 -> /BYTES/h3 meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "j"/7.000000000,0 -> /BYTES/j7 data: "k"/5.000000000,0 -> /BYTES/k5 +stats: key_count=9 key_bytes=210 val_count=16 val_bytes=242 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=249 gc_bytes_age=34691 intent_count=3 intent_bytes=57 separated_intent_count=3 intent_age=279 # Run gets for all keys at all timestamps, with tombstones and intents. run ok diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter similarity index 95% rename from pkg/storage/testdata/mvcc_histories/range_key_iter rename to pkg/storage/testdata/mvcc_histories/range_tombstone_iter index ca429f91f01b..f1ac69d8c57a 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter @@ -1,4 +1,4 @@ -# Tests range key handling in MVCC iterators. +# Tests range tombstone handling in MVCC iterators. # # Sets up the following dataset, where x is tombstone, o-o is range tombstone, [] is intent. # @@ -12,25 +12,25 @@ # 1 o---------------------------------------o # a b c d e f g h i j k l m n o # -run ok -put_rangekey k=a end=k ts=1 +run stats ok +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 +del_range_ts k=f end=h ts=3 put k=f ts=4 v=f4 put k=g ts=4 v=g4 -put_rangekey k=c end=g ts=5 +del_range_ts k=c end=g ts=5 put k=f ts=6 v=f6 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 with t=A txn_begin ts=7 put k=a v=a7 @@ -40,9 +40,57 @@ with t=A put k=m v=l7 put k=o v=n7 ---- +>> del_range_ts k=a end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del k=b ts=4 del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=4 v=d4 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 live_count=-2 live_bytes=-42 gc_bytes_age=+7513 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=g ts=5 +stats: range_key_count=+2 range_key_bytes=+71 range_val_count=+7 live_count=-3 live_bytes=-63 gc_bytes_age=+12822 +>> 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=-190 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=4 del: "h": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> put k=k ts=5 v=k5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=m end=n ts=3 localTs=2 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2522 +>> put k=a v=a7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-192 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=d v=d7 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=+93 +>> put k=j v=j7 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=+93 +>> put k=l v=l7 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=+93 +>> put k=m v=l7 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=+93 +>> put k=o v=n7 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=+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-b}/[1.000000000,0=/] @@ -78,6 +126,7 @@ meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 data: "m"/7.000000000,0 -> /BYTES/l7 meta: "o"/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: "o"/7.000000000,0 -> /BYTES/n7 +stats: key_count=12 key_bytes=252 val_count=19 val_bytes=400 range_key_count=8 range_key_bytes=167 range_val_count=15 range_val_bytes=13 live_count=9 live_bytes=477 gc_bytes_age=34303 intent_count=6 intent_bytes=114 separated_intent_count=6 intent_age=558 # Iterate across the entire span for all key types, and without intents. run ok diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental similarity index 96% rename from pkg/storage/testdata/mvcc_histories/range_key_iter_incremental rename to pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental index 94be0cf80003..9843aa830673 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_incremental +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_incremental @@ -1,4 +1,4 @@ -# Tests range key handling in MVCCIncrementalIterator. It does not support +# Tests range tombstone handling in MVCCIncrementalIterator. It does not support # reverse iteration, as it does not implement MVCCIterator. # # Sets up the following dataset, where x is tombstone, o-o is range tombstone, [] is intent. @@ -13,25 +13,25 @@ # 1 o---------------------------------------o # a b c d e f g h i j k l m n o # -run ok -put_rangekey k=a end=k ts=1 +run stats ok +del_range_ts k=a end=k ts=1 put k=a ts=2 v=a2 del k=a ts=4 -put_rangekey k=b end=d ts=3 +del_range_ts k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 +del_range_ts k=f end=h ts=3 put k=f ts=4 v=f4 -put_rangekey k=c end=g ts=5 +del_range_ts k=c end=g ts=5 put k=f ts=6 v=f6 put k=g ts=4 v=g4 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 -put_rangekey k=m end=n ts=3 localTs=2 +del_range_ts k=m end=n ts=3 localTs=2 with t=A txn_begin ts=7 put k=a v=a7 @@ -43,9 +43,57 @@ with t=B put k=d v=d8 put k=m v=m8 ---- +>> del_range_ts k=a end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del k=b ts=4 del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=4 v=d4 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 live_count=-2 live_bytes=-42 gc_bytes_age=+7513 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=g ts=5 +stats: range_key_count=+2 range_key_bytes=+71 range_val_count=+7 live_count=-3 live_bytes=-63 gc_bytes_age=+12822 +>> 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=-190 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=h ts=4 del: "h": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> put k=k ts=5 v=k5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=m end=n ts=3 localTs=2 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 range_val_bytes=+13 gc_bytes_age=+2522 +>> put k=a v=a7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-192 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=j v=j7 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=+93 +>> put k=l v=l7 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=+93 +>> put k=o v=o7 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=+93 +>> put k=d v=d8 t=B +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=+92 +>> put k=m v=m8 t=B +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=+92 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=8.000000000,0 wto=false gul=0,0 rangekey: {a-b}/[1.000000000,0=/] @@ -81,6 +129,7 @@ meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=8.000000000,0 data: "m"/8.000000000,0 -> /BYTES/m8 meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +stats: key_count=12 key_bytes=252 val_count=19 val_bytes=400 range_key_count=8 range_key_bytes=167 range_val_count=15 range_val_bytes=13 live_count=9 live_bytes=477 gc_bytes_age=34303 intent_count=6 intent_bytes=114 separated_intent_count=6 intent_age=556 # Iterate across the entire span for all key types. run ok diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter_nextkey_null_regression b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_nextkey_null_regression similarity index 84% rename from pkg/storage/testdata/mvcc_histories/range_key_iter_nextkey_null_regression rename to pkg/storage/testdata/mvcc_histories/range_tombstone_iter_nextkey_null_regression index 5c76cfbf97b6..aa33439fbb78 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter_nextkey_null_regression +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_nextkey_null_regression @@ -1,11 +1,11 @@ -# This is a contrived scenario which makes sure NextKey() handles range keys -# starting at key.Next() correctly. +# This is a contrived scenario which makes sure NextKey() handles range +# tombstones starting at key.Next() correctly. run ok put k=a ts=2 v=a2 put k=a ts=3 v=a3 put k=b ts=1 v=b1 -put_rangekey k=a end=+a ts=5 -put_rangekey k=+a end=b ts=4 +del_range_ts k=a end=+a ts=5 +del_range_ts k=+a end=b ts=4 ---- >> at end: rangekey: a{-\x00}/[5.000000000,0=/] diff --git a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_point_synthesis similarity index 97% rename from pkg/storage/testdata/mvcc_histories/range_key_point_synthesis rename to pkg/storage/testdata/mvcc_histories/range_tombstone_iter_point_synthesis index 6b885f705e24..e025578e4c80 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_iter_point_synthesis @@ -13,11 +13,11 @@ # a b c d e f g h i j k l m n o p # run ok -put_rangekey k=a end=f ts=1 -put_rangekey k=h end=k ts=1 -put_rangekey k=b end=d ts=3 -put_rangekey k=n end=o ts=3 -put_rangekey k=l end=o ts=5 +del_range_ts k=a end=f ts=1 +del_range_ts k=h end=k ts=1 +del_range_ts k=b end=d ts=3 +del_range_ts k=n end=o ts=3 +del_range_ts k=l end=o ts=5 put k=a ts=2 v=a2 del k=a ts=4 del k=b ts=4 @@ -26,11 +26,11 @@ put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 localTs=4 +del_range_ts k=f end=h ts=3 localTs=4 put k=f ts=4 v=f4 -put k=f ts=6 v=f6 put k=g ts=4 v=g4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 +put k=f ts=6 v=f6 put k=h ts=3 v=h3 put k=k ts=5 v=k5 with t=A @@ -46,7 +46,7 @@ rangekey: {a-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] -rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0={localTs=4.000000000,0}/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] rangekey: {h-k}/[1.000000000,0=/] rangekey: {l-n}/[5.000000000,0=/] rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] @@ -95,11 +95,11 @@ iter_scan: "e"/3.000000000,0=/BYTES/e3 iter_scan: "f"/6.000000000,0=/BYTES/f6 iter_scan: "f"/5.000000000,0=/ iter_scan: "f"/4.000000000,0=/BYTES/f4 -iter_scan: "f"/3.000000000,0={localTs=4.000000000,0}/ +iter_scan: "f"/3.000000000,0=/ iter_scan: "f"/2.000000000,0=/BYTES/f2 iter_scan: "g"/5.000000000,0=/ iter_scan: "g"/4.000000000,0=/BYTES/g4 -iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_scan: "g"/3.000000000,0=/ iter_scan: "g"/2.000000000,0=/BYTES/g2 iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ @@ -126,11 +126,11 @@ iter_scan: "j"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000 iter_scan: "h"/1.000000000,0=/ iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "g"/2.000000000,0=/BYTES/g2 -iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_scan: "g"/3.000000000,0=/ iter_scan: "g"/4.000000000,0=/BYTES/g4 iter_scan: "g"/5.000000000,0=/ iter_scan: "f"/2.000000000,0=/BYTES/f2 -iter_scan: "f"/3.000000000,0={localTs=4.000000000,0}/ +iter_scan: "f"/3.000000000,0=/ iter_scan: "f"/4.000000000,0=/BYTES/f4 iter_scan: "f"/5.000000000,0=/ iter_scan: "f"/6.000000000,0=/BYTES/f6 @@ -452,7 +452,7 @@ iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 iter_seek_ge: "f"/6.000000000,0=/BYTES/f6 iter_seek_ge: "f"/5.000000000,0=/ iter_seek_ge: "f"/4.000000000,0=/BYTES/f4 -iter_seek_ge: "f"/3.000000000,0={localTs=4.000000000,0}/ +iter_seek_ge: "f"/3.000000000,0=/ iter_seek_ge: "f"/2.000000000,0=/BYTES/f2 iter_seek_ge: "g"/5.000000000,0=/ @@ -468,7 +468,7 @@ iter_seek_ge k=g ts=1 iter_seek_ge: "g"/5.000000000,0=/ iter_seek_ge: "g"/5.000000000,0=/ iter_seek_ge: "g"/4.000000000,0=/BYTES/g4 -iter_seek_ge: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_seek_ge: "g"/3.000000000,0=/ iter_seek_ge: "g"/2.000000000,0=/BYTES/g2 iter_seek_ge: "h"/3.000000000,0=/BYTES/h3 @@ -723,7 +723,7 @@ iter_seek_lt k=f ts=6 iter_seek_lt k=f ts=7 ---- iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 -iter_seek_lt: "f"/3.000000000,0={localTs=4.000000000,0}/ +iter_seek_lt: "f"/3.000000000,0=/ iter_seek_lt: "f"/4.000000000,0=/BYTES/f4 iter_seek_lt: "f"/5.000000000,0=/ iter_seek_lt: "f"/6.000000000,0=/BYTES/f6 @@ -740,7 +740,7 @@ iter_seek_lt k=g ts=5 iter_seek_lt k=g ts=6 ---- iter_seek_lt: "g"/2.000000000,0=/BYTES/g2 -iter_seek_lt: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_seek_lt: "g"/3.000000000,0=/ iter_seek_lt: "g"/4.000000000,0=/BYTES/g4 iter_seek_lt: "g"/5.000000000,0=/ iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 @@ -888,7 +888,7 @@ iter_seek_lt: "f"/2.000000000,0=/BYTES/f2 iter_scan: "f"/2.000000000,0=/BYTES/f2 iter_scan: "g"/5.000000000,0=/ iter_scan: "g"/4.000000000,0=/BYTES/g4 -iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_scan: "g"/3.000000000,0=/ iter_scan: "g"/2.000000000,0=/BYTES/g2 iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ @@ -905,8 +905,8 @@ iter_new types=pointsAndRanges pointSynthesis iter_seek_lt k=g ts=2 iter_scan ---- -iter_seek_lt: "g"/3.000000000,0={localTs=4.000000000,0}/ -iter_scan: "g"/3.000000000,0={localTs=4.000000000,0}/ +iter_seek_lt: "g"/3.000000000,0=/ +iter_scan: "g"/3.000000000,0=/ iter_scan: "g"/2.000000000,0=/BYTES/g2 iter_scan: "h"/3.000000000,0=/BYTES/h3 iter_scan: "h"/1.000000000,0=/ diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans_complex b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans_complex index 125b7b029765..9fceedd2aabf 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans_complex +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans_complex @@ -12,12 +12,12 @@ # 1 o-------------------o o-----------o # a b c d e f g h i j k l m n o p # -run ok -put_rangekey k=a end=f ts=1 -put_rangekey k=h end=k ts=1 -put_rangekey k=b end=d ts=3 -put_rangekey k=n end=o ts=3 -put_rangekey k=l end=o ts=5 +run stats ok +del_range_ts k=a end=f ts=1 +del_range_ts k=h end=k ts=1 +del_range_ts k=b end=d ts=3 +del_range_ts k=n end=o ts=3 +del_range_ts k=l end=o ts=5 put k=a ts=2 v=a2 del k=a ts=4 del k=b ts=4 @@ -26,11 +26,11 @@ put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 put k=g ts=2 v=g2 -put_rangekey k=f end=h ts=3 localTs=4 +del_range_ts k=f end=h ts=3 localTs=4 put k=f ts=4 v=f4 -put k=f ts=6 v=f6 put k=g ts=4 v=g4 -put_rangekey k=c end=h ts=5 +del_range_ts k=c end=h ts=5 +put k=f ts=6 v=f6 put k=h ts=3 v=h3 put k=k ts=5 v=k5 with t=A @@ -39,15 +39,61 @@ with t=A put k=f v=f7 put k=j v=j7 ---- +>> del_range_ts k=a end=f ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=h end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del_range_ts k=n end=o ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1261 +>> del_range_ts k=l end=o ts=5 +stats: range_key_count=+1 range_key_bytes=+22 range_val_count=+2 gc_bytes_age=+2082 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del k=b ts=4 del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=2 v=d2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=d ts=4 v=d4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 gc_bytes_age=+1824 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 localTs=4 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=h ts=5 +stats: range_key_count=+1 range_key_bytes=+49 range_val_count=+5 live_count=-4 live_bytes=-84 gc_bytes_age=+12665 +>> 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=-190 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=k ts=5 v=k5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=d v=d7 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=+93 +>> put k=f v=f7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=j v=j7 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=+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-b}/[1.000000000,0=/] rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] -rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0={localTs=4.000000000,0}/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] rangekey: {h-k}/[1.000000000,0=/] rangekey: {l-n}/[5.000000000,0=/] rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] @@ -70,6 +116,7 @@ data: "h"/3.000000000,0 -> /BYTES/h3 meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "j"/7.000000000,0 -> /BYTES/j7 data: "k"/5.000000000,0 -> /BYTES/k5 +stats: key_count=9 key_bytes=210 val_count=16 val_bytes=242 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=249 gc_bytes_age=34691 intent_count=3 intent_bytes=57 separated_intent_count=3 intent_age=279 # Forward scans at all timestamps. run ok From aaac54c6aec66957df5f1b2a9181b6ba9f9f89df Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 23 Sep 2022 12:40:51 +0000 Subject: [PATCH 5/5] storage: add `TestMVCCHistories` cases This patch adds additional test cases for `TestMVCCHistories`. These were primarily designed by deliberately introducing bugs in MVCC code that did not cause existing test cases to fail. Release note: None --- .../mvcc_histories/delete_range_predicate | 27 +- .../delete_range_predicate_complex | 606 ++++++++++++++++++ ...range_tombstone_stats_intent_resolve_abort | 325 ++++++---- ...ange_tombstone_stats_intent_resolve_commit | 377 +++++++---- ...ange_tombstone_stats_intent_resolve_pushed | 385 ++++++----- ...tombstone_stats_intent_resolve_range_abort | 274 ++++---- ...ombstone_stats_intent_resolve_range_commit | 326 ++++++---- ...ombstone_stats_intent_resolve_range_pushed | 334 ++++++---- .../range_tombstone_stats_intent_rewrite | 557 +++++++++------- .../range_tombstone_writes_idempotent | 43 ++ 10 files changed, 2237 insertions(+), 1017 deletions(-) create mode 100644 pkg/storage/testdata/mvcc_histories/delete_range_predicate_complex diff --git a/pkg/storage/testdata/mvcc_histories/delete_range_predicate b/pkg/storage/testdata/mvcc_histories/delete_range_predicate index 043f1368e22c..fab880c3ae7b 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range_predicate +++ b/pkg/storage/testdata/mvcc_histories/delete_range_predicate @@ -70,7 +70,8 @@ meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 data: "i"/7.000000000,0 -> /BYTES/i7 stats: key_count=8 key_bytes=160 val_count=12 val_bytes=111 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=3 live_bytes=111 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 -# error on intent, no tombstones should be written +# Error on intent, no tombstones should be written. We try both the +# point tombstone and range tombstone paths. run stats error del_range_pred k=a end=p ts=6 startTime=1 ---- @@ -95,6 +96,30 @@ data: "i"/7.000000000,0 -> /BYTES/i7 stats: key_count=8 key_bytes=160 val_count=12 val_bytes=111 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=3 live_bytes=111 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 error: (*roachpb.WriteIntentError:) conflicting intents on "i" +run stats error +del_range_pred k=i end=+i ts=6 startTime=1 +---- +>> del_range_pred k=i end=+i ts=6 startTime=1 +stats: no change +>> at end: +rangekey: {f-h\x00}/[5.000000000,0=/] +rangekey: {k-p}/[4.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/3.000000000,0 -> /BYTES/b3 +data: "d"/5.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> /BYTES/e2 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/4.000000000,0 -> /BYTES/h4 +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +stats: key_count=8 key_bytes=160 val_count=12 val_bytes=111 range_key_count=2 range_key_bytes=27 range_val_count=2 live_count=3 live_bytes=111 gc_bytes_age=17863 intent_count=1 intent_bytes=19 separated_intent_count=1 intent_age=93 +error: (*roachpb.WriteIntentError:) conflicting intents on "i" + # error encountering point key at d5. # a tombstone should not get written at c5 or e5, since # DeleteRange didn't flush before reaching d5. diff --git a/pkg/storage/testdata/mvcc_histories/delete_range_predicate_complex b/pkg/storage/testdata/mvcc_histories/delete_range_predicate_complex new file mode 100644 index 000000000000..406238d482dc --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/delete_range_predicate_complex @@ -0,0 +1,606 @@ +# Tests MVCCPredicateDeleteRange with a more complex dataset. +# +# Sets up the following dataset, where x is tombstone, o-o is range tombstone, [] is intent. +# +# T +# 6 f6 x p6 +# 5 o-------------------o i5 o-----------o +# 4 x x d4 f4 g4 +# 3 o-------o e3 o-------oh3 o---o +# 2 a2 d2 f2 g2 j2 k2 +# 1 o-------------------o o-----------o +# a b c d e f g h i j k l m n o p +# +run stats ok +del_range_ts k=a end=f ts=1 +del_range_ts k=h end=k ts=1 +del_range_ts k=b end=d ts=3 +del_range_ts k=n end=o ts=3 +del_range_ts k=l end=o ts=5 +put k=a ts=2 v=a2 +del k=a ts=4 +del k=b ts=4 +put k=d ts=2 v=d2 +put k=d ts=4 v=d4 +put k=e ts=3 v=e3 +put k=f ts=2 v=f2 +put k=g ts=2 v=g2 +put k=i ts=5 v=i5 +put k=j ts=2 v=j2 +del k=j ts=6 +put k=k ts=2 v=k2 +del_range_ts k=f end=h ts=3 localTs=4 +put k=f ts=4 v=f4 +put k=g ts=4 v=g4 +del_range_ts k=c end=h ts=5 +put k=f ts=6 v=f6 +put k=h ts=3 v=h3 +put k=p ts=6 v=p6 +---- +>> del_range_ts k=a end=f ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=h end=k ts=1 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1287 +>> del_range_ts k=b end=d ts=3 +stats: range_key_count=+2 range_key_bytes=+35 range_val_count=+3 gc_bytes_age=+3439 +>> del_range_ts k=n end=o ts=3 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 gc_bytes_age=+1261 +>> del_range_ts k=l end=o ts=5 +stats: range_key_count=+1 range_key_bytes=+22 range_val_count=+2 gc_bytes_age=+2082 +>> put k=a ts=2 v=a2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=a ts=4 +del: "a": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3168 +>> del k=b ts=4 +del: "b": found key false +stats: key_count=+1 key_bytes=+14 val_count=+1 gc_bytes_age=+1344 +>> put k=d ts=2 v=d2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=d ts=4 v=d4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 gc_bytes_age=+1824 +>> put k=e ts=3 v=e3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=f ts=2 v=f2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=g ts=2 v=g2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=i ts=5 v=i5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=j ts=2 v=j2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del k=j ts=6 +del: "j": found key true +stats: key_bytes=+12 val_count=+1 live_count=-1 live_bytes=-21 gc_bytes_age=+3102 +>> put k=k ts=2 v=k2 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> del_range_ts k=f end=h ts=3 localTs=4 +stats: range_key_count=+1 range_key_bytes=+13 range_val_count=+1 live_count=-2 live_bytes=-42 gc_bytes_age=+5335 +>> put k=f ts=4 v=f4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> put k=g ts=4 v=g4 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> del_range_ts k=c end=h ts=5 +stats: range_key_count=+1 range_key_bytes=+49 range_val_count=+5 live_count=-4 live_bytes=-84 gc_bytes_age=+12665 +>> 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=-190 +>> put k=h ts=3 v=h3 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> put k=p ts=6 v=p6 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Delete the entire span, using both point and range tombstones. +run stats ok +del_range_pred k=a end=z ts=10 startTime=0 rangeThreshold=10 +---- +>> del_range_pred k=a end=z ts=10 startTime=0 rangeThreshold=10 +stats: key_bytes=+60 val_count=+5 live_count=-5 live_bytes=-105 gc_bytes_age=+14850 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/10.000000000,0 -> / +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/10.000000000,0 -> / +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/10.000000000,0 -> / +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/10.000000000,0 -> / +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/10.000000000,0 -> / +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=286 val_count=22 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 gc_bytes_age=51066 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: key_bytes=-60 val_count=-5 live_count=+5 live_bytes=+105 gc_bytes_age=-14850 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +run stats ok +del_range_pred k=a end=z ts=10 startTime=0 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=0 rangeThreshold=1 +stats: range_key_count=+2 range_key_bytes=+63 range_val_count=+6 live_count=-5 live_bytes=-105 gc_bytes_age=+15024 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[10.000000000,0=/ 1.000000000,0=/] +rangekey: {k-l}/[10.000000000,0=/] +rangekey: {l-n}/[10.000000000,0=/ 5.000000000,0=/] +rangekey: {n-o}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {o-p\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=10 range_key_bytes=221 range_val_count=20 gc_bytes_age=51240 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-2 range_key_bytes=-63 range_val_count=-6 live_count=+5 live_bytes=+105 gc_bytes_age=-15024 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Range tombstone deletion of times (5-10]. +run stats ok +del_range_pred k=a end=z ts=10 startTime=5 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=5 rangeThreshold=1 +stats: range_key_count=+2 range_key_bytes=+47 range_val_count=+4 live_count=-2 live_bytes=-42 gc_bytes_age=+8123 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: f{-\x00}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {f\x00-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: p{-\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=10 range_key_bytes=205 range_val_count=18 live_count=3 live_bytes=63 gc_bytes_age=44339 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-2 range_key_bytes=-47 range_val_count=-4 live_count=+2 live_bytes=+42 gc_bytes_age=-8123 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Range tombstone deletion of times (5-10]. +run stats ok +del_range_pred k=a end=z ts=10 startTime=5 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=5 rangeThreshold=1 +stats: range_key_count=+2 range_key_bytes=+47 range_val_count=+4 live_count=-2 live_bytes=-42 gc_bytes_age=+8123 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: f{-\x00}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {f\x00-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: p{-\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=10 range_key_bytes=205 range_val_count=18 live_count=3 live_bytes=63 gc_bytes_age=44339 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-2 range_key_bytes=-47 range_val_count=-4 live_count=+2 live_bytes=+42 gc_bytes_age=-8123 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Range tombstone deletion of times (4-10]. +run stats ok +del_range_pred k=a end=z ts=10 startTime=4 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=4 rangeThreshold=1 +stats: range_key_count=+4 range_key_bytes=+84 range_val_count=+7 live_count=-3 live_bytes=-63 gc_bytes_age=+13550 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: f{-\x00}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {f\x00-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-i}/[1.000000000,0=/] +rangekey: i{-\x00}/[10.000000000,0=/ 1.000000000,0=/] +rangekey: {i\x00-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: p{-\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=12 range_key_bytes=242 range_val_count=21 live_count=2 live_bytes=42 gc_bytes_age=49766 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-4 range_key_bytes=-84 range_val_count=-7 live_count=+3 live_bytes=+63 gc_bytes_age=-13550 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Range tombstone deletion of times (3-10]. +run stats ok +del_range_pred k=a end=z ts=10 startTime=3 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=3 rangeThreshold=1 +stats: range_key_count=+4 range_key_bytes=+84 range_val_count=+7 live_count=-3 live_bytes=-63 gc_bytes_age=+13550 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: f{-\x00}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {f\x00-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-i}/[1.000000000,0=/] +rangekey: i{-\x00}/[10.000000000,0=/ 1.000000000,0=/] +rangekey: {i\x00-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: p{-\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=12 range_key_bytes=242 range_val_count=21 live_count=2 live_bytes=42 gc_bytes_age=49766 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-4 range_key_bytes=-84 range_val_count=-7 live_count=+3 live_bytes=+63 gc_bytes_age=-13550 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 + +# Range tombstone deletion of times (2-10]. +run stats ok +del_range_pred k=a end=z ts=10 startTime=2 rangeThreshold=1 +---- +>> del_range_pred k=a end=z ts=10 startTime=2 rangeThreshold=1 +stats: range_key_count=+2 range_key_bytes=+47 range_val_count=+4 live_count=-4 live_bytes=-84 gc_bytes_age=+11860 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[10.000000000,0=/ 5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-i\x00}/[10.000000000,0=/ 1.000000000,0=/] +rangekey: {i\x00-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: p{-\x00}/[10.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=10 range_key_bytes=205 range_val_count=18 live_count=1 live_bytes=21 gc_bytes_age=48076 + +run stats ok +clear_time_range k=a end=z ts=10 targetTs=6 +---- +>> clear_time_range k=a end=z ts=10 targetTs=6 +stats: range_key_count=-2 range_key_bytes=-47 range_val_count=-4 live_count=+4 live_bytes=+84 gc_bytes_age=-11860 +>> at end: +rangekey: {a-b}/[1.000000000,0=/] +rangekey: {b-c}/[3.000000000,0=/ 1.000000000,0=/] +rangekey: {c-d}/[5.000000000,0=/ 3.000000000,0=/ 1.000000000,0=/] +rangekey: {d-f}/[5.000000000,0=/ 1.000000000,0=/] +rangekey: {f-h}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {h-k}/[1.000000000,0=/] +rangekey: {l-n}/[5.000000000,0=/] +rangekey: {n-o}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.000000000,0 -> / +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "d"/2.000000000,0 -> /BYTES/d2 +data: "e"/3.000000000,0 -> /BYTES/e3 +data: "f"/6.000000000,0 -> /BYTES/f6 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "f"/2.000000000,0 -> /BYTES/f2 +data: "g"/4.000000000,0 -> /BYTES/g4 +data: "g"/2.000000000,0 -> /BYTES/g2 +data: "h"/3.000000000,0 -> /BYTES/h3 +data: "i"/5.000000000,0 -> /BYTES/i5 +data: "j"/6.000000000,0 -> / +data: "j"/2.000000000,0 -> /BYTES/j2 +data: "k"/2.000000000,0 -> /BYTES/k2 +data: "p"/6.000000000,0 -> /BYTES/p6 +stats: key_count=11 key_bytes=226 val_count=17 val_bytes=98 range_key_count=8 range_key_bytes=158 range_val_count=14 live_count=5 live_bytes=105 gc_bytes_age=36216 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 index 65f85ce3dbb9..1a9794ac305a 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_abort @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,115 +88,131 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 >> 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 +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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=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 +meta: "c"/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: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "f"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +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 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 run stats ok with t=A status=ABORTED @@ -214,43 +236,51 @@ with t=A status=ABORTED 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 +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=-93 >> 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 +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_bytes=-48 gc_bytes_age=-1767 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +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=-93 >> 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 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5766 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: key_bytes=-12 val_count=-1 val_bytes=-48 live_count=+1 live_bytes=+21 gc_bytes_age=-7533 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=f t=A status=ABORTED -stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6852 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6777 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=g 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 +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=-93 >> 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=+194 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +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=-93 >> 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=+198 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-94 +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=-93 >> resolve_intent k=j 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 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5766 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=k t=A status=ABORTED -stats: key_bytes=-12 val_count=-1 val_bytes=-48 gc_bytes_age=-5634 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-94 +stats: key_bytes=-12 val_count=-1 val_bytes=-48 gc_bytes_age=-5572 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=l t=A status=ABORTED -stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6852 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6777 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> 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 +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=-93 >> 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 +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_bytes=-48 gc_bytes_age=-1767 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: key_bytes=-12 val_count=-1 val_bytes=-55 live_count=-1 live_bytes=-69 gc_bytes_age=+188 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-48 gc_bytes_age=-5766 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: key_bytes=-12 val_count=-1 val_bytes=-48 live_count=+1 live_bytes=+21 gc_bytes_age=-7533 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=r t=A status=ABORTED -stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6860 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: key_bytes=-12 val_count=-1 val_bytes=-61 gc_bytes_age=-6787 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> at end: -rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] data: "b"/1.000000000,0 -> /BYTES/b1 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ data: "e"/1.000000000,0 -> /BYTES/e1 @@ -259,8 +289,55 @@ data: "h"/1.000000000,0 -> /BYTES/h1 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ data: "k"/1.000000000,0 -> /BYTES/k1 data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "n"/5.000000000,0 -> /BYTES/n5 -data: "o"/5.000000000,0 -> {localTs=0.000000009,0}/ -data: "q"/5.000000000,0 -> /BYTES/q5 -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=4 live_bytes=84 gc_bytes_age=19673 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=4 live_bytes=84 gc_bytes_age=41454 + +# Make sure missing intents are handled correctly, both across point keys, point +# tombstones, range tombstones, and empty keyspans. +run stats ok +with t=A status=ABORTED + resolve_intent k=aaa + resolve_intent k=a + resolve_intent k=d + resolve_intent k=ggg + resolve_intent k=g + resolve_intent k=j +---- +>> resolve_intent k=aaa t=A status=ABORTED +stats: no change +>> resolve_intent k=a t=A status=ABORTED +stats: no change +>> resolve_intent k=d t=A status=ABORTED +stats: no change +>> resolve_intent k=ggg t=A status=ABORTED +stats: no change +>> resolve_intent k=g t=A status=ABORTED +stats: no change +>> resolve_intent k=j t=A status=ABORTED +stats: no change +>> at end: +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "k"/1.000000000,0 -> /BYTES/k1 +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=4 live_bytes=84 gc_bytes_age=41454 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 index 0b0f6525433b..3686d0e258d1 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_commit @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,115 +88,131 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 >> 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 +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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=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 +meta: "c"/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: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "f"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +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 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 run stats ok with t=A status=COMMITTED @@ -214,71 +236,144 @@ with t=A status=COMMITTED 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=f t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=l t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 live_bytes=-48 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=r t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4512 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4464 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 +>> at end: +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "b"/7.000000000,0 -> /BYTES/b7 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "d"/7.000000000,0 -> / +data: "e"/7.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "g"/7.000000000,0 -> /BYTES/g7 +data: "h"/7.000000000,0 -> /BYTES/h7 +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/7.000000000,0 -> /BYTES/i7 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "j"/7.000000000,0 -> / +data: "k"/7.000000000,0 -> / +data: "k"/1.000000000,0 -> /BYTES/k1 +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "m"/7.000000000,0 -> /BYTES/m7 +data: "n"/7.000000000,0 -> /BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/7.000000000,0 -> / +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=189 gc_bytes_age=62311 + +# Make sure missing intents are handled correctly, both across point keys, point +# tombstones, range tombstones, and empty keyspans. +run stats ok +with t=A status=COMMITTED + resolve_intent k=aaa + resolve_intent k=a + resolve_intent k=d + resolve_intent k=ggg + resolve_intent k=g + resolve_intent k=j +---- +>> resolve_intent k=aaa t=A status=COMMITTED +stats: no change +>> resolve_intent k=a t=A status=COMMITTED +stats: no change +>> resolve_intent k=d t=A status=COMMITTED +stats: no change +>> resolve_intent k=ggg t=A status=COMMITTED +stats: no change +>> resolve_intent k=g t=A status=COMMITTED +stats: no change +>> resolve_intent k=j t=A status=COMMITTED +stats: no change >> 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 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "b"/7.000000000,0 -> /BYTES/b7 data: "b"/1.000000000,0 -> /BYTES/b1 -data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "d"/6.000000000,0 -> / -data: "e"/6.000000000,0 -> / +data: "d"/7.000000000,0 -> / +data: "e"/7.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "g"/6.000000000,0 -> /BYTES/g6 -data: "h"/6.000000000,0 -> /BYTES/h6 +data: "g"/7.000000000,0 -> /BYTES/g7 +data: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/7.000000000,0 -> /BYTES/i7 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "j"/6.000000000,0 -> / -data: "k"/6.000000000,0 -> / +data: "j"/7.000000000,0 -> / +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "m"/6.000000000,0 -> /BYTES/m6 +data: "m"/7.000000000,0 -> /BYTES/m7 +data: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,0}/ -data: "p"/6.000000000,0 -> / -data: "q"/6.000000000,0 -> / -data: "q"/5.000000000,0 -> /BYTES/q5 -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=189 gc_bytes_age=40767 +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/7.000000000,0 -> / +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=189 gc_bytes_age=62311 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 index 1208ea92bc4b..3432d167a729 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_resolve_pushed @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,119 +88,135 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 >> 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 +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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=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 +meta: "c"/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: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "f"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +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 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 run stats ok with t=A status=COMMITTED - txn_advance ts=7 + txn_advance ts=8 resolve_intent k=a resolve_intent k=b resolve_intent k=c @@ -215,72 +237,147 @@ with t=A status=COMMITTED 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 gc_bytes_age=-19 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3282 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3301 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=f t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4539 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4491 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3282 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3282 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=l t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4539 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4491 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 gc_bytes_age=-19 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 live_bytes=-35 intent_count=-1 intent_bytes=-19 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3282 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> 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 +stats: val_bytes=-35 gc_bytes_age=-3301 intent_count=-1 intent_bytes=-12 separated_intent_count=-1 intent_age=-93 >> resolve_intent k=r t=A status=COMMITTED -stats: val_bytes=-48 gc_bytes_age=-4539 intent_count=-1 intent_bytes=-25 separated_intent_count=-1 intent_age=-94 +stats: val_bytes=-48 gc_bytes_age=-4491 intent_count=-1 intent_bytes=-25 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=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 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/a7 +data: "b"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/b7 data: "b"/1.000000000,0 -> /BYTES/b1 -data: "c"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/c6 +data: "c"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "d"/7.000000000,0 -> {localTs=6.000000000,0}/ -data: "e"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "d"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/8.000000000,0 -> {localTs=7.000000000,0}/ data: "e"/1.000000000,0 -> /BYTES/e1 -data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/8.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "g"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/g6 -data: "h"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/h6 +data: "g"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/g7 +data: "h"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -data: "i"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/i6 +data: "i"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/i7 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "j"/7.000000000,0 -> {localTs=6.000000000,0}/ -data: "k"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "j"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/8.000000000,0 -> {localTs=7.000000000,0}/ data: "k"/1.000000000,0 -> /BYTES/k1 -data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/8.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 -> {localTs=0.000000009,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=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=306 gc_bytes_age=47780 +data: "m"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/m7 +data: "n"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=306 gc_bytes_age=69246 + +# Make sure missing intents are handled correctly, both across point keys, point +# tombstones, range tombstones, and empty keyspans. +run stats ok +with t=A status=COMMITTED + txn_advance ts=9 + resolve_intent k=aaa + resolve_intent k=a + resolve_intent k=d + resolve_intent k=ggg + resolve_intent k=g + resolve_intent k=j +---- +>> resolve_intent k=aaa t=A status=COMMITTED +stats: no change +>> resolve_intent k=a t=A status=COMMITTED +stats: no change +>> resolve_intent k=d t=A status=COMMITTED +stats: no change +>> resolve_intent k=ggg t=A status=COMMITTED +stats: no change +>> resolve_intent k=g t=A status=COMMITTED +stats: no change +>> resolve_intent k=j t=A status=COMMITTED +stats: no change +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=9.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/a7 +data: "b"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/b7 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/c7 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "d"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "g"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/g7 +data: "h"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/h7 +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/i7 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "j"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/1.000000000,0 -> /BYTES/k1 +data: "l"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "m"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/m7 +data: "n"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=306 gc_bytes_age=69246 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 index b86ded5783f7..46c37f664b12 100644 --- 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 @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,123 +88,177 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 +data: "h"/1.000000000,0 -> /BYTES/h1 +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 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / +data: "k"/1.000000000,0 -> /BYTES/k1 +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 + +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=-966 live_count=-5 live_bytes=-537 gc_bytes_age=-61033 intent_count=-18 intent_bytes=-318 separated_intent_count=-18 intent_age=-1674 >> 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 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 -> /BYTES/h1 -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 -> {localTs=0.000000009,0}/ -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 -> / -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 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=4 live_bytes=84 gc_bytes_age=41454 +# A second intent resolution should be a noop. 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=-966 live_count=-5 live_bytes=-537 gc_bytes_age=-61702 intent_count=-18 intent_bytes=-318 separated_intent_count=-18 intent_age=-1692 +stats: no change >> at end: -rangekey: {g-s}/[4.000000000,0=/ 3.000000000,0=/] +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] data: "b"/1.000000000,0 -> /BYTES/b1 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ data: "e"/1.000000000,0 -> /BYTES/e1 @@ -207,8 +267,8 @@ data: "h"/1.000000000,0 -> /BYTES/h1 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ data: "k"/1.000000000,0 -> /BYTES/k1 data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "n"/5.000000000,0 -> /BYTES/n5 -data: "o"/5.000000000,0 -> {localTs=0.000000009,0}/ -data: "q"/5.000000000,0 -> /BYTES/q5 -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=4 live_bytes=84 gc_bytes_age=19673 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=12 key_bytes=168 val_count=12 val_bytes=96 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=4 live_bytes=84 gc_bytes_age=41454 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 index 46884356abfb..7109b81b078d 100644 --- 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 @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,151 +88,223 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "c"/7.000000000,0 -> /BYTES/c7 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 +data: "h"/1.000000000,0 -> /BYTES/h1 +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 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / +data: "k"/1.000000000,0 -> /BYTES/k1 +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 + +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=-40176 intent_count=-18 intent_bytes=-318 separated_intent_count=-18 intent_age=-1674 >> 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 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> /BYTES/a7 +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=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"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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: "d"/7.000000000,0 -> / +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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: "g"/7.000000000,0 -> /BYTES/g7 +data: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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"/7.000000000,0 -> /BYTES/i7 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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 -> / +data: "j"/7.000000000,0 -> / +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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: "m"/7.000000000,0 -> /BYTES/m7 +data: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/7.000000000,0 -> / +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=189 gc_bytes_age=62311 +# A second intent resolution should be a noop. 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=-318 separated_intent_count=-18 intent_age=-1692 +stats: no change >> 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 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/7.000000000,0 -> /BYTES/a7 +data: "b"/7.000000000,0 -> /BYTES/b7 data: "b"/1.000000000,0 -> /BYTES/b1 -data: "c"/6.000000000,0 -> /BYTES/c6 +data: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "d"/6.000000000,0 -> / -data: "e"/6.000000000,0 -> / +data: "d"/7.000000000,0 -> / +data: "e"/7.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "g"/6.000000000,0 -> /BYTES/g6 -data: "h"/6.000000000,0 -> /BYTES/h6 +data: "g"/7.000000000,0 -> /BYTES/g7 +data: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -data: "i"/6.000000000,0 -> /BYTES/i6 +data: "i"/7.000000000,0 -> /BYTES/i7 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "j"/6.000000000,0 -> / -data: "k"/6.000000000,0 -> / +data: "j"/7.000000000,0 -> / +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "m"/6.000000000,0 -> /BYTES/m6 +data: "m"/7.000000000,0 -> /BYTES/m7 +data: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,0}/ -data: "p"/6.000000000,0 -> / -data: "q"/6.000000000,0 -> / -data: "q"/5.000000000,0 -> /BYTES/q5 -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=189 gc_bytes_age=40767 +data: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/7.000000000,0 -> / +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=198 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=189 gc_bytes_age=62311 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 index 4ec1dda0cbad..e4dfab375ab1 100644 --- 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 @@ -3,20 +3,22 @@ # # (x is tombstone, o---o is range tombstone, [] is intent) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,153 +88,227 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 >> 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 +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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=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 +meta: "c"/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: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "f"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +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 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 run stats ok -txn_advance t=A ts=7 +txn_advance t=A ts=8 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=-669 live_bytes=-315 gc_bytes_age=-33595 intent_count=-18 intent_bytes=-318 separated_intent_count=-18 intent_age=-1692 +stats: val_bytes=-669 live_bytes=-315 gc_bytes_age=-33241 intent_count=-18 intent_bytes=-318 separated_intent_count=-18 intent_age=-1674 >> 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 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/a7 +data: "b"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/b7 data: "b"/1.000000000,0 -> /BYTES/b1 -data: "c"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/c6 +data: "c"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "d"/7.000000000,0 -> {localTs=6.000000000,0}/ -data: "e"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "d"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/8.000000000,0 -> {localTs=7.000000000,0}/ data: "e"/1.000000000,0 -> /BYTES/e1 -data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/8.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "g"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/g6 -data: "h"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/h6 +data: "g"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/g7 +data: "h"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -data: "i"/7.000000000,0 -> {localTs=6.000000000,0}/BYTES/i6 +data: "i"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/i7 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -data: "j"/7.000000000,0 -> {localTs=6.000000000,0}/ -data: "k"/7.000000000,0 -> {localTs=6.000000000,0}/ +data: "j"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/8.000000000,0 -> {localTs=7.000000000,0}/ data: "k"/1.000000000,0 -> /BYTES/k1 -data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/8.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 -> {localTs=0.000000009,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=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=306 gc_bytes_age=47780 +data: "m"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/m7 +data: "n"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=306 gc_bytes_age=69246 + +# A second intent resolution should be a noop. +run stats ok +txn_advance t=A ts=9 +resolve_intent_range t=A k=a end=z status=COMMITTED +---- +>> resolve_intent_range t=A k=a end=z status=COMMITTED +stats: no change +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=9.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +data: "a"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/a7 +data: "b"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/b7 +data: "b"/1.000000000,0 -> /BYTES/b1 +data: "c"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/c7 +data: "c"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "d"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "f"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "g"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/g7 +data: "h"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/h7 +data: "h"/1.000000000,0 -> /BYTES/h1 +data: "i"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/i7 +data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "j"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "k"/1.000000000,0 -> /BYTES/k1 +data: "l"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "l"/1.000000000,0 -> {localTs=0.000000009,0}/ +data: "m"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/m7 +data: "n"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/n7 +data: "n"/6.000000000,0 -> /BYTES/n6 +data: "o"/8.000000000,0 -> {localTs=7.000000000,0}/BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +data: "p"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/8.000000000,0 -> {localTs=7.000000000,0}/ +data: "q"/6.000000000,0 -> /BYTES/q6 +data: "r"/8.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=393 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=306 gc_bytes_age=69246 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite index ce4bd8a3e774..a4752bd25c7f 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_stats_intent_rewrite @@ -1,22 +1,24 @@ -# Tests MVCC stats calculations when rewriting intents. Intermediate states are +# 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) # -# 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 +# 7 [a7][b7][c7][x] [x] [x] [g7][h7][i7][x] [x] [x] [m7][n7][o7][x] [x] [x] +# 6 n6 x q6 x +# 5 o-----------------------------------------------o +# 4 o------o o------o o------o 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. Additionally, it sets a -# local timestamp for some tombstones, to ensure non-empty tombstone values are -# handled correctly. +# This uses multiple range tombstones, since the lowest is the one that matters +# for point key GCBytesAge, and to try to provoke bugs when stepping from a lone +# intent with no previous value and landing on a bare range key. 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. Additionally, it sets a local timestamp for some tombstones, to +# ensure non-empty tombstone values are handled correctly. # # TODO(erikgrinaker): This is probably better handled by randomized or # generative testing, since the combinations are getting unwieldy. But it'll do @@ -33,29 +35,33 @@ with ts=1 put k=k v=k1 del k=l localTs=0.9 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_range_ts k=ggg end=i ts=4 +del_range_ts k=jjj end=l ts=4 +del_range_ts k=mmm end=o ts=4 +del_range_ts k=ppp end=r ts=4 +del_range_ts k=g end=s ts=5 +with ts=6 + put k=n v=n6 del k=o localTs=0.9 - put k=q v=q5 + put k=q v=q6 del k=r localTs=0.9 with t=A - txn_begin ts=6 - put k=a v=a6 - put k=b v=b6 - put k=c v=c6 + txn_begin ts=7 + put k=a v=a7 + put k=b v=b7 + put k=c v=c7 del k=d del k=e del k=f localTs=5.9 - put k=g v=g6 - put k=h v=h6 - put k=i v=i6 + put k=g v=g7 + put k=h v=h7 + put k=i v=i7 del k=j del k=k del k=l localTs=5.9 - put k=m v=m6 - put k=n v=n6 - put k=o v=o6 + put k=m v=m7 + put k=n v=n7 + put k=o v=o7 del k=p del k=q del k=r localTs=5.9 @@ -82,335 +88,388 @@ del: "l": found key false stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2277 >> 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 +>> del_range_ts k=ggg end=i ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=jjj end=l ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=mmm end=o ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=ppp end=r ts=4 +stats: range_key_count=+2 range_key_bytes=+39 range_val_count=+3 gc_bytes_age=+3768 +>> del_range_ts k=g end=s ts=5 +stats: range_key_bytes=+81 range_val_count=+9 gc_bytes_age=+7615 +>> put k=n v=n6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=o localTs=0.9 ts=5 +>> del k=o localTs=0.9 ts=6 del: "o": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> put k=q v=q5 ts=5 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=q v=q6 ts=6 stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 ->> del k=r localTs=0.9 ts=5 +>> del k=r localTs=0.9 ts=6 del: "r": found key false -stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2185 ->> 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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+9 gc_bytes_age=+2162 +>> put k=a v=a7 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=+93 +>> put k=b v=b7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=c v=c7 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=+93 >> del k=d t=A del: "d": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=e t=A del: "e": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=f localTs=5.9 t=A del: "f": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 ->> put k=g v=g6 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=h v=h6 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=i v=i6 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=g v=g7 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=+93 +>> put k=h v=h7 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=+93 +>> put k=i v=i7 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=+93 >> del k=j t=A del: "j": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=k t=A del: "k": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5634 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 gc_bytes_age=+5572 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=l localTs=5.9 t=A del: "l": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6852 intent_count=+1 intent_bytes=+25 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 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6777 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 +>> put k=m v=m7 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=+93 +>> put k=n v=n7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_bytes=+48 gc_bytes_age=+1767 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 +>> put k=o v=o7 t=A +stats: key_bytes=+12 val_count=+1 val_bytes=+55 live_count=+1 live_bytes=+69 gc_bytes_age=-188 intent_count=+1 intent_bytes=+19 separated_intent_count=+1 intent_age=+93 >> del k=p t=A del: "p": found key false -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 +stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+48 gc_bytes_age=+5766 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=q t=A del: "q": found key true -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 +stats: key_bytes=+12 val_count=+1 val_bytes=+48 live_count=-1 live_bytes=-21 gc_bytes_age=+7533 intent_count=+1 intent_bytes=+12 separated_intent_count=+1 intent_age=+93 >> del k=r localTs=5.9 t=A del: "r": found key false -stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6860 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+94 +stats: key_bytes=+12 val_count=+1 val_bytes=+61 gc_bytes_age=+6787 intent_count=+1 intent_bytes=+25 separated_intent_count=+1 intent_age=+93 >> 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 +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: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/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: "a"/7.000000000,0 -> /BYTES/a7 +meta: "b"/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: "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=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 +meta: "c"/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: "c"/7.000000000,0 -> /BYTES/c7 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> / +meta: "e"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +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=0 ts=6.000000000,0 min=0,0 seq=0} ts=6.000000000,0 del=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true -data: "f"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "f"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "f"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/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: "g"/7.000000000,0 -> /BYTES/g7 +meta: "h"/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: "h"/7.000000000,0 -> /BYTES/h7 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +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 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "j"/7.000000000,0 -> / +meta: "k"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "k"/7.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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=13 mergeTs= txnDidNotUpdateMeta=true -data: "l"/6.000000000,0 -> {localTs=5.000000009,0}/ +meta: "l"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "l"/7.000000000,0 -> {localTs=5.000000009,0}/ data: "l"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "m"/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: "m"/7.000000000,0 -> /BYTES/m7 +meta: "n"/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: "n"/7.000000000,0 -> /BYTES/n7 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 -> {localTs=0.000000009,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=13 mergeTs= txnDidNotUpdateMeta=true -data: "r"/6.000000000,0 -> {localTs=5.000000009,0}/ -data: "r"/5.000000000,0 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=81375 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1692 +meta: "o"/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: "o"/7.000000000,0 -> /BYTES/o7 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "p"/7.000000000,0 -> / +meta: "q"/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=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=true +data: "q"/7.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/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=true klen=12 vlen=13 mergeTs= txnDidNotUpdateMeta=true +data: "r"/7.000000000,0 -> {localTs=5.000000009,0}/ +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1062 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=102487 intent_count=18 intent_bytes=318 separated_intent_count=18 intent_age=1674 # Rewrite the same keys at a higher timestamp. run stats ok -with t=A ts=7 +with t=A ts=8 txn_advance txn_restart - put k=a v=a7 - put k=b v=b7 - put k=c v=c7 + put k=a v=a8 + put k=b v=b8 + put k=c v=c8 del k=d del k=e del k=f - put k=g v=g7 - put k=h v=h7 - put k=i v=i7 + put k=g v=g8 + put k=h v=h8 + put k=i v=i8 del k=j del k=k del k=l - put k=m v=m7 - put k=n v=n7 - put k=o v=o7 + put k=m v=m8 + put k=n v=n8 + put k=o v=o8 del k=p del k=q del k=r ---- ->> put k=a v=a7 t=A ts=7 +>> put k=a v=a8 t=A ts=8 stats: intent_age=-1 ->> put k=b v=b7 t=A ts=7 +>> put k=b v=b8 t=A ts=8 stats: gc_bytes_age=-19 intent_age=-1 ->> put k=c v=c7 t=A ts=7 +>> put k=c v=c8 t=A ts=8 stats: intent_age=-1 ->> del k=d t=A ts=7 +>> del k=d t=A ts=8 del: "d": found key false stats: gc_bytes_age=-62 intent_age=-1 ->> del k=e t=A ts=7 +>> del k=e t=A ts=8 del: "e": found key true stats: gc_bytes_age=-81 intent_age=-1 ->> del k=f t=A ts=7 +>> del k=f t=A ts=8 del: "f": found key false -stats: val_bytes=-13 gc_bytes_age=-1284 intent_bytes=-13 intent_age=-1 ->> put k=g v=g7 t=A ts=7 +stats: val_bytes=-13 gc_bytes_age=-1271 intent_bytes=-13 intent_age=-1 +>> put k=g v=g8 t=A ts=8 stats: intent_age=-1 ->> put k=h v=h7 t=A ts=7 +>> put k=h v=h8 t=A ts=8 stats: intent_age=-1 ->> put k=i v=i7 t=A ts=7 +>> put k=i v=i8 t=A ts=8 stats: intent_age=-1 ->> del k=j t=A ts=7 +>> del k=j t=A ts=8 del: "j": found key false stats: gc_bytes_age=-62 intent_age=-1 ->> del k=k t=A ts=7 +>> del k=k t=A ts=8 del: "k": found key false stats: gc_bytes_age=-62 intent_age=-1 ->> del k=l t=A ts=7 +>> del k=l t=A ts=8 del: "l": found key false -stats: val_bytes=-13 gc_bytes_age=-1284 intent_bytes=-13 intent_age=-1 ->> put k=m v=m7 t=A ts=7 +stats: val_bytes=-13 gc_bytes_age=-1271 intent_bytes=-13 intent_age=-1 +>> put k=m v=m8 t=A ts=8 stats: intent_age=-1 ->> put k=n v=n7 t=A ts=7 +>> put k=n v=n8 t=A ts=8 stats: gc_bytes_age=-19 intent_age=-1 ->> put k=o v=o7 t=A ts=7 +>> put k=o v=o8 t=A ts=8 stats: intent_age=-1 ->> del k=p t=A ts=7 +>> del k=p t=A ts=8 del: "p": found key false stats: gc_bytes_age=-62 intent_age=-1 ->> del k=q t=A ts=7 +>> del k=q t=A ts=8 del: "q": found key true stats: gc_bytes_age=-81 intent_age=-1 ->> del k=r t=A ts=7 +>> del k=r t=A ts=8 del: "r": found key false -stats: val_bytes=-13 gc_bytes_age=-1284 intent_bytes=-13 intent_age=-1 +stats: val_bytes=-13 gc_bytes_age=-1271 intent_bytes=-13 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 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=8.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "a"/8.000000000,0 -> /BYTES/a8 +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "b"/8.000000000,0 -> /BYTES/b8 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 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "c"/8.000000000,0 -> /BYTES/c8 data: "c"/1.000000000,0 -> {localTs=0.000000009,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 -> / +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "d"/8.000000000,0 -> / +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "e"/8.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 -> / +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "f"/8.000000000,0 -> / data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -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 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "g"/8.000000000,0 -> /BYTES/g8 +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "h"/8.000000000,0 -> /BYTES/h8 data: "h"/1.000000000,0 -> /BYTES/h1 -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 +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "i"/8.000000000,0 -> /BYTES/i8 data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -> / -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "j"/8.000000000,0 -> / +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "k"/8.000000000,0 -> / data: "k"/1.000000000,0 -> /BYTES/k1 -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: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "l"/8.000000000,0 -> / data: "l"/1.000000000,0 -> {localTs=0.000000009,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 -> {localTs=0.000000009,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 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1023 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=77075 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1674 +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "m"/8.000000000,0 -> /BYTES/m8 +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "n"/8.000000000,0 -> /BYTES/n8 +data: "n"/6.000000000,0 -> /BYTES/n6 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "o"/8.000000000,0 -> /BYTES/o8 +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "p"/8.000000000,0 -> / +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "q"/8.000000000,0 -> / +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=8.000000000,0 min=0,0 seq=0} ts=8.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "r"/8.000000000,0 -> / +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1023 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=98226 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1656 # Rewrite keys<->tombstones at a higher timestamp. run stats ok -with t=A ts=8 +with t=A ts=9 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 + put k=d v=d9 + put k=e v=e9 + put k=f v=f9 del k=g del k=h del k=i - put k=j v=j8 - put k=k v=k8 - put k=l v=l8 + put k=j v=j9 + put k=k v=k9 + put k=l v=l9 + del k=m + del k=n + del k=o + put k=p v=p9 + put k=q v=q9 + put k=r v=r9 ---- ->> del k=a t=A ts=8 +>> del k=a t=A ts=9 del: "a": found key false -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=+5642 intent_bytes=-7 intent_age=-1 +>> del k=b t=A ts=9 del: "b": found key true -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=+5623 intent_bytes=-7 intent_age=-1 +>> del k=c t=A ts=9 del: "c": found key false -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=+5642 intent_bytes=-7 intent_age=-1 +>> put k=d v=d9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> put k=e v=e9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5723 intent_bytes=+7 intent_age=-1 +>> put k=f v=f9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> del k=g t=A ts=9 del: "g": found key false -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=+5642 intent_bytes=-7 intent_age=-1 +>> del k=h t=A ts=9 del: "h": found key false -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=+5642 intent_bytes=-7 intent_age=-1 +>> del k=i t=A ts=9 del: "i": found key false -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 +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5642 intent_bytes=-7 intent_age=-1 +>> put k=j v=j9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> put k=k v=k9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> put k=l v=l9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> del k=m t=A ts=9 +del: "m": found key false +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5642 intent_bytes=-7 intent_age=-1 +>> del k=n t=A ts=9 +del: "n": found key true +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5623 intent_bytes=-7 intent_age=-1 +>> del k=o t=A ts=9 +del: "o": found key false +stats: val_bytes=-7 live_count=-1 live_bytes=-69 gc_bytes_age=+5642 intent_bytes=-7 intent_age=-1 +>> put k=p v=p9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 intent_bytes=+7 intent_age=-1 +>> put k=q v=q9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5723 intent_bytes=+7 intent_age=-1 +>> put k=r v=r9 t=A ts=9 +stats: val_bytes=+7 live_count=+1 live_bytes=+69 gc_bytes_age=-5704 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 -> / +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=9.000000000,0 wto=false gul=0,0 +rangekey: g{-gg}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ggg-i}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {i-jjj}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {jjj-l}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {l-mmm}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {mmm-o}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {o-ppp}/[5.000000000,0=/ 3.000000000,0=/] +rangekey: {ppp-r}/[5.000000000,0=/ 4.000000000,0=/ 3.000000000,0=/] +rangekey: {r-s}/[5.000000000,0=/ 3.000000000,0=/] +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "a"/9.000000000,0 -> / +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "b"/9.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 -> / +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "c"/9.000000000,0 -> / data: "c"/1.000000000,0 -> {localTs=0.000000009,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 +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "d"/9.000000000,0 -> /BYTES/d9 +meta: "e"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "e"/9.000000000,0 -> /BYTES/e9 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 +meta: "f"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "f"/9.000000000,0 -> /BYTES/f9 data: "f"/1.000000000,0 -> {localTs=0.000000009,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 -> / -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 -> / +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "g"/9.000000000,0 -> / +meta: "h"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "h"/9.000000000,0 -> / data: "h"/1.000000000,0 -> /BYTES/h1 -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 -> / +meta: "i"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "i"/9.000000000,0 -> / data: "i"/1.000000000,0 -> {localTs=0.000000009,0}/ -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 -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: "j"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "j"/9.000000000,0 -> /BYTES/j9 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "k"/9.000000000,0 -> /BYTES/k9 data: "k"/1.000000000,0 -> /BYTES/k1 -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: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "l"/9.000000000,0 -> /BYTES/l9 data: "l"/1.000000000,0 -> {localTs=0.000000009,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 -> {localTs=0.000000009,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 -> {localTs=0.000000009,0}/ -stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1023 range_key_count=1 range_key_bytes=22 range_val_count=2 live_count=9 live_bytes=621 gc_bytes_age=76665 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1662 +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "m"/9.000000000,0 -> / +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "n"/9.000000000,0 -> / +data: "n"/6.000000000,0 -> /BYTES/n6 +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=true klen=12 vlen=0 mergeTs= txnDidNotUpdateMeta=false +data: "o"/9.000000000,0 -> / +data: "o"/6.000000000,0 -> {localTs=0.000000009,0}/ +meta: "p"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "p"/9.000000000,0 -> /BYTES/p9 +meta: "q"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "q"/9.000000000,0 -> /BYTES/q9 +data: "q"/6.000000000,0 -> /BYTES/q6 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=2 ts=9.000000000,0 min=0,0 seq=0} ts=9.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=false +data: "r"/9.000000000,0 -> /BYTES/r9 +data: "r"/6.000000000,0 -> {localTs=0.000000009,0}/ +stats: key_count=18 key_bytes=396 val_count=30 val_bytes=1023 range_key_count=9 range_key_bytes=250 range_val_count=22 live_count=9 live_bytes=621 gc_bytes_age=97592 intent_count=18 intent_bytes=279 separated_intent_count=18 intent_age=1638 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent index 245168267137..545c704bb640 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_writes_idempotent @@ -114,6 +114,49 @@ data: "n"/3.000000000,0 -> / meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 +# Writing at the timestamp of a point or range key errors. +run error +del_range_ts k=e end=f ts=4 idempotent +---- +>> at end: +rangekey: {a-d}/[3.000000000,0=/] +rangekey: {k-m}/[4.000000000,0=/] +rangekey: {m-o}/[4.000000000,0=/ 2.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "c"/4.000000000,0 -> /BYTES/c4 +data: "e"/4.000000000,0 -> /BYTES/e4 +data: "f"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +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: "k"/3.000000000,0 -> /BYTES/k3 +data: "m"/1.000000000,0 -> /BYTES/m1 +data: "n"/3.000000000,0 -> / +meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=l end=m ts=4 idempotent +---- +>> at end: +rangekey: {a-d}/[3.000000000,0=/] +rangekey: {k-m}/[4.000000000,0=/] +rangekey: {m-o}/[4.000000000,0=/ 2.000000000,0=/] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "c"/4.000000000,0 -> /BYTES/c4 +data: "e"/4.000000000,0 -> /BYTES/e4 +data: "f"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +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: "k"/3.000000000,0 -> /BYTES/k3 +data: "m"/1.000000000,0 -> /BYTES/m1 +data: "n"/3.000000000,0 -> / +meta: "p"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/p0 mergeTs= txnDidNotUpdateMeta=false +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "l" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 + # Writing below intents error. run error del_range_ts k=i end=j ts=3 idempotent