Skip to content

Commit

Permalink
storage: add stats tracking for MVCC history tests
Browse files Browse the repository at this point in the history
This patch adds a `run` keyword `stats` for `TestMVCCHistories` which
outputs MVCC statistics during a test. It outputs the final MVCC stats
at the end of a test, and also the evaluated stats delta of each
command which is cross-checked against computed stats.

Release note: None
  • Loading branch information
erikgrinaker committed Jun 2, 2022
1 parent daf3b48 commit 1ecba31
Show file tree
Hide file tree
Showing 16 changed files with 487 additions and 71 deletions.
117 changes: 98 additions & 19 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ package storage
import (
"context"
"fmt"
"regexp"
"sort"
"strconv"
"strings"
"testing"
Expand Down Expand Up @@ -43,6 +45,8 @@ import (
//
// The input files use the following DSL:
//
// run [ok|trace|stats|error]
//
// txn_begin t=<name> [ts=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]]
// txn_remove t=<name>
// txn_restart t=<name>
Expand Down Expand Up @@ -97,6 +101,9 @@ func TestMVCCHistories(t *testing.T) {
// Everything reads/writes under the same prefix.
span := roachpb.Span{Key: keys.LocalMax, EndKey: roachpb.KeyMax}

// Timestamp for MVCC stats calculations, in nanoseconds.
const statsTS = 100e9

datadriven.Walk(t, testutils.TestDataPath(t, "mvcc_histories"), func(t *testing.T, path string) {
// We start from a clean slate in every test file.
engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */))
Expand Down Expand Up @@ -167,18 +174,14 @@ func TestMVCCHistories(t *testing.T) {
// It stops upon the first error encountered, if any.
//
// Options:
// "trace" means detail each operation in the output.
// "error" means expect an error to occur. The specific error type/
// message to expect is spelled out in the expected output.
// - trace: emit intermediate results after each operation.
// - stats: emit MVCC statistics for each operation and at the end.
// - error: expect an error to occur. The specific error type/ message
// to expect is spelled out in the expected output.
//
trace := false
if e.hasArg("trace") {
trace = true
}
expectError := false
if e.hasArg("error") {
expectError = true
}
trace := e.hasArg("trace")
stats := e.hasArg("stats")
expectError := e.hasArg("error")

// buf will accumulate the actual output, which the
// datadriven driver will use to compare to the expected
Expand Down Expand Up @@ -292,7 +295,7 @@ func TestMVCCHistories(t *testing.T) {
txnChange = txnChange || cmd.typ == typTxnUpdate
dataChange = dataChange || cmd.typ == typDataUpdate

if trace {
if trace || (stats && cmd.typ == typDataUpdate) {
// If tracing is also requested by the datadriven input,
// we'll trace the statement in the actual results too.
buf.Printf(">> %s", d.Cmd)
Expand All @@ -302,7 +305,13 @@ func TestMVCCHistories(t *testing.T) {
_ = buf.WriteByte('\n')
}

var msInitial enginepb.MVCCStats
if stats {
msInitial = computeStats(e.t, e.engine, span.Key, span.EndKey, statsTS)
}

// Run the command.
e.ms = &enginepb.MVCCStats{}
foundErr = cmd.fn(e)

if trace {
Expand All @@ -312,6 +321,21 @@ func TestMVCCHistories(t *testing.T) {
reportResults(cmd.typ == typTxnUpdate, cmd.typ == typDataUpdate)
}

if stats && cmd.typ == typDataUpdate {
// If stats are enabled, emit evaluated stats returned by the
// command, and compare them with the real computed stats diff.
msEval := *e.ms
msEval.AgeTo(statsTS)
buf.Printf("stats: %s\n", formatStats(msEval, true))

msDiff := computeStats(e.t, e.engine, span.Key, span.EndKey, statsTS)
msDiff.Subtract(msInitial)
if msEval != msDiff {
e.t.Errorf("MVCC stats mismatch for %q at %s\nReturned: %s\nExpected: %s",
d.Cmd, d.Pos, formatStats(msEval, true), formatStats(msDiff, true))
}
}

if foundErr != nil {
// An error occurred. Stop the script prematurely.
break
Expand All @@ -327,6 +351,12 @@ func TestMVCCHistories(t *testing.T) {
reportResults(txnChange, dataChange)
}

// Calculate and output final stats if requested and the data changed.
if stats && dataChange {
ms := computeStats(t, e.engine, span.Key, span.EndKey, statsTS)
buf.Printf("stats: %s\n", formatStats(ms, false))
}

signalError := e.t.Errorf
if txnChange || dataChange {
// We can't recover from an error and continue
Expand Down Expand Up @@ -560,7 +590,7 @@ func (e *evalCtx) resolveIntent(
intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key})
intent.Status = resolveStatus
intent.ClockWhilePending = roachpb.ObservedTimestamp{Timestamp: clockWhilePending}
_, err := MVCCResolveWriteIntent(e.ctx, rw, nil, intent)
_, err := MVCCResolveWriteIntent(e.ctx, rw, e.ms, intent)
return err
}

Expand Down Expand Up @@ -613,7 +643,7 @@ func cmdCPut(e *evalCtx) error {
resolve, resolveStatus := e.getResolve()

return e.withWriter("cput", func(rw ReadWriter) error {
if err := MVCCConditionalPut(e.ctx, rw, nil, key, ts, localTs, val, expVal, behavior, txn); err != nil {
if err := MVCCConditionalPut(e.ctx, rw, e.ms, key, ts, localTs, val, expVal, behavior, txn); err != nil {
return err
}
if resolve {
Expand All @@ -630,7 +660,7 @@ func cmdDelete(e *evalCtx) error {
localTs := hlc.ClockTimestamp(e.getTsWithName("localTs"))
resolve, resolveStatus := e.getResolve()
return e.withWriter("del", func(rw ReadWriter) error {
if err := MVCCDelete(e.ctx, rw, nil, key, ts, localTs, txn); err != nil {
if err := MVCCDelete(e.ctx, rw, e.ms, key, ts, localTs, txn); err != nil {
return err
}
if resolve {
Expand All @@ -654,7 +684,7 @@ func cmdDeleteRange(e *evalCtx) error {
resolve, resolveStatus := e.getResolve()
return e.withWriter("del_range", func(rw ReadWriter) error {
deleted, resumeSpan, num, err := MVCCDeleteRange(
e.ctx, rw, nil, key, endKey, int64(max), ts, localTs, txn, returnKeys)
e.ctx, rw, e.ms, key, endKey, int64(max), ts, localTs, txn, returnKeys)
if err != nil {
return err
}
Expand Down Expand Up @@ -729,7 +759,7 @@ func cmdIncrement(e *evalCtx) error {
resolve, resolveStatus := e.getResolve()

return e.withWriter("increment", func(rw ReadWriter) error {
curVal, err := MVCCIncrement(e.ctx, rw, nil, key, ts, localTs, txn, inc)
curVal, err := MVCCIncrement(e.ctx, rw, e.ms, key, ts, localTs, txn, inc)
if err != nil {
return err
}
Expand All @@ -746,7 +776,7 @@ func cmdMerge(e *evalCtx) error {
val := e.getVal()
ts := e.getTs(nil)
return e.withWriter("merge", func(rw ReadWriter) error {
return MVCCMerge(e.ctx, rw, nil, key, ts, val)
return MVCCMerge(e.ctx, rw, e.ms, key, ts, val)
})
}

Expand All @@ -761,7 +791,7 @@ func cmdPut(e *evalCtx) error {
resolve, resolveStatus := e.getResolve()

return e.withWriter("put", func(rw ReadWriter) error {
if err := MVCCPut(e.ctx, rw, nil, key, ts, localTs, val, txn); err != nil {
if err := MVCCPut(e.ctx, rw, e.ms, key, ts, localTs, val, txn); err != nil {
return err
}
if resolve {
Expand Down Expand Up @@ -840,6 +870,54 @@ func cmdScan(e *evalCtx) error {
return err
}

// formatStats formats MVCC stats.
func formatStats(ms enginepb.MVCCStats, delta bool) string {
// Split stats into field pairs. Subindex 1 is key, 2 is value.
fields := regexp.MustCompile(`(\w+):(-?\d+)`).FindAllStringSubmatch(ms.String(), -1)

// Sort some fields in preferred order, keeping the rest as-is at the end.
//
// TODO(erikgrinaker): Consider just reordering the MVCCStats struct fields
// instead, which determines the order of MVCCStats.String().
order := []string{"key_count", "key_bytes", "val_count", "val_bytes",
"live_count", "live_bytes", "gc_bytes_age",
"intent_count", "intent_bytes", "separated_intent_count", "intent_age"}
sort.SliceStable(fields, func(i, j int) bool {
for _, name := range order {
if fields[i][1] == name {
return true
} else if fields[j][1] == name {
return false
}
}
return false
})

// Format and output fields.
var s string
for _, field := range fields {
key, value := field[1], field[2]

// Always skip zero-valued fields and LastUpdateNanos.
if value == "0" || key == "last_update_nanos" {
continue
}

if len(s) > 0 {
s += " "
}
s += key + "="
if delta && value[0] != '-' {
s += "+" // prefix unsigned deltas with +
}
s += value
}
if len(s) == 0 && delta {
return "no change"
}
return s
}

// evalCtx stored the current state of the environment of a running
// script.
type evalCtx struct {
Expand All @@ -854,6 +932,7 @@ type evalCtx struct {
td *datadriven.TestData
txns map[string]*roachpb.Transaction
txnCounter uint128.Uint128
ms *enginepb.MVCCStats
}

func newEvalCtx(ctx context.Context, engine Engine) *evalCtx {
Expand Down
25 changes: 20 additions & 5 deletions pkg/storage/testdata/mvcc_histories/conditional_put
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,14 @@ error: (*roachpb.ConditionFailedError:) unexpected value: <nil>

# Do a conditional put with expectation that the value is completely missing; will succeed.

run ok
run stats ok
cput k=k v=v ts=123,2
----
>> cput k=k v=v ts=123,2
stats: key_count=+1 key_bytes=+14 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20
>> at end:
data: "k"/123.000000000,2 -> /BYTES/v
stats: key_count=1 key_bytes=14 val_count=1 val_bytes=6 live_count=1 live_bytes=20

# Another conditional put expecting value missing will fail, now that value1 is written.

Expand All @@ -42,12 +45,15 @@ error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\00

# Move to an empty value. Will succeed.

run ok
run stats ok
cput k=k v= cond=v ts=123,5
----
>> cput k=k v= cond=v ts=123,5
stats: key_bytes=+12 val_count=+1 val_bytes=+5 live_bytes=-1 gc_bytes_age=-414
>> at end:
data: "k"/123.000000000,5 -> /BYTES/
data: "k"/123.000000000,2 -> /BYTES/v
stats: key_count=1 key_bytes=26 val_count=2 val_bytes=11 live_count=1 live_bytes=19 gc_bytes_age=-414

# Move key2 (which does not exist) to from value1 to value2.
# Expect it to fail since it does not exist with value1.
Expand All @@ -63,13 +69,16 @@ error: (*roachpb.ConditionFailedError:) unexpected value: <nil>
# Move key2 (which does not yet exist) to from value1 to value2, but
# allowing for it not existing.

run ok
run stats ok
cput k=k2 v=v2 cond=v ts=123,7 allow_missing
----
>> cput k=k2 v=v2 cond=v ts=123,7 allow_missing
stats: key_count=+1 key_bytes=+15 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+22
>> at end:
data: "k"/123.000000000,5 -> /BYTES/
data: "k"/123.000000000,2 -> /BYTES/v
data: "k2"/123.000000000,7 -> /BYTES/v2
stats: key_count=2 key_bytes=41 val_count=3 val_bytes=18 live_count=2 live_bytes=41 gc_bytes_age=-414

# Try to move key2 (which has value2) from value1 to empty. Expect error.

Expand All @@ -84,26 +93,32 @@ error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\00

# Try to move key2 (which has value2) from value2 to empty. Expect success.

run ok
run stats ok
cput k=k2 v= cond=v2 allow_missing ts=123,9
----
>> cput k=k2 v= cond=v2 allow_missing ts=123,9
stats: key_bytes=+12 val_count=+1 val_bytes=+5 live_bytes=-2 gc_bytes_age=-437
>> at end:
data: "k"/123.000000000,5 -> /BYTES/
data: "k"/123.000000000,2 -> /BYTES/v
data: "k2"/123.000000000,9 -> /BYTES/
data: "k2"/123.000000000,7 -> /BYTES/v2
stats: key_count=2 key_bytes=53 val_count=4 val_bytes=23 live_count=2 live_bytes=39 gc_bytes_age=-851

# Now move to value2 from expected empty value.

run ok
run stats ok
cput k=k v=v2 cond= ts=123,10
----
>> cput k=k v=v2 cond= ts=123,10
stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_bytes=+2 gc_bytes_age=-391
>> at end:
data: "k"/123.000000000,10 -> /BYTES/v2
data: "k"/123.000000000,5 -> /BYTES/
data: "k"/123.000000000,2 -> /BYTES/v
data: "k2"/123.000000000,9 -> /BYTES/
data: "k2"/123.000000000,7 -> /BYTES/v2
stats: key_count=2 key_bytes=65 val_count=5 val_bytes=30 live_count=2 live_bytes=41 gc_bytes_age=-1242

# Verify we get value2 as expected.

Expand Down
Loading

0 comments on commit 1ecba31

Please sign in to comment.