Skip to content

Commit

Permalink
workload: extend the command-line interface
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
knz committed Jul 29, 2019
1 parent 45a1f49 commit d35e070
Show file tree
Hide file tree
Showing 3 changed files with 235 additions and 47 deletions.
136 changes: 136 additions & 0 deletions pkg/workload/cli/format.go
Original file line number Diff line number Diff line change
@@ -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) {}
76 changes: 76 additions & 0 deletions pkg/workload/cli/format_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
70 changes: 23 additions & 47 deletions pkg/workload/cli/run.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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...)
Expand All @@ -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)
Expand All @@ -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())
}
Expand All @@ -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()
Expand All @@ -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
Expand All @@ -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 {
Expand Down

0 comments on commit d35e070

Please sign in to comment.