Skip to content

Commit 089c226

Browse files
craig[bot]csgourav
andcommitted
Merge #143926
143926: roachtest: Add unit labels in clusterstatscollector aggregate output r=sambhav-jain-16 a=csgourav This change enhances the clusterstatscollector's aggregated metrics by adding two new descriptive fields, Unit and is_higher_better, which are used in plotting aggregated metrics. Updates the way BenchmarkFns are specified in tests, those are the ones responsible for generating one summarized value for one test run. Epic: none Fixes: CRDB-48909 Co-authored-by: Gourav Kumar <[email protected]>
2 parents 24e135f + 17bdf42 commit 089c226

File tree

8 files changed

+154
-52
lines changed

8 files changed

+154
-52
lines changed

pkg/cmd/roachtest/clusterstats/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@ go_test(
6363
"@com_github_prometheus_client_golang//api/prometheus/v1:prometheus",
6464
"@com_github_prometheus_common//model",
6565
"@com_github_stretchr_testify//require",
66+
"//pkg/cmd/roachtest/roachtestutil",
6667
],
6768
)
6869

pkg/cmd/roachtest/clusterstats/exporter.go

Lines changed: 63 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ type StatExporter interface {
7272
from time.Time,
7373
to time.Time,
7474
queries []AggQuery,
75-
benchmarkFns ...func(map[string]StatSummary) (string, float64),
75+
benchmarkFns ...func(map[string]StatSummary) *roachtestutil.AggregatedMetric,
7676
) (*ClusterStatRun, error)
7777
}
7878

@@ -93,8 +93,9 @@ type StatSummary struct {
9393
// stat information collected during the run. This struct is mirrored in
9494
// cockroachdb/roachperf for deserialization.
9595
type ClusterStatRun struct {
96-
Total map[string]float64 `json:"total"`
97-
Stats map[string]StatSummary `json:"stats"`
96+
Total map[string]float64 `json:"total"`
97+
Stats map[string]StatSummary `json:"stats"`
98+
BenchmarkMetrics map[string]roachtestutil.AggregatedMetric `json:"-"` // Not serialized to JSON
9899
}
99100

100101
// statsWriter writes the stats buffer to the file. This is used in unit test
@@ -137,13 +138,47 @@ func (r *ClusterStatRun) serializeOpenmetricsOutRun(
137138
return statsWriter(ctx, t, c, report, dest)
138139
}
139140

141+
// createReport returns a ClusterStatRun struct that encompases the results of
142+
// the run.
143+
func createReport(
144+
summaries map[string]StatSummary,
145+
summaryStats map[string]float64,
146+
benchmarkMetrics map[string]roachtestutil.AggregatedMetric,
147+
) *ClusterStatRun {
148+
testRun := ClusterStatRun{
149+
Stats: make(map[string]StatSummary),
150+
BenchmarkMetrics: benchmarkMetrics,
151+
}
152+
153+
for tag, summary := range summaries {
154+
testRun.Stats[tag] = summary
155+
}
156+
testRun.Total = summaryStats
157+
return &testRun
158+
}
159+
160+
// serializeOpenmetricsReport serializes the passed in statistics into an openmetrics
161+
// parseable performance artifact format.
140162
func serializeOpenmetricsReport(r ClusterStatRun, labelString *string) (*bytes.Buffer, error) {
141163
var buffer bytes.Buffer
142164

143165
// Emit summary metrics from Total
144166
for metricName, value := range r.Total {
145167
buffer.WriteString(roachtestutil.GetOpenmetricsGaugeType(metricName))
146-
buffer.WriteString(fmt.Sprintf("%s{%s} %f %d\n", util.SanitizeMetricName(metricName), *labelString, value, timeutil.Now().UTC().Unix()))
168+
169+
// Add labels from benchmark metrics if available
170+
additionalLabels := ""
171+
if benchmarkMetric, ok := r.BenchmarkMetrics[metricName]; ok {
172+
additionalLabels += fmt.Sprintf(",unit=\"%s\"", util.SanitizeValue(benchmarkMetric.Unit))
173+
additionalLabels += fmt.Sprintf(",is_higher_better=\"%t\"", benchmarkMetric.IsHigherBetter)
174+
}
175+
176+
buffer.WriteString(fmt.Sprintf("%s{%s%s} %f %d\n",
177+
util.SanitizeMetricName(metricName),
178+
*labelString,
179+
additionalLabels,
180+
value,
181+
timeutil.Now().UTC().Unix()))
147182
}
148183

149184
// Emit histogram metrics from Stats
@@ -179,20 +214,6 @@ func serializeOpenmetricsReport(r ClusterStatRun, labelString *string) (*bytes.B
179214
return &buffer, nil
180215
}
181216

182-
// createReport returns a ClusterStatRun struct that encompases the results of
183-
// the run.
184-
func createReport(
185-
summaries map[string]StatSummary, summaryStats map[string]float64,
186-
) *ClusterStatRun {
187-
testRun := ClusterStatRun{Stats: make(map[string]StatSummary)}
188-
189-
for tag, summary := range summaries {
190-
testRun.Stats[tag] = summary
191-
}
192-
testRun.Total = summaryStats
193-
return &testRun
194-
}
195-
196217
// Export collects, serializes and saves a roachperf file, with statistics
197218
// collect from - to time, for the AggQuery(s) given. The format is described
198219
// in the doc.go and the AggQuery definition. In addition to the AggQuery(s),
@@ -208,20 +229,37 @@ func (cs *clusterStatCollector) Export(
208229
from time.Time,
209230
to time.Time,
210231
queries []AggQuery,
211-
benchmarkFns ...func(summaries map[string]StatSummary) (string, float64),
232+
benchmarkFns ...func(summaries map[string]StatSummary) *roachtestutil.AggregatedMetric,
212233
) (testRun *ClusterStatRun, err error) {
213234
l := t.L()
214235
summaries := cs.collectSummaries(ctx, l, Interval{From: from, To: to}, queries)
215236

216-
summaryValues := make(map[string]float64)
217-
for _, scalarFn := range benchmarkFns {
218-
t, result := scalarFn(summaries)
219-
summaryValues[t] = result
237+
// Cache this value to avoid calling the function multiple times
238+
isOpenMetricsEnabled := t.ExportOpenmetrics()
239+
240+
// Initialize benchmarkMetrics as nil when OpenMetrics is disabled
241+
var benchmarkMetrics map[string]roachtestutil.AggregatedMetric
242+
if isOpenMetricsEnabled {
243+
benchmarkMetrics = make(map[string]roachtestutil.AggregatedMetric)
244+
}
245+
246+
// Summary values for total are always collected
247+
summaryValues := map[string]float64{}
248+
for _, benchMarkFn := range benchmarkFns {
249+
benchmarkMetric := benchMarkFn(summaries)
250+
if benchmarkMetric != nil {
251+
summaryValues[benchmarkMetric.Name] = float64(benchmarkMetric.Value)
252+
253+
// Only populate BenchmarkMetrics when OpenMetrics export is enabled
254+
if isOpenMetricsEnabled {
255+
benchmarkMetrics[benchmarkMetric.Name] = *benchmarkMetric
256+
}
257+
}
220258
}
221259

222-
testRun = createReport(summaries, summaryValues)
260+
testRun = createReport(summaries, summaryValues, benchmarkMetrics)
223261
if !dryRun {
224-
err = testRun.SerializeOutRun(ctx, t, c, t.ExportOpenmetrics())
262+
err = testRun.SerializeOutRun(ctx, t, c, isOpenMetricsEnabled)
225263
}
226264
return testRun, err
227265
}

pkg/cmd/roachtest/clusterstats/exporter_test.go

Lines changed: 25 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ import (
1818

1919
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
2020
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
21+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2122
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2223
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2324
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
@@ -199,9 +200,15 @@ var (
199200
}
200201

201202
benchMarkFn = func(totalKey string, totalValue float64) func(
202-
summaries map[string]StatSummary) (string, float64) {
203-
return func(summaries map[string]StatSummary) (string, float64) {
204-
return totalKey, totalValue
203+
summaries map[string]StatSummary) *roachtestutil.AggregatedMetric {
204+
return func(summaries map[string]StatSummary) *roachtestutil.AggregatedMetric {
205+
return &roachtestutil.AggregatedMetric{
206+
Name: totalKey,
207+
Value: roachtestutil.MetricPoint(totalValue),
208+
Unit: "count",
209+
IsHigherBetter: true,
210+
AdditionalLabels: nil,
211+
}
205212
}
206213
}
207214
)
@@ -347,13 +354,21 @@ func TestExport(t *testing.T) {
347354
benchMarkFn("t3", 404),
348355
)
349356
require.Nil(t, err)
350-
require.Equal(t, ClusterStatRun{
351-
Stats: map[string]StatSummary{
352-
fooStat.Query: makeExpectedTs(fooTS, fooAggQuery, 5, "1", "2", "3"),
353-
barStat.Query: makeExpectedTs(barTS, barAggQuery, 5, "1", "2", "3"),
354-
},
355-
Total: map[string]float64{"t1": 203, "t3": 404},
356-
}, *testRun)
357+
358+
// Compare individual fields instead of the whole struct
359+
require.Equal(t, map[string]float64{"t1": 203, "t3": 404}, testRun.Total)
360+
require.Equal(t, map[string]StatSummary{
361+
fooStat.Query: makeExpectedTs(fooTS, fooAggQuery, 5, "1", "2", "3"),
362+
barStat.Query: makeExpectedTs(barTS, barAggQuery, 5, "1", "2", "3"),
363+
}, testRun.Stats)
364+
365+
// Verify BenchmarkMetrics
366+
require.NotNil(t, testRun.BenchmarkMetrics)
367+
require.Len(t, testRun.BenchmarkMetrics, 2)
368+
require.Contains(t, testRun.BenchmarkMetrics, "t1")
369+
require.Contains(t, testRun.BenchmarkMetrics, "t3")
370+
require.Equal(t, float64(203), float64(testRun.BenchmarkMetrics["t1"].Value))
371+
require.Equal(t, float64(404), float64(testRun.BenchmarkMetrics["t3"].Value))
357372
})
358373
t.Run("multi tag, multi stat, 5 ticks with openmetrics false", func(t *testing.T) {
359374
c := getClusterStatCollector(ctx, ctrl, []expectPromRangeQuery{

pkg/cmd/roachtest/clusterstats/openmetrics_expected.txt

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
# TYPE t1 gauge
2-
t1{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id"} 203.000000 1727690493
2+
t1{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id",unit="count",is_higher_better="true"} 203.000000 1727690493
33
# TYPE t3 gauge
4-
t3{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id"} 404.000000 1727690493
4+
t3{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id",unit="count",is_higher_better="true"} 404.000000 1727690493
55
# TYPE foo_count gauge
66
foo_count{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id",agg_tag="sum(foo)"} 123.000000 1608854400
77
foo_count{cloud="gce",owner="roachtest_mock",test="mock_name",test_run_id="mock_id",agg_tag="sum(foo)"} 246.000000 1608854410

pkg/cmd/roachtest/tests/allocation_bench.go

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ import (
1616
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/clusterstats"
1717
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
1818
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
19+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
1920
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2021
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2122
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
@@ -395,28 +396,46 @@ func runAllocationBenchSample(
395396
true, /* dryRun */
396397
startTime, endTime,
397398
joinSummaryQueries(resourceMinMaxSummary, overloadMaxSummary, rebalanceCostSummary),
398-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
399+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
399400
ret, name := 0.0, "cpu(%)"
400401
if stat, ok := stats[cpuStat.Query]; ok {
401402
ret = roundFraction(arithmeticMean(stat.Value), 1, 2)
402403
}
403-
return name, ret
404+
return &roachtestutil.AggregatedMetric{
405+
Name: name,
406+
Value: roachtestutil.MetricPoint(ret),
407+
Unit: "percent",
408+
IsHigherBetter: false,
409+
AdditionalLabels: nil,
410+
}
404411
},
405-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
412+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
406413
ret, name := 0.0, "write(%)"
407414
if stat, ok := stats[ioWriteStat.Query]; ok {
408415
ret = roundFraction(arithmeticMean(stat.Value), 1, 2)
409416
}
410-
return name, ret
417+
return &roachtestutil.AggregatedMetric{
418+
Name: name,
419+
Value: roachtestutil.MetricPoint(ret),
420+
Unit: "percent",
421+
IsHigherBetter: false,
422+
AdditionalLabels: nil,
423+
}
411424
},
412-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
425+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
413426
rebalanceMb := 0.0
414427
values := stats[rebalanceSnapshotSentStat.Query].Value
415428
if len(values) > 0 {
416429
startMB, endMB := values[0], values[len(values)-1]
417430
rebalanceMb = roundFraction(endMB-startMB, 1024, 2)
418431
}
419-
return "cost(gb)", rebalanceMb
432+
return &roachtestutil.AggregatedMetric{
433+
Name: "cost(gb)",
434+
Value: roachtestutil.MetricPoint(rebalanceMb),
435+
Unit: "GB",
436+
IsHigherBetter: false,
437+
AdditionalLabels: nil,
438+
}
420439
},
421440
)
422441
}

pkg/cmd/roachtest/tests/allocator.go

Lines changed: 18 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -120,11 +120,25 @@ func registerAllocator(r registry.Registry) {
120120
// up-replication began, until the last rebalance action taken.
121121
// The up replication time, is the time taken to up-replicate
122122
// alone, not considering post up-replication rebalancing.
123-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
124-
return "t-balance(s)", endTime.Sub(startTime).Seconds() - allocatorStableSeconds
123+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
124+
balanceTime := endTime.Sub(startTime).Seconds() - allocatorStableSeconds
125+
return &roachtestutil.AggregatedMetric{
126+
Name: "t-balance(s)",
127+
Value: roachtestutil.MetricPoint(balanceTime),
128+
Unit: "seconds",
129+
IsHigherBetter: false,
130+
AdditionalLabels: nil,
131+
}
125132
},
126-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
127-
return "t-uprepl(s)", replicateTime.Sub(startTime).Seconds()
133+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
134+
upReplTime := replicateTime.Sub(startTime).Seconds()
135+
return &roachtestutil.AggregatedMetric{
136+
Name: "t-uprepl(s)",
137+
Value: roachtestutil.MetricPoint(upReplTime),
138+
Unit: "seconds",
139+
IsHigherBetter: false,
140+
AdditionalLabels: nil,
141+
}
128142
},
129143
)
130144
return err

pkg/cmd/roachtest/tests/cdc.go

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -143,15 +143,22 @@ func (ct *cdcTester) startStatsCollection() func() {
143143
startTime,
144144
endTime,
145145
[]clusterstats.AggQuery{sqlServiceLatencyAgg, changefeedThroughputAgg, cpuUsageAgg},
146-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
146+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
147147
// TODO(jayant): update this metric to be more accurate.
148148
// It may be worth plugging in real latency values from the latency
149149
// verifier here in the future for more accuracy. However, it may not be
150150
// worth the added complexity. Since latency verifier failures will show
151151
// up as roachtest failures, we don't need to make them very apparent in
152152
// roachperf. Note that other roachperf stats, such as the aggregate stats
153153
// above, will be accurate.
154-
return "Total Run Time (mins)", endTime.Sub(startTime).Minutes()
154+
duration := endTime.Sub(startTime).Minutes()
155+
return &roachtestutil.AggregatedMetric{
156+
Name: "Total Run Time (mins)",
157+
Value: roachtestutil.MetricPoint(duration),
158+
Unit: "minutes",
159+
IsHigherBetter: false,
160+
AdditionalLabels: nil,
161+
}
155162
},
156163
)
157164
if err != nil {

pkg/cmd/roachtest/tests/online_restore.go

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/clusterstats"
1818
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
1919
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
20+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2021
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2122
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2223
"github.com/cockroachdb/cockroach/pkg/jobs"
@@ -366,13 +367,13 @@ func exportStats(ctx context.Context, rd restoreDriver, restoreStats restoreStat
366367
restoreStats.workloadStartTime,
367368
endTime,
368369
[]clusterstats.AggQuery{sqlServiceLatencyP95Agg, queriesThroughputAgg},
369-
func(stats map[string]clusterstats.StatSummary) (string, float64) {
370+
func(stats map[string]clusterstats.StatSummary) *roachtestutil.AggregatedMetric {
370371
var timeToHealth time.Time
371372
healthyLatencyRatio := 1.25
372373
n := len(stats[latencyQueryKey].Value)
373374
rd.t.L().Printf("aggregating latency over %d data points", n)
374375
if n == 0 {
375-
return "", 0
376+
return nil // Return nil for no data points
376377
}
377378
healthyLatency := stats[latencyQueryKey].Value[n-1]
378379
latestHealthyValue := healthyLatency
@@ -394,7 +395,14 @@ func exportStats(ctx context.Context, rd restoreDriver, restoreStats restoreStat
394395
description := "Time to within 1.25x of regular p95 latency (mins)"
395396
rd.t.L().Printf("%s: %.2f minutes, compared to link + download phase time %.2f", description, rto, fullRestoreTime)
396397
rd.t.L().Printf("Latency at Recovery Time %.0f ms; at end of test %.0f ms", latestHealthyValue, healthyLatency)
397-
return description, rto
398+
399+
return &roachtestutil.AggregatedMetric{
400+
Name: description,
401+
Value: roachtestutil.MetricPoint(rto),
402+
Unit: "minutes",
403+
IsHigherBetter: false,
404+
AdditionalLabels: nil,
405+
}
398406
},
399407
)
400408
if err != nil {

0 commit comments

Comments
 (0)