Skip to content

Commit

Permalink
storage: add MVCCIncrementalIterator.RangeKeysIgnoringTime()
Browse files Browse the repository at this point in the history
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. The additional
`RangeKeysIgnoringTime()` and `RangeKeyChangedIgnoringTime()` methods
provide 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
  • Loading branch information
erikgrinaker committed Aug 20, 2022
1 parent 290926e commit 1b064b4
Show file tree
Hide file tree
Showing 5 changed files with 166 additions and 108 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
35 changes: 12 additions & 23 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -2622,16 +2622,8 @@ func MVCCClearTimeRange(
break
}

// Because we're using NextIgnoringTime() to look for older keys, it's
// possible that the iterator will surface range keys outside of the time
// bounds, so we need to do additional filtering here.
//
// TODO(erikgrinaker): Consider a Clone() variant that can reuse a buffer.
// See also TODO in Clone() to use a single allocation for all byte
// slices.
rangeKeys := iter.RangeKeys()
rangeKeys.Trim(startTime.Next(), endTime)
clearRangeKeys = rangeKeys.Clone()
clearRangeKeys = iter.RangeKeys().Clone()
}

if hasPoint, _ := iter.HasPointAndRange(); !hasPoint {
Expand Down Expand Up @@ -2662,7 +2654,7 @@ func MVCCClearTimeRange(
// cleared key, then we didn't restore it after all, but we must still
// adjust the stats for the range tombstone.
if !restoredMeta.Deleted {
if v, ok := iter.RangeKeys().FirstAbove(k.Timestamp); ok {
if v, ok := iter.RangeKeysIgnoringTime().FirstAbove(k.Timestamp); ok {
if v.Timestamp.LessEq(clearedMeta.Timestamp.ToTimestamp()) {
restoredMeta.Deleted = true
restoredMeta.KeyBytes = 0
Expand Down Expand Up @@ -2700,14 +2692,9 @@ func MVCCClearTimeRange(
// we cleared or a different range tombstone below the one we cleared.
if !v.IsTombstone() {
if v, ok := clearRangeKeys.FirstAbove(k.Timestamp); ok {
// TODO(erikgrinaker): We have to fetch the complete set of range keys
// as seen by this key -- these may or may not be filtered by timestamp
// depending on whether we did a NextIgnoringTime(), so we have to fetch
// the entire set rather than using clearedRangeKeys. We should optimize
// this somehow.
if !clearedMetaKey.Key.Equal(k.Key) ||
!clearedMeta.Timestamp.ToTimestamp().LessEq(v.Timestamp) {
if !iter.RangeKeys().HasBetween(v.Timestamp.Prev(), k.Timestamp) {
if !iter.RangeKeysIgnoringTime().HasBetween(k.Timestamp, v.Timestamp.Prev()) {
ms.Add(enginepb.MVCCStats{
LastUpdateNanos: v.Timestamp.WallTime,
LiveCount: 1,
Expand Down Expand Up @@ -2921,15 +2908,17 @@ func MVCCPredicateDeleteRange(
//
// 2) The latest key is live, matches the predicates, and has a
// timestamp below EndTime.
continueRun := func(k MVCCKey, iter SimpleMVCCIterator,
continueRun := func(k MVCCKey, iter *MVCCIncrementalIterator,
) (toContinue bool, isPointTombstone bool, isRangeTombstone bool, err error) {
hasPointKey, hasRangeKey := iter.HasPointAndRange()
// We need to see the full, unfiltered set of range keys, ignoring time
// bounds. The RangeKeysIgnoringTime() call is cheap.
hasPointKey, _ := iter.HasPointAndRange()
rangeKeys := iter.RangeKeysIgnoringTime()
hasRangeKey := !rangeKeys.IsEmpty()

if hasRangeKey {
// TODO (msbutler): cache the range keys while the range bounds remain
// constant, since iter.RangeKeys() is expensive. Manual caching may not be necessary if
// https://github.com/cockroachdb/cockroach/issues/84379 lands.
newestRangeKey := iter.RangeKeys().Versions[0].Timestamp
if endTime.LessEq(newestRangeKey) {
newestRangeKey := rangeKeys.Newest()
if endTime.LessEq(rangeKeys.Newest()) {
return false, false, false, roachpb.NewWriteTooOldError(
endTime, newestRangeKey.Next(), k.Key.Clone())
}
Expand Down
51 changes: 46 additions & 5 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1703,6 +1703,13 @@ func printIter(e *evalCtx) {
e.results.buf.Printf("%s:", e.td.Cmd)
defer e.results.buf.Printf("\n")

// MVCCIncrementalIterator has odd behavior following a NextIgnoringTime()
// call, so we detect this and adjust expectations.
var incrIterIgnoringTime bool
if incrIter, ok := e.bareIter().(*MVCCIncrementalIterator); ok {
incrIterIgnoringTime = incrIter.ignoringTime
}

ok, err := e.iter.Valid()
if err != nil {
e.results.buf.Printf(" err=%v", err)
Expand All @@ -1715,7 +1722,9 @@ func printIter(e *evalCtx) {
}
hasPoint, hasRange := e.iter.HasPointAndRange()
if !hasPoint && !hasRange {
e.t.Fatalf("valid iterator at %s without point nor range keys", e.iter.UnsafeKey())
if !incrIterIgnoringTime || e.mvccIncrementalIter().RangeKeysIgnoringTime().IsEmpty() {
e.t.Fatalf("valid iterator at %s without point nor range keys", e.iter.UnsafeKey())
}
}

if hasPoint {
Expand Down Expand Up @@ -1750,6 +1759,29 @@ func printIter(e *evalCtx) {
e.results.buf.Printf("]")
}

if incrIterIgnoringTime {
rangeKeys := e.mvccIncrementalIter().RangeKeysIgnoringTime()
if !rangeKeys.Equal(e.iter.RangeKeys()) {
e.results.buf.Printf(" (+%s/[", rangeKeys.Bounds)
for i, version := range rangeKeys.Versions {
value, err := DecodeMVCCValue(version.Value)
if err != nil {
e.Fatalf("%v", err)
}
if i > 0 {
e.results.buf.Printf(" ")
}
e.results.buf.Printf("%s=%s", version.Timestamp, value)
}
e.results.buf.Printf("]")
if e.mvccIncrementalIter().RangeKeyChangedIgnoringTime() {
e.results.buf.Printf(" !")
}
e.results.buf.Printf(")")

}
}

if checkAndUpdateRangeKeyChanged(e) {
e.results.buf.Printf(" !")
}
Expand All @@ -1759,10 +1791,19 @@ func checkAndUpdateRangeKeyChanged(e *evalCtx) bool {
rangeKeyChanged := e.iter.RangeKeyChanged()
rangeKeys := e.iter.RangeKeys()

// For MVCCIncrementalIterator, Next(Key)IgnoringTime() may reveal additional
// range key versions, but RangeKeyChanged only applies to the filtered set.
if incrIter, ok := e.bareIter().(*MVCCIncrementalIterator); ok && incrIter.ignoringTime {
rangeKeys = incrIter.rangeKeys
if incrIter, ok := e.bareIter().(*MVCCIncrementalIterator); ok {
// For MVCCIncrementalIterator, make sure RangeKeyChangedIgnoringTime() fires
// whenever RangeKeyChanged() does. The inverse is not true.
rangeKeyChangedIgnoringTime := incrIter.RangeKeyChangedIgnoringTime()
if rangeKeyChanged && !rangeKeyChangedIgnoringTime {
e.t.Fatalf("RangeKeyChanged=%t but RangeKeyChangedIgnoringTime=%t",
rangeKeyChanged, incrIter.RangeKeyChangedIgnoringTime())
}
// If RangeKeyChangedIgnoringTime() fires, and RangeKeyChanged() doesn't,
// then RangeKeys() must be empty.
if rangeKeyChangedIgnoringTime && !rangeKeyChanged && !rangeKeys.IsEmpty() {
e.t.Fatalf("RangeKeyChangedIgnoringTime without RangeKeyChanged, but RangeKeys not empty")
}
}

if rangeKeyChanged != !rangeKeys.Equal(e.iterRangeKeys) {
Expand Down
68 changes: 51 additions & 17 deletions pkg/storage/mvcc_incremental_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,10 @@ type MVCCIncrementalIterator struct {
// positioning operation.
rangeKeyChanged bool

// rangeKeyChangedIgnoringTime is true if i.rangeKeysIgnoringTime changed
// during the previous positioning operation.
rangeKeyChangedIgnoringTime bool

// ignoringTime is true if the iterator is currently ignoring time bounds,
// i.e. following a call to NextIgnoringTime().
ignoringTime bool
Expand Down Expand Up @@ -251,6 +255,7 @@ func (i *MVCCIncrementalIterator) SeekGE(startKey MVCCKey) {
i.iter.SeekGE(startKey)
i.advance(true /* seeked */)
i.rangeKeyChanged = !prevRangeKey.Equal(i.rangeKeys.Bounds.Key) // Is there a better way?
i.rangeKeyChangedIgnoringTime = i.rangeKeyChanged
}

// Close implements SimpleMVCCIterator.
Expand Down Expand Up @@ -476,8 +481,8 @@ func (i *MVCCIncrementalIterator) updateRangeKeys() (bool, bool) {
// intent policy is MVCCIncrementalIterIntentPolicyError.
func (i *MVCCIncrementalIterator) advance(seeked bool) {
i.ignoringTime = false
i.rangeKeyChanged = false
hadRange := !i.rangeKeys.IsEmpty()
i.rangeKeyChanged, i.rangeKeyChangedIgnoringTime = false, false
hadRange, hadRangeIgnoringTime := !i.rangeKeys.IsEmpty(), !i.rangeKeysIgnoringTime.IsEmpty()
for {
if !i.updateValid() {
return
Expand All @@ -499,9 +504,12 @@ func (i *MVCCIncrementalIterator) advance(seeked bool) {
var newRangeKey bool
if rangeKeyChanged {
i.hasPoint, i.hasRange = i.updateRangeKeys()
i.rangeKeyChanged = hadRange || i.hasRange // !hasRange → !hasRange is no change
newRangeKey = i.hasRange

// NB: !hasRange → !hasRange is not a change.
i.rangeKeyChanged = hadRange || i.hasRange
i.rangeKeyChangedIgnoringTime = hadRangeIgnoringTime || !i.rangeKeysIgnoringTime.IsEmpty()

// If we're on a visible, bare range key then we're done. If the range key
// was filtered out by the time bounds (the !hasPoint && !hasRange case),
// then we move on to the next key.
Expand Down Expand Up @@ -581,38 +589,54 @@ func (i *MVCCIncrementalIterator) UnsafeKey() MVCCKey {
}

// HasPointAndRange implements SimpleMVCCIterator.
//
// This only returns hasRange=true if there are filtered range keys present.
// Thus, it is possible for this to return hasPoint=false,hasRange=false
// following a NextIgnoringTime() call if positioned on a bare, filtered
// range key. In this case, the range keys are available via
// RangeKeysIgnoringTime().
func (i *MVCCIncrementalIterator) HasPointAndRange() (bool, bool) {
if i.ignoringTime {
return i.iter.HasPointAndRange()
}
return i.hasPoint, i.hasRange
}

// RangeBounds implements SimpleMVCCIterator.
//
// This only returns the filtered range key bounds. Thus, if a
// NextIgnoringTime() call moves onto an otherwise hidden range key, this will
// still return an empty span. These hidden range keys are available via
// RangeKeysIgnoringTime().
func (i *MVCCIncrementalIterator) RangeBounds() roachpb.Span {
if i.ignoringTime {
return i.rangeKeysIgnoringTime.Bounds
}
return i.rangeKeys.Bounds
}

// RangeKeys implements SimpleMVCCIterator.
func (i *MVCCIncrementalIterator) RangeKeys() MVCCRangeKeyStack {
if i.ignoringTime {
return i.rangeKeysIgnoringTime
}
return i.rangeKeys
}

// RangeKeysIgnoringTime returns the range keys at the current position,
// ignoring time bounds. This call is cheap, so callers do not need to perform
// their own caching.
func (i *MVCCIncrementalIterator) RangeKeysIgnoringTime() MVCCRangeKeyStack {
return i.rangeKeysIgnoringTime
}

// RangeKeyChanged implements SimpleMVCCIterator.
//
// RangeKeyChanged only applies to the filtered set of range keys. If an
// IgnoringTime() operation reveals addition range keys or versions, these do
// not trigger RangeKeyChanged().
// IgnoringTime() operation reveals additional range keys or versions, these do
// not trigger RangeKeyChanged(). See also RangeKeyChangedIgnoringTime().
func (i *MVCCIncrementalIterator) RangeKeyChanged() bool {
return i.rangeKeyChanged
}

// RangeKeyChangedIgnoringTime is like RangeKeyChanged, but returns true if the
// range keys returned by RangeKeysIgnoringTime() changed since the previous
// positioning operation -- in particular, after a Next(Key)IgnoringTime() call.
func (i *MVCCIncrementalIterator) RangeKeyChangedIgnoringTime() bool {
return i.rangeKeyChangedIgnoringTime
}

// UnsafeValue implements SimpleMVCCIterator.
func (i *MVCCIncrementalIterator) UnsafeValue() []byte {
if !i.hasPoint {
Expand All @@ -625,7 +649,7 @@ func (i *MVCCIncrementalIterator) UnsafeValue() []byte {
// intent policy.
func (i *MVCCIncrementalIterator) updateIgnoreTime() {
i.ignoringTime = true
i.rangeKeyChanged = false
i.rangeKeyChanged, i.rangeKeyChangedIgnoringTime = false, false
hadRange := !i.rangeKeys.IsEmpty()
for {
if !i.updateValid() {
Expand All @@ -635,6 +659,7 @@ func (i *MVCCIncrementalIterator) updateIgnoreTime() {
if i.iter.RangeKeyChanged() {
i.hasPoint, i.hasRange = i.updateRangeKeys()
i.rangeKeyChanged = hadRange || i.hasRange // !hasRange → !hasRange is no change
i.rangeKeyChangedIgnoringTime = true
if !i.hasPoint {
i.meta.Reset()
return
Expand Down Expand Up @@ -672,7 +697,16 @@ func (i *MVCCIncrementalIterator) updateIgnoreTime() {
// Intents within and outside the (StartTime, EndTime] time range are handled
// according to the iterator policy.
//
// RangeKeyChanged() will only fire if the time bound range keys change.
// NB: Range key methods only respect the filtered set of range keys. To access
// unfiltered range keys, use RangeKeysIgnoringTime(). This implies that if this
// call steps onto a range key that's entirely outside of the time bounds:
//
// * 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.
func (i *MVCCIncrementalIterator) NextIgnoringTime() {
i.iter.Next()
i.updateIgnoreTime()
Expand All @@ -683,7 +717,7 @@ func (i *MVCCIncrementalIterator) NextIgnoringTime() {
// forward. Intents within and outside the (StartTime, EndTime] time range are
// handled according to the iterator policy.
//
// RangeKeyChanged() will only fire if the time bound range keys change.
// NB: See NextIgnoringTime comment for important details about range keys.
func (i *MVCCIncrementalIterator) NextKeyIgnoringTime() {
i.iter.NextKey()
i.updateIgnoreTime()
Expand Down
Loading

0 comments on commit 1b064b4

Please sign in to comment.