Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86259: storage: implement `RangeKeyChanged()` for `MVCCIncrementalIterator` r=tbg a=erikgrinaker

**storage: use `RangeKeyChanged()` in `MVCCIncrementalIterator`**

This patch uses `RangeKeyChanged()` to detect changes to range keys in
`MVCCIncrementalIterator`. There are no functional changes.

Some quick benchmarks, using catchup scans:

```
name                                                                 old time/op  new time/op  delta
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=0-24      538ms ± 1%   535ms ± 1%     ~     (p=0.211 n=10+9)
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=1-24      690ms ± 0%   590ms ± 1%  -14.56%  (p=0.000 n=9+10)
CatchUpScan/mixed-case/withDiff=true/perc=0.00/numRangeKeys=100-24    743ms ± 1%   646ms ± 1%  -13.13%  (p=0.000 n=9+9)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=0-24     794ms ± 1%   794ms ± 0%     ~     (p=0.579 n=10+10)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=1-24     966ms ± 0%   911ms ± 1%   -5.72%  (p=0.000 n=10+10)
CatchUpScan/mixed-case/withDiff=false/perc=0.00/numRangeKeys=100-24   974ms ± 0%   920ms ± 0%   -5.51%  (p=0.000 n=10+10)
```

Release justification: bug fixes and low-risk updates to new functionality.

Release note: None

**storage: implement `RangeKeyChanged()` for `MVCCIncrementalIterator`**

This patch implements `RangeKeyChanged()` for `MVCCIncrementalIterator`.
It only fires if the time bound range keys change, not if a
`Next(Key)IgnoringTime()` operation reveals additional range keys.

Resolves #86105.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None
  
**storage: add `MVCCIncrementalIterator.RangeKeysIgnoringTime()`**

This patch changes `MVCCIncrementalIterator` range key behavior
following a `Next(Key)IgnoringTime()` call. Previously, range key
methods would then expose unfiltered range keys. Now, the standard range
key methods only apply to filtered range keys, and an additional
`RangeKeysIgnoringTime()` method provides access to unfiltered range
keys.

This implies that if such a call steps onto a range key that's entirely
outside of the time bounds then:

* `HasPointAndRange()` will return `false`,`false` if on a bare range
  key.

* `RangeKeyChanged()` will not fire, unless stepping off of a range key
  within the time bounds.

* `RangeBounds()` and `RangeKeys()` will return empty results.

This is done to avoid conditional range key handling following these
calls, except for the exact sites where the caller is interested in
the unfiltered range keys.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86440: storage: use concrete `pebbleIterator` in `intentInterleavingIter` r=tbg a=erikgrinaker

**storage: tweak `unsafeMVCCIterator` construction**

Release justification: non-production code changes

Release note: None

**storage: inline some `intentInterleavingIter` methods**

This patch splits up `maybeSkipIntentRangeKeys()` and
`maybeSuppressRangeKeyChanged()` to allow for mid-stack inlining.

I doubt that the gains are as large as these microbenchmarks claim, and
there's a fair bit of variance between runs, but it can't hurt.

```
name                                                                         old time/op    new time/op    delta
MVCCScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24               5.37µs ± 2%    5.43µs ± 2%  +1.13%  (p=0.041 n=10+10)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24             38.2µs ± 2%    38.2µs ± 2%    ~     (p=0.971 n=10+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24           2.79ms ± 2%    2.71ms ± 2%  -2.59%  (p=0.000 n=10+10)
MVCCReverseScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24        5.99µs ± 1%    5.99µs ± 2%    ~     (p=0.541 n=10+10)
MVCCReverseScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24      51.7µs ± 3%    52.1µs ± 1%    ~     (p=0.631 n=10+10)
MVCCReverseScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24    3.88ms ± 2%    3.87ms ± 1%    ~     (p=0.897 n=10+8)
MVCCComputeStats_Pebble/valueSize=32/numRangeKeys=0-24                          158ms ± 1%     155ms ± 1%  -2.34%  (p=0.000 n=10+9)
```

Touches #82559.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None
  
**storage: use concrete `pebbleIterator` in `intentInterleavingIter`**

Since `intentInterleavingIter` always constructs the underlying
iterators from the given reader, and these readers always construct
`*pebbleIterator`, it can use the concrete type rather than interfaces
for both iterators. This avoids dynamic dispatch, yielding a decent
performance improvement.

Unfortunately, this requires disabling `unsafeMVCCIterator` inside
`intentInterleavingIter`. This wasn't always enabled anyway, since it
was omitted both for the engine iterator and when using an engine
directly (which doesn't have consistent iterators).

```
name                                                                         old time/op    new time/op    delta
MVCCScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24               5.43µs ± 2%    5.45µs ± 2%    ~     (p=0.566 n=10+10)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24             38.2µs ± 2%    37.0µs ± 1%  -3.02%  (p=0.000 n=10+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24           2.71ms ± 2%    2.66ms ± 1%  -1.83%  (p=0.000 n=10+9)
MVCCReverseScan_Pebble/rows=1/versions=1/valueSize=64/numRangeKeys=0-24        5.99µs ± 2%    5.86µs ± 2%  -2.15%  (p=0.000 n=10+10)
MVCCReverseScan_Pebble/rows=100/versions=1/valueSize=64/numRangeKeys=0-24      52.1µs ± 1%    50.2µs ± 2%  -3.77%  (p=0.000 n=10+10)
MVCCReverseScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24    3.87ms ± 1%    3.83ms ± 1%  -1.26%  (p=0.000 n=8+10)
MVCCComputeStats_Pebble/valueSize=32/numRangeKeys=0-24                          155ms ± 1%     155ms ± 1%    ~     (p=0.842 n=9+10)
```

Touches #82559.

Release justification: low risk, high benefit changes to existing functionality

Release note: None

86478: storage: use concrete `pebbleIterator` in `verifyingIterator` r=tbg a=erikgrinaker

**storage: add Pebble SST iterator benchmarks**

Release justification: non-production code changes

Release note: None
  
**storage: use concrete `pebbleIterator` in `verifyingIterator`**

Gives a slight performance boost, since it avoid dynamic dispatch:

```
name                                                  old time/op  new time/op  delta
SSTIterator/keys=1/variant=pebble/verify=true-24      42.8µs ± 1%  42.4µs ± 1%  -0.79%  (p=0.043 n=10+10)
SSTIterator/keys=100/variant=pebble/verify=true-24    61.8µs ± 1%  60.7µs ± 1%  -1.64%  (p=0.000 n=10+10)
SSTIterator/keys=10000/variant=pebble/verify=true-24  1.91ms ± 0%  1.88ms ± 0%  -1.79%  (p=0.000 n=10+10)
```

An attempt was also made at using `RangeKeyChanged()` instead of
`HasPointAndRange()`, but this had no effect.

Touches #83051.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86513: storage: use `RangeKeyChanged` in `ReadAsOfIterator` r=tbg a=erikgrinaker

This patch uses `RangeKeyChanged()` to detect range keys in
`ReadAsOfIterator`, and caches them to improve performance.
It also fixes a bug where the iterator would fail to detect tombstones
with a non-empty `MVCCValueHeader`.

Resolves #84714.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86514: storage: use `RangeKeyChanged()` in `MVCCDeleteRangeUsingTombstone()` r=tbg a=erikgrinaker

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86529: storage: omit unnecessary range key calls during intent resolution r=tbg a=erikgrinaker

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
craig[bot] and erikgrinaker committed Aug 22, 2022
7 parents 0e861ec + 1b064b4 + e209dc1 + a86ed19 + b184a71 + cc25f74 + 9061c76 commit 9c7db33
Show file tree
Hide file tree
Showing 16 changed files with 897 additions and 681 deletions.
20 changes: 7 additions & 13 deletions pkg/kv/kvserver/rangefeed/catchup_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
type simpleCatchupIter interface {
storage.SimpleMVCCIterator
NextIgnoringTime()
RangeKeysIgnoringTime() storage.MVCCRangeKeyStack
}

type simpleCatchupIterAdapter struct {
Expand All @@ -44,6 +45,10 @@ func (i simpleCatchupIterAdapter) NextIgnoringTime() {
i.SimpleMVCCIterator.Next()
}

func (i simpleCatchupIterAdapter) RangeKeysIgnoringTime() storage.MVCCRangeKeyStack {
return i.SimpleMVCCIterator.RangeKeys()
}

var _ simpleCatchupIter = simpleCatchupIterAdapter{}

// CatchUpIterator is an iterator for catchup-scans.
Expand Down Expand Up @@ -145,8 +150,6 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err

// Emit any new MVCC range tombstones when their start key is encountered.
// Range keys can currently only be MVCC range tombstones.
// We need to verify that the range tombstone is visible at the catch-up
// timestamp, since we might have come here after a call to NextIgnoringTime.
//
// TODO(erikgrinaker): Find a faster/better way to detect range key changes
// that doesn't involve constant comparisons. Pebble probably already knows,
Expand All @@ -161,11 +164,6 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err
// Emit events for these MVCC range tombstones, in chronological order.
versions := i.RangeKeys().Versions
for j := len(versions) - 1; j >= 0; j-- {
if !i.startTime.LessEq(versions[j].Timestamp) {
// This range tombstone isn't visible by this catch-up scan.
continue
}

var span roachpb.Span
a, span.Key = a.Copy(rangeBounds.Key, 0)
a, span.EndKey = a.Copy(rangeBounds.EndKey, 0)
Expand Down Expand Up @@ -285,12 +283,8 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err
}
// If an MVCC range tombstone exists between this value and the next
// one, we don't emit the value after all -- it should be a tombstone.
//
// TODO(erikgrinaker): We can't save range keys when we detect changes
// to rangeKeysStart above, because NextIgnoringTime() could reveal
// additional MVCC range tombstones below StartTime that cover this
// point. We need to find a more performant way to handle this.
if !hasRange || !i.RangeKeys().HasBetween(ts, reorderBuf[l].Val.Value.Timestamp) {
// The RangeKeysIgnoringTime() call is cheap, no need for caching.
if !i.RangeKeysIgnoringTime().HasBetween(ts, reorderBuf[l].Val.Value.Timestamp) {
// TODO(sumeer): find out if it is deliberate that we are not populating
// PrevValue.Timestamp.
reorderBuf[l].Val.PrevValue.RawBytes = val
Expand Down
10 changes: 7 additions & 3 deletions pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,9 +494,13 @@ func BenchmarkCheckSSTConflicts(b *testing.B) {
func BenchmarkSSTIterator(b *testing.B) {
for _, numKeys := range []int{1, 100, 10000} {
b.Run(fmt.Sprintf("keys=%d", numKeys), func(b *testing.B) {
for _, verify := range []bool{false, true} {
b.Run(fmt.Sprintf("verify=%t", verify), func(b *testing.B) {
runSSTIterator(b, numKeys, verify)
for _, variant := range []string{"legacy", "pebble"} {
b.Run(fmt.Sprintf("variant=%s", variant), func(b *testing.B) {
for _, verify := range []bool{false, true} {
b.Run(fmt.Sprintf("verify=%t", verify), func(b *testing.B) {
runSSTIterator(b, variant, numKeys, verify)
})
}
})
}
})
Expand Down
22 changes: 20 additions & 2 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1845,7 +1845,7 @@ func runCheckSSTConflicts(
}
}

func runSSTIterator(b *testing.B, numKeys int, verify bool) {
func runSSTIterator(b *testing.B, variant string, numKeys int, verify bool) {
keyBuf := append(make([]byte, 0, 64), []byte("key-")...)
value := MVCCValue{Value: roachpb.MakeValueFromBytes(bytes.Repeat([]byte("a"), 128))}

Expand All @@ -1861,9 +1861,27 @@ func runSSTIterator(b *testing.B, numKeys int, verify bool) {
}
sstWriter.Close()

var makeSSTIterator func(data []byte, verify bool) (SimpleMVCCIterator, error)
switch variant {
case "legacy":
makeSSTIterator = func(data []byte, verify bool) (SimpleMVCCIterator, error) {
return NewMemSSTIterator(data, verify)
}
case "pebble":
makeSSTIterator = func(data []byte, verify bool) (SimpleMVCCIterator, error) {
return NewPebbleMemSSTIterator(data, verify, IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
LowerBound: keys.MinKey,
UpperBound: keys.MaxKey,
})
}
default:
b.Fatalf("unknown variant %q", variant)
}

b.ResetTimer()
for i := 0; i < b.N; i++ {
iter, err := NewMemSSTIterator(sstFile.Bytes(), verify)
iter, err := makeSSTIterator(sstFile.Bytes(), verify)
if err != nil {
b.Fatal(err)
}
Expand Down
117 changes: 76 additions & 41 deletions pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ type intentInterleavingIter struct {
constraint intentInterleavingIterConstraint

// iter is for iterating over MVCC keys and interleaved intents.
iter MVCCIterator
iter *pebbleIterator // MVCCIterator
// The valid value from iter.Valid() after the last positioning call.
iterValid bool
// When iterValid = true, this contains the result of iter.UnsafeKey(). We
Expand All @@ -96,7 +96,7 @@ type intentInterleavingIter struct {

// intentIter is for iterating over separated intents, so that
// intentInterleavingIter can make them look as if they were interleaved.
intentIter EngineIterator
intentIter *pebbleIterator // EngineIterator
intentIterState pebble.IterValidityState
// The decoded key from the lock table. This is an unsafe key
// in that it is only valid when intentIter has not been
Expand Down Expand Up @@ -255,17 +255,21 @@ func newIntentInterleavingIterator(reader Reader, opts IterOptions) MVCCIterator
// constrainedToGlobal.
intentOpts.UpperBound = keys.LockTableSingleKeyEnd
}

// All readers given to intentInterleavingIter construct pebbleIterators, so
// we can use the concrete type here to avoid the cost of dynamic dispatch.
//
// Note that we can reuse intentKeyBuf, intentLimitKeyBuf after
// NewEngineIterator returns.
intentIter := reader.NewEngineIterator(intentOpts)
intentIter := reader.NewEngineIterator(intentOpts).(*pebbleIterator)

// The creation of these iterators can race with concurrent mutations, which
// may make them inconsistent with each other. So we clone here, to ensure
// consistency (certain Reader implementations already ensure consistency,
// and we use that when possible to save allocations).
var iter MVCCIterator
var iter *pebbleIterator
if reader.ConsistentIterators() {
iter = reader.NewMVCCIterator(MVCCKeyIterKind, opts)
iter = maybeUnwrapUnsafeIter(reader.NewMVCCIterator(MVCCKeyIterKind, opts)).(*pebbleIterator)
} else {
iter = newPebbleIteratorByCloning(
intentIter.GetRawIter(), opts, StandardDurability, reader.SupportsRangeKeys())
Expand Down Expand Up @@ -345,45 +349,54 @@ func (i *intentInterleavingIter) makeLowerLimitKey() roachpb.Key {
// NB: This is called before computePos(), and can't rely on intentCmp.
//
// REQUIRES: i.dir > 0
//
// gcassert:inline
func (i *intentInterleavingIter) maybeSkipIntentRangeKey() error {
if util.RaceEnabled && i.dir < 0 {
i.err = errors.AssertionFailedf("maybeSkipIntentRangeKey called in reverse")
i.valid = false
return i.err
}
if i.iterValid && i.intentKey != nil {
if hasPoint, hasRange := i.iter.HasPointAndRange(); hasRange && !hasPoint {
// iter may be on a bare range key that will cover the provisional value,
// in which case we can step onto it. We guard against emitting the wrong
// range key for the intent if the provisional value turns out to be
// missing by:
//
// 1. Before we step, make sure iter isn't ahead of intentIter. We have
// to do a key comparison anyway in case intentIter is ahead of iter.
// 2. After we step, make sure we're on a point key covered by a range key.
// We don't need a key comparison (but do so under race), because if
// the provisional value is missing then we'll either land on a
// different point key below the range key (which will emit the
// correct range key), or we'll land on a different bare range key.
//
// TODO(erikgrinaker): in cases where we don't step iter, we can save
// the result of the comparison in i.intentCmp to avoid another one.
if intentCmp := i.intentKey.Compare(i.iterKey.Key); intentCmp < 0 {
i.err = errors.Errorf("iter ahead of provisional value for intent %s (at %s)",
i.intentKey, i.iterKey)
return i.doMaybeSkipIntentRangeKey()
}
return nil
}

// doMaybeSkipIntentRangeKey is a helper for maybeSkipIntentRangeKey(), which
// allows mid-stack inlining of the former.
func (i *intentInterleavingIter) doMaybeSkipIntentRangeKey() error {
if hasPoint, hasRange := i.iter.HasPointAndRange(); hasRange && !hasPoint {
// iter may be on a bare range key that will cover the provisional value,
// in which case we can step onto it. We guard against emitting the wrong
// range key for the intent if the provisional value turns out to be
// missing by:
//
// 1. Before we step, make sure iter isn't ahead of intentIter. We have
// to do a key comparison anyway in case intentIter is ahead of iter.
// 2. After we step, make sure we're on a point key covered by a range key.
// We don't need a key comparison (but do so under race), because if
// the provisional value is missing then we'll either land on a
// different point key below the range key (which will emit the
// correct range key), or we'll land on a different bare range key.
//
// TODO(erikgrinaker): in cases where we don't step iter, we can save
// the result of the comparison in i.intentCmp to avoid another one.
if intentCmp := i.intentKey.Compare(i.iterKey.Key); intentCmp < 0 {
i.err = errors.Errorf("iter ahead of provisional value for intent %s (at %s)",
i.intentKey, i.iterKey)
i.valid = false
return i.err
} else if intentCmp == 0 {
i.iter.Next()
if err := i.tryDecodeKey(); err != nil {
return err
}
hasPoint, hasRange = i.iter.HasPointAndRange()
if !hasPoint || !hasRange || (util.RaceEnabled && !i.iterKey.Key.Equal(i.intentKey)) {
i.err = errors.Errorf("iter not on provisional value for intent %s", i.intentKey)
i.valid = false
return i.err
} else if intentCmp == 0 {
i.iter.Next()
if err := i.tryDecodeKey(); err != nil {
return err
}
hasPoint, hasRange = i.iter.HasPointAndRange()
if !hasPoint || !hasRange || (util.RaceEnabled && !i.iterKey.Key.Equal(i.intentKey)) {
i.err = errors.Errorf("iter not on provisional value for intent %s", i.intentKey)
i.valid = false
return i.err
}
}
}
}
Expand All @@ -394,16 +407,24 @@ func (i *intentInterleavingIter) maybeSkipIntentRangeKey() error {
// direction if the underlying iterator has moved past an intent onto a
// different range key that should not be surfaced yet. Must be called after
// computePos().
//
// gcassert:inline
func (i *intentInterleavingIter) maybeSuppressRangeKeyChanged() {
if util.RaceEnabled && i.dir > 0 {
panic(errors.AssertionFailedf("maybeSuppressRangeKeyChanged called in forward direction"))
}
if i.rangeKeyChanged && i.isCurAtIntentIterReverse() && i.intentCmp > 0 &&
i.iter.RangeBounds().EndKey.Compare(i.intentKey) <= 0 {
i.rangeKeyChanged = false
// NB: i.intentCmp implies isCurAtIntentIterReverse(), but cheaper.
if i.rangeKeyChanged && i.intentCmp > 0 {
i.doMaybeSuppressRangeKeyChanged()
}
}

// doMaybeSuppressRangeKeyChanges is a helper for maybeSuppressRangeKeyChanged
// which allows mid-stack inlining of the former.
func (i *intentInterleavingIter) doMaybeSuppressRangeKeyChanged() {
i.rangeKeyChanged = i.iter.RangeBounds().EndKey.Compare(i.intentKey) > 0
}

func (i *intentInterleavingIter) SeekGE(key MVCCKey) {
i.dir = +1
i.valid = true
Expand Down Expand Up @@ -1245,7 +1266,7 @@ func (i *intentInterleavingIter) SupportsPrev() bool {
return true
}

// unsageMVCCIterator is used in RaceEnabled test builds to randomly inject
// unsafeMVCCIterator is used in RaceEnabled test builds to randomly inject
// changes to unsafe keys retrieved from MVCCIterators.
type unsafeMVCCIterator struct {
MVCCIterator
Expand All @@ -1254,8 +1275,22 @@ type unsafeMVCCIterator struct {
rawMVCCKeyBuf []byte
}

func wrapInUnsafeIter(iter MVCCIterator) MVCCIterator {
return &unsafeMVCCIterator{MVCCIterator: iter}
// gcassert:inline
func maybeWrapInUnsafeIter(iter MVCCIterator) MVCCIterator {
if util.RaceEnabled {
return &unsafeMVCCIterator{MVCCIterator: iter}
}
return iter
}

// gcassert:inline
func maybeUnwrapUnsafeIter(iter MVCCIterator) MVCCIterator {
if util.RaceEnabled {
if unsafeIter, ok := iter.(*unsafeMVCCIterator); ok {
return unsafeIter.MVCCIterator
}
}
return iter
}

var _ MVCCIterator = &unsafeMVCCIterator{}
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/intent_interleaving_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@ func TestIntentInterleavingIter(t *testing.T) {
if d.HasArg("prefix") {
d.ScanArgs(t, "prefix", &opts.Prefix)
}
iter := wrapInUnsafeIter(newIntentInterleavingIterator(eng, opts))
iter := maybeWrapInUnsafeIter(newIntentInterleavingIterator(eng, opts))
var b strings.Builder
defer iter.Close()
// pos is the original <file>:<lineno> prefix computed by
Expand Down
16 changes: 9 additions & 7 deletions pkg/storage/multi_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,21 +114,21 @@ func populateBatch(t *testing.T, batch Batch, input string) {
}
k := []byte{input[i]}
ts := hlc.Timestamp{WallTime: int64(input[i+1])}
var v []byte
var v MVCCValue
if i+1 < len(input) && input[i+1] == 'M' {
ts = hlc.Timestamp{}
v = nil
} else if i+2 < len(input) && input[i+2] == 'X' {
v = nil
i++
} else {
v = []byte{input[i+1]}
v.Value.SetString(string(input[i+1]))
}
i += 2
if ts.IsEmpty() {
require.NoError(t, batch.PutUnversioned(k, v))
vRaw, err := EncodeMVCCValue(v)
require.NoError(t, err)
require.NoError(t, batch.PutUnversioned(k, vRaw))
} else {
require.NoError(t, batch.PutRawMVCC(MVCCKey{Key: k, Timestamp: ts}, v))
require.NoError(t, batch.PutMVCC(MVCCKey{Key: k, Timestamp: ts}, v))
}
}
}
Expand All @@ -154,7 +154,9 @@ func iterateSimpleMultiIter(t *testing.T, it SimpleMVCCIterator, subtest iterSub
output.WriteRune('M')
} else {
output.WriteByte(byte(it.UnsafeKey().Timestamp.WallTime))
if len(it.UnsafeValue()) == 0 {
v, err := DecodeMVCCValue(it.UnsafeValue())
require.NoError(t, err)
if v.IsTombstone() {
output.WriteRune('X')
}
}
Expand Down
Loading

0 comments on commit 9c7db33

Please sign in to comment.