Skip to content

Commit

Permalink
db: add BenchmarkIteratorScan
Browse files Browse the repository at this point in the history
Add a simple microbenchmark that benchmarks the construction of an iterator and
a scan across the entirety of a database's keyspace. Add variances for key
count, read-amplification and the kinds of keys iterated over.

Besides general utility in measuring iterator scan performance, it's expected
to be useful in optimizing the current iterator slow down with range-keys
enabled.
  • Loading branch information
jbowens committed May 5, 2022
1 parent 782d102 commit 6d97548
Show file tree
Hide file tree
Showing 5 changed files with 170 additions and 1 deletion.
46 changes: 46 additions & 0 deletions internal/testkeys/testdata/divvy
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
divvy alpha=1 portions=3
----
a d g j m p s v y
b e h k n q t w z
c f i l o r u x

divvy alpha=1 portions=1
----
a b c d e f g h i j k l m n o p q r s t u v w x y z

divvy alpha=1 portions=5
----
a f k p u z
b g l q v
c h m r w
d i n s x
e j o t y

divvy alpha=2 portions=26
----
a az by cx dw ev fu gt hs ir jq kp lo mn nm ol pk qj ri sh tg uf ve wd xc yb za
aa b bz cy dx ew fv gu ht is jr kq lp mo nn om pl qk rj si th ug vf we xd yc zb
ab ba c cz dy ex fw gv hu it js kr lq mp no on pm ql rk sj ti uh vg wf xe yd zc
ac bb ca d dz ey fx gw hv iu jt ks lr mq np oo pn qm rl sk tj ui vh wg xf ye zd
ad bc cb da e ez fy gx hw iv ju kt ls mr nq op po qn rm sl tk uj vi wh xg yf ze
ae bd cc db ea f fz gy hx iw jv ku lt ms nr oq pp qo rn sm tl uk vj wi xh yg zf
af be cd dc eb fa g gz hy ix jw kv lu mt ns or pq qp ro sn tm ul vk wj xi yh zg
ag bf ce dd ec fb ga h hz iy jx kw lv mu nt os pr qq rp so tn um vl wk xj yi zh
ah bg cf de ed fc gb ha i iz jy kx lw mv nu ot ps qr rq sp to un vm wl xk yj zi
ai bh cg df ee fd gc hb ia j jz ky lx mw nv ou pt qs rr sq tp uo vn wm xl yk zj
aj bi ch dg ef fe gd hc ib ja k kz ly mx nw ov pu qt rs sr tq up vo wn xm yl zk
ak bj ci dh eg ff ge hd ic jb ka l lz my nx ow pv qu rt ss tr uq vp wo xn ym zl
al bk cj di eh fg gf he id jc kb la m mz ny ox pw qv ru st ts ur vq wp xo yn zm
am bl ck dj ei fh gg hf ie jd kc lb ma n nz oy px qw rv su tt us vr wq xp yo zn
an bm cl dk ej fi gh hg if je kd lc mb na o oz py qx rw sv tu ut vs wr xq yp zo
ao bn cm dl ek fj gi hh ig jf ke ld mc nb oa p pz qy rx sw tv uu vt ws xr yq zp
ap bo cn dm el fk gj hi ih jg kf le md nc ob pa q qz ry sx tw uv vu wt xs yr zq
aq bp co dn em fl gk hj ii jh kg lf me nd oc pb qa r rz sy tx uw vv wu xt ys zr
ar bq cp do en fm gl hk ij ji kh lg mf ne od pc qb ra s sz ty ux vw wv xu yt zs
as br cq dp eo fn gm hl ik jj ki lh mg nf oe pd qc rb sa t tz uy vx ww xv yu zt
at bs cr dq ep fo gn hm il jk kj li mh ng of pe qd rc sb ta u uz vy wx xw yv zu
au bt cs dr eq fp go hn im jl kk lj mi nh og pf qe rd sc tb ua v vz wy xx yw zv
av bu ct ds er fq gp ho in jm kl lk mj ni oh pg qf re sd tc ub va w wz xy yx zw
aw bv cu dt es fr gq hp io jn km ll mk nj oi ph qg rf se td uc vb wa x xz yy zx
ax bw cv du et fs gr hq ip jo kn lm ml nk oj pi qh rg sf te ud vc wb xa y yz zy
ay bx cw dv eu ft gs hr iq jp ko ln mm nl ok pj qi rh sg tf ue vd wc xb ya z zz
26 changes: 25 additions & 1 deletion internal/testkeys/testkeys.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,16 @@ type Keyspace interface {
key(buf []byte, i int) int
}

// Divvy divides the provided keyspace into N equal portions, containing
// disjoint keys evenly distributed across the keyspace.
func Divvy(ks Keyspace, n int) []Keyspace {
ret := make([]Keyspace, n)
for i := 0; i < n; i++ {
ret[i] = ks.Slice(i, ks.Count()).EveryN(n)
}
return ret
}

// Alpha constructs a keyspace consisting of all keys containing characters a-z,
// with at most `maxLength` characters.
func Alpha(maxLength int) Keyspace {
Expand Down Expand Up @@ -223,7 +233,21 @@ type alphabet struct {
}

func (a alphabet) Count() int {
return (keyCount(len(a.alphabet), a.maxLength) - a.headSkip - a.tailSkip) / a.increment
// Calculate the total number of keys, ignoring the increment.
total := (keyCount(len(a.alphabet), a.maxLength) - a.headSkip - a.tailSkip)

// The increment dictates that we take every N keys, where N = a.increment.
// Consider a total containing the 5 keys:
// a b c d e
// ^ ^ ^
// If the increment is 2, this keyspace includes 'a', 'c' and 'e'. After
// dividing by the increment, there may be remainder. If there is, there's
// one additional key in the alphabet.
count := total / a.increment
if total%a.increment > 0 {
count++
}
return count
}

func (a alphabet) MaxLen() int {
Expand Down
23 changes: 23 additions & 0 deletions internal/testkeys/testkeys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,10 @@ package testkeys

import (
"bytes"
"fmt"
"testing"

"github.com/cockroachdb/pebble/internal/datadriven"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -175,6 +177,27 @@ func TestSuffixLen(t *testing.T) {
}
}

func TestDivvy(t *testing.T) {
var buf bytes.Buffer
datadriven.RunTest(t, "testdata/divvy", func(d *datadriven.TestData) string {
buf.Reset()
switch d.Cmd {
case "divvy":
var alphaLen, portions int
d.ScanArgs(t, "alpha", &alphaLen)
d.ScanArgs(t, "portions", &portions)

input := Alpha(alphaLen)
for _, ks := range Divvy(input, portions) {
fmt.Fprintln(&buf, keyspaceToString(ks))
}
return buf.String()
default:
return fmt.Sprintf("unrecognized command %q", d.Cmd)
}
})
}

func keyspaceToString(ks Keyspace) string {
var buf bytes.Buffer
b := make([]byte, ks.MaxLen())
Expand Down
62 changes: 62 additions & 0 deletions iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/pebble/internal/datadriven"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -1762,3 +1763,64 @@ func BenchmarkBlockPropertyFilter(b *testing.B) {
})
}
}

func BenchmarkIteratorScan(b *testing.B) {
const maxPrefixLen = 8
keyBuf := make([]byte, maxPrefixLen+testkeys.MaxSuffixLen)
rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))

for _, keyCount := range []int{100, 1000, 10000} {
for _, readAmp := range []int{1, 3, 7, 10} {
func() {
opts := &Options{
FS: vfs.NewMem(),
FormatMajorVersion: FormatNewest,
}
opts.Experimental.RangeKeys = new(RangeKeysArena)
opts.DisableAutomaticCompactions = true
d, err := Open("", opts)
require.NoError(b, err)
defer func() { require.NoError(b, d.Close()) }()

// Take the very large keyspace consisting of alphabetic
// characters of lengths up to `maxPrefixLen` and reduce it down
// to `keyCount` keys by picking every 1 key every `keyCount` keys.
keys := testkeys.Alpha(maxPrefixLen)
keys = keys.EveryN(keys.Count() / keyCount)
if keys.Count() < keyCount {
b.Fatalf("expected %d keys, found %d", keyCount, keys.Count())
}

// Portion the keys into `readAmp` overlapping key sets.
for _, ks := range testkeys.Divvy(keys, readAmp) {
batch := d.NewBatch()
for i := 0; i < ks.Count(); i++ {
n := testkeys.WriteKeyAt(keyBuf[:], ks, i, int(rng.Uint64n(100)))
batch.Set(keyBuf[:n], keyBuf[:n], nil)
}
require.NoError(b, batch.Commit(nil))
require.NoError(b, d.Flush())
}
// Each level is a sublevel.
m := d.Metrics()
require.Equal(b, readAmp, m.ReadAmp())

for _, keyTypes := range []IterKeyType{IterKeyTypePointsOnly, IterKeyTypePointsAndRanges} {
iterOpts := IterOptions{KeyTypes: keyTypes}
b.Run(fmt.Sprintf("keys=%d,r-amp=%d,key-types=%s", keyCount, readAmp, keyTypes), func(b *testing.B) {
for i := 0; i < b.N; i++ {
b.StartTimer()
iter := d.NewIter(&iterOpts)
valid := iter.First()
for valid {
valid = iter.Next()
}
b.StopTimer()
require.NoError(b, iter.Close())
}
})
}
}()
}
}
}
14 changes: 14 additions & 0 deletions options.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,20 @@ const (
IterKeyTypePointsAndRanges
)

// String implements fmt.Stringer.
func (t IterKeyType) String() string {
switch t {
case IterKeyTypePointsOnly:
return "points-only"
case IterKeyTypeRangesOnly:
return "ranges-only"
case IterKeyTypePointsAndRanges:
return "points-and-ranges"
default:
panic(fmt.Sprintf("unknown key type %d", t))
}
}

// IterOptions hold the optional per-query parameters for NewIter.
//
// Like Options, a nil *IterOptions is valid and means to use the default
Expand Down

0 comments on commit 6d97548

Please sign in to comment.