Skip to content

Commit

Permalink
storage: add gc benchmarks with multiple range tombstones
Browse files Browse the repository at this point in the history
Previously we only had tests that verify GC performance with no range
tombstones or with a single range tombstone under all the data.
This commit extends the benchmark to have many tombstones interleaving
data at different keys.

Release note: None
  • Loading branch information
aliher1911 committed Sep 26, 2022
1 parent 2c18ed3 commit eec6d5a
Show file tree
Hide file tree
Showing 2 changed files with 108 additions and 22 deletions.
16 changes: 16 additions & 0 deletions pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"math/rand"
"testing"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -56,6 +57,21 @@ func setupMVCCInMemPebbleWithSeparatedIntents(b testing.TB) Engine {
return peb
}

func setupPebbleInMemPebbleForLatestRelease(b testing.TB, _ string) Engine {
ctx := context.Background()
s := cluster.MakeClusterSettings()
if err := clusterversion.Initialize(ctx, clusterversion.TestingBinaryVersion,
&s.SV); err != nil {
b.Fatalf("failed to set current cluster version: %+v", err)
}

peb, err := Open(ctx, InMemory(), CacheSize(testCacheSize), Settings(s))
if err != nil {
b.Fatalf("could not create new in-mem pebble instance: %+v", err)
}
return peb
}

func BenchmarkMVCCScan_Pebble(b *testing.B) {
skip.UnderShort(b)
ctx := context.Background()
Expand Down
114 changes: 92 additions & 22 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,20 +57,29 @@ func BenchmarkMVCCGarbageCollect(b *testing.B) {
keySizes := []int{128}
valSizes := []int{128}
numKeys := []int{1, 1024}

versionConfigs := []struct {
total int
toDelete []int
total int
rangeTombstoneKeys []int
toDelete []int
}{
{2, []int{1}},
{1024, []int{1, 16, 32, 512, 1015, 1023}},
{
total: 2,
rangeTombstoneKeys: []int{0, 1, 2},
toDelete: []int{1},
},
{
total: 1024,
rangeTombstoneKeys: []int{0, 1, 16, 32},
toDelete: []int{1, 16, 32, 512, 1015, 1023},
},
}
numRangeTombstones := []int{0, 1}
updateStats := []bool{false, true}
engineMakers := []struct {
name string
create engineMaker
}{
{"pebble", setupMVCCInMemPebble},
{"pebble", setupPebbleInMemPebbleForLatestRelease},
}

ctx := context.Background()
Expand All @@ -86,7 +95,7 @@ func BenchmarkMVCCGarbageCollect(b *testing.B) {
b.Run(fmt.Sprintf("numVersions=%d", versions.total), func(b *testing.B) {
for _, toDelete := range versions.toDelete {
b.Run(fmt.Sprintf("deleteVersions=%d", toDelete), func(b *testing.B) {
for _, rangeTombstones := range numRangeTombstones {
for _, rangeTombstones := range versions.rangeTombstoneKeys {
b.Run(fmt.Sprintf("numRangeTs=%d", rangeTombstones), func(b *testing.B) {
for _, stats := range updateStats {
b.Run(fmt.Sprintf("updateStats=%t", stats), func(b *testing.B) {
Expand Down Expand Up @@ -1588,35 +1597,96 @@ func runMVCCGarbageCollect(
// the returned slice that affects the oldest 'deleteVersions' versions. The
// first write for each key will be at `ts+(1,0)`, the second one
// at `ts+(1,1)`, etc.
// If numRangeKeys is set to 1 then range tombstone will be written at ts.
//
// Write 'numRangeKeys' covering random intervals of keys using set of keys
// from the set described above. One range key is always below all point keys.
// Range tombstones are spread evenly covering versions space inserted between
// every m point versions.
// Range tombstones use timestamps starting from 'ts+(0,1)` and increase wall
// clock between version and if there are more range tombstones than
// numVersions, then logical clock is used to distinguish between them.
//
// NB: a real invocation of MVCCGarbageCollect typically has most of the keys
// in sorted order. Here they will be ordered randomly.

pointKeyTs := func(index int) hlc.Timestamp {
return ts.Add(int64(index+1), 0)
}

rangeKeyTs := func(index, numVersions, numRangeKeys int) hlc.Timestamp {
wallTime := index * numVersions / numRangeKeys
base := wallTime * numRangeKeys / numVersions
logical := 1 + index - base
return ts.Add(int64(wallTime), int32(logical))
}

setup := func() (gcKeys []roachpb.GCRequest_GCKey) {
batch := eng.NewBatch()
if opts.numRangeKeys > 1 {
b.Fatal("Invalid bench data config. Number of range keys can be 0 or 1")
pointKeys := make([]roachpb.Key, opts.numKeys)
for i := 0; i < opts.numKeys; i++ {
pointKeys[i] = randutil.RandBytes(rng, opts.keyBytes)
gcKeys = append(gcKeys, roachpb.GCRequest_GCKey{
Timestamp: pointKeyTs(opts.deleteVersions - 1),
Key: pointKeys[i],
})
}
if opts.numRangeKeys == 1 {
if err := MVCCDeleteRangeUsingTombstone(ctx, batch, nil, keys.LocalMax, keys.MaxKey,
ts, hlc.ClockTimestamp{}, nil, nil, true, 0, nil); err != nil {
// When keys are deleted using range tombstones MVCC operations perform
// smart checks to drop them altogether if there's no underlying data.
// We want to prevent that for testing purposes and we add a key at lower
// timestamp prior to putting first tombstone and subsequently remove it
// from the batch at the end of setup.
if opts.numRangeKeys > 0 {
if err := MVCCPut(ctx, batch, nil, keys.LocalMax, ts, hlc.ClockTimestamp{}, val, nil); err != nil {
b.Fatal(err)
}
}
for i := 0; i < opts.numKeys; i++ {
key := randutil.RandBytes(rng, opts.keyBytes)
if opts.deleteVersions > 0 {
gcKeys = append(gcKeys, roachpb.GCRequest_GCKey{
Timestamp: ts.Add(1, int32(opts.deleteVersions-1)),
Key: key,
})
rtsVersion := 0
for version := 0; ; version++ {
pts := pointKeyTs(version)
// Insert range keys until we reach next point key or run out of range
// key versions.
for ; rtsVersion < opts.numRangeKeys; rtsVersion++ {
rts := rangeKeyTs(rtsVersion, opts.numVersions, opts.numRangeKeys)
if pts.LessEq(rts) {
break
}
startKey := keys.LocalMax
endKey := keys.MaxKey
if rtsVersion > 0 && opts.numKeys > 1 {
for {
startKey = pointKeys[rng.Intn(opts.numKeys)]
endKey = pointKeys[rng.Intn(opts.numKeys)]
switch startKey.Compare(endKey) {
case 0:
continue
case 1:
startKey, endKey = endKey, startKey
case -1:
}
break
}
}
if err := MVCCDeleteRangeUsingTombstone(ctx, batch, nil, startKey, endKey,
rts, hlc.ClockTimestamp{}, nil, nil, true, 0, nil); err != nil {
b.Fatal(err)
}
}
if version == opts.numVersions {
break
}
for j := 0; j < opts.numVersions; j++ {
if err := MVCCPut(ctx, batch, nil, key, ts.Add(1, int32(j)), hlc.ClockTimestamp{}, val, nil); err != nil {
for _, key := range pointKeys {
if err := MVCCPut(ctx, batch, nil, key, pts, hlc.ClockTimestamp{}, val,
nil); err != nil {
b.Fatal(err)
}
}
}
// Cleanup key used for preventing range tombstone elision optimisation.
if opts.numRangeKeys > 0 {
if err := batch.ClearMVCC(MVCCKey{Key: keys.LocalMax, Timestamp: ts}); err != nil {
b.Fatal(err)
}
}
if err := batch.Commit(false); err != nil {
b.Fatal(err)
}
Expand Down

0 comments on commit eec6d5a

Please sign in to comment.