From d35e07012949d06c8074e76b4a5f269633468192 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Thu, 30 May 2019 11:44:48 +0200 Subject: [PATCH] workload: extend the command-line interface Release note (cli change): The `cockroach workload` command now supports additional command-line parameters to customize the output, in order to facilitate the integration with 3rd party testing tools: - for tools that wish to observe the metrics more frequently than every second, a new flag `--display-every` is now supported, which can be used to specify the period between metric reports. This applies to both the JSON and textual output. - for tools that require a different output format than the default, a new `--display-format` argument is supported. For now only the formats "simple" (original output format) and "incremental-raw" (RFC3339 timestamps, no summary row) are supported. --- pkg/workload/cli/format.go | 136 ++++++++++++++++++++++++++++++++ pkg/workload/cli/format_test.go | 76 ++++++++++++++++++ pkg/workload/cli/run.go | 70 ++++++---------- 3 files changed, 235 insertions(+), 47 deletions(-) create mode 100644 pkg/workload/cli/format.go create mode 100644 pkg/workload/cli/format_test.go diff --git a/pkg/workload/cli/format.go b/pkg/workload/cli/format.go new file mode 100644 index 000000000000..54acabe1f40a --- /dev/null +++ b/pkg/workload/cli/format.go @@ -0,0 +1,136 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package cli + +import ( + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/workload/histogram" +) + +// outputFormat is the interface used to output results incrementally +// during a workload run. +type outputFormat interface { + // rampDone is called once when the ramp-up period completes, if + // configured. + rampDone() + // outputError is called when an error is encountered. + outputError(err error) + // outputTick is called when the main loop considers it useful + // to emit one row of results. + outputTick(startElapsed time.Duration, t histogram.Tick) + // outputTotal is called at the end, using the main histogram + // collector. + outputTotal(startElapsed time.Duration, t histogram.Tick) + // outputResult is called at the end, using the result histogram + // collector. + outputResult(startElapsed time.Duration, t histogram.Tick) +} + +// textFormatter produces output meant for quick parsing by humans. The +// data is printed as fixed-width columns. Summary rows +// are printed at the end. +type textFormatter struct { + i int + numErr int +} + +func (f *textFormatter) rampDone() { + f.i = 0 +} + +func (f *textFormatter) outputError(_ error) { + f.numErr++ +} + +func (f *textFormatter) outputTick(startElapsed time.Duration, t histogram.Tick) { + if f.i%20 == 0 { + fmt.Println("_elapsed___errors__ops/sec(inst)___ops/sec(cum)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)") + } + f.i++ + fmt.Printf("%7.1fs %8d %14.1f %14.1f %8.1f %8.1f %8.1f %8.1f %s\n", + startElapsed.Seconds(), + f.numErr, + float64(t.Hist.TotalCount())/t.Elapsed.Seconds(), + float64(t.Cumulative.TotalCount())/startElapsed.Seconds(), + time.Duration(t.Hist.ValueAtQuantile(50)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(95)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(99)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(100)).Seconds()*1000, + t.Name, + ) +} + +const totalHeader = "\n_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)" + +func (f *textFormatter) outputTotal(startElapsed time.Duration, t histogram.Tick) { + f.outputFinal(startElapsed, t, "__total") +} + +func (f *textFormatter) outputResult(startElapsed time.Duration, t histogram.Tick) { + f.outputFinal(startElapsed, t, "__result") +} + +func (f *textFormatter) outputFinal( + startElapsed time.Duration, t histogram.Tick, titleSuffix string, +) { + fmt.Println(totalHeader + titleSuffix) + if t.Cumulative == nil { + return + } + if t.Cumulative.TotalCount() == 0 { + return + } + fmt.Printf("%7.1fs %8d %14d %14.1f %8.1f %8.1f %8.1f %8.1f %8.1f %s\n", + startElapsed.Seconds(), + f.numErr, + t.Cumulative.TotalCount(), + float64(t.Cumulative.TotalCount())/startElapsed.Seconds(), + time.Duration(t.Cumulative.Mean()).Seconds()*1000, + time.Duration(t.Cumulative.ValueAtQuantile(50)).Seconds()*1000, + time.Duration(t.Cumulative.ValueAtQuantile(95)).Seconds()*1000, + time.Duration(t.Cumulative.ValueAtQuantile(99)).Seconds()*1000, + time.Duration(t.Cumulative.ValueAtQuantile(100)).Seconds()*1000, + t.Name, + ) +} + +// rawFormatter produces output that is machine-readable. The time is +// printed using absolute timestamps. No summary row is printed at the +// end. +type rawFormatter struct { + numErr int +} + +func (f *rawFormatter) rampDone() {} + +func (f *rawFormatter) outputError(_ error) { + f.numErr++ +} + +func (f *rawFormatter) outputTick(startElapsed time.Duration, t histogram.Tick) { + fmt.Printf("%s %d %.2f %.2f %.2f %.2f %.2f %.2f %s\n", + t.Now.UTC().Format(time.RFC3339Nano), + f.numErr, + float64(t.Hist.TotalCount())/t.Elapsed.Seconds(), + float64(t.Cumulative.TotalCount())/startElapsed.Seconds(), + time.Duration(t.Hist.ValueAtQuantile(50)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(95)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(99)).Seconds()*1000, + time.Duration(t.Hist.ValueAtQuantile(100)).Seconds()*1000, + t.Name, + ) +} + +func (f *rawFormatter) outputTotal(startElapsed time.Duration, t histogram.Tick) {} + +func (f *rawFormatter) outputResult(startElapsed time.Duration, t histogram.Tick) {} diff --git a/pkg/workload/cli/format_test.go b/pkg/workload/cli/format_test.go new file mode 100644 index 000000000000..1e85695b95fb --- /dev/null +++ b/pkg/workload/cli/format_test.go @@ -0,0 +1,76 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package cli + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/workload/histogram" +) + +func Example_text_formatter() { + testFormatter(&textFormatter{}) + + // output: + // _elapsed___errors__ops/sec(inst)___ops/sec(cum)__p50(ms)__p95(ms)__p99(ms)_pMax(ms) + // 1.0s 0 1.0 1.0 503.3 503.3 503.3 503.3 read + // 2.0s 0 0.5 1.0 335.5 335.5 335.5 335.5 read + // + // _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total + // 3.0s 0 2 0.7 411.0 335.5 503.3 503.3 503.3 read + // + // _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__result + // 4.0s 0 2 0.5 411.0 335.5 503.3 503.3 503.3 woo +} + +func Example_raw_formatter() { + testFormatter(&rawFormatter{}) + + // output: + // 0001-01-01T00:00:01Z 0 1.00 1.00 503.32 503.32 503.32 503.32 read + // 0001-01-01T00:00:02Z 0 0.50 1.00 335.54 335.54 335.54 335.54 read +} + +func testFormatter(formatter outputFormat) { + reg := histogram.NewRegistry(time.Second) + + start := time.Time{} + + reg.GetHandle().Get("read").Record(time.Second / 2) + reg.Tick(func(t histogram.Tick) { + // Make output deterministic. + t.Elapsed = time.Second + t.Now = start.Add(t.Elapsed) + + formatter.outputTick(time.Second, t) + }) + + reg.GetHandle().Get("read").Record(time.Second / 3) + reg.Tick(func(t histogram.Tick) { + // ditto. + t.Elapsed = 2 * time.Second + t.Now = start.Add(t.Elapsed) + + formatter.outputTick(2*time.Second, t) + }) + + resultTick := histogram.Tick{Name: "woo"} + reg.Tick(func(t histogram.Tick) { + // ditto. + t.Elapsed = 3 * time.Second + t.Now = start.Add(t.Elapsed) + + formatter.outputTotal(3*time.Second, t) + resultTick.Now = t.Now + resultTick.Cumulative = t.Cumulative + }) + formatter.outputResult(4*time.Second, resultTick) +} diff --git a/pkg/workload/cli/run.go b/pkg/workload/cli/run.go index 372892fde3ad..4cf1e567eb65 100644 --- a/pkg/workload/cli/run.go +++ b/pkg/workload/cli/run.go @@ -56,6 +56,10 @@ var pprofport = sharedFlags.Int("pprofport", 33333, "Port for pprof endpoint.") var dataLoader = sharedFlags.String("data-loader", `INSERT`, "How to load initial table data. Options are INSERT and IMPORT") +var displayEvery = runFlags.Duration("display-every", time.Second, "How much time between every one-line activity reports.") + +var displayFormat = runFlags.String("display-format", "simple", "Output display format (simple, incremental-raw)") + var histograms = runFlags.String( "histograms", "", "File to write per-op incremental and cumulative histogram data.") @@ -306,6 +310,16 @@ func startPProfEndPoint(ctx context.Context) { func runRun(gen workload.Generator, urls []string, dbName string) error { ctx := context.Background() + var formatter outputFormat + switch *displayFormat { + case "simple": + formatter = &textFormatter{} + case "incremental-raw": + formatter = &rawFormatter{} + default: + return errors.Errorf("unknown display format: %s", *displayFormat) + } + startPProfEndPoint(ctx) initDB, err := gosql.Open(`cockroach`, strings.Join(urls, ` `)) if err != nil { @@ -401,8 +415,7 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { } }() - var numErr int - ticker := time.NewTicker(time.Second) + ticker := time.NewTicker(*displayEvery) defer ticker.Stop() done := make(chan os.Signal, 3) signal.Notify(done, exitSignals...) @@ -429,11 +442,11 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { jsonEnc = json.NewEncoder(jsonF) } - everySecond := log.Every(time.Second) - for i := 0; ; { + everySecond := log.Every(*displayEvery) + for { select { case err := <-errCh: - numErr++ + formatter.outputError(err) if *tolerateErrors { if everySecond.ShouldLog() { log.Error(ctx, err) @@ -445,21 +458,7 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { case <-ticker.C: startElapsed := timeutil.Since(start) reg.Tick(func(t histogram.Tick) { - if i%20 == 0 { - fmt.Println("_elapsed___errors__ops/sec(inst)___ops/sec(cum)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)") - } - i++ - fmt.Printf("%8s %8d %14.1f %14.1f %8.1f %8.1f %8.1f %8.1f %s\n", - time.Duration(startElapsed.Seconds()+0.5)*time.Second, - numErr, - float64(t.Hist.TotalCount())/t.Elapsed.Seconds(), - float64(t.Cumulative.TotalCount())/startElapsed.Seconds(), - time.Duration(t.Hist.ValueAtQuantile(50)).Seconds()*1000, - time.Duration(t.Hist.ValueAtQuantile(95)).Seconds()*1000, - time.Duration(t.Hist.ValueAtQuantile(99)).Seconds()*1000, - time.Duration(t.Hist.ValueAtQuantile(100)).Seconds()*1000, - t.Name, - ) + formatter.outputTick(startElapsed, t) if jsonEnc != nil && rampDone == nil { _ = jsonEnc.Encode(t.Snapshot()) } @@ -470,7 +469,7 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { case <-rampDone: rampDone = nil start = timeutil.Now() - i = 0 + formatter.rampDone() reg.Tick(func(t histogram.Tick) { t.Cumulative.Reset() t.Hist.Reset() @@ -481,32 +480,11 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { if ops.Close != nil { ops.Close(ctx) } - const totalHeader = "\n_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)" - fmt.Println(totalHeader + `__total`) - startElapsed := timeutil.Since(start) - printTotalHist := func(t histogram.Tick) { - if t.Cumulative == nil { - return - } - if t.Cumulative.TotalCount() == 0 { - return - } - fmt.Printf("%7.1fs %8d %14d %14.1f %8.1f %8.1f %8.1f %8.1f %8.1f %s\n", - startElapsed.Seconds(), numErr, - t.Cumulative.TotalCount(), - float64(t.Cumulative.TotalCount())/startElapsed.Seconds(), - time.Duration(t.Cumulative.Mean()).Seconds()*1000, - time.Duration(t.Cumulative.ValueAtQuantile(50)).Seconds()*1000, - time.Duration(t.Cumulative.ValueAtQuantile(95)).Seconds()*1000, - time.Duration(t.Cumulative.ValueAtQuantile(99)).Seconds()*1000, - time.Duration(t.Cumulative.ValueAtQuantile(100)).Seconds()*1000, - t.Name, - ) - } + startElapsed := timeutil.Since(start) resultTick := histogram.Tick{Name: ops.ResultHist} reg.Tick(func(t histogram.Tick) { - printTotalHist(t) + formatter.outputTotal(startElapsed, t) if jsonEnc != nil { // Note that we're outputting the delta from the last tick. The // cumulative histogram can be computed by merging all of the @@ -522,9 +500,7 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { } } }) - - fmt.Println(totalHeader + `__result`) - printTotalHist(resultTick) + formatter.outputResult(startElapsed, resultTick) if h, ok := gen.(workload.Hookser); ok { if h.Hooks().PostRun != nil {