Skip to content

Commit d187376

Browse files
committed
db: add probe tests to TestIterHistories
This commit adds probe tests to iterator_histories, also adds unit tests for the new optimization.
1 parent ca74226 commit d187376

File tree

13 files changed

+589
-134
lines changed

13 files changed

+589
-134
lines changed

file_cache.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -732,7 +732,7 @@ func (h *fileCacheHandle) newPointIter(
732732
ValueFetcher: internalOpts.blobValueFetcher,
733733
References: blobReferences,
734734
},
735-
MaximumSuffixProperty: sstable.MaxTestKeysSuffixProperty{},
735+
MaximumSuffixProperty: opts.GetMaximumSuffixProperty(),
736736
})
737737
}
738738
if err != nil {

internal/base/internal.go

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -96,8 +96,7 @@ const (
9696
//InternalKeyKindRollbackXID InternalKeyKind = 12
9797
//InternalKeyKindNoop InternalKeyKind = 13
9898
//InternalKeyKindColumnFamilyRangeDelete InternalKeyKind = 14
99-
InternalKeyKindSyntheticKey InternalKeyKind = 14
100-
InternalKeyKindRangeDelete InternalKeyKind = 15
99+
InternalKeyKindRangeDelete InternalKeyKind = 15
101100
//InternalKeyKindColumnFamilyBlobIndex InternalKeyKind = 16
102101
//InternalKeyKindBlobIndex InternalKeyKind = 17
103102

@@ -148,6 +147,9 @@ const (
148147
// appear amongst other key kinds in a batch (with the exception of alongside
149148
// InternalKeyKindIngestSST), or in an sstable.
150149
InternalKeyKindExcise InternalKeyKind = 24
150+
// InternalKeyKindSyntheticKey is a key used to mark synthetic keys in the
151+
// sstable. This is used to perform optimization during SeekPrefixGE.
152+
InternalKeyKindSyntheticKey InternalKeyKind = 25
151153

152154
// This maximum value isn't part of the file format. Future extensions may
153155
// increase this value.
@@ -158,7 +160,7 @@ const (
158160
// which sorts 'less than or equal to' any other valid internalKeyKind, when
159161
// searching for any kind of internal key formed by a certain user key and
160162
// seqNum.
161-
InternalKeyKindMax InternalKeyKind = 24
163+
InternalKeyKindMax InternalKeyKind = 25
162164

163165
// InternalKeyKindMaxForSSTable is the largest valid key kind that can exist
164166
// in an SSTable. This should usually equal InternalKeyKindMax, except
@@ -201,7 +203,6 @@ var internalKeyKindNames = []string{
201203
InternalKeyKindMerge: "MERGE",
202204
InternalKeyKindLogData: "LOGDATA",
203205
InternalKeyKindSingleDelete: "SINGLEDEL",
204-
InternalKeyKindSyntheticKey: "SYNTHETIC",
205206
InternalKeyKindRangeDelete: "RANGEDEL",
206207
InternalKeyKindSeparator: "SEPARATOR",
207208
InternalKeyKindSetWithDelete: "SETWITHDEL",
@@ -211,6 +212,7 @@ var internalKeyKindNames = []string{
211212
InternalKeyKindIngestSST: "INGESTSST",
212213
InternalKeyKindDeleteSized: "DELSIZED",
213214
InternalKeyKindExcise: "EXCISE",
215+
InternalKeyKindSyntheticKey: "SYNTHETIC",
214216
InternalKeyKindInvalid: "INVALID",
215217
}
216218

@@ -309,7 +311,6 @@ func MakeExclusiveSentinelKey(kind InternalKeyKind, userKey []byte) InternalKey
309311
var kindsMap = map[string]InternalKeyKind{
310312
"DEL": InternalKeyKindDelete,
311313
"SINGLEDEL": InternalKeyKindSingleDelete,
312-
"SYNTHETIC": InternalKeyKindSyntheticKey,
313314
"RANGEDEL": InternalKeyKindRangeDelete,
314315
"LOGDATA": InternalKeyKindLogData,
315316
"SET": InternalKeyKindSet,
@@ -323,6 +324,7 @@ var kindsMap = map[string]InternalKeyKind{
323324
"INGESTSST": InternalKeyKindIngestSST,
324325
"DELSIZED": InternalKeyKindDeleteSized,
325326
"EXCISE": InternalKeyKindExcise,
327+
"SYNTHETIC": InternalKeyKindSyntheticKey,
326328
}
327329

328330
// ParseSeqNum parses the string representation of a sequence number.

internal/base/internal_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ func TestInvalidInternalKey(t *testing.T) {
4141
"\x01\x02\x03\x04\x05\x06\x07",
4242
"foo",
4343
"foo\x08\x07\x06\x05\x04\x03\x02",
44-
"foo\x19\x07\x06\x05\x04\x03\x02\x01",
44+
"foo\x20\x07\x06\x05\x04\x03\x02\x01",
4545
}
4646
for _, tc := range testCases {
4747
k := DecodeInternalKey([]byte(tc))

internal/itertest/testdata/probes

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,33 @@ bar#2,SET:value
2121
bar#2,SET:value
2222
bar#2,SET:value
2323

24+
# Create probes to exercise the UserKey predicate. On 2 SeekGE Index return a KV
25+
# with user key "bar"; on other ops return a KV with user key "foo". Then if
26+
# the returned KV's user key is "bar", override the result with a specific KV.
27+
# In this case "ok"
28+
29+
new
30+
(If (And OpSeekGE (OnIndex 2)) (ReturnKV "bar.SET.2" "value") (ReturnKV "foo.SET.1" "value"))
31+
(If (UserKey "bar") (ReturnKV "ok.SET.1" "ok") noop)
32+
----
33+
34+
iter
35+
first
36+
next
37+
seek-ge foo
38+
prev
39+
seek-ge foo
40+
next
41+
seek-ge foo
42+
----
43+
foo#1,SET:value
44+
foo#1,SET:value
45+
foo#1,SET:value
46+
foo#1,SET:value
47+
foo#1,SET:value
48+
foo#1,SET:value
49+
ok#1,SET:ok
50+
2451
# Create an identical probe as above, and then wrap it in a "noop" probe.
2552
# A noop probe does nothing; it passes through the child iterator's results
2653
# verbatim, so the iteration results should be the same.

iterator_histories_test.go

Lines changed: 47 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@ package pebble
55

66
import (
77
"bytes"
8+
"context"
89
"fmt"
910
"path/filepath"
1011
"strconv"
@@ -15,6 +16,8 @@ import (
1516
"github.com/cockroachdb/errors"
1617
"github.com/cockroachdb/pebble/internal/base"
1718
"github.com/cockroachdb/pebble/internal/invariants"
19+
"github.com/cockroachdb/pebble/internal/itertest"
20+
"github.com/cockroachdb/pebble/internal/manifest"
1821
"github.com/cockroachdb/pebble/internal/testkeys"
1922
"github.com/cockroachdb/pebble/internal/testutils"
2023
"github.com/cockroachdb/pebble/sstable"
@@ -38,6 +41,7 @@ func TestIterHistories(t *testing.T) {
3841
var buf bytes.Buffer
3942
iters := map[string]*Iterator{}
4043
batches := map[string]*Batch{}
44+
parser := itertest.NewParser()
4145
newIter := func(name string, reader Reader, o *IterOptions) *Iterator {
4246
it, _ := reader.NewIter(o)
4347
iters[name] = it
@@ -285,9 +289,11 @@ func TestIterHistories(t *testing.T) {
285289
delete(batches, name)
286290
return fmt.Sprintf("committed %d keys\n", count)
287291
case "combined-iter":
288-
o := &IterOptions{KeyTypes: IterKeyTypePointsAndRanges}
292+
o := &IterOptions{KeyTypes: IterKeyTypePointsAndRanges, MaximumSuffixProperty: sstable.MaxTestKeysSuffixProperty{}}
289293
var reader Reader = d
290294
var name string
295+
pointProbes := make(map[base.TableNum][]itertest.Probe)
296+
rangeDelProbes := make(map[base.TableNum][]keyspanProbe)
291297
for _, arg := range td.CmdArgs {
292298
switch arg.Key {
293299
case "mask-suffix":
@@ -348,6 +354,40 @@ func TestIterHistories(t *testing.T) {
348354
}()
349355
case "use-l6-filter":
350356
o.UseL6Filters = true
357+
case "probe-points":
358+
fileNumStr := arg.Vals[0]
359+
i, err := strconv.Atoi(fileNumStr)
360+
if err != nil {
361+
require.NoError(t, err)
362+
}
363+
pointProbes[base.TableNum(i)] = itertest.MustParseProbes(parser, arg.Vals[1:]...)
364+
case "probe-rangedels":
365+
fileNumStr := arg.Vals[0]
366+
i, err := strconv.Atoi(fileNumStr)
367+
if err != nil {
368+
require.NoError(t, err)
369+
}
370+
rangeDelProbes[base.TableNum(i)] = parseKeyspanProbes(arg.Vals[1:]...)
371+
}
372+
}
373+
// If we have probes, override d.newIters to attach them
374+
if (len(pointProbes) > 0 || len(rangeDelProbes) > 0) && reader == d {
375+
oldNewIters := d.newIters
376+
d.newIters = func(
377+
ctx context.Context, file *manifest.TableMetadata, iopts *IterOptions,
378+
iio internalIterOpts, kinds iterKinds,
379+
) (iterSet, error) {
380+
set, err := oldNewIters(ctx, file, iopts, iio, kinds)
381+
if err != nil {
382+
return set, err
383+
}
384+
if probes := pointProbes[file.TableNum]; len(probes) > 0 {
385+
set.point = itertest.Attach(set.point, itertest.ProbeState{Comparer: testkeys.Comparer, Log: &buf}, probes...)
386+
}
387+
if rangedelProbes := rangeDelProbes[file.TableNum]; len(rangedelProbes) > 0 {
388+
set.rangeDeletion = attachKeyspanProbes(set.rangeDeletion, keyspanProbeContext{log: &buf}, rangedelProbes...)
389+
}
390+
return set, nil
351391
}
352392
}
353393
var iter *Iterator
@@ -370,7 +410,12 @@ func TestIterHistories(t *testing.T) {
370410
if err != nil {
371411
return err.Error()
372412
}
373-
return runIterCmd(td, iter, name == "" /* close iter */)
413+
out := runIterCmd(td, iter, name == "" /* close iter */)
414+
// Append probe logs if any
415+
if len(pointProbes) > 0 && buf.Len() > 0 {
416+
out += buf.String()
417+
}
418+
return out
374419
case "rangekey-iter":
375420
name := pluckStringCmdArg(td, "name")
376421
iter := newIter(name, d, &IterOptions{KeyTypes: IterKeyTypeRangesOnly})

level_iter.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,7 @@ func (l *levelIter) init(
154154
if len(opts.PointKeyFilters) == 0 {
155155
l.tableOpts.PointKeyFilters = l.filtersBuf[:0:1]
156156
}
157+
l.tableOpts.MaximumSuffixProperty = opts.MaximumSuffixProperty
157158
l.tableOpts.UseL6Filters = opts.UseL6Filters
158159
l.tableOpts.Category = opts.Category
159160
l.tableOpts.layer = l.layer

options.go

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,9 @@ type BlockPropertyCollector = sstable.BlockPropertyCollector
6767
// BlockPropertyFilter exports the sstable.BlockPropertyFilter type.
6868
type BlockPropertyFilter = base.BlockPropertyFilter
6969

70+
// MaximumSuffixProperty exports the sstable.MaximumSuffixProperty type.
71+
type MaximumSuffixProperty = sstable.MaximumSuffixProperty
72+
7073
// ShortAttributeExtractor exports the base.ShortAttributeExtractor type.
7174
type ShortAttributeExtractor = base.ShortAttributeExtractor
7275

@@ -218,6 +221,9 @@ type IterOptions struct {
218221
// implies obsolete points should not be hidden.
219222
snapshotForHideObsoletePoints base.SeqNum
220223

224+
// MaximumSuffixProperty is the maximum suffix property for the iterator.
225+
// This is used to perform the synthetic key optimization.
226+
MaximumSuffixProperty MaximumSuffixProperty
221227
// NB: If adding new Options, you must account for them in iterator
222228
// construction and Iterator.SetOptions.
223229
}
@@ -238,6 +244,14 @@ func (o *IterOptions) GetUpperBound() []byte {
238244
return o.UpperBound
239245
}
240246

247+
// GetMaximumSuffixProperty returns the MaximumSuffixProperty.
248+
func (o *IterOptions) GetMaximumSuffixProperty() MaximumSuffixProperty {
249+
if o == nil {
250+
return nil
251+
}
252+
return o.MaximumSuffixProperty
253+
}
254+
241255
func (o *IterOptions) pointKeys() bool {
242256
if o == nil {
243257
return true

sstable/block_property_test_utils.go

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,12 +141,15 @@ func testKeysSuffixToInterval(suffix []byte) BlockInterval {
141141

142142
type MaxTestKeysSuffixProperty struct{}
143143

144+
// Name is part of the MaxTestKeysSuffixProperty interface.
144145
func (testprop MaxTestKeysSuffixProperty) Name() string {
145146
return `pebble.internal.testkeys.suffixes`
146147
}
147148

149+
// Extract is part of the MaxTestKeysSuffixProperty interface used to extract the
150+
// latest suffix from the block property.
148151
func (testprop MaxTestKeysSuffixProperty) Extract(
149-
encodedProperty []byte,
152+
dst []byte, encodedProperty []byte,
150153
) (suffix []byte, ok bool, err error) {
151154
if len(encodedProperty) <= 1 {
152155
return nil, false, nil
@@ -158,6 +161,7 @@ func (testprop MaxTestKeysSuffixProperty) Extract(
158161
} else if interval.IsEmpty() {
159162
return nil, false, nil
160163
}
161-
ret := strconv.AppendUint([]byte("@"), (interval.Upper - 1), 10)
162-
return ret, true, nil
164+
dst = append(dst, '@')
165+
dst = strconv.AppendUint(dst, (interval.Upper - 1), 10)
166+
return dst, true, nil
163167
}

sstable/reader.go

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,8 @@ type ReadEnv struct {
7878
// only be set when Virtual is non-nil.
7979
IsSharedIngested bool
8080
Block block.ReadEnv
81-
InternalBounds *base.InternalKeyBounds
81+
// InternalBounds is the bounds of the sstable. This is currently used during synthetic key optimization.
82+
InternalBounds *base.InternalKeyBounds
8283
}
8384

8485
var NoReadEnv = ReadEnv{}
@@ -106,15 +107,23 @@ type IterOptions struct {
106107
MaximumSuffixProperty MaximumSuffixProperty
107108
}
108109

110+
// MaximumSuffixProperty is an interface for the maximum suffix property.
111+
// This is used to perform the synthetic key optimization.
109112
type MaximumSuffixProperty interface {
113+
// Name returns the name of the maximum suffix property.
110114
Name() string
111-
Extract(encodedProperty []byte) (suffix []byte, ok bool, err error)
115+
// Extract extracts the maximum suffix from the encoded property and
116+
// appends it to the provided dst slice, if it exists. If no such property is found,
117+
// Extract returns ok=false. The Extract method must NOT mutate the encoded property.
118+
Extract(dst []byte, encodedProperty []byte) (suffix []byte, ok bool, err error)
112119
}
113120

114-
type SyntheticKey struct {
121+
// syntheticKey is a struct that contains the synthetic key and the seek key.
122+
type syntheticKey struct {
115123
kv base.InternalKV
116124
seekKey []byte
117125
atSyntheticKey bool
126+
maxSuffixBuf []byte
118127
}
119128

120129
// NewPointIter returns an iterator for the point keys in the table.

0 commit comments

Comments
 (0)