From d5d2eb7aad8c0eaeab1446e2a9e46859dc8d7e97 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Thu, 9 Feb 2023 10:21:40 -0500 Subject: [PATCH] storage: randomly mangle range key buffers in test Randomly mangle buffers for range key bounds, suffixes and values when the iterator becomes invalid or RangeKeyChanged(). Additionally, fix two bugs surfaced by this mangling: The first bug was in pebbleMVCCScanner: The scanner could step an iterator positioned at a bare range key into an exhausted position but continue to make use of pointers to the previous iterator position's range key buffers. The second bug was in CheckSSTConflicts: The extIter was repositioned without saving or re-retrieving the range keys. Epic: None Release note: None --- pkg/storage/engine.go | 8 - .../mvcc_history_metamorphic_iterator_test.go | 10 +- pkg/storage/mvcc_history_test.go | 11 +- pkg/storage/mvcc_key.go | 7 + pkg/storage/pebble_mvcc_scanner.go | 68 +++++-- pkg/storage/pebbleiter/BUILD.bazel | 1 + pkg/storage/pebbleiter/crdb_test_on.go | 182 +++++++++++++++++- pkg/storage/sst.go | 5 + .../mvcc_histories/range_tombstone_gets | 107 +++++++++- 9 files changed, 360 insertions(+), 39 deletions(-) diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 05caa963cc0f..198bbbd4ae43 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -1606,14 +1606,6 @@ func assertSimpleMVCCIteratorInvariants(iter SimpleMVCCIterator) error { } } - } else { - // Bounds and range keys must be empty. - if bounds := iter.RangeBounds(); !bounds.Equal(roachpb.Span{}) { - return errors.AssertionFailedf("hasRange=false but RangeBounds=%s", bounds) - } - if r := iter.RangeKeys(); !r.IsEmpty() || !r.Bounds.Equal(roachpb.Span{}) { - return errors.AssertionFailedf("hasRange=false but RangeKeys=%s", r) - } } if hasPoint { value, err := iter.UnsafeValue() diff --git a/pkg/storage/mvcc_history_metamorphic_iterator_test.go b/pkg/storage/mvcc_history_metamorphic_iterator_test.go index ded8e8ea1e0a..db2fe3ca0185 100644 --- a/pkg/storage/mvcc_history_metamorphic_iterator_test.go +++ b/pkg/storage/mvcc_history_metamorphic_iterator_test.go @@ -173,7 +173,7 @@ func (m *metamorphicIterator) moveAround() { } hasPoint, _ := m.it.HasPointAndRange() - rangeKeys := m.it.RangeKeys().Clone() + rangeKeys := rangeKeysIfExist(m.it).Clone() var rangeKeysIgnoringTime storage.MVCCRangeKeyStack if iit != nil { rangeKeysIgnoringTime = iit.RangeKeysIgnoringTime() @@ -225,7 +225,7 @@ func (m *metamorphicIterator) moveAround() { if m.it.UnsafeKey().Equal(cur) { break // made it } - printfln("step: %s %s [changed=%t]", m.it.UnsafeKey(), m.it.RangeKeys(), m.it.RangeKeyChanged()) + printfln("step: %s %s [changed=%t]", m.it.UnsafeKey(), rangeKeysIfExist(m.it), m.it.RangeKeyChanged()) if iit != nil { // If we're an incremental iterator with time bounds, and `cur` is not within bounds, // would miss it if we used Next. So call NextIgnoringTime unconditionally. @@ -248,7 +248,7 @@ func (m *metamorphicIterator) moveAround() { valid, err := m.it.Valid() require.Nil(m.t, err) require.True(m.t, valid, "unable to recover original position following SeekLT") - printfln("rev-step: %s %s [changed=%t]", m.it.UnsafeKey(), m.it.RangeKeys(), m.it.RangeKeyChanged()) + printfln("rev-step: %s %s [changed=%t]", m.it.UnsafeKey(), rangeKeysIfExist(m.it), m.it.RangeKeyChanged()) if m.it.UnsafeKey().Equal(cur) { printfln("done") break // made it @@ -269,13 +269,13 @@ func (m *metamorphicIterator) moveAround() { rangeKeysIgnoringTime2 = iit.RangeKeysIgnoringTime() } printfln("recovered position: %s hasPoint=%t, rangeKeys=%s, rangeKeysIgnoringTime=%s", - m.it.UnsafeKey(), hasPoint2, m.it.RangeKeys(), rangeKeysIgnoringTime2) + m.it.UnsafeKey(), hasPoint2, rangeKeysIfExist(m.it), rangeKeysIgnoringTime2) } // Back where we started and hopefully in an indistinguishable state. // When the stack is empty, sometimes it's a nil slice and sometimes zero // slice. A similar problem exists with MVCCRangeKeyVersion.Value. Sidestep // them by comparing strings. - require.Equal(m.t, fmt.Sprint(rangeKeys), fmt.Sprint(m.it.RangeKeys())) + require.Equal(m.t, fmt.Sprint(rangeKeys), fmt.Sprint(rangeKeysIfExist(m.it))) if iit != nil { require.Equal(m.t, fmt.Sprint(rangeKeysIgnoringTime), fmt.Sprint(iit.RangeKeysIgnoringTime())) } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index e06a204eb040..91940216eedd 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1982,9 +1982,18 @@ func printIter(e *evalCtx) { } } +func rangeKeysIfExist(it storage.SimpleMVCCIterator) storage.MVCCRangeKeyStack { + if valid, err := it.Valid(); !valid || err != nil { + return storage.MVCCRangeKeyStack{} + } else if _, hasRange := it.HasPointAndRange(); !hasRange { + return storage.MVCCRangeKeyStack{} + } + return it.RangeKeys() +} + func checkAndUpdateRangeKeyChanged(e *evalCtx) bool { rangeKeyChanged := e.iter.RangeKeyChanged() - rangeKeys := e.iter.RangeKeys() + rangeKeys := rangeKeysIfExist(e.iter) if incrIter := e.tryMVCCIncrementalIter(); incrIter != nil { // For MVCCIncrementalIterator, make sure RangeKeyChangedIgnoringTime() fires diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 3ab8404e9c52..41ff457304dd 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -512,6 +512,13 @@ type MVCCRangeKeyVersion struct { Value []byte } +// CloneInto copies the version into the provided destination +// MVCCRangeKeyVersion, reusing and overwriting its value slice. +func (v MVCCRangeKeyVersion) CloneInto(dst *MVCCRangeKeyVersion) { + dst.Timestamp = v.Timestamp + dst.Value = append(dst.Value[:0], v.Value...) +} + // AsRangeKey returns an MVCCRangeKey for the given version. Byte slices // are shared with the stack. func (s MVCCRangeKeyStack) AsRangeKey(v MVCCRangeKeyVersion) MVCCRangeKey { diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 59c70a11cf16..cae9a358c8a3 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -441,14 +442,15 @@ type pebbleMVCCScanner struct { // cur* variables store the "current" record we're pointing to. Updated in // updateCurrent. Note that the timestamp can be clobbered in the case of // adding an intent from the intent history but is otherwise meaningful. - curUnsafeKey MVCCKey - curRawKey []byte - curUnsafeValue MVCCValue - curRawValue pebble.LazyValue - curRangeKeys MVCCRangeKeyStack - savedRangeKeys MVCCRangeKeyStack - results results - intents pebble.Batch + curUnsafeKey MVCCKey + curRawKey []byte + curUnsafeValue MVCCValue + curRawValue pebble.LazyValue + curRangeKeys MVCCRangeKeyStack + savedRangeKeys MVCCRangeKeyStack + savedRangeKeyVers MVCCRangeKeyVersion + results results + intents pebble.Batch // mostRecentTS stores the largest timestamp observed that is equal to or // above the scan timestamp. Only applicable if failOnMoreRecent is true. If // set and no other error is hit, a WriteToOld error will be returned from @@ -546,6 +548,25 @@ func (p *pebbleMVCCScanner) get(ctx context.Context) { if !p.iterValid() { return } + + // Unlike scans, if tombstones are enabled, we synthesize point tombstones + // for MVCC range tombstones even if there is no existing point key below + // it. These are often needed for e.g. conflict checks. However, both + // processRangeKeys and getOne may need to advance the iterator, + // moving away from range key we originally landed on. If we're in tombstone + // mode and there's a range key, save the most recent visible value so that + // we can use it to synthesize a tombstone if we fail to find a KV. + var hadMVCCRangeTombstone bool + if p.tombstones { + if _, hasRange := p.parent.HasPointAndRange(); hasRange { + rangeKeys := p.parent.RangeKeys() + if rkv, ok := rangeKeys.FirstAtOrBelow(p.ts); ok { + hadMVCCRangeTombstone = true + rkv.CloneInto(&p.savedRangeKeyVers) + } + } + } + var added bool if p.processRangeKeys(true /* seeked */, false /* reverse */) { if p.updateCurrent() { @@ -553,13 +574,14 @@ func (p *pebbleMVCCScanner) get(ctx context.Context) { } } p.maybeFailOnMoreRecent() - // Unlike scans, if tombstones are enabled, we synthesize point tombstones for - // MVCC range tombstones even if there is no existing point key below it. - // These are often needed for e.g. conflict checks. - if p.tombstones && !added && p.err == nil { - if rkv, ok := p.coveredByRangeKey(hlc.MinTimestamp); ok { - p.addSynthetic(ctx, p.curRangeKeys.Bounds.Key, rkv) - } + + // In tombstone mode, if there was no existing point key we may need to + // synthesize a point tombstone if we saved a range key before + // Unlike scans, if tombstones are enabled, we synthesize point tombstones + // for MVCC range tombstones even if there is no existing point key below + // it. These are often needed for e.g. conflict checks. + if p.tombstones && hadMVCCRangeTombstone && !added && p.err == nil { + p.addSynthetic(ctx, p.start, p.savedRangeKeyVers) } } @@ -1658,6 +1680,9 @@ func (p *pebbleMVCCScanner) iterNext() bool { p.parent.Next() // We don't need to process range key changes here, because curRangeKeys // already contains the range keys at this position from before the peek. + if buildutil.CrdbTestBuild { + p.assertOwnedRangeKeys() + } if !p.iterValid() { return false } @@ -1834,3 +1859,16 @@ func (p *pebbleMVCCScanner) intentsRepr() []byte { } return p.intents.Repr() } + +// assertOwnedRangeKeys asserts that p.curRangeKeys is empty, or backed by +// p.savedRangeKeys's buffers. +func (p *pebbleMVCCScanner) assertOwnedRangeKeys() { + if p.curRangeKeys.IsEmpty() { + return + } + // NB: We compare on the EndKey in case the start key is /Min, the empty + // key. + if &p.curRangeKeys.Bounds.EndKey[0] != &p.savedRangeKeys.Bounds.EndKey[0] { + panic(errors.AssertionFailedf("current range keys are not scanner-owned")) + } +} diff --git a/pkg/storage/pebbleiter/BUILD.bazel b/pkg/storage/pebbleiter/BUILD.bazel index c3b54f46f659..ab46ad360509 100644 --- a/pkg/storage/pebbleiter/BUILD.bazel +++ b/pkg/storage/pebbleiter/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/storage/pebbleiter", visibility = ["//visibility:public"], deps = [ + "//pkg/util", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_pebble//:pebble", ], diff --git a/pkg/storage/pebbleiter/crdb_test_on.go b/pkg/storage/pebbleiter/crdb_test_on.go index cdb738e06c20..e5605329b0b5 100644 --- a/pkg/storage/pebbleiter/crdb_test_on.go +++ b/pkg/storage/pebbleiter/crdb_test_on.go @@ -15,7 +15,9 @@ package pebbleiter import ( "math/rand" + "time" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" ) @@ -29,13 +31,14 @@ type Iterator = *assertionIter // MaybeWrap returns the provided Pebble iterator, wrapped with double close // detection. func MaybeWrap(iter *pebble.Iterator) Iterator { - return &assertionIter{Iterator: iter} + return &assertionIter{Iterator: iter, closedCh: make(chan struct{})} } // assertionIter wraps a *pebble.Iterator with assertion checking. type assertionIter struct { *pebble.Iterator - closed bool + closed bool + closedCh chan struct{} // unsafeBufs hold buffers used for returning values with short lifetimes to // the caller. To assert that the client is respecting the lifetimes, // assertionIter mangles the buffers as soon as the associated lifetime @@ -53,6 +56,33 @@ type assertionIter struct { key [2][]byte val [2][]byte } + rangeKeyBufs struct { + idx int + bufs [2]rangeKeyBuf + } +} + +type rangeKeyBuf struct { + start []byte + end []byte + keys []pebble.RangeKeyData + + // bgCh is used in race builds to synchronize with a separate goroutine + // performing background buffer mangling. If non-nil, a separate mangling + // goroutine is active and periodically mangling this buffer. When the + // buffer is next used, maybeSaveAndMangleRangeKeyBufs performs a + // synchronous send to the channel to signal that the mangling goroutine + // should exit. + bgCh chan struct{} +} + +func (b *rangeKeyBuf) mangle() { + zero(b.start) + zero(b.end) + for k := range b.keys { + zero(b.keys[k].Suffix) + zero(b.keys[k].Value) + } } func (i *assertionIter) Clone(cloneOpts pebble.CloneOptions) (Iterator, error) { @@ -68,6 +98,7 @@ func (i *assertionIter) Close() error { panic(errors.AssertionFailedf("pebble.Iterator already closed")) } i.closed = true + close(i.closedCh) return i.Iterator.Close() } @@ -96,63 +127,98 @@ func (i *assertionIter) LazyValue() pebble.LazyValue { return i.Iterator.LazyValue() } +func (i *assertionIter) RangeBounds() ([]byte, []byte) { + if !i.Valid() { + panic(errors.AssertionFailedf("RangeBounds() called on !Valid() pebble.Iterator")) + } + if _, hasRange := i.Iterator.HasPointAndRange(); !hasRange { + panic(errors.AssertionFailedf("RangeBounds() called on pebble.Iterator without range keys")) + } + // See maybeSaveAndMangleRangeKeyBufs for where these are saved. + j := i.rangeKeyBufs.idx + return i.rangeKeyBufs.bufs[j].start, i.rangeKeyBufs.bufs[j].end +} + +func (i *assertionIter) RangeKeys() []pebble.RangeKeyData { + if !i.Valid() { + panic(errors.AssertionFailedf("RangeKeys() called on !Valid() pebble.Iterator")) + } + if _, hasRange := i.Iterator.HasPointAndRange(); !hasRange { + panic(errors.AssertionFailedf("RangeKeys() called on pebble.Iterator without range keys")) + } + // See maybeSaveAndMangleRangeKeyBufs for where these are saved. + return i.rangeKeyBufs.bufs[i.rangeKeyBufs.idx].keys +} + func (i *assertionIter) First() bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.First() } func (i *assertionIter) SeekGE(key []byte) bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.SeekGE(key) } func (i *assertionIter) SeekGEWithLimit(key []byte, limit []byte) pebble.IterValidityState { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.SeekGEWithLimit(key, limit) } func (i *assertionIter) SeekPrefixGE(key []byte) bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.SeekPrefixGE(key) } func (i *assertionIter) Next() bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.Next() } func (i *assertionIter) NextWithLimit(limit []byte) pebble.IterValidityState { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.NextWithLimit(limit) } func (i *assertionIter) NextPrefix() bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.NextPrefix() } func (i *assertionIter) Last() bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.Last() } func (i *assertionIter) SeekLT(key []byte) bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.SeekLT(key) } func (i *assertionIter) SeekLTWithLimit(key []byte, limit []byte) pebble.IterValidityState { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.SeekLTWithLimit(key, limit) } func (i *assertionIter) Prev() bool { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.Prev() } func (i *assertionIter) PrevWithLimit(limit []byte) pebble.IterValidityState { i.maybeMangleBufs() + defer i.maybeSaveAndMangleRangeKeyBufs() return i.Iterator.PrevWithLimit(limit) } @@ -162,14 +228,116 @@ func (i *assertionIter) PrevWithLimit(limit []byte) pebble.IterValidityState { func (i *assertionIter) maybeMangleBufs() { if rand.Intn(2) == 0 { idx := i.unsafeBufs.idx - for _, b := range [...][]byte{i.unsafeBufs.key[idx], i.unsafeBufs.val[idx]} { - for i := range b { - b[i] = 0 - } - } + zero(i.unsafeBufs.key[idx]) + zero(i.unsafeBufs.val[idx]) if rand.Intn(2) == 0 { // Switch to a new buffer for the next iterator position. i.unsafeBufs.idx = (i.unsafeBufs.idx + 1) % 2 } } } + +// maybeSaveAndMangleRangeKeyBufs is invoked at the end of every iterator +// operation. It saves the range keys to buffers owned by `assertionIter` and +// with random probability mangles any buffers previously returned to the user. +func (i *assertionIter) maybeSaveAndMangleRangeKeyBufs() { + // If RangeKeyChanged()=false, the pebble.Iterator contract guarantees that + // any buffers previously returned through RangeBounds() and RangeKeys() are + // still valid. + // + // NB: Only permitted to call RangeKeyChanged() if Valid(). + valid := i.Iterator.Valid() + if valid && !i.Iterator.RangeKeyChanged() { + return + } + // INVARIANT: !Valid() || RangeKeyChanged() + + // The previous range key buffers are no longer guaranteed to be stable. + // Randomly zero them to ensure we catch bugs where they're reused. + idx := i.rangeKeyBufs.idx + mangleBuf := &i.rangeKeyBufs.bufs[idx] + if rand.Intn(2) == 0 { + mangleBuf.mangle() + } + // If the new iterator position has range keys, copy them to our buffers. + if !valid { + return + } + if _, hasRange := i.Iterator.HasPointAndRange(); !hasRange { + return + } + switchBuffers := rand.Intn(2) == 0 + if switchBuffers { + // Switch to a new buffer for the new range key state. + i.rangeKeyBufs.idx = (idx + 1) % 2 + + // In race builds, mangle the old buffer from another goroutine. This is + // nice because the race detector can tell us where the improper read is + // originating. Otherwise, we're relying on the improper read + // manifesting a test assertion failure, which may be far from the + // problematic access of an unsafe buffer. + if util.RaceEnabled { + ch := make(chan struct{}) + mangleBuf.bgCh = ch + ticker := time.NewTicker(5 * time.Millisecond) + go func() { + defer ticker.Stop() + mangleBuf.mangle() + for { + select { + case <-i.closedCh: + return + case <-ch: + return + case <-ticker.C: + mangleBuf.mangle() + } + } + }() + + // If the buffer we're about to use is being mangled in the + // background, synchronize with the goroutine doing the mangling by + // sending to its channel. After the synchronous channel send, we're + // guaranteed the goroutine will not mangle the buffer again and + // we're free to use it. + if prevMangleBuf := &i.rangeKeyBufs.bufs[i.rangeKeyBufs.idx]; prevMangleBuf.bgCh != nil { + prevMangleBuf.bgCh <- struct{}{} + prevMangleBuf.bgCh = nil + } + } + } + start, end := i.Iterator.RangeBounds() + rangeKeys := i.Iterator.RangeKeys() + + buf := &i.rangeKeyBufs.bufs[i.rangeKeyBufs.idx] + buf.start = append(buf.start[:0], start...) + buf.end = append(buf.end[:0], end...) + if len(rangeKeys) > cap(buf.keys) { + buf.keys = make([]pebble.RangeKeyData, len(rangeKeys)) + } else { + buf.keys = buf.keys[:len(rangeKeys)] + } + for k := range rangeKeys { + // Preserve nil-ness returned by Pebble to ensure we're testing exactly + // what Pebble will return in production. + copyWithMatchingNilness(&buf.keys[k].Suffix, rangeKeys[k].Suffix) + copyWithMatchingNilness(&buf.keys[k].Value, rangeKeys[k].Value) + } +} + +func zero(b []byte) { + for i := range b { + b[i] = 0x00 + } +} + +func copyWithMatchingNilness(dst *[]byte, src []byte) { + if src == nil { + *dst = nil + return + } + if *dst == nil { + *dst = make([]byte, 0, len(src)) + } + *dst = append((*dst)[:0], src...) +} diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index b0fbf1103880..a009de597ccd 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -678,6 +678,11 @@ func CheckSSTConflicts( statsDiff.Add(updateStatsOnRangeKeyMerge(sstRangeKeys.Bounds.Key, sstRangeKeys.Versions)) } extIter.SeekGE(savedExtKey) + // After seeking, the old buffers have been invalidated. + // Re-retrieve the buffers. + if extHasRange { + extRangeKeys = extIter.RangeKeys() + } } if extRangeKeysChanged && !sstPrevRangeKeys.IsEmpty() && sstPrevRangeKeys.Bounds.Overlaps(extRangeKeys.Bounds) { // Because we always re-seek the extIter after every sstIter step, diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets index c77d8cfe7559..1494f0c3efa5 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_gets +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_gets @@ -3,13 +3,13 @@ # Sets up the following dataset, where x is tombstone, o-o is range tombstone, [] is intent. # # T -# 6 [e6] +# 6 [e6] o6 # 5 f5 # 4 o-----------------------o o-------o [j-l)@4 has localTs=3 # 3 x d3 f3 -# 2 o---------------o h2 +# 2 o---------------o h2 o--------o # 1 a1 x c1 f1 -# a b c d e f g h i j k l +# a b c d e f g h i j k l m n o p # run ok put k=a ts=1 v=a1 @@ -21,12 +21,14 @@ del k=a ts=3 put k=d ts=3 v=d3 put k=f ts=3 v=f3 put k=h ts=2 v=h2 +del_range_ts k=n end=p ts=2 del_range_ts k=c end=i ts=4 put k=f ts=5 v=f5 del_range_ts k=j end=l ts=4 localTs=3 with t=A txn_begin k=e ts=6 put k=e v=e6 +put k=o ts=6 v=o6 ---- del: "b": found key false del: "a": found key false @@ -36,6 +38,7 @@ rangekey: {a-c}/[2.000000000,0=/] rangekey: {c-e}/[4.000000000,0=/ 2.000000000,0=/] rangekey: {e-i}/[4.000000000,0=/] rangekey: {j-l}/[4.000000000,0={localTs=3.000000000,0}/] +rangekey: {n-p}/[2.000000000,0=/] data: "a"/3.000000000,0 -> / data: "a"/1.000000000,0 -> /BYTES/a1 data: "b"/1.000000000,0 -> / @@ -47,6 +50,7 @@ data: "f"/5.000000000,0 -> /BYTES/f5 data: "f"/3.000000000,0 -> /BYTES/f3 data: "f"/1.000000000,0 -> /BYTES/f1 data: "h"/2.000000000,0 -> /BYTES/h2 +data: "o"/6.000000000,0 -> /BYTES/o6 # Run gets for all keys and all timestamps. run ok @@ -241,6 +245,87 @@ get: "k" -> get: "k" -> / @4.000000000,0 get: "k" -> / @4.000000000,0 +run ok +get k=l ts=1 +get k=l ts=2 +get k=l ts=3 +get k=l ts=4 +get k=l ts=5 +get k=l ts=6 +get k=l ts=1 tombstones +get k=l ts=2 tombstones +get k=l ts=3 tombstones +get k=l ts=4 tombstones +get k=l ts=5 tombstones +get k=l ts=6 tombstones +---- +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> +get: "l" -> + +run ok +get k=n ts=1 +get k=n ts=2 +get k=n ts=3 +get k=n ts=4 +get k=n ts=5 +get k=n ts=6 +get k=n ts=1 tombstones +get k=n ts=2 tombstones +get k=n ts=3 tombstones +get k=n ts=4 tombstones +get k=n ts=5 tombstones +get k=n ts=6 tombstones +---- +get: "n" -> +get: "n" -> +get: "n" -> +get: "n" -> +get: "n" -> +get: "n" -> +get: "n" -> +get: "n" -> / @2.000000000,0 +get: "n" -> / @2.000000000,0 +get: "n" -> / @2.000000000,0 +get: "n" -> / @2.000000000,0 +get: "n" -> / @2.000000000,0 + +run ok +get k=o ts=1 +get k=o ts=2 +get k=o ts=3 +get k=o ts=4 +get k=o ts=5 +get k=o ts=6 +get k=o ts=1 tombstones +get k=o ts=2 tombstones +get k=o ts=3 tombstones +get k=o ts=4 tombstones +get k=o ts=5 tombstones +get k=o ts=6 tombstones +---- +get: "o" -> +get: "o" -> +get: "o" -> +get: "o" -> +get: "o" -> +get: "o" -> /BYTES/o6 @6.000000000,0 +get: "o" -> +get: "o" -> / @2.000000000,0 +get: "o" -> / @2.000000000,0 +get: "o" -> / @2.000000000,0 +get: "o" -> / @2.000000000,0 +get: "o" -> /BYTES/o6 @6.000000000,0 + # failOnMoreRecent: c run error get k=c ts=1 failOnMoreRecent @@ -397,3 +482,19 @@ get k=k ts=1 globalUncertaintyLimit=4 localUncertaintyLimit=3 ---- get: "k" -> error: (*kvpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 1.000000000,0 encountered previous write with future timestamp 4.000000000,0 (local=3.000000000,0) within uncertainty interval `t <= (local=3.000000000,0, global=0,0)`; observed timestamps: [] + +# Test a particular case where: +# - a tombstone must be synthesized due to a range tombstone +# - the uncertainty limit is such that we must check for uncertainty +# - the only point at the key is not visible at the read timstamp, but is also +# not outside the uncertainty limit +# +# In these circumstances, the scanner will seekVersion to find the first visible +# key (there is none), invalidating the underlying Pebble iterator. Although the +# underlying Pebble iterator has been invalidated, the scanner should still +# succeed in synthesizing a tombstone at the range key timestamp retrieved +# before the iterator was invalidated. +run ok +get k=o ts=5 tombstones globalUncertaintyLimit=6 localUncertaintyLimit=5 +---- +get: "o" -> / @2.000000000,0