Skip to content

Commit

Permalink
db: Add BenchmarkQueueWorkload
Browse files Browse the repository at this point in the history
BenchmarkQueueWorkload benchmarks a workload consisting of multiple queues
that are all being processed at the same time. Processing a queue entails either
appending to the end of the queue (a Set operation) or deleting from the start of
the queue (a Delete operation). The goal is to detect cases where we see a large
buildup of point tombstones at the beginning of each queue, which leads to the
slowdown of SeekGE(<start of queue>). To that end, the test subbenchmarks
a series of configurations that each 1) process the queues a certain number of times
and then 2) benchmark both the queue processing throughput and SeekGE performance.
See https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
for more information.
anish-shanbhag committed Jul 22, 2024
1 parent 72c3f55 commit 89e5644
Showing 1 changed file with 160 additions and 0 deletions.
160 changes: 160 additions & 0 deletions iterator_test.go
Original file line number Diff line number Diff line change
@@ -2721,3 +2721,163 @@ func BenchmarkSeekPrefixTombstones(b *testing.B) {
iter.SeekPrefixGE(seekKey)
}
}

func runBenchmarkQueueWorkload(b *testing.B, deleteRatio float32, initOps int, valueSize int) {
const queueCount = 8
// These should be large enough to assign a unique key to each item in the
// queue.
const maxQueueIDLen = 1
const maxItemLen = 7
const maxKeyLen = maxQueueIDLen + 1 + maxItemLen
queueIDKeyspace := testkeys.Alpha(maxQueueIDLen)
itemKeyspace := testkeys.Alpha(maxItemLen)
key := make([]byte, maxKeyLen)
val := make([]byte, valueSize)
rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))

getKey := func(q int, i int) []byte {
n := testkeys.WriteKey(key, queueIDKeyspace, int64(q))
key[n] = '/'
prefixLen := n + 1
n = testkeys.WriteKey(key[prefixLen:], itemKeyspace, int64(i))
return key[:prefixLen+n]
}

type Queue struct {
start int
end int // exclusive
}
var queues = make([]*Queue, queueCount)
for i := 0; i < queueCount; i++ {
queues[i] = &Queue{}
}

o := (&Options{
DisableWAL: true,
FS: vfs.NewMem(),
Comparer: testkeys.Comparer,
FormatMajorVersion: FormatNewest,
}).EnsureDefaults()

d, err := Open("", o)
require.NoError(b, err)

processQueueOnce := func(batch *Batch) {
for {
// Randomly pick a queue to process.
q := rng.Intn(queueCount)
queue := queues[q]

isDelete := rng.Float32() < deleteRatio

if isDelete {
// Only process the queue if it's not empty. Otherwise, retry
// with a different queue.
if queue.start != queue.end {
require.NoError(b, batch.Delete(getKey(q, queue.start), nil))
queue.start = (queue.start + 1) % int(itemKeyspace.Count())
break
}
} else {
// Append to the queue.
require.NoError(b, batch.Set(getKey(q, queue.end), val, nil))
queue.end = (queue.end + 1) % int(itemKeyspace.Count())
break
}
}
}

// First, process queues initialOps times.
batch := d.NewBatch()
for i := 0; i < initOps; i++ {
processQueueOnce(batch)
// Use a large batch size to speed up initialization.
if batch.Len() >= 10<<24 /* 167 MiB */ {
require.NoError(b, batch.Commit(NoSync))
batch = d.NewBatch()
}
}
require.NoError(b, batch.Commit(NoSync))
// Manually flush in case the last batch was small.
_, err = d.AsyncFlush()
require.NoError(b, err)

waitForCompactions := func() {
d.mu.Lock()
// NB: Wait for table stats because some compaction types rely
// on table stats to be collected.
d.waitTableStats()
for d.mu.compact.compactingCount > 0 {
d.mu.compact.cond.Wait()
d.waitTableStats()
}
d.mu.Unlock()
}

waitForCompactions()

// Log the number of tombstones and live keys in each level after
// background compactions are complete.
b.Log("LSM after compactions:")
firstIter, _ := d.NewIter(nil)
firstIter.First()
lastIter, _ := d.NewIter(nil)
lastIter.Last()
stats, _ := d.ScanStatistics(context.Background(), firstIter.Key(), lastIter.Key(), ScanStatisticsOptions{})
require.NoError(b, firstIter.Close())
require.NoError(b, lastIter.Close())
metrics := d.Metrics()
for i := 0; i < numLevels; i++ {
numTombstones := stats.Levels[i].KindsCount[base.InternalKeyKindDelete]
numSets := stats.Levels[i].KindsCount[base.InternalKeyKindSet]
numTables := metrics.Levels[i].NumFiles
if numSets > 0 {
b.Logf("L%d: %d tombstones, %d sets, %d sstables\n", i, numTombstones, numSets, numTables)
}
}

// Seek to the start of each queue.
b.Run("seek", func(b *testing.B) {
for i := 0; i < b.N; i++ {
for q := 0; q < queueCount; q++ {
iter, _ := d.NewIter(nil)
iter.SeekGE(getKey(q, 0))
require.NoError(b, iter.Close())
}
}
})

require.NoError(b, d.Close())
}

// BenchmarkQueueWorkload benchmarks a workload consisting of multiple queues
// that are all being processed at the same time. Processing a queue entails
// either appending to the end of the queue (a Set operation) or deleting from
// the start of the queue (a Delete operation). The goal is to detect cases
// where we see a large buildup of point tombstones at the beginning of each
// queue, which leads to the slowdown of SeekGE(<start of queue>). To that end,
// the test subbenchmarks a series of configurations that each 1) process the
// queues a certain number of times and then 2) benchmark both the queue
// processing throughput and SeekGE performance. See
// https://github.com/facebook/rocksdb/wiki/Implement-Queue-Service-Using-RocksDB
// for more information.
func BenchmarkQueueWorkload(b *testing.B) {
// The portion of processing ops that will be deletes for each subbenchmark.
var deleteRatios = []float32{0.1, 0.3, 0.5}
// The number of times queues will be processed before running each
// subbenchmark.
var initOps = []int{400_000, 800_000, 1_200_000, 2_000_000, 3_500_000, 5_000_000, 7_500_000, 10_000_000, 50_000_000}
// We vary the value size to identify how compaction behaves when the
// relative sizes of tombstones and the keys they delete are different.
var valueSizes = []int{128, 2048}

for _, deleteRatio := range deleteRatios {
for _, valueSize := range valueSizes {
for _, numInitOps := range initOps {
b.Run(fmt.Sprintf("initial_ops=%d/deleteRatio=%.2f/valueSize=%d", numInitOps, deleteRatio, valueSize), func(b *testing.B) {
runBenchmarkQueueWorkload(b, deleteRatio, numInitOps, valueSize)
})
}
}
}
}

0 comments on commit 89e5644

Please sign in to comment.