Skip to content

Commit 477c799

Browse files
craig[bot]tbg
andcommitted
142933: roachtest: create go bench output in sysbench r=tbg a=tbg This makes it easier to compare sysbench results using the standard benchmark comparison facilities afforded to us by the excellent `benchstat`[^1]. Previously, we'd resort to bash snippets to manufacture a Go benchmark line. Now, they're simply a part of the test artifacts and (after having run against both SHAs as many times as desired) you can run ``` find ./artifacts/<beforesha> -name bench.txt | xargs cat > before.txt find ./artifacts/<aftersha> -name bench.txt | xargs cat > after.txt benchstat before.txt after.txt ``` Here's an example `bench.txt`: ``` BenchmarkSysbenchSettings/a=oltp_read_write/nodes=3/cpu=8/conc=64 1 27732.02 queries/sec 1386.60 txns/sec 11.92 ms/min 46.15 ms/avg 70.55 ms/p95 157.01 ms/max ``` [^1]: https://pkg.go.dev/golang.org/x/perf/cmd/benchstat Epic: CRDB-42584 Release note: None 142935: tests: also collect run-only mutex profile in sysbench r=tbg a=tbg This extends `Benchmark{,Parallel}Sysbench`'s functionality to create a delta profile for allocations to the mutex profiler. Epic: CRDB-42584 Release note: none Co-authored-by: Tobias Grieger <[email protected]>
3 parents 5350a82 + 43787c6 + 5634542 commit 477c799

File tree

4 files changed

+225
-25
lines changed

4 files changed

+225
-25
lines changed

pkg/cmd/roachtest/tests/BUILD.bazel

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -342,6 +342,8 @@ go_library(
342342
"@org_golang_google_protobuf//proto",
343343
"@org_golang_x_exp//maps",
344344
"@org_golang_x_oauth2//clientcredentials",
345+
"@org_golang_x_text//cases",
346+
"@org_golang_x_text//language",
345347
],
346348
)
347349

@@ -353,6 +355,7 @@ go_test(
353355
"drt_test.go",
354356
"query_comparison_util_test.go",
355357
"restore_test.go",
358+
"sysbench_test.go",
356359
"tpcc_test.go",
357360
":mocks_drt", # keep
358361
],

pkg/cmd/roachtest/tests/sysbench.go

Lines changed: 94 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"encoding/json"
1313
"fmt"
1414
"os"
15+
"path/filepath"
1516
"regexp"
1617
"strings"
1718
"time"
@@ -29,6 +30,8 @@ import (
2930
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
3031
"github.com/cockroachdb/errors"
3132
"github.com/stretchr/testify/require"
33+
"golang.org/x/text/cases"
34+
"golang.org/x/text/language"
3235
)
3336

3437
type sysbenchWorkload int
@@ -239,7 +242,22 @@ func runSysbench(ctx context.Context, t test.Test, c cluster.Cluster, opts sysbe
239242
return errors.Errorf("no SQL statistics found in sysbench output:\n%s", result.Stdout)
240243
}
241244
t.L().Printf("sysbench results:\n%s", result.Stdout[idx:])
242-
return exportSysbenchResults(t, c, result.Stdout, start, opts)
245+
246+
if err := exportSysbenchResults(t, c, result.Stdout, start, opts); err != nil {
247+
return err
248+
}
249+
250+
// Also produce standard Go benchmark output. This can be used to run
251+
// benchstat comparisons.
252+
goBenchOutput, err := sysbenchToGoBench(t.Name(), result.Stdout[idx:])
253+
if err != nil {
254+
return err
255+
}
256+
if err := os.WriteFile(filepath.Join(t.ArtifactsDir(), "bench.txt"), []byte(goBenchOutput), 0666); err != nil {
257+
return err
258+
}
259+
260+
return nil
243261
}
244262
if opts.usePostgres {
245263
if err := runWorkload(ctx); err != nil {
@@ -515,3 +533,78 @@ func detectSysbenchCrash(result install.RunResultDetails) (string, bool) {
515533
}
516534
return "", false
517535
}
536+
537+
// sysbenchToGoBench converts sysbench output into Go benchmark format.
538+
func sysbenchToGoBench(name string, result string) (string, error) {
539+
// Extract key metrics from sysbench output using regex patterns.
540+
var qps, tps string
541+
var minLat, avgLat, p95Lat, maxLat string
542+
543+
// Parse transactions per second.
544+
m := regexp.MustCompile(`transactions:\s+\d+\s+\(([\d.]+)\s+per sec`).FindStringSubmatch(result)
545+
if len(m) <= 1 {
546+
return "", errors.New("failed to parse transactions per second")
547+
}
548+
tps = m[1]
549+
550+
// Parse queries per second.
551+
m = regexp.MustCompile(`queries:\s+\d+\s+\(([\d.]+)\s+per sec`).FindStringSubmatch(result)
552+
if len(m) <= 1 {
553+
return "", errors.New("failed to parse queries per second")
554+
}
555+
qps = m[1]
556+
557+
// Parse each latency metric using a loop.
558+
metrics := map[string]*string{
559+
"min": &minLat,
560+
"avg": &avgLat,
561+
"max": &maxLat,
562+
"95th percentile": &p95Lat,
563+
}
564+
for metric, ptr := range metrics {
565+
pattern := fmt.Sprintf(`%s:\s+([\d.]+)`, metric)
566+
m = regexp.MustCompile(pattern).FindStringSubmatch(result)
567+
if len(m) <= 1 {
568+
return "", errors.Newf("failed to parse %s latency", metric)
569+
}
570+
*ptr = m[1]
571+
}
572+
573+
// Process the test name.
574+
parts := strings.Split(name, "/")
575+
if len(parts) == 0 {
576+
return "", errors.New("empty test name")
577+
}
578+
579+
// Normalize first segment (e.g. "sysbench-settings" -> "SysbenchSettings").
580+
firstPart := parts[0]
581+
// Split on non-alphanumeric characters.
582+
words := regexp.MustCompile(`[^a-zA-Z0-9]+`).Split(firstPart, -1)
583+
// Capitalize each word and join them.
584+
var sb strings.Builder
585+
for _, word := range words {
586+
if word == "" {
587+
continue
588+
}
589+
sb.WriteString(cases.Title(language.Und).String(strings.ToLower(word)))
590+
}
591+
firstPart = sb.String()
592+
593+
// Build the benchmark name.
594+
benchName := "Benchmark" + firstPart
595+
596+
// Add remaining parts, using auto-assigned keys only for parts without keys.
597+
nextKey := 'a'
598+
for _, part := range parts[1:] {
599+
if strings.Contains(part, "=") {
600+
benchName += "/" + part
601+
} else {
602+
benchName += fmt.Sprintf("/%s=%s", string(nextKey), part)
603+
nextKey++
604+
}
605+
}
606+
607+
// Return formatted benchmark string with all metrics.
608+
return fmt.Sprintf("%s\t1\t%s queries/sec\t%s txns/sec\t%s ms/min\t%s ms/avg\t%s ms/p95\t%s ms/max",
609+
benchName, qps, tps, minLat, avgLat, p95Lat, maxLat), nil
610+
}
Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
// Copyright 2025 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the CockroachDB Software License
4+
// included in the /LICENSE file.
5+
6+
package tests
7+
8+
import (
9+
"testing"
10+
11+
"github.com/stretchr/testify/require"
12+
)
13+
14+
func TestSysbenchToGoBench(t *testing.T) {
15+
testCases := []struct {
16+
name string
17+
result string
18+
benchmarkName string
19+
expectedOutput string
20+
expectError bool
21+
}{
22+
{
23+
name: "valid",
24+
result: `SQL statistics:
25+
queries performed:
26+
read: 10711078
27+
write: 3060308
28+
other: 1530154
29+
total: 15301540
30+
transactions: 765077 (1275.05 per sec.)
31+
queries: 15301540 (25501.03 per sec.)
32+
ignored errors: 0 (0.00 per sec.)
33+
reconnects: 0 (0.00 per sec.)
34+
35+
General statistics:
36+
total time: 600.0347s
37+
total number of events: 765077
38+
39+
Latency (ms):
40+
min: 11.86
41+
avg: 50.19
42+
max: 276.58
43+
95th percentile: 81.48
44+
sum: 38399834.47
45+
46+
Threads fairness:
47+
events (avg/stddev): 11954.3281/1290.50
48+
execution time (avg/stddev): 599.9974/0.01`,
49+
benchmarkName: "sysbench-settings/oltp_read_write/nodes=3/cpu=8/conc=64",
50+
expectedOutput: "BenchmarkSysbenchSettings/a=oltp_read_write/nodes=3/cpu=8/conc=64\t1\t25501.03 queries/sec\t1275.05 txns/sec\t11.86 ms/min\t50.19 ms/avg\t81.48 ms/p95\t276.58 ms/max",
51+
expectError: false,
52+
},
53+
{
54+
name: "bad-input",
55+
result: `something`,
56+
benchmarkName: "sysbench-settings/oltp_read_write/nodes=3/cpu=8/conc=64",
57+
expectedOutput: "",
58+
expectError: true,
59+
},
60+
}
61+
62+
for _, tc := range testCases {
63+
t.Run(tc.name, func(t *testing.T) {
64+
output, err := sysbenchToGoBench(tc.benchmarkName, tc.result)
65+
66+
if tc.expectError {
67+
require.Error(t, err)
68+
} else {
69+
require.NoError(t, err)
70+
require.Equal(t, tc.expectedOutput, output)
71+
}
72+
})
73+
}
74+
}

pkg/sql/tests/sysbench_test.go

Lines changed: 54 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -984,57 +984,87 @@ func (f doneFn) Stop(b testing.TB) {
984984
}
985985

986986
func startAllocsProfile(b testing.TB) doneFn {
987-
out := benchmemFile(b)
988-
if out == "" {
989-
return func(tb testing.TB) {}
987+
outAllocs := testProfileFile(b, "memprofile")
988+
diffAllocs := diffProfile(b, func() []byte {
989+
if outAllocs == "" {
990+
return nil
991+
}
992+
p := runtimepprof.Lookup("allocs")
993+
var buf bytes.Buffer
994+
995+
runtime.GC()
996+
require.NoError(b, p.WriteTo(&buf, 0))
997+
998+
return buf.Bytes()
999+
})
1000+
1001+
outMutex := testProfileFile(b, "mutexprofile")
1002+
diffMutex := diffProfile(b, func() []byte {
1003+
if outMutex == "" {
1004+
return nil
1005+
}
1006+
p := runtimepprof.Lookup("mutex")
1007+
var buf bytes.Buffer
1008+
require.NoError(b, p.WriteTo(&buf, 0))
1009+
return buf.Bytes()
1010+
})
1011+
1012+
return func(b testing.TB) {
1013+
if sl := diffAllocs(b); len(sl) > 0 {
1014+
require.NoError(b, os.WriteFile(outAllocs, sl, 0644))
1015+
}
1016+
if sl := diffMutex(b); len(sl) > 0 {
1017+
require.NoError(b, os.WriteFile(outMutex, sl, 0644))
1018+
}
9901019
}
1020+
}
9911021

1022+
func diffProfile(b testing.TB, take func() []byte) func(testing.TB) []byte {
9921023
// The below is essentially cribbed from pprof.go in net/http/pprof.
993-
p := runtimepprof.Lookup("allocs")
994-
var buf bytes.Buffer
995-
runtime.GC()
996-
require.NoError(b, p.WriteTo(&buf, 0))
997-
pBase, err := profile.ParseData(buf.Bytes())
1024+
1025+
baseBytes := take()
1026+
if baseBytes == nil {
1027+
return func(tb testing.TB) []byte { return nil }
1028+
}
1029+
pBase, err := profile.ParseData(baseBytes)
9981030
require.NoError(b, err)
9991031

1000-
return func(b testing.TB) {
1001-
runtime.GC()
1002-
var buf bytes.Buffer
1003-
require.NoError(b, p.WriteTo(&buf, 0))
1004-
pNew, err := profile.ParseData(buf.Bytes())
1032+
return func(b testing.TB) []byte {
1033+
pNew, err := profile.ParseData(take())
10051034
require.NoError(b, err)
10061035
pBase.Scale(-1)
10071036
pMerged, err := profile.Merge([]*profile.Profile{pBase, pNew})
10081037
require.NoError(b, err)
10091038
pMerged.TimeNanos = pNew.TimeNanos
10101039
pMerged.DurationNanos = pNew.TimeNanos - pBase.TimeNanos
10111040

1012-
buf = bytes.Buffer{}
1041+
buf := bytes.Buffer{}
10131042
require.NoError(b, pMerged.Write(&buf))
1014-
require.NoError(b, os.WriteFile(out, buf.Bytes(), 0644))
1043+
return buf.Bytes()
10151044
}
10161045
}
10171046

1018-
// If -test.benchmem is passed, also write a base alloc profile when the
1019-
// setup is done. This can be used via `pprof -base` to show only the
1020-
// allocs during run (excluding the setup).
1047+
// If -test.<something> is passed for the flag corresponding to the given
1048+
// profile, also write a base profile when the setup is done. This can be used
1049+
// via `pprof -base` to show only the samples from during run (excluding the
1050+
// setup).
10211051
//
1022-
// The file name for the base profile will be derived from -test.memprofile, and
1052+
// The file name for the base profile will be derived from -test.<flag>, and
10231053
// will contain it as a prefix (mod the file extension).
1024-
func benchmemFile(b testing.TB) string {
1054+
func testProfileFile(b testing.TB, flagWithoutTestPrefix string) string {
10251055
b.Helper()
1026-
var benchMemFile string
1056+
var flagFile string
10271057
var outputDir string
1028-
require.NoError(b, sniffarg.DoEnv("test.memprofile", &benchMemFile))
1058+
require.NoError(b, sniffarg.DoEnv("test."+flagWithoutTestPrefix, &flagFile))
10291059
require.NoError(b, sniffarg.DoEnv("test.outputdir", &outputDir))
10301060

1031-
if benchMemFile == "" {
1061+
if flagFile == "" {
10321062
return ""
10331063
}
10341064

10351065
saniRE := regexp.MustCompile(`\W+`)
10361066
saniName := saniRE.ReplaceAllString(strings.TrimPrefix(b.Name(), "Benchmark"), "_")
1037-
dest := strings.Replace(benchMemFile, ".", "_"+saniName+".", 1)
1067+
dest := strings.Replace(flagFile, ".", "_"+saniName+".", 1)
10381068
if outputDir != "" {
10391069
dest = filepath.Join(outputDir, dest)
10401070
}

0 commit comments

Comments
 (0)