diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index a92b50a04867..ae09fe0adc51 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "pebble_iterator.go", "pebble_merge.go", "pebble_mvcc_scanner.go", + "point_synthesizing_iter.go", "replicas_storage.go", "resource_limiter.go", "row_counter.go", diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 83bf5f6a5b9d..13b046b0430f 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -703,7 +703,8 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera // // In tombstones mode, if the most recent value is a deletion tombstone, the // result will be a non-nil roachpb.Value whose RawBytes field is nil. -// Otherwise, a deletion tombstone results in a nil roachpb.Value. +// Otherwise, a deletion tombstone results in a nil roachpb.Value. Range +// tombstones will be emitted as if they were point tombstones. // // In inconsistent mode, if an intent is encountered, it will be placed in the // dedicated return parameter. By contrast, in consistent mode, an intent will @@ -724,7 +725,15 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera func MVCCGet( ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions, ) (*roachpb.Value, *roachpb.Intent, error) { - iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{Prefix: true}) + var rangeKeyMaskingBelow hlc.Timestamp + if !opts.Tombstones { + rangeKeyMaskingBelow = timestamp + } + iter := newPointSynthesizingIter(newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + RangeKeyMaskingBelow: rangeKeyMaskingBelow, + }), true /* emitOnSeek */) defer iter.Close() value, intent, err := mvccGet(ctx, iter, key, timestamp, opts) return value.ToPointer(), intent, err @@ -2730,7 +2739,9 @@ type MVCCScanResult struct { // In tombstones mode, if the most recent value for a key is a deletion // tombstone, the scan result will contain a roachpb.KeyValue for that key whose // RawBytes field is nil. Otherwise, the key-value pair will be omitted from the -// result entirely. +// result entirely. If a point key was deleted by an MVCC range tombstone, +// a synthesized point tombstone is returned -- range tombstones by themselves +// are not surfaced (in particular if they don't cover any point keys). // // When scanning inconsistently, any encountered intents will be placed in the // dedicated result parameter. By contrast, when scanning consistently, any @@ -2753,7 +2764,16 @@ func MVCCScan( timestamp hlc.Timestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { - iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey}) + var rangeKeyMaskingBelow hlc.Timestamp + if !opts.Tombstones { + rangeKeyMaskingBelow = timestamp + } + iter := newPointSynthesizingIter(newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{ + KeyTypes: IterKeyTypePointsWithRanges, + LowerBound: key, + UpperBound: endKey, + RangeKeyMaskingBelow: rangeKeyMaskingBelow, + }), false /* emitOnSeek */) defer iter.Close() return mvccScanToKvs(ctx, iter, key, endKey, timestamp, opts) } @@ -2766,6 +2786,7 @@ func MVCCScanToBytes( timestamp hlc.Timestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { + // TODO(erikgrinaker): Update this. iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey}) defer iter.Close() return mvccScanToBytes(ctx, iter, key, endKey, timestamp, opts) diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 0807d744daa0..3ec5b4544f46 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -1002,6 +1002,9 @@ func cmdIterNew(e *evalCtx) error { MVCCIterator: r.NewMVCCIterator(kind, opts), closeReader: closeReader, } + if e.hasArg("pointSynthesis") { + e.iter = newPointSynthesizingIter(e.iter, false) + } return nil } diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 2452b2b4e79b..7b8e63612bce 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -62,6 +62,12 @@ func (k MVCCKey) Next() MVCCKey { } } +// Clone returns a copy of the key. +func (k MVCCKey) Clone() MVCCKey { + k.Key = k.Key.Clone() + return k +} + // Compare returns -1 if this key is less than the given key, 0 if they're // equal, or 1 if this is greater. Comparison is by key,timestamp, where larger // timestamps sort before smaller ones except empty ones which sort first (like diff --git a/pkg/storage/point_synthesizing_iter.go b/pkg/storage/point_synthesizing_iter.go new file mode 100644 index 000000000000..b4d96a80cc73 --- /dev/null +++ b/pkg/storage/point_synthesizing_iter.go @@ -0,0 +1,269 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/uuid" +) + +// pointSynthesizingIter wraps an MVCCIterator, and synthesizes point tombstones +// for range tombstones above/below existing point keys. It does not emit range +// keys at all, since these would appear to conflict with the synthesized point +// keys. +type pointSynthesizingIter struct { + iter MVCCIterator + rangeKeys []MVCCRangeKey + rangeKeysIdx int + rangeKeysStart roachpb.Key + // emitOnSeek will cause a SeekGE() call to emit synthetic points for the seek + // key even if it has no existing point keys. + emitOnSeek bool + // curKey is the current key position of the synthesizing iterator. This may + // be out of sync with the point iterator (when there are no further real + // point versions for the current key). + curKey roachpb.Key + // atPoint is true if the synthesizing iterator is positioned on the real + // point key. In that case, rangeKeysIdx points to the next range key below + // the point key, or past the end of rangeKeys if there are none. + // + // If atPoint is false, then the point iterator will be positioned on the next + // point key after the current range tombstone, which can either be an older + // version of the current key or a different key (or invalid if exhausted). + atPoint bool +} + +var _ MVCCIterator = new(pointSynthesizingIter) + +func newPointSynthesizingIter(iter MVCCIterator, emitOnSeek bool) *pointSynthesizingIter { + return &pointSynthesizingIter{ + iter: iter, + emitOnSeek: emitOnSeek, + } +} + +// updateWithFirstPointKey scans to the first point key and updates the iterator +// for its position. +func (i *pointSynthesizingIter) updateWithFirstPointKey() { + for { + if ok, err := i.iter.Valid(); !ok || err != nil { + break + } + if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + break + } + i.iter.Next() + } + i.updatePosition() +} + +// maybeUpdateWithBareRangeKey updates the iterator with synthetic point keys +// for the current range key if it is a bare range key and there are no point +// keys overlapping the range key for the current key. It will move ahead to +// check that and leave the iterator at the next key position regardless. +func (i *pointSynthesizingIter) maybeUpdateWithBareRangeKey() bool { + if ok, err := i.iter.Valid(); !ok || err != nil { + return false + } + hasPoint, hasRange := i.iter.HasPointAndRange() + if hasPoint || !hasRange { + return false + } + i.updatePosition() + + i.iter.Next() + if ok, err := i.iter.Valid(); !ok || err != nil { + return true + } + if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + if i.iter.UnsafeKey().Key.Equal(i.curKey) { + // We found a point key for the seek key, update the position again. + i.updatePosition() + } + } + return true +} + +// updatePosition will update the iterator position to the newest version of the +// current point iterator's key. +func (i *pointSynthesizingIter) updatePosition() { + if ok, err := i.iter.Valid(); !ok || err != nil { + i.curKey = nil + i.rangeKeys = nil + return + } + + hasPoint, hasRange := i.iter.HasPointAndRange() + + if hasRange { + if rangeStart, _ := i.iter.RangeBounds(); !rangeStart.Equal(i.rangeKeysStart) { + i.rangeKeys = i.iter.RangeKeys() + i.rangeKeysStart = rangeStart.Clone() + } + } + + key := i.iter.UnsafeKey() + i.curKey = key.Key.Clone() + i.rangeKeysIdx = 0 + i.atPoint = hasPoint && (len(i.rangeKeys) == 0 || key.Timestamp.IsEmpty() || + i.rangeKeys[0].Timestamp.LessEq(key.Timestamp)) +} + +func (i *pointSynthesizingIter) SeekGE(key MVCCKey) { + i.iter.SeekGE(key) + if !i.emitOnSeek || !i.maybeUpdateWithBareRangeKey() { + i.updateWithFirstPointKey() + } +} + +func (i *pointSynthesizingIter) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) { + i.iter.SeekIntentGE(key, txnUUID) + i.updateWithFirstPointKey() +} + +func (i *pointSynthesizingIter) Next() { + if i.atPoint { + // Pass by the current point key and onto the next one. This may be a + // different version of the current key, or a different key entirely. + i.atPoint = false + i.iter.Next() + } else { + // Move onto the next range key. This may be below the current point key, + // we'll find out below. + i.rangeKeysIdx++ + } + var key MVCCKey + if ok, _ := i.iter.Valid(); ok { + if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { + key = i.iter.UnsafeKey() + } + } + if len(key.Key) > 0 && key.Key.Equal(i.curKey) && (i.rangeKeysIdx >= len(i.rangeKeys) || + key.Timestamp.IsEmpty() || i.rangeKeys[i.rangeKeysIdx].Timestamp.LessEq(key.Timestamp)) { + // If the iter point key is at the same position as us and newer than the current + // range key, then position on the point key. + i.atPoint = true + } else if i.rangeKeysIdx >= len(i.rangeKeys) { + // If we've exhausted the range keys then synthesize points for the current point key, + // which must now be a different key from curKey. + i.updateWithFirstPointKey() + } + // Otherwise, we're now on the correct range key. +} + +func (i *pointSynthesizingIter) NextKey() { + i.iter.NextKey() + i.updateWithFirstPointKey() +} + +func (i *pointSynthesizingIter) SeekLT(key MVCCKey) { + panic("not implemented") +} + +func (i *pointSynthesizingIter) Prev() { + panic("not implemented") +} + +func (i *pointSynthesizingIter) Valid() (bool, error) { + if !i.atPoint && i.rangeKeysIdx < len(i.rangeKeys) { + return true, nil + } + return i.iter.Valid() +} + +func (i *pointSynthesizingIter) HasPointAndRange() (bool, bool) { + ok, _ := i.Valid() + return ok, false +} + +func (i *pointSynthesizingIter) RangeBounds() (roachpb.Key, roachpb.Key) { + return nil, nil +} + +func (i *pointSynthesizingIter) RangeKeys() []MVCCRangeKey { + return nil +} + +func (i *pointSynthesizingIter) Key() MVCCKey { + return i.UnsafeKey().Clone() +} + +func (i *pointSynthesizingIter) UnsafeKey() MVCCKey { + if i.atPoint { + return i.iter.UnsafeKey() + } + if len(i.curKey) == 0 || i.rangeKeysIdx >= len(i.rangeKeys) { + return MVCCKey{} + } + return MVCCKey{ + Key: i.curKey, + Timestamp: i.rangeKeys[i.rangeKeysIdx].Timestamp, + } +} + +func (i *pointSynthesizingIter) UnsafeRawKey() []byte { + if i.atPoint { + return i.iter.UnsafeRawKey() + } + return EncodeMVCCKeyPrefix(i.curKey) +} + +func (i *pointSynthesizingIter) UnsafeRawMVCCKey() []byte { + if i.atPoint { + return i.iter.UnsafeRawMVCCKey() + } + return EncodeMVCCKey(i.UnsafeKey()) +} + +func (i *pointSynthesizingIter) Value() []byte { + v := i.UnsafeValue() + if v != nil { + v = append([]byte{}, v...) + } + return v +} + +func (i *pointSynthesizingIter) UnsafeValue() []byte { + if i.atPoint { + return i.iter.UnsafeValue() + } + return nil +} + +func (i *pointSynthesizingIter) Close() { + i.iter.Close() +} + +func (i *pointSynthesizingIter) ValueProto(msg protoutil.Message) error { + panic("not implemented") +} + +func (i *pointSynthesizingIter) ComputeStats(start, end roachpb.Key, nowNanos int64) (enginepb.MVCCStats, error) { + panic("not implemented") +} + +func (i *pointSynthesizingIter) FindSplitKey(start, end, minSplitKey roachpb.Key, targetSize int64) (MVCCKey, error) { + panic("not implemented") +} + +func (i *pointSynthesizingIter) SetUpperBound(key roachpb.Key) { + panic("not implemented") +} + +func (i *pointSynthesizingIter) Stats() IteratorStats { + return i.iter.Stats() +} + +func (i *pointSynthesizingIter) SupportsPrev() bool { + panic("not implemented") +} diff --git a/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis new file mode 100644 index 000000000000..a2251eee806d --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_key_point_synthesis @@ -0,0 +1,101 @@ +# Tests pointSynthesizingIter. +# +# Sets up following dataset, where x is tombstone, o-o is range tombstone, [] is intent. +# +# T +# 7 [d7] [j7] +# 6 f6 +# 5 o---------------o k5 +# 4 x x d4 f4 g4 +# 3 o-------o e3 o-------oh3 +# 2 a2 f2 g2 +# 1 o---------------------------------------o +# a b c d e f g h i j k +# +run ok +put_rangekey k=a end=k ts=1 +put_rangekey k=b end=d ts=3 +put k=a ts=2 v=a2 +del k=a ts=4 +del k=b ts=4 +put k=d ts=4 v=d4 +put k=e ts=3 v=e3 +put k=f ts=2 v=f2 +put k=g ts=2 v=g2 +put_rangekey k=f end=h ts=3 +put k=f ts=4 v=f4 +put k=f ts=6 v=f6 +put k=g ts=4 v=g4 +put_rangekey k=c end=g ts=5 +put k=h ts=3 v=h3 +put k=k ts=5 v=k5 +with t=A + txn_begin ts=7 + put k=d v=d7 + put k=j v=j7 +---- +>> 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-g}/[5.000000000,0 3.000000000,0 1.000000000,0] +rangekey: {g-h}/[3.000000000,0 1.000000000,0] +rangekey: {h-k}/[1.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +data: "b"/4.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=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +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 +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 + +# Iterate across the entire span. +run ok +iter_new types=pointsAndRanges pointSynthesis +iter_seek_ge k=a +iter_scan +---- +iter_seek_ge: "a"/4.000000000,0=/ +iter_scan: "a"/4.000000000,0=/ +iter_scan: "a"/2.000000000,0=/BYTES/a2 +iter_scan: "a"/1.000000000,0=/ +iter_scan: "b"/4.000000000,0=/ +iter_scan: "b"/3.000000000,0=/ +iter_scan: "b"/1.000000000,0=/ +iter_scan: "d"/0,0=txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +iter_scan: "d"/7.000000000,0=/BYTES/d7 +iter_scan: "d"/5.000000000,0=/ +iter_scan: "d"/4.000000000,0=/BYTES/d4 +iter_scan: "d"/1.000000000,0=/ +iter_scan: "e"/5.000000000,0=/ +iter_scan: "e"/3.000000000,0=/BYTES/e3 +iter_scan: "e"/1.000000000,0=/ +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=/ +iter_scan: "f"/2.000000000,0=/BYTES/f2 +iter_scan: "f"/1.000000000,0=/ +iter_scan: "g"/4.000000000,0=/BYTES/g4 +iter_scan: "g"/3.000000000,0=/ +iter_scan: "g"/2.000000000,0=/BYTES/g2 +iter_scan: "g"/1.000000000,0=/ +iter_scan: "h"/3.000000000,0=/BYTES/h3 +iter_scan: "h"/1.000000000,0=/ +iter_scan: "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 +iter_scan: "j"/7.000000000,0=/BYTES/j7 +iter_scan: "j"/1.000000000,0=/ +iter_scan: "k"/5.000000000,0=/BYTES/k5 +iter_scan: "k"/1.000000000,0=/ +iter_scan: . diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_scans b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans new file mode 100644 index 000000000000..05b5500f2097 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_scans @@ -0,0 +1,249 @@ +# Tests MVCC scans and gets across range tombstones. +# +# Sets up following dataset, where x is tombstone, o-o is range tombstone, [] is intent. +# +# T +# 6 [e6] +# 5 f5 +# 4 o------------------o +# 3 x d3 f3 +# 2 o---------------o +# 1 a1 x c1 f1 +# a b c d e f g h +# +run ok +put k=a ts=1 v=a1 +del k=b ts=1 +put k=c ts=1 v=c1 +put k=f ts=1 v=f1 +del_range_ts k=a end=e ts=2 +del k=a ts=3 +put k=d ts=3 v=d3 +put k=f ts=3 v=f3 +del_range_ts k=c end=h ts=4 +put k=f ts=5 v=f5 +with t=A + txn_begin ts=6 + put k=e v=e6 +---- +>> 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: {a-c}/[2.000000000,0] +rangekey: {c-e}/[4.000000000,0 2.000000000,0] +rangekey: {e-h}/[4.000000000,0] +data: "a"/3.000000000,0 -> / +data: "a"/1.000000000,0 -> /BYTES/a1 +data: "b"/1.000000000,0 -> / +data: "c"/1.000000000,0 -> /BYTES/c1 +data: "d"/3.000000000,0 -> /BYTES/d3 +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=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true +data: "e"/6.000000000,0 -> /BYTES/e6 +data: "f"/5.000000000,0 -> /BYTES/f5 +data: "f"/3.000000000,0 -> /BYTES/f3 +data: "f"/1.000000000,0 -> /BYTES/f1 + +# Run non-tombstone scans at all timestamps. +run ok +scan k=a end=z ts=1 +---- +scan: "a" -> /BYTES/a1 @1.000000000,0 +scan: "c" -> /BYTES/c1 @1.000000000,0 +scan: "f" -> /BYTES/f1 @1.000000000,0 + +run ok +scan k=a end=z ts=2 +---- +scan: "f" -> /BYTES/f1 @1.000000000,0 + +run ok +scan k=a end=z ts=3 +---- +scan: "d" -> /BYTES/d3 @3.000000000,0 +scan: "f" -> /BYTES/f3 @3.000000000,0 + +run ok +scan k=a end=z ts=4 +---- +scan: "a"-"z" -> + +run ok +scan k=a end=z ts=5 +---- +scan: "f" -> /BYTES/f5 @5.000000000,0 + +run ok +scan k=a end=z ts=6 inconsistent +---- +scan: "a" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} +scan: "f" -> /BYTES/f5 @5.000000000,0 + +# Run tombstone scans at all timestamps. +run ok +scan k=a end=z ts=1 tombstones +---- +scan: "a" -> /BYTES/a1 @1.000000000,0 +scan: "b" -> / @1.000000000,0 +scan: "c" -> /BYTES/c1 @1.000000000,0 +scan: "f" -> /BYTES/f1 @1.000000000,0 + +run ok +scan k=a end=z ts=2 tombstones +---- +scan: "a" -> / @2.000000000,0 +scan: "b" -> / @2.000000000,0 +scan: "c" -> / @2.000000000,0 +scan: "d" -> / @2.000000000,0 +scan: "f" -> /BYTES/f1 @1.000000000,0 + +run ok +scan k=a end=z ts=3 tombstones +---- +scan: "a" -> / @3.000000000,0 +scan: "b" -> / @2.000000000,0 +scan: "c" -> / @2.000000000,0 +scan: "d" -> /BYTES/d3 @3.000000000,0 +scan: "f" -> /BYTES/f3 @3.000000000,0 + +run ok +scan k=a end=z ts=4 tombstones +---- +scan: "a" -> / @3.000000000,0 +scan: "b" -> / @2.000000000,0 +scan: "c" -> / @4.000000000,0 +scan: "d" -> / @4.000000000,0 +scan: "e" -> / @4.000000000,0 +scan: "f" -> / @4.000000000,0 + +run ok +scan k=a end=z ts=5 tombstones +---- +scan: "a" -> / @3.000000000,0 +scan: "b" -> / @2.000000000,0 +scan: "c" -> / @4.000000000,0 +scan: "d" -> / @4.000000000,0 +scan: "e" -> / @4.000000000,0 +scan: "f" -> /BYTES/f5 @5.000000000,0 + +run ok +scan k=a end=z ts=6 tombstones inconsistent +---- +scan: "a" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} +scan: "a" -> / @3.000000000,0 +scan: "b" -> / @2.000000000,0 +scan: "c" -> / @4.000000000,0 +scan: "d" -> / @4.000000000,0 +scan: "e" -> / @4.000000000,0 +scan: "f" -> /BYTES/f5 @5.000000000,0 + +# Try some corresponding gets. +run ok +get k=a ts=1 +get k=a ts=2 +get k=a ts=3 +get k=a ts=1 tombstones +get k=a ts=2 tombstones +get k=a ts=3 tombstones +---- +get: "a" -> /BYTES/a1 @1.000000000,0 +get: "a" -> +get: "a" -> +get: "a" -> /BYTES/a1 @1.000000000,0 +get: "a" -> / @2.000000000,0 +get: "a" -> / @3.000000000,0 + +run ok +get k=b ts=1 +get k=b ts=2 +get k=b ts=3 +get k=b ts=1 tombstones +get k=b ts=2 tombstones +get k=b ts=3 tombstones +---- +get: "b" -> +get: "b" -> +get: "b" -> +get: "b" -> / @1.000000000,0 +get: "b" -> / @2.000000000,0 +get: "b" -> / @2.000000000,0 + +run ok +get k=c ts=1 +get k=c ts=2 +get k=c ts=3 +get k=c ts=4 +get k=c ts=5 +get k=c ts=1 tombstones +get k=c ts=2 tombstones +get k=c ts=3 tombstones +get k=c ts=4 tombstones +get k=c ts=5 tombstones +---- +get: "c" -> /BYTES/c1 @1.000000000,0 +get: "c" -> +get: "c" -> +get: "c" -> +get: "c" -> +get: "c" -> /BYTES/c1 @1.000000000,0 +get: "c" -> / @2.000000000,0 +get: "c" -> / @2.000000000,0 +get: "c" -> / @4.000000000,0 +get: "c" -> / @4.000000000,0 + +run ok +get k=d ts=1 +get k=d ts=2 +get k=d ts=3 +get k=d ts=4 +get k=d ts=5 +get k=d ts=1 tombstones +get k=d ts=2 tombstones +get k=d ts=3 tombstones +get k=d ts=4 tombstones +get k=d ts=5 tombstones +---- +get: "d" -> +get: "d" -> +get: "d" -> /BYTES/d3 @3.000000000,0 +get: "d" -> +get: "d" -> +get: "d" -> +get: "d" -> / @2.000000000,0 +get: "d" -> /BYTES/d3 @3.000000000,0 +get: "d" -> / @4.000000000,0 +get: "d" -> / @4.000000000,0 + +run ok +get k=e ts=3 inconsistent +get k=e ts=4 inconsistent +get k=e ts=5 inconsistent +get k=e ts=6 inconsistent +get k=e ts=3 tombstones inconsistent +get k=e ts=4 tombstones inconsistent +get k=e ts=5 tombstones inconsistent +get k=e ts=6 tombstones inconsistent +---- +get: "e" -> +get: "e" -> +get: "e" -> +get: "e" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} +get: "e" -> +get: "e" -> +get: "e" -> / @4.000000000,0 +get: "e" -> / @4.000000000,0 +get: "e" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=6.000000000,0 min=0,0 seq=0} +get: "e" -> / @4.000000000,0 + +run ok +get k=g ts=3 +get k=g ts=4 +get k=g ts=5 +get k=g ts=3 tombstones +get k=g ts=4 tombstones +get k=g ts=5 tombstones +---- +get: "g" -> +get: "g" -> +get: "g" -> +get: "g" -> +get: "g" -> / @4.000000000,0 +get: "g" -> / @4.000000000,0