Skip to content

Commit

Permalink
storage: use RangeKeyChanged in ReadAsOfIterator
Browse files Browse the repository at this point in the history
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`.

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

Release note: None
  • Loading branch information
erikgrinaker committed Aug 21, 2022
1 parent 0c6ac53 commit b184a71
Show file tree
Hide file tree
Showing 3 changed files with 47 additions and 38 deletions.
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
61 changes: 31 additions & 30 deletions pkg/storage/read_as_of_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@ type ReadAsOfIterator struct {

// err tracks if iterating to the current key returned an error
err error

// newestRangeTombstone contains the timestamp of the latest range
// tombstone below asOf at the current position, if any.
newestRangeTombstone hlc.Timestamp
}

var _ SimpleMVCCIterator = &ReadAsOfIterator{}
Expand All @@ -51,7 +55,7 @@ func (f *ReadAsOfIterator) SeekGE(originalKey MVCCKey) {
synthetic := MVCCKey{Key: originalKey.Key, Timestamp: f.asOf}
f.iter.SeekGE(synthetic)

if f.advance(); f.valid && f.UnsafeKey().Less(originalKey) {
if f.advance(true /* seeked */); f.valid && f.UnsafeKey().Less(originalKey) {
// The following is true:
// originalKey.Key == f.UnsafeKey &&
// f.asOf timestamp (if set) >= current timestamp > originalKey timestamp.
Expand Down Expand Up @@ -83,7 +87,7 @@ func (f *ReadAsOfIterator) Next() {
// call before any calls to NextKey().
func (f *ReadAsOfIterator) NextKey() {
f.iter.NextKey()
f.advance()
f.advance(false /* seeked */)
}

// UnsafeKey returns the current key, but the memory is invalidated on the next
Expand Down Expand Up @@ -129,43 +133,40 @@ func (f *ReadAsOfIterator) updateValid() bool {
// advance moves past keys with timestamps later than f.asOf and skips MVCC keys
// whose latest value (subject to f.asOF) has been deleted by a point or range
// tombstone.
func (f *ReadAsOfIterator) advance() {
func (f *ReadAsOfIterator) advance(seeked bool) {
for {
if ok := f.updateValid(); !ok {
return
}

key := f.iter.UnsafeKey()

if f.asOf.Less(key.Timestamp) {
// Skip keys above the asOf timestamp, regardless of type of key (e.g. point or range)
f.iter.Next()
} else if hasPoint, hasRange := f.iter.HasPointAndRange(); !hasPoint && hasRange {
// Bare range keys get surfaced before the point key, even though the
// point key shadows it; thus, because we can infer range key information
// when the point key surfaces, skip over the bare range key, and reason
// about shadowed keys at the surfaced point key.
//
// E.g. Scanning the keys below:
// 2 a2
// 1 o---o
// a b
//
// would result in two surfaced keys:
// {a-b}@1;
// a2, {a-b}@1
// Detect range tombstones, and step forward to the next key if on a bare
// range key.
if seeked || f.iter.RangeKeyChanged() {
seeked = false
hasPoint, hasRange := f.iter.HasPointAndRange()
f.newestRangeTombstone = hlc.Timestamp{}
if hasRange {
if v, ok := f.iter.RangeKeys().FirstBelow(f.asOf); ok {
f.newestRangeTombstone = v.Timestamp
}
}
if !hasPoint {
f.iter.Next()
continue
}
}

// Process point keys.
if key := f.iter.UnsafeKey(); f.asOf.Less(key.Timestamp) {
// Skip keys above the asOf timestamp.
f.iter.Next()
} else if len(f.iter.UnsafeValue()) == 0 {
} else if value, err := DecodeMVCCValue(f.iter.UnsafeValue()); err != nil {
f.valid, f.err = false, err
return
} else if value.IsTombstone() {
// Skip to the next MVCC key if we find a point tombstone.
f.iter.NextKey()
} else if !hasRange {
// On a valid key without a range key
return
// TODO (msbutler): ensure this caches range key values (#84379) before
// the 22.2 branch cut, else we face a steep perf cliff for RESTORE with
// range keys.
} else if f.iter.RangeKeys().HasBetween(key.Timestamp, f.asOf) {
} else if key.Timestamp.LessEq(f.newestRangeTombstone) {
// The latest range key, as of system time, shadows the latest point key.
// This key is therefore deleted as of system time.
f.iter.NextKey()
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/sst_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
)

func runTestSSTIterator(t *testing.T, iter SimpleMVCCIterator, allKVs []MVCCKeyValue) {
Expand Down Expand Up @@ -85,12 +86,17 @@ func TestSSTIterator(t *testing.T) {
var allKVs []MVCCKeyValue
maxWallTime := 10
for i := 0; i < maxWallTime; i++ {
var v MVCCValue
v.Value.SetBytes([]byte{'a', byte(i)})
vRaw, err := EncodeMVCCValue(v)
require.NoError(t, err)

kv := MVCCKeyValue{
Key: MVCCKey{
Key: []byte{'A' + byte(i)},
Timestamp: hlc.Timestamp{WallTime: int64(i)},
},
Value: []byte{'a' + byte(i)},
Value: vRaw,
}
if err := sst.Put(kv.Key, kv.Value); err != nil {
t.Fatalf("%+v", err)
Expand Down

0 comments on commit b184a71

Please sign in to comment.