Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96380: storage: add scan benchmark with resolved intents and fix pebbleMVCCS… r=sumeerbhola a=sumeerbhola

…canner.itersBeforeSeek

This commit restores the lower-bound of 1 on
pebbleMVCCScanner.itersBeforeSeek. This has no effect on the added benchmark since on master (unlike v22.2) we use Pebble's Iterator.NextPrefix for the common case of stepping to the next roachpb.Key. itersBeforeSeek continues to be used for seeking to a particular version and for reverse scans.

The added benchmark has 7 levels and resolved intents where both the Set and SingleDelete of the intent are present in Pebble. It tries to trick pebbleMVCCScanner with having keys with many versions in the beginning of the scan. Benchmark results:

```
BenchmarkMVCCScannerWithIntentsAndVersions-10    	4000	    316177 ns/op	  133119 B/op	      28 allocs/op

stats: (interface (dir, seek, step): (fwd, 2, 1999), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 3110), (rev, 0, 0)), (internal-stats: (block-bytes: (total 15 K, cached 15 K)), (points: (count 3.1 K, key-bytes 88 K, value-bytes 60 K, tombstoned 0)))
```
Informs cockroachdb#96361

Epic: none

Release note: None

Co-authored-by: sumeerbhola <[email protected]>
  • Loading branch information
craig[bot] and sumeerbhola committed Feb 3, 2023
2 parents dd97d0c + 6f01e09 commit 44d9f3c
Show file tree
Hide file tree
Showing 2 changed files with 163 additions and 2 deletions.
156 changes: 156 additions & 0 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"math/rand"
"os"
"path/filepath"
"sort"
"testing"
"time"

Expand All @@ -40,6 +41,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uint128"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/sstable"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -1813,3 +1816,156 @@ func makeBenchRowKey(b *testing.B, buf []byte, id int, columnFamily uint32) roac
}
return keys.MakeFamilyKey(buf, columnFamily)
}

// Benchmark with 7 levels (these are L0 sub-levels, but are similar to
// normal levels in using levelIter inside Pebble) each with one file. 1000 roachpb.Keys
// with the first 10 keys having 6 versions, and the remaining with 1 version.
// Each key is Put using a transactional write, so an intent is written too.
// The lowest level has only the intents and corresponding provisional value.
// The next higher level has the intent resolution of the next lower level,
// and its own Puts (i.e., intents and provisional value). This means all
// SingleDelete, Set pairs for the intents are separated into 2 files (in 2
// levels). And the first 10 keys with 6 versions have each of their versions
// in separate files (in separate levels). Each iteration is a MVCCScan over
// all these keys reading at a timestamp higher than the latest version.
//
// This benchmark is intended to behave akin to a real LSM with many levels,
// where the intent have been deleted but the deletes have not been compacted
// away. See #96361 for more motivation.
func BenchmarkMVCCScannerWithIntentsAndVersions(b *testing.B) {
skip.UnderShort(b, "setting up takes too long")
defer log.Scope(b).Close(b)

st := cluster.MakeTestingClusterSettings()
ctx := context.Background()
eng, err := Open(ctx, InMemory(), st, CacheSize(testCacheSize),
func(cfg *engineConfig) error {
cfg.Opts.DisableAutomaticCompactions = true
return nil
})
require.NoError(b, err)
defer eng.Close()
value := roachpb.MakeValueFromString("value")
numVersions := 6
txnIDCount := 2 * numVersions
adjustTxnID := func(txnID int) int {
// Assign txn IDs in a deterministic way that will mimic the end result of
// random assignment -- the live intent is centered between dead intents,
// when we have separated intents.
if txnID%2 == 0 {
txnID = txnIDCount - txnID
}
return txnID
}
const totalNumKeys = 1000
var prevTxn roachpb.Transaction
var numPrevKeys int
for i := 1; i <= numVersions+1; i++ {
lockUpdate := roachpb.LockUpdate{
Txn: prevTxn.TxnMeta,
Status: roachpb.COMMITTED,
}
txnID := adjustTxnID(i)
txnUUID := uuid.FromUint128(uint128.FromInts(0, uint64(txnID)))
ts := hlc.Timestamp{WallTime: int64(i)}
txn := roachpb.Transaction{
TxnMeta: enginepb.TxnMeta{
ID: txnUUID,
Key: []byte("foo"),
WriteTimestamp: ts,
MinTimestamp: ts,
},
Status: roachpb.PENDING,
ReadTimestamp: ts,
GlobalUncertaintyLimit: ts,
}
prevTxn = txn
batch := eng.NewBatch()
// Resolve the previous intents.
for j := 0; j < numPrevKeys; j++ {
key := makeKey(nil, j)
lu := lockUpdate
lu.Key = key
found, _, _, err := MVCCResolveWriteIntent(
ctx, batch, nil, lu, MVCCResolveWriteIntentOptions{})
require.Equal(b, true, found)
require.NoError(b, err)
}
numKeys := totalNumKeys
if i == numVersions+1 {
numKeys = 0
} else if i != 1 {
numKeys = 10
}
// Put the keys for this iteration.
for j := 0; j < numKeys; j++ {
key := makeKey(nil, j)
require.NoError(b, MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, &txn))
}
numPrevKeys = numKeys
// Read the keys from the Batch and write them to a sstable to ingest.
reader := batch.(*pebbleBatch).batch.Reader()
kind, key, value, ok := reader.Next()
type kvPair struct {
key []byte
kind pebble.InternalKeyKind
value []byte
}
var kvPairs []kvPair
for ; ok; kind, key, value, ok = reader.Next() {
kvPairs = append(kvPairs, kvPair{key: key, kind: kind, value: value})
}
sort.Slice(kvPairs, func(i, j int) bool {
cmp := EngineKeyCompare(kvPairs[i].key, kvPairs[j].key)
if cmp == 0 {
// Should not happen since we resolve in a different batch from the
// one where we wrote the intent.
b.Fatalf("found equal user keys in same batch")
}
return cmp < 0
})
sstFileName := fmt.Sprintf("tmp-ingest-%d", i)
sstFile, err := eng.Create(sstFileName)
require.NoError(b, err)
// No improvement with v3 since the multiple versions are in different
// files.
format := sstable.TableFormatPebblev2
opts := DefaultPebbleOptions().MakeWriterOptions(0, format)
writer := sstable.NewWriter(sstFile, opts)
for _, kv := range kvPairs {
require.NoError(b, writer.Add(
pebble.InternalKey{UserKey: kv.key, Trailer: uint64(kv.kind)}, kv.value))
}
require.NoError(b, writer.Close())
batch.Close()
require.NoError(b, eng.IngestExternalFiles(ctx, []string{sstFileName}))
}
for i := 0; i < b.N; i++ {
rw := eng.NewReadOnly(StandardDurability)
ts := hlc.Timestamp{WallTime: int64(numVersions) + 5}
startKey := makeKey(nil, 0)
endKey := makeKey(nil, totalNumKeys+1)
iter := newMVCCIterator(
rw, ts, false, false, IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
LowerBound: startKey,
UpperBound: endKey,
},
)
res, err := mvccScanToKvs(ctx, iter, startKey, endKey,
hlc.Timestamp{WallTime: int64(numVersions) + 5}, MVCCScanOptions{})
if err != nil {
b.Fatal(err)
}
if res.NumKeys != totalNumKeys {
b.Fatalf("expected %d keys, and found %d", totalNumKeys, res.NumKeys)
}
if i == 0 {
// This is to understand the results.
stats := iter.Stats()
fmt.Printf("stats: %s\n", stats.Stats.String())
}
iter.Close()
rw.Close()
}
}
9 changes: 7 additions & 2 deletions pkg/storage/pebble_mvcc_scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -687,8 +687,13 @@ func (p *pebbleMVCCScanner) incrementItersBeforeSeek() {
// Decrements itersBeforeSeek while ensuring it stays positive.
func (p *pebbleMVCCScanner) decrementItersBeforeSeek() {
p.itersBeforeSeek--
if p.itersBeforeSeek < 0 {
p.itersBeforeSeek = 0
if p.itersBeforeSeek < 1 {
if maxItersBeforeSeek > 0 {
p.itersBeforeSeek = 1
} else if p.itersBeforeSeek < 0 {
// maxItersBeforeSeek == 0 && p.itersBeforeSeek < 0.
p.itersBeforeSeek = 0
}
}
}

Expand Down

0 comments on commit 44d9f3c

Please sign in to comment.