Skip to content

Commit

Permalink
rowblk: add cockroach microbenchmarks
Browse files Browse the repository at this point in the history
Add microbenchmarks for block building and iteration that use Cockroach MVCC
keys. These microbenchmarks will provide a comparison point for colblk
equivalents.
  • Loading branch information
jbowens committed Aug 5, 2024
1 parent cda4471 commit 65445b8
Show file tree
Hide file tree
Showing 2 changed files with 175 additions and 0 deletions.
46 changes: 46 additions & 0 deletions internal/crdbtest/crdbtest.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,13 @@ import (
"bytes"
"cmp"
"encoding/binary"
"fmt"
"slices"
"time"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"golang.org/x/exp/rand"
)

const withWall = 9
Expand Down Expand Up @@ -303,3 +307,45 @@ func getKeyPartFromEngineKey(engineKey []byte) (key []byte, ok bool) {
// Key excludes the sentinel byte.
return engineKey[:keyPartEnd], true
}

// KeyConfig configures the shape of the random keys generated.
type KeyConfig struct {
PrefixAlphabetLen int // Number of bytes in the alphabet used for the prefix.
PrefixLenShared int // Number of bytes shared by all key prefixes.
PrefixLen int // Number of bytes in the prefix.
BaseWallTime uint64 // Smallest MVCC WallTime.
Logical uint32 // MVCC logical time for all keys.
}

func (cfg KeyConfig) String() string {
return fmt.Sprintf("AlphaLen=%d,Shared=%d,PrefixLen=%d,Logical=%d",
cfg.PrefixAlphabetLen, cfg.PrefixLenShared, cfg.PrefixLen, cfg.Logical)
}

// RandomKVs constructs count random KVs with the provided parameters.
func RandomKVs(rng *rand.Rand, count int, cfg KeyConfig, valueLen int) (keys, vals [][]byte) {
sharedPrefix := make([]byte, cfg.PrefixLenShared)
for i := 0; i < len(sharedPrefix); i++ {
sharedPrefix[i] = byte(rng.Intn(cfg.PrefixAlphabetLen) + 'a')
}

keys = make([][]byte, count)
vals = make([][]byte, count)
for i := range keys {
keys[i] = randCockroachKey(rng, cfg, sharedPrefix)
vals[i] = make([]byte, valueLen)
rng.Read(vals[i])
}
slices.SortFunc(keys, Compare)
return keys, vals
}

func randCockroachKey(rng *rand.Rand, cfg KeyConfig, blockPrefix []byte) []byte {
key := make([]byte, 0, cfg.PrefixLen+MaxSuffixLen)
key = append(key, blockPrefix...)
wallTime := cfg.BaseWallTime + rng.Uint64n(uint64(time.Hour))
for len(key) < cfg.PrefixLen {
key = append(key, byte(rng.Intn(cfg.PrefixAlphabetLen)+'a'))
}
return EncodeTimestamp(key, wallTime, cfg.Logical)
}
129 changes: 129 additions & 0 deletions sstable/rowblk/rowblk_bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"time"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/crdbtest"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/sstable/block"
"golang.org/x/exp/rand"
Expand Down Expand Up @@ -213,3 +214,131 @@ func BenchmarkBlockIterPrev(b *testing.B) {
}
}
}

func BenchmarkCockroachDataBlockWriter(b *testing.B) {
for _, alphaLen := range []int{4, 8, 26} {
for _, lenSharedPct := range []float64{0.25, 0.5} {
for _, prefixLen := range []int{8, 32, 128} {
lenShared := int(float64(prefixLen) * lenSharedPct)
for _, valueLen := range []int{8, 128, 1024} {
keyConfig := crdbtest.KeyConfig{
PrefixAlphabetLen: alphaLen,
PrefixLen: prefixLen,
PrefixLenShared: lenShared,
Logical: 0,
BaseWallTime: uint64(time.Now().UnixNano()),
}
b.Run(fmt.Sprintf("%s,valueLen=%d", keyConfig, valueLen), func(b *testing.B) {
benchmarkCockroachDataBlockWriter(b, keyConfig, valueLen)
})
}
}
}
}
}

func benchmarkCockroachDataBlockWriter(b *testing.B, keyConfig crdbtest.KeyConfig, valueLen int) {
const targetBlockSize = 32 << 10
seed := uint64(time.Now().UnixNano())
rng := rand.New(rand.NewSource(seed))
keys, values := crdbtest.RandomKVs(rng, targetBlockSize/valueLen, keyConfig, valueLen)

var w Writer
w.RestartInterval = 16
b.ResetTimer()
for i := 0; i < b.N; i++ {
w.Reset()
var j int
var prevKeyLen int
for w.EstimatedSize() < targetBlockSize {
ik := base.MakeInternalKey(keys[j], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet)
var samePrefix bool
if j > 0 {
samePrefix = bytes.Equal(keys[j], keys[j-1])
}
w.AddWithOptionalValuePrefix(
ik, false, values[j], prevKeyLen, true, block.InPlaceValuePrefix(samePrefix), samePrefix)
j++
prevKeyLen = len(ik.UserKey)
}
w.Finish()
}
}

func BenchmarkCockroachDataBlockIter(b *testing.B) {
for _, alphaLen := range []int{4, 8, 26} {
for _, lenSharedPct := range []float64{0.25, 0.5} {
for _, prefixLen := range []int{8, 32, 128} {
lenShared := int(float64(prefixLen) * lenSharedPct)
for _, logical := range []uint32{0, 1} {
for _, valueLen := range []int{8, 128, 1024} {
keyConfig := crdbtest.KeyConfig{
PrefixAlphabetLen: alphaLen,
PrefixLen: prefixLen,
PrefixLenShared: lenShared,
Logical: logical,
BaseWallTime: uint64(time.Now().UnixNano()),
}
b.Run(fmt.Sprintf("%s,value=%d", keyConfig, valueLen),
func(b *testing.B) {
benchmarkCockroachDataBlockIter(b, keyConfig, valueLen)
})
}
}
}
}
}
}

func benchmarkCockroachDataBlockIter(b *testing.B, keyConfig crdbtest.KeyConfig, valueLen int) {
const targetBlockSize = 32 << 10
seed := uint64(time.Now().UnixNano())
rng := rand.New(rand.NewSource(seed))
keys, values := crdbtest.RandomKVs(rng, targetBlockSize/valueLen, keyConfig, valueLen)

var w Writer
w.RestartInterval = 16
var count int
var prevKeyLen int
for w.EstimatedSize() < targetBlockSize {
ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet)
var samePrefix bool
if count > 0 {
samePrefix = bytes.Equal(keys[count], keys[count-1])
}
w.AddWithOptionalValuePrefix(
ik, false, values[count], prevKeyLen, true, block.InPlaceValuePrefix(samePrefix), samePrefix)
count++
prevKeyLen = len(ik.UserKey)
}
serializedBlock := w.Finish()
var it Iter
it.Init(crdbtest.Compare, crdbtest.Split, serializedBlock, block.NoTransforms)
avgRowSize := float64(len(serializedBlock)) / float64(count)

b.Run("Next", func(b *testing.B) {
kv := it.First()
b.ResetTimer()
for i := 0; i < b.N; i++ {
if kv == nil {
kv = it.First()
} else {
kv = it.Next()
}
}
b.StopTimer()
b.ReportMetric(avgRowSize, "bytes/row")
})
b.Run("SeekGE", func(b *testing.B) {
rng := rand.New(rand.NewSource(seed))
b.ResetTimer()
for i := 0; i < b.N; i++ {
k := keys[rng.Intn(count)]
if kv := it.SeekGE(k, base.SeekGEFlagsNone); kv == nil {
b.Fatalf("%q not found", k)
}
}
b.StopTimer()
b.ReportMetric(avgRowSize, "bytes/row")
})
}

0 comments on commit 65445b8

Please sign in to comment.