From 483f844249cb53dd6be68f688c3aa4b2869abd0f Mon Sep 17 00:00:00 2001 From: Klaus Post Date: Thu, 30 Jan 2025 09:37:28 -0800 Subject: [PATCH] Make data collection aggregated throughout (#362) --- cli/analyze.go | 112 ++++- cli/benchclient.go | 53 +- cli/benchmark.go | 285 +++++++---- cli/benchserver.go | 293 ++++++++--- cli/cli.go | 6 +- cli/clientmode.go | 2 +- cli/cmp.go | 16 +- cli/flags.go | 39 +- cli/print.go | 12 - cli/progress-bar.go | 126 ----- cli/ui.go | 285 +++++++++++ go.mod | 15 +- go.sum | 31 +- main.go | 9 + pkg/aggregate/aggregate.go | 6 +- pkg/aggregate/collector.go | 175 +++++++ pkg/aggregate/live.go | 949 +++++++++++++++++++++++++++++++++++ pkg/aggregate/mapasslice.go | 129 +++++ pkg/aggregate/requests.go | 328 +++++++++--- pkg/aggregate/throughput.go | 253 +++++++++- pkg/aggregate/ttfb.go | 91 +++- pkg/bench/benchmark.go | 47 +- pkg/bench/category.go | 73 +++ pkg/bench/category_string.go | 25 + pkg/bench/collector.go | 64 ++- pkg/bench/delete.go | 11 +- pkg/bench/fanout.go | 5 +- pkg/bench/get.go | 11 +- pkg/bench/list.go | 11 +- pkg/bench/mixed.go | 9 +- pkg/bench/multipart.go | 18 +- pkg/bench/ops.go | 150 ++++-- pkg/bench/put.go | 5 +- pkg/bench/retention.go | 9 +- pkg/bench/s3zip.go | 9 +- pkg/bench/select.go | 9 +- pkg/bench/snowball.go | 5 +- pkg/bench/stat.go | 12 +- pkg/bench/versioned.go | 9 +- 39 files changed, 3055 insertions(+), 642 deletions(-) delete mode 100644 cli/progress-bar.go create mode 100644 cli/ui.go create mode 100644 pkg/aggregate/collector.go create mode 100644 pkg/aggregate/live.go create mode 100644 pkg/aggregate/mapasslice.go create mode 100644 pkg/bench/category.go create mode 100644 pkg/bench/category_string.go diff --git a/cli/analyze.go b/cli/analyze.go index 0cf755b1..9b2fe714 100644 --- a/cli/analyze.go +++ b/cli/analyze.go @@ -18,6 +18,8 @@ package cli import ( + "bufio" + "bytes" "encoding/json" "errors" "fmt" @@ -85,6 +87,10 @@ var analyzeFlags = []cli.Flag{ Name: serverFlagName, Usage: "When running benchmarks open a webserver to fetch results remotely, eg: localhost:7762", }, + cli.BoolFlag{ + Name: "full", + Usage: "Record full analysis data with every request stored. Default will aggregate data.", + }, } var analyzeCmd = cli.Command{ @@ -117,31 +123,54 @@ func mainAnalyze(ctx *cli.Context) error { if len(args) > 1 { console.Fatal("Only one benchmark file can be given") } - zstdDec, _ := zstd.NewReader(nil) - defer zstdDec.Close() monitor := api.NewBenchmarkMonitor(ctx.String(serverFlagName)) defer monitor.Done() - log := console.Printf + log := func(format string, data ...interface{}) { + console.Eraseline() + console.Printf("\r"+format, data...) + } if globalQuiet { log = nil } for _, arg := range args { - var input io.Reader - if arg == "-" { - input = os.Stdin + rc, isAggregate := openInput(arg) + defer rc.Close() + if !ctx.Bool("full") || isAggregate { + var final aggregate.Realtime + if isAggregate { + if err := json.NewDecoder(rc).Decode(&final); err != nil { + fatalIf(probe.NewError(err), "Unable to parse input") + } + if log != nil { + log("Loading %q", arg) + } + } else { + opCh := make(chan bench.Operation, 10000) + go func() { + err := bench.StreamOperationsFromCSV(rc, false, ctx.Int("analyze.offset"), ctx.Int("analyze.limit"), log, opCh) + fatalIf(probe.NewError(err), "Unable to parse input") + }() + final = *aggregate.Live(opCh, nil, "") + } + rep := final.Report(aggregate.ReportOptions{ + Details: true, + Color: !globalNoColor, + }) + if globalJSON { + b, err := json.MarshalIndent(final, "", " ") + fatalIf(probe.NewError(err), "Unable to parse input") + fmt.Println(string(b)) + } else { + console.Println("\n") + console.Println(rep.String()) + } } else { - f, err := os.Open(arg) - fatalIf(probe.NewError(err), "Unable to open input file") - defer f.Close() - input = f + ops, err := bench.OperationsFromCSV(rc, true, ctx.Int("analyze.offset"), ctx.Int("analyze.limit"), log) + fatalIf(probe.NewError(err), "Unable to parse input") + console.Println("") + printAnalysis(ctx, os.Stdout, ops) + monitor.OperationsReady(ops, strings.TrimSuffix(filepath.Base(arg), ".csv.zst"), commandLine(ctx)) } - err := zstdDec.Reset(input) - fatalIf(probe.NewError(err), "Unable to read input") - ops, err := bench.OperationsFromCSV(zstdDec, true, ctx.Int("analyze.offset"), ctx.Int("analyze.limit"), log) - fatalIf(probe.NewError(err), "Unable to parse input") - - printAnalysis(ctx, ops) - monitor.OperationsReady(ops, strings.TrimSuffix(filepath.Base(arg), ".csv.zst"), commandLine(ctx)) } return nil } @@ -234,7 +263,46 @@ func printMixedOpAnalysis(ctx *cli.Context, aggr aggregate.Aggregated, details b } } -func printAnalysis(ctx *cli.Context, o bench.Operations) { +func openInput(arg string) (rc io.ReadCloser, isJSON bool) { + var input io.Reader + var fileClose func() error + if arg == "-" { + input = os.Stdin + } else { + f, err := os.Open(arg) + fatalIf(probe.NewError(err), "Unable to open input file") + fileClose = f.Close + input = f + } + z, err := zstd.NewReader(input) + fatalIf(probe.NewError(err), "could not read from input") + + buf := bufio.NewReader(z) + v, err := buf.Peek(1) + fatalIf(probe.NewError(err), "could not read from input") + + return readCloser{ + Reader: buf, + closeFn: func() error { + z.Close() + if fileClose != nil { + return fileClose() + } + return nil + }, + }, bytes.Equal(v, []byte("{")) +} + +type readCloser struct { + io.Reader + closeFn func() error +} + +func (rc readCloser) Close() error { + return rc.closeFn() +} + +func printAnalysis(ctx *cli.Context, w io.Writer, o bench.Operations) { details := ctx.Bool("analyze.v") var wrSegs io.Writer prefiltered := false @@ -294,6 +362,12 @@ func printAnalysis(ctx *cli.Context, o bench.Operations) { return } + preOutput := color.Output + color.Output = w + defer func() { + color.Output = preOutput + }() + if aggr.Mixed { printMixedOpAnalysis(ctx, aggr, details) return @@ -522,7 +596,7 @@ func printRequestAnalysis(_ *cli.Context, ops aggregate.Operation, details bool) console.SetColor("Print", color.New(color.FgHiWhite)) console.Println("\nRequests by host:") - for _, ep := range reqs.HostNames { + for _, ep := range reqs.HostNames.Slice() { reqs := eps[ep] if reqs.Requests <= 1 { continue diff --git a/cli/benchclient.go b/cli/benchclient.go index 86933c95..30a21f7a 100644 --- a/cli/benchclient.go +++ b/cli/benchclient.go @@ -28,6 +28,7 @@ import ( "github.com/minio/cli" "github.com/minio/pkg/v3/console" + "github.com/minio/warp/pkg/aggregate" "github.com/minio/warp/pkg/bench" "github.com/minio/websocket" ) @@ -38,6 +39,7 @@ type clientReplyType string const ( clientRespBenchmarkStarted clientReplyType = "benchmark_started" clientRespStatus clientReplyType = "benchmark_status" + clientRespAborted clientReplyType = "abort_requested" clientRespOps clientReplyType = "ops" ) @@ -50,9 +52,10 @@ type clientReply struct { Started bool `json:"started"` Finished bool `json:"finished"` } `json:"stage_info"` - Type clientReplyType `json:"type"` - Err string `json:"err,omitempty"` - Ops bench.Operations `json:"ops,omitempty"` + Type clientReplyType `json:"type"` + Err string `json:"err,omitempty"` + Ops bench.Operations `json:"ops,omitempty"` + Update *aggregate.Realtime `json:"update,omitempty"` } // executeBenchmark will execute the benchmark and return any error. @@ -237,6 +240,20 @@ func serveWs(w http.ResponseWriter, r *http.Request) { close(info.start) }() resp.Type = clientRespStatus + case serverReqAbortStage: + activeBenchmarkMu.Lock() + ab := activeBenchmark + activeBenchmarkMu.Unlock() + resp.Type = clientRespAborted + if ab == nil { + break + } + console.Infoln("Aborting stage", req.Stage) + ab.Lock() + if cancel := ab.info[req.Stage].cancelFn; cancel != nil { + cancel() + } + ab.Unlock() case serverReqStageStatus: activeBenchmarkMu.Lock() ab := activeBenchmark @@ -249,6 +266,7 @@ func serveWs(w http.ResponseWriter, r *http.Request) { ab.Lock() err := ab.err stageInfo := ab.info + updates := ab.updates ab.Unlock() if err != nil { resp.Err = err.Error() @@ -270,6 +288,12 @@ func serveWs(w http.ResponseWriter, r *http.Request) { resp.StageInfo.Custom = info.custom default: } + if req.UpdateReq != nil && updates != nil { + u := make(chan *aggregate.Realtime, 1) + req.UpdateReq.C = u + updates <- *req.UpdateReq + resp.Update = <-u + } case serverReqSendOps: activeBenchmarkMu.Lock() ab := activeBenchmark @@ -278,6 +302,19 @@ func serveWs(w http.ResponseWriter, r *http.Request) { resp.Err = "no benchmark running" break } + ab.Lock() + updates := ab.updates + ab.Unlock() + if req.UpdateReq != nil && updates != nil { + u := make(chan *aggregate.Realtime, 1) + req.UpdateReq.C = u + updates <- *req.UpdateReq + select { + case <-time.After(time.Second): + resp.Err = "timeout fetching update" + case resp.Update = <-u: + } + } resp.Type = clientRespOps ab.Lock() resp.Ops = ab.results @@ -325,15 +362,7 @@ func runCommand(ctx *cli.Context, c *cli.Command) (err error) { }() } - if c.Before != nil { - err = c.Before(ctx) - if err != nil { - fmt.Fprintln(ctx.App.Writer, err) - fmt.Fprintln(ctx.App.Writer) - return err - } - } - + // We do not run c.Before, since it only updates global flags, which we don't want to modify. if c.Action == nil { return errors.New("no action") } diff --git a/cli/benchmark.go b/cli/benchmark.go index a9dbd0ab..4006e303 100644 --- a/cli/benchmark.go +++ b/cli/benchmark.go @@ -18,9 +18,11 @@ package cli import ( + "bytes" "context" "crypto/rand" "encoding/binary" + "encoding/json" "errors" "fmt" "io" @@ -29,13 +31,14 @@ import ( "sync" "time" - "github.com/cheggaaa/pb" + tea "github.com/charmbracelet/bubbletea" "github.com/klauspost/compress/zstd" "github.com/minio/cli" "github.com/minio/madmin-go/v3" "github.com/minio/mc/pkg/probe" "github.com/minio/pkg/v3/console" "github.com/minio/warp/api" + "github.com/minio/warp/pkg/aggregate" "github.com/minio/warp/pkg/bench" ) @@ -62,7 +65,7 @@ var benchFlags = []cli.Flag{ cli.DurationFlag{ Name: "autoterm.dur", Usage: "Minimum duration where output must have been stable to allow automatic termination.", - Value: 10 * time.Second, + Value: 15 * time.Second, }, cli.Float64Flag{ Name: "autoterm.pct", @@ -97,81 +100,47 @@ func runBench(ctx *cli.Context, b bench.Benchmark) error { activeBenchmarkMu.Lock() ab := activeBenchmark activeBenchmarkMu.Unlock() - b.GetCommon().Error = printError + c := b.GetCommon() + c.Error = printError if ab != nil { - b.GetCommon().ClientIdx = ab.clientIdx + c.ClientIdx = ab.clientIdx return runClientBenchmark(ctx, b, ab) } - if done, err := runServerBenchmark(ctx, b); done || err != nil { // Close all extra output channels so the benchmark will terminate - for _, out := range b.GetCommon().ExtraOut { + for _, out := range c.ExtraOut { close(out) } fatalIf(probe.NewError(err), "Error running remote benchmark") return nil } + var ui ui + if !globalQuiet && !globalJSON { + go ui.Run() + } + + retrieveOps, updates := addCollector(ctx, b) + c.UpdateStatus = ui.SetSubText monitor := api.NewBenchmarkMonitor(ctx.String(serverFlagName)) - monitor.SetLnLoggers(printInfo, printError) + monitor.SetLnLoggers(func(data ...interface{}) { + ui.SetSubText(strings.TrimRight(fmt.Sprintln(data...), "\r\n.")) + }, printError) defer monitor.Done() - monitor.InfoLn("Preparing server.") - pgDone := make(chan struct{}) - c := b.GetCommon() + monitor.InfoLn("Preparing server") c.Clear = !ctx.Bool("noclear") if ctx.Bool("autoterm") { - // TODO: autoterm cannot be used when in client/server mode c.AutoTermDur = ctx.Duration("autoterm.dur") c.AutoTermScale = ctx.Float64("autoterm.pct") / 100 } - if !globalQuiet && !globalJSON { - c.PrepareProgress = make(chan float64, 1) - const pgScale = 10000 - pg := newProgressBar(pgScale, pb.U_NO) - pg.ShowCounters = false - pg.ShowElapsedTime = false - pg.ShowSpeed = false - pg.ShowTimeLeft = false - pg.ShowFinalTime = true - go func() { - defer close(pgDone) - defer pg.Finish() - tick := time.NewTicker(time.Millisecond * 125) - defer tick.Stop() - pg.Set(-1) - pg.SetCaption("Preparing: ") - newVal := int64(-1) - for { - select { - case <-tick.C: - current := pg.Get() - if current != newVal { - pg.Set64(newVal) - pg.Update() - } - monitor.InfoQuietln(fmt.Sprintf("Preparation: %0.0f%% done...", float64(newVal)/float64(100))) - case pct, ok := <-c.PrepareProgress: - if !ok { - pg.Set64(pgScale) - if newVal > 0 { - pg.Update() - } - return - } - newVal = int64(pct * pgScale) - } - } - }() - } else { - close(pgDone) - } + c.PrepareProgress = make(chan float64, 1) + ui.StartPrepare("Preparing", c.PrepareProgress, updates) err := b.Prepare(context.Background()) fatalIf(probe.NewError(err), "Error preparing server") if c.PrepareProgress != nil { close(c.PrepareProgress) - <-pgDone } if ap, ok := b.(AfterPreparer); ok { @@ -191,14 +160,19 @@ func runBench(ctx *cli.Context, b bench.Benchmark) error { tStart = startTime } } - + if u := ui.updates.Load(); u != nil { + *u <- aggregate.UpdateReq{Reset: true} + } benchDur := ctx.Duration("duration") + ui.StartBenchmark("Benchmarking", tStart, tStart.Add(benchDur), updates) ctx2, cancel := context.WithDeadline(context.Background(), tStart.Add(benchDur)) defer cancel() + ui.cancelFn.Store(&cancel) start := make(chan struct{}) go func() { + monitor.InfoLn("Pausing before benchmark") <-time.After(time.Until(tStart)) - monitor.InfoLn("Benchmark starting...") + monitor.InfoLn("Running from " + time.Now().Format(time.RFC3339)) close(start) }() @@ -210,73 +184,91 @@ func runBench(ctx *cli.Context, b bench.Benchmark) error { prof, err := startProfiling(ctx2, ctx) fatalIf(probe.NewError(err), "Unable to start profile.") - monitor.InfoLn("Starting benchmark in ", time.Until(tStart).Round(time.Second), "...") - pgDone = make(chan struct{}) - if !globalQuiet && !globalJSON { - pg := newProgressBar(int64(benchDur), pb.U_DURATION) - go func() { - defer close(pgDone) - defer pg.Finish() - pg.SetCaption("Benchmarking:") - tick := time.NewTicker(time.Millisecond * 125) - defer tick.Stop() - done := ctx2.Done() - for { - select { - case t := <-tick.C: - elapsed := t.Sub(tStart) - if elapsed < 0 { - continue - } - pg.Set64(int64(elapsed)) - pg.Update() - monitor.InfoQuietln(fmt.Sprintf("Running benchmark: %0.0f%%...", 100*float64(elapsed)/float64(benchDur))) - case <-done: - pg.Set64(int64(benchDur)) - pg.Update() - return - } - } - }() - } else { - close(pgDone) - } - ops, _ := b.Start(ctx2, start) + monitor.InfoLn("Starting benchmark in", time.Until(tStart).Round(time.Second)) + b.Start(ctx2, start) + c.Collector.Close() cancel() - <-pgDone - // Previous context is canceled, create a new... - monitor.InfoLn("Saving benchmark data...") ctx2 = context.Background() - ops.SortByStartTime() - ops.SetClientID(cID) prof.stop(ctx2, ctx, fileName+".profiles.zip") - if len(ops) > 0 { - f, err := os.Create(fileName + ".csv.zst") + // Previous context is canceled, create a new... + monitor.InfoLn("Saving benchmark data") + if ops := retrieveOps(); len(ops) > 0 { + ops.SortByStartTime() + ops.SetClientID(cID) + + if len(ops) > 0 { + f, err := os.Create(fileName + ".csv.zst") + if err != nil { + monitor.Errorln("Unable to write benchmark data:", err) + } else { + func() { + defer f.Close() + enc, err := zstd.NewWriter(f, zstd.WithEncoderLevel(zstd.SpeedBetterCompression)) + fatalIf(probe.NewError(err), "Unable to compress benchmark output") + + defer enc.Close() + err = ops.CSV(enc, commandLine(ctx)) + fatalIf(probe.NewError(err), "Unable to write benchmark output") + + monitor.InfoLn(fmt.Sprintf("\nBenchmark data written to %q\n", fileName+".csv.zst")) + }() + } + } + monitor.OperationsReady(ops, fileName, commandLine(ctx)) + var buf bytes.Buffer + printAnalysis(ctx, &buf, ops) + ui.Update(tea.Quit()) + ui.Wait() + fmt.Println(buf.String()) + } else if updates != nil { + finalCh := make(chan *aggregate.Realtime, 1) + updates <- aggregate.UpdateReq{Final: true, C: finalCh} + final := <-finalCh + final.Commandline = commandLine(ctx) + final.WarpVersion = GlobalVersion + final.WarpDate = GlobalDate + final.WarpCommit = GlobalCommit + f, err := os.Create(fileName + ".json.zst") if err != nil { monitor.Errorln("Unable to write benchmark data:", err) } else { func() { defer f.Close() enc, err := zstd.NewWriter(f, zstd.WithEncoderLevel(zstd.SpeedBetterCompression)) - fatalIf(probe.NewError(err), "Unable to compress benchmark output") + if err != nil { + monitor.Errorln("Unable to compress benchmark data:", err) + } defer enc.Close() - err = ops.CSV(enc, commandLine(ctx)) - fatalIf(probe.NewError(err), "Unable to write benchmark output") + js := json.NewEncoder(enc) + js.SetIndent("", " ") + err = js.Encode(final) + if err != nil { + monitor.Errorln("Unable to write benchmark data:", err) + } - monitor.InfoLn(fmt.Sprintf("Benchmark data written to %q\n", fileName+".csv.zst")) + monitor.InfoLn(fmt.Sprintf("\nBenchmark data written to %q\n\n", fileName+".json.zst")) }() } + rep := final.Report(aggregate.ReportOptions{ + Details: ctx.Bool("analyze.v"), + Color: !globalNoColor, + }) + + ui.Update(tea.Quit()) + ui.Wait() + fmt.Println("") + fmt.Println(rep) } - monitor.OperationsReady(ops, fileName, commandLine(ctx)) - printAnalysis(ctx, ops) if !ctx.Bool("keep-data") && !ctx.Bool("noclear") { + ui.SetPhase("Cleanup") monitor.InfoLn("Starting cleanup...") b.Cleanup(context.Background()) } monitor.InfoLn("Cleanup Done.") + ui.Wait() return nil } @@ -292,6 +284,7 @@ type clientBenchmark struct { info map[benchmarkStage]stageInfo stage benchmarkStage results bench.Operations + updates chan<- aggregate.UpdateReq clientIdx int sync.Mutex } @@ -300,6 +293,8 @@ type stageInfo struct { start chan struct{} done chan struct{} custom map[string]string + stageCtx context.Context + cancelFn context.CancelFunc startRequested bool } @@ -310,9 +305,12 @@ func (c *clientBenchmark) init(ctx context.Context) { c.info = make(map[benchmarkStage]stageInfo, len(benchmarkStages)) c.ctx, c.cancel = context.WithCancel(ctx) for _, stage := range benchmarkStages { + sCtx, sCancel := context.WithCancel(ctx) c.info[stage] = stageInfo{ - start: make(chan struct{}), - done: make(chan struct{}), + start: make(chan struct{}), + done: make(chan struct{}), + stageCtx: sCtx, + cancelFn: sCancel, } } } @@ -337,7 +335,7 @@ func (c *clientBenchmark) waitForStage(s benchmarkStage) error { // waitForStage waits for the stage to be ready and updates the stage when it is func (c *clientBenchmark) stageDone(s benchmarkStage, err error, custom map[string]string) { - console.Infoln(s, "done...") + console.Infoln("Stage", s, "done...") if err != nil { console.Errorln(err.Error()) } @@ -350,6 +348,9 @@ func (c *clientBenchmark) stageDone(s benchmarkStage, err error, custom map[stri if info.done != nil { close(info.done) } + if info.cancelFn != nil { + info.cancelFn() + } c.info[s] = info c.Unlock() } @@ -379,19 +380,29 @@ func runClientBenchmark(ctx *cli.Context, b bench.Benchmark, cb *clientBenchmark if err != nil { return err } + + retrieveOps, updates := addCollector(ctx, b) common := b.GetCommon() + common.UpdateStatus = func(s string) { + console.Infoln(s) + } + defer common.Collector.Close() + cb.Lock() - start := cb.info[stageBenchmark].start - ctx2, cancel := context.WithCancel(cb.ctx) - defer cancel() + benchStage := cb.info[stageBenchmark] + start := benchStage.start + cb.updates = updates cb.Unlock() - err = b.Prepare(ctx2) + err = b.Prepare(cb.info[stagePrepare].stageCtx) cb.stageDone(stagePrepare, err, common.Custom) if err != nil { return err } + ctx2, cancel := benchStage.stageCtx, benchStage.cancelFn + defer cancel() + // Start after waiting a second or until we reached the start time. benchDur := ctx.Duration("duration") go func() { @@ -422,7 +433,8 @@ func runClientBenchmark(ctx *cli.Context, b bench.Benchmark, cb *clientBenchmark fileName = fmt.Sprintf("%s-%s-%s-%s", appName, ctx.Command.Name, time.Now().Format("2006-01-02[150405]"), cID) } - ops, err := b.Start(ctx2, start) + err = b.Start(ctx2, start) + ops := retrieveOps() cb.Lock() cb.results = ops cb.Unlock() @@ -450,7 +462,37 @@ func runClientBenchmark(ctx *cli.Context, b bench.Benchmark, cb *clientBenchmark console.Infof("Benchmark data written to %q\n", fileName+".csv.zst") }() } + } else if updates != nil { + finalCh := make(chan *aggregate.Realtime, 1) + updates <- aggregate.UpdateReq{Final: true, C: finalCh} + final := <-finalCh + final.Commandline = commandLine(ctx) + final.WarpVersion = GlobalVersion + final.WarpDate = GlobalDate + final.WarpCommit = GlobalCommit + f, err := os.Create(fileName + ".json.zst") + if err != nil { + console.Errorln("Unable to write benchmark data:", err) + } else { + func() { + defer f.Close() + enc, err := zstd.NewWriter(f, zstd.WithEncoderLevel(zstd.SpeedBetterCompression)) + if err != nil { + console.Errorln("Unable to compress benchmark data:", err) + } + + defer enc.Close() + js := json.NewEncoder(enc) + js.SetIndent("", " ") + err = js.Encode(final) + if err != nil { + console.Errorln("Unable to write benchmark data:", err) + } + console.Infoln(fmt.Sprintf("\nBenchmark data written to %q\n\n", fileName+".json.zst")) + }() + } } + common.Collector.Close() err = cb.waitForStage(stageCleanup) if err != nil { @@ -458,13 +500,35 @@ func runClientBenchmark(ctx *cli.Context, b bench.Benchmark, cb *clientBenchmark } if !ctx.Bool("keep-data") && !ctx.Bool("noclear") { console.Infoln("Starting cleanup...") - b.Cleanup(context.Background()) + b.Cleanup(cb.info[stageCleanup].stageCtx) } cb.stageDone(stageCleanup, nil, common.Custom) return nil } +func addCollector(ctx *cli.Context, b bench.Benchmark) (bench.OpsCollector, chan<- aggregate.UpdateReq) { + // Add collectors + common := b.GetCommon() + + if !ctx.Bool("full") { + updates := make(chan aggregate.UpdateReq, 1000) + c := aggregate.LiveCollector(context.Background(), updates, pRandASCII(4)) + c.AddOutput(common.ExtraOut...) + common.Collector = c + return bench.EmptyOpsCollector, updates + } + if common.DiscardOutput { + common.Collector = bench.NewNullCollector() + common.Collector.AddOutput(common.ExtraOut...) + return bench.EmptyOpsCollector, nil + } + var retrieveOps bench.OpsCollector + common.Collector, retrieveOps = bench.NewOpsCollector() + common.Collector.AddOutput(common.ExtraOut...) + return retrieveOps, nil +} + type runningProfiles struct { client *madmin.AdminClient } @@ -570,6 +634,9 @@ func parseLocalTime(s string) time.Time { now := time.Now() y, m, d := now.Date() t = t.AddDate(y, int(m)-1, d-1) + if t.Before(time.Now()) { + t = t.Add(24 * time.Hour) + } return t } diff --git a/cli/benchserver.go b/cli/benchserver.go index a1129c7d..70619187 100644 --- a/cli/benchserver.go +++ b/cli/benchserver.go @@ -19,6 +19,7 @@ package cli import ( "context" + "encoding/json" "errors" "fmt" "net/url" @@ -26,12 +27,15 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" + tea "github.com/charmbracelet/bubbletea" "github.com/klauspost/compress/zstd" "github.com/minio/cli" "github.com/minio/mc/pkg/probe" "github.com/minio/warp/api" + "github.com/minio/warp/pkg/aggregate" "github.com/minio/warp/pkg/bench" "github.com/minio/websocket" ) @@ -45,6 +49,7 @@ const ( serverReqBenchmark serverRequestOp = "benchmark" serverReqStartStage serverRequestOp = "start_stage" serverReqStageStatus serverRequestOp = "stage_status" + serverReqAbortStage serverRequestOp = "stage_abort" serverReqSendOps serverRequestOp = "send_ops" ) @@ -79,10 +84,12 @@ type serverRequest struct { Command string `json:"command"` Args cli.Args `json:"args"` } - StartTime time.Time `json:"start_time"` - Operation serverRequestOp `json:"op"` - Stage benchmarkStage `json:"stage"` - ClientIdx int `json:"client_idx"` + StartTime time.Time `json:"start_time"` + Operation serverRequestOp `json:"op"` + Stage benchmarkStage `json:"stage"` + ClientIdx int `json:"client_idx"` + Aggregate bool `json:"aggregate"` + UpdateReq *aggregate.UpdateReq `json:"update_req,omitempty"` } // runServerBenchmark will run a benchmark server if requested. @@ -92,18 +99,30 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { return false, nil } + if ctx.Bool("autoterm") && ctx.Bool("full") { + return true, errors.New("use of -autoterm cannot be used with --full on remote benchmarks") + } + + var ui ui + if !globalQuiet && !globalJSON { + go ui.Run() + } + conns := newConnections(parseHosts(ctx.String("warp-client"), false)) if len(conns.hosts) == 0 { return true, errors.New("no hosts") } - conns.info = printInfo + infoLn := func(data ...interface{}) { + ui.SetSubText(strings.TrimRight(fmt.Sprintln(data...), "\r\n.")) + } + + conns.info = infoLn conns.errLn = printError defer conns.closeAll() monitor := api.NewBenchmarkMonitor(ctx.String(serverFlagName)) + monitor.SetLnLoggers(infoLn, printError) defer monitor.Done() - monitor.SetLnLoggers(printInfo, printError) - infoLn := monitor.InfoLn - errorLn := monitor.Errorln + errorLn := printError var allOps bench.Operations @@ -122,7 +141,7 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { "host": func(flag cli.Flag) (string, error) { hostsIn := flag.String() if !strings.Contains(hostsIn, "file:") { - return flagToJSON(ctx, flag) + return flagToJSON(ctx, flag, "host") } // This is a file, we will read it locally and expand. hosts := parseHosts(hostsIn, false) @@ -139,15 +158,16 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { req.Benchmark.Flags = make(map[string]string) for _, flag := range ctx.Command.Flags { - if _, ok := excludeFlags[flag.GetName()]; ok { + name := strings.Split(flag.GetName(), ",")[0] + if _, ok := excludeFlags[name]; ok { continue } - if ctx.IsSet(flag.GetName()) { + if ctx.IsSet(name) { var err error - if t := transformFlags[flag.GetName()]; t != nil { - req.Benchmark.Flags[flag.GetName()], err = t(flag) + if t := transformFlags[name]; t != nil { + req.Benchmark.Flags[name], err = t(flag) } else { - req.Benchmark.Flags[flag.GetName()], err = flagToJSON(ctx, flag) + req.Benchmark.Flags[name], err = flagToJSON(ctx, flag, name) } if err != nil { return true, err @@ -159,6 +179,7 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { } // Connect to hosts, send benchmark requests. + ui.StartPrepare("Preparing", nil, nil) for i := range conns.hosts { resp, err := conns.roundTrip(i, req) fatalIf(probe.NewError(err), "Unable to send benchmark info to warp client") @@ -168,11 +189,12 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { infoLn("Client ", conns.hostName(i), " connected...") // Assume ok. } - infoLn("All clients connected...") + infoLn("All clients connected. Preparing benchmark...") common := b.GetCommon() + _ = conns.startStageAll(stagePrepare, time.Now().Add(time.Second), true) - err := conns.waitForStage(stagePrepare, true, common) + err := conns.waitForStage(context.Background(), stagePrepare, true, common, nil) if err != nil { fatalIf(probe.NewError(err), "Failed to prepare") } @@ -184,17 +206,38 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { infoLn("All clients prepared...") const benchmarkWait = 3 * time.Second - + var updates chan aggregate.UpdateReq + if !ctx.Bool("full") { + updates = make(chan aggregate.UpdateReq, 10) + } prof, err := startProfiling(context.Background(), ctx) if err != nil { return true, err } + tStart := time.Now().Add(benchmarkWait) + benchDur := ctx.Duration("duration") err = conns.startStageAll(stageBenchmark, time.Now().Add(benchmarkWait), false) if err != nil { errorLn("Failed to start all clients", err) } - infoLn("Running benchmark on all clients...") - err = conns.waitForStage(stageBenchmark, false, common) + ui.StartBenchmark("Benchmarking", tStart, tStart.Add(benchDur), updates) + ui.SetSubText("Press 'q' to abort benchmark and retrieve partial results") + + benchCtx, cancel := context.WithCancel(context.Background()) + ui.cancelFn.Store(&cancel) + defer cancel() + if ctx.Bool("autoterm") { + if ctx.Bool("full") { + return true, errors.New("use of -autoterm cannot be combined with -full on remote benchmarks") + } + common.AutoTermDur = ctx.Duration("autoterm.dur") + common.AutoTermScale = ctx.Float64("autoterm.pct") / 100 + if common.AutoTermDur > 0 { + benchCtx = aggregate.AutoTerm(benchCtx, "", common.AutoTermScale, int(common.AutoTermDur.Seconds()+0.999), common.AutoTermDur, updates) + } + } + + err = conns.waitForStage(benchCtx, stageBenchmark, false, common, updates) if err != nil { errorLn("Failed to keep connection to all clients", err) } @@ -205,47 +248,89 @@ func runServerBenchmark(ctx *cli.Context, b bench.Benchmark) (bool, error) { } prof.stop(context.Background(), ctx, fileName+".profiles.zip") - infoLn("Done. Downloading operations...") - downloaded := conns.downloadOps() - switch len(downloaded) { - case 0: - case 1: - allOps = downloaded[0] - default: - threads := uint16(0) - for _, ops := range downloaded { - threads = ops.OffsetThreads(threads) - allOps = append(allOps, ops...) + ui.SetPhase("Downloading Operations") + if updates == nil { + downloaded := conns.downloadOps() + switch len(downloaded) { + case 0: + case 1: + allOps = downloaded[0] + default: + threads := uint16(0) + for _, ops := range downloaded { + threads = ops.OffsetThreads(threads) + allOps = append(allOps, ops...) + } } - } - if len(allOps) > 0 { - allOps.SortByStartTime() - f, err := os.Create(fileName + ".csv.zst") + if len(allOps) > 0 { + allOps.SortByStartTime() + f, err := os.Create(fileName + ".csv.zst") + if err != nil { + errorLn("Unable to write benchmark data:", err) + } else { + func() { + defer f.Close() + enc, err := zstd.NewWriter(f, zstd.WithEncoderLevel(zstd.SpeedBetterCompression)) + fatalIf(probe.NewError(err), "Unable to compress benchmark output") + + defer enc.Close() + err = allOps.CSV(enc, commandLine(ctx)) + fatalIf(probe.NewError(err), "Unable to write benchmark output") + + infoLn(fmt.Sprintf("Benchmark data written to %q\n", fileName+".csv.zst")) + }() + } + } + monitor.OperationsReady(allOps, fileName, commandLine(ctx)) + ui.Update(tea.Quit()) + ui.Wait() + printAnalysis(ctx, os.Stdout, allOps) + } else { + final := conns.downloadAggr() + if final.Total.TotalRequests == 0 { + return true, errors.New("no operations received") + } + final.Commandline = commandLine(ctx) + final.WarpVersion = GlobalVersion + final.WarpDate = GlobalDate + final.WarpCommit = GlobalCommit + f, err := os.Create(fileName + ".json.zst") if err != nil { - errorLn("Unable to write benchmark data:", err) + monitor.Errorln("Unable to write benchmark data:", err) } else { func() { defer f.Close() enc, err := zstd.NewWriter(f, zstd.WithEncoderLevel(zstd.SpeedBetterCompression)) fatalIf(probe.NewError(err), "Unable to compress benchmark output") - defer enc.Close() - err = allOps.CSV(enc, commandLine(ctx)) + js := json.NewEncoder(enc) + js.SetIndent("", " ") + err = js.Encode(final) fatalIf(probe.NewError(err), "Unable to write benchmark output") - infoLn(fmt.Sprintf("Benchmark data written to %q\n", fileName+".csv.zst")) + monitor.InfoLn(fmt.Sprintf("Benchmark data written to %q\n", fileName+".json.zst")) }() } + rep := final.Report(aggregate.ReportOptions{ + Details: ctx.Bool("analyze.v"), + Color: !globalNoColor, + }) + ui.Update(tea.Quit()) + ui.Wait() + fmt.Println("") + fmt.Println(rep) } - monitor.OperationsReady(allOps, fileName, commandLine(ctx)) - printAnalysis(ctx, allOps) + + ui.SetPhase("Cleanup") + monitor.InfoLn("Starting cleanup...") + b.Cleanup(context.Background()) err = conns.startStageAll(stageCleanup, time.Now(), false) if err != nil { errorLn("Failed to clean up all clients", err) } - err = conns.waitForStage(stageCleanup, false, common) + err = conns.waitForStage(context.Background(), stageCleanup, false, common, nil) if err != nil { errorLn("Failed to keep connection to all clients", err) } @@ -413,7 +498,7 @@ func (c *connections) startStage(i int, t time.Time, stage benchmarkStage) error c.errorF("Client %v returned error: %v\n", c.hostName(i), resp.Err) return errors.New(resp.Err) } - c.info("Client ", c.hostName(i), ": Requested stage ", stage, " start...") + c.info("Client ", c.hostName(i), ": Requested stage", stage, "start...") return nil } @@ -422,7 +507,7 @@ func (c *connections) startStageAll(stage benchmarkStage, startAt time.Time, fai var wg sync.WaitGroup var gerr error var mu sync.Mutex - c.info("Requesting stage ", stage, " start...") + c.info("Requesting stage", stage, "start...") for i, conn := range c.ws { if conn == nil { @@ -484,10 +569,66 @@ func (c *connections) downloadOps() []bench.Operations { return res } +// downloadOps will download operations from all connected clients. +// If an error is encountered the result will be ignored. +func (c *connections) downloadAggr() aggregate.Realtime { + var wg sync.WaitGroup + var mu sync.Mutex + c.info("Downloading operations...") + var res aggregate.Realtime + for i, conn := range c.ws { + if conn == nil { + continue + } + wg.Add(1) + go func(i int) { + defer wg.Done() + resp, err := c.roundTrip(i, serverRequest{Operation: serverReqSendOps, Aggregate: true, UpdateReq: &aggregate.UpdateReq{Final: true}}) + if err != nil { + c.errorF("Client %v download returned error: %v\n", c.hostName(i), resp.Err) + return + } + if resp.Err != "" { + c.errorF("Client %v returned error: %v\n", c.hostName(i), resp.Err) + return + } + c.info("Client ", c.hostName(i), ": Operations downloaded.") + mu.Lock() + res.Merge(resp.Update) + mu.Unlock() + }(i) + } + wg.Wait() + return res +} + // waitForStage will wait for stage completion on all clients. -func (c *connections) waitForStage(stage benchmarkStage, failOnErr bool, common *bench.Common) error { +func (c *connections) waitForStage(ctx context.Context, stage benchmarkStage, failOnErr bool, common *bench.Common, updates <-chan aggregate.UpdateReq) error { var wg sync.WaitGroup var mu sync.Mutex + var lastUpdate []atomic.Pointer[aggregate.Realtime] + var done chan struct{} + if updates != nil { + lastUpdate = make([]atomic.Pointer[aggregate.Realtime], len(c.ws)) + done = make(chan struct{}) + defer close(done) + go func() { + for { + select { + case <-done: + return + case r := <-updates: + var m aggregate.Realtime + for i := range lastUpdate { + if u := lastUpdate[i].Load(); u != nil { + m.Merge(u) + } + } + r.C <- &m + } + } + }() + } for i, conn := range c.ws { if conn == nil { // log? @@ -496,10 +637,33 @@ func (c *connections) waitForStage(stage benchmarkStage, failOnErr bool, common wg.Add(1) go func(i int) { defer wg.Done() + didCancel := false for { + if !didCancel && ctx != nil && ctx.Err() != nil { + c.info("Client ", c.hostName(i), ": Sending cancellation to stage", stage, "...") + req := serverRequest{ + Operation: serverReqAbortStage, + Stage: stage, + } + _, err := c.roundTrip(i, req) + if err != nil { + c.disconnect(i) + if failOnErr { + fatalIf(probe.NewError(err), "Stage failed.") + } + c.errLn(err) + return + } + didCancel = true + continue + } req := serverRequest{ Operation: serverReqStageStatus, Stage: stage, + Aggregate: updates != nil, + } + if updates != nil { + req.UpdateReq = &aggregate.UpdateReq{} } resp, err := c.roundTrip(i, req) if err != nil { @@ -518,6 +682,9 @@ func (c *connections) waitForStage(stage benchmarkStage, failOnErr bool, common c.errorF("Client %v returned error: %v\n", c.hostName(i), resp.Err) return } + if updates != nil { + lastUpdate[i].Store(resp.Update) + } if resp.StageInfo.Finished { // Merge custom if len(resp.StageInfo.Custom) > 0 { @@ -530,7 +697,7 @@ func (c *connections) waitForStage(stage benchmarkStage, failOnErr bool, common } mu.Unlock() } - c.info("Client ", c.hostName(i), ": Finished stage ", stage, "...") + c.info("Client", c.hostName(i), ": Finished stage", stage, "...") return } time.Sleep(time.Second) @@ -542,42 +709,42 @@ func (c *connections) waitForStage(stage benchmarkStage, failOnErr bool, common } // flagToJSON converts a flag to a representation that can be reversed into the flag. -func flagToJSON(ctx *cli.Context, flag cli.Flag) (string, error) { +func flagToJSON(ctx *cli.Context, flag cli.Flag, name string) (string, error) { switch flag.(type) { case cli.StringFlag: - if ctx.IsSet(flag.GetName()) { - return ctx.String(flag.GetName()), nil + if ctx.IsSet(name) { + return ctx.String(name), nil } case cli.BoolFlag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Bool(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Bool(name)), nil } case cli.Int64Flag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Int64(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Int64(name)), nil } case cli.IntFlag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Int(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Int(name)), nil } case cli.DurationFlag: - if ctx.IsSet(flag.GetName()) { - return ctx.Duration(flag.GetName()).String(), nil + if ctx.IsSet(name) { + return ctx.Duration(name).String(), nil } case cli.UintFlag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Uint(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Uint(name)), nil } case cli.Uint64Flag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Uint64(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Uint64(name)), nil } case cli.Float64Flag: - if ctx.IsSet(flag.GetName()) { - return fmt.Sprint(ctx.Float64(flag.GetName())), nil + if ctx.IsSet(name) { + return fmt.Sprint(ctx.Float64(name)), nil } default: - if ctx.IsSet(flag.GetName()) { + if ctx.IsSet(name) { return "", fmt.Errorf("unhandled flag type: %T", flag) } } diff --git a/cli/cli.go b/cli/cli.go index 4b2c0c33..be67afa7 100644 --- a/cli/cli.go +++ b/cli/cli.go @@ -43,6 +43,10 @@ var ( globalJSON = false // Json flag set via command line globalDebug = false // Debug flag set via command line globalNoColor = false // No Color flag set via command line + + GlobalVersion string + GlobalCommit string + GlobalDate string ) const ( @@ -165,7 +169,7 @@ func registerApp(name string, appCmds []cli.Command) *cli.App { cfg := &mprofile.Config{ Path: ctx.String("pprofdir"), UseTempPath: false, - Quiet: false, + Quiet: ctx.Bool("quiet") || ctx.Bool("json"), MemProfileRate: 4096, MemProfileType: "heap", CloserHook: nil, diff --git a/cli/clientmode.go b/cli/clientmode.go index 14efc3ea..0a5c36d3 100644 --- a/cli/clientmode.go +++ b/cli/clientmode.go @@ -70,7 +70,7 @@ func mainClient(ctx *cli.Context) error { fatal(errInvalidArgument(), "Too many parameters") } http.HandleFunc("/ws", serveWs) - console.Infoln("Listening on", addr) + console.Infoln("Listening on", addr, "Press Ctrl+C to exit.") fatalIf(probe.NewError(http.ListenAndServe(addr, nil)), "Unable to start client") return nil } diff --git a/cli/cmp.go b/cli/cmp.go index c042d0b1..e7ddddf7 100644 --- a/cli/cmp.go +++ b/cli/cmp.go @@ -18,12 +18,12 @@ package cli import ( + "errors" "io" "os" "time" "github.com/fatih/color" - "github.com/klauspost/compress/zstd" "github.com/minio/cli" "github.com/minio/mc/pkg/probe" "github.com/minio/pkg/v3/console" @@ -55,19 +55,17 @@ func mainCmp(ctx *cli.Context) error { checkAnalyze(ctx) checkCmp(ctx) args := ctx.Args() - zstdDec, _ := zstd.NewReader(nil) - defer zstdDec.Close() log := console.Printf if globalQuiet { log = nil } readOps := func(s string) bench.Operations { - f, err := os.Open(s) - fatalIf(probe.NewError(err), "Unable to open input file") - defer f.Close() - err = zstdDec.Reset(f) - fatalIf(probe.NewError(err), "Unable to read input") - ops, err := bench.OperationsFromCSV(zstdDec, true, ctx.Int("analyze.offset"), ctx.Int("analyze.limit"), log) + rc, isAggreated := openInput(s) + defer rc.Close() + if isAggreated { + fatalIf(probe.NewError(errors.New("aggregated compare not available yet")), "Aggregated compare not available yet") + } + ops, err := bench.OperationsFromCSV(rc, true, ctx.Int("analyze.offset"), ctx.Int("analyze.limit"), log) fatalIf(probe.NewError(err), "Unable to parse input") return ops } diff --git a/cli/flags.go b/cli/flags.go index 8d1c8073..72aedd52 100644 --- a/cli/flags.go +++ b/cli/flags.go @@ -20,6 +20,7 @@ package cli import ( "fmt" "os" + "strings" "sync" "github.com/minio/cli" @@ -130,11 +131,11 @@ func setGlobals(quiet, debug, json, noColor bool) { func commandLine(ctx *cli.Context) string { s := os.Args[0] + " " + ctx.Command.Name for _, flag := range ctx.Command.Flags { - val, err := flagToJSON(ctx, flag) + name := strings.Split(flag.GetName(), ",")[0] + val, err := flagToJSON(ctx, flag, name) if err != nil || val == "" { continue } - name := flag.GetName() switch name { case "access-key", "secret-key", "influxdb": val = "*REDACTED*" @@ -269,6 +270,11 @@ var ioFlags = []cli.Flag{ Value: 0, Usage: "Rate limit each instance to this number of requests per second (0 to disable)", }, + cli.BoolFlag{ + Name: "stdout", + Usage: "Send operations to stdout", + Hidden: true, + }, cli.StringFlag{ Name: "lookup", Usage: "Force requests to be 'host' for host-style or 'path' for path-style lookup. Default will attempt autodetect based on remote host name.", @@ -296,6 +302,32 @@ func getCommon(ctx *cli.Context, src func() generator.Source) bench.Common { extra = append(extra, in) } } + statusln := func(s string) { + console.Eraseline() + console.Print(s) + } + if globalQuiet { + statusln = func(_ string) {} + } + + if ctx.Bool("stdout") { + globalQuiet = true + statusln = func(_ string) {} + so := make(chan bench.Operation, 1000) + go func() { + i := 0 + var errState bool + for op := range so { + if errState { + continue + } + errState = op.WriteCSV(os.Stdout, i) != nil + i++ + } + }() + extra = append(extra, so) + } + noOps := ctx.Bool("stress") rpsLimit := ctx.Float64("rps-limit") var rpsLimiter *rate.Limiter @@ -311,9 +343,10 @@ func getCommon(ctx *cli.Context, src func() generator.Source) bench.Common { Bucket: ctx.String("bucket"), Location: ctx.String("region"), PutOpts: putOpts(ctx), - DiscardOutput: ctx.Bool("stress"), + DiscardOutput: noOps, ExtraOut: extra, RpsLimiter: rpsLimiter, Transport: clientTransport(ctx), + UpdateStatus: statusln, } } diff --git a/cli/print.go b/cli/print.go index f9e2d146..d2dde76e 100644 --- a/cli/print.go +++ b/cli/print.go @@ -46,18 +46,6 @@ type errorMessage struct { var printMu sync.Mutex -func printInfo(data ...interface{}) { - printMu.Lock() - defer printMu.Unlock() - w, _ := pb.GetTerminalWidth() - if w > 0 { - fmt.Print("\r", strings.Repeat(" ", w), "\r") - } else { - data = append(data, "\n") - } - console.Info(data...) -} - func printError(data ...interface{}) { printMu.Lock() defer printMu.Unlock() diff --git a/cli/progress-bar.go b/cli/progress-bar.go deleted file mode 100644 index 5795758e..00000000 --- a/cli/progress-bar.go +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Warp (C) 2019-2020 MinIO, Inc. - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -package cli - -import ( - "runtime" - "strings" - "time" - - "github.com/cheggaaa/pb" - "github.com/fatih/color" - "github.com/minio/pkg/v3/console" -) - -// progress extender. -type progressBar struct { - *pb.ProgressBar -} - -// newProgressBar - instantiate a progress bar. -func newProgressBar(total int64, units pb.Units) *progressBar { - // Progress bar specific theme customization. - console.SetColor("Bar", color.New(color.FgGreen, color.Bold)) - - pgbar := progressBar{} - - // get the new original progress bar. - bar := pb.New64(total) - - // Set new human friendly print units. - bar.SetUnits(units) - - // Refresh rate for progress bar is set to 125 milliseconds. - bar.SetRefreshRate(time.Millisecond * 125) - - // Do not print a newline by default handled, it is handled manually. - bar.NotPrint = true - - // Show current speed is true. - bar.ShowSpeed = false - bar.ShowTimeLeft = false - bar.ManualUpdate = true - - // Custom callback with colorized bar. - bar.Callback = func(s string) { - printMu.Lock() - defer printMu.Unlock() - console.Print(console.Colorize("Bar", "\r"+s+"\r")) - } - - // Use different unicodes for Linux, OS X and Windows. - switch runtime.GOOS { - case "linux", "windows": - // Need to add '\x00' as delimiter for unicode characters. - bar.Format("┃\x00▓\x00█\x00░\x00┃") - case "darwin": - // Need to add '\x00' as delimiter for unicode characters. - bar.Format(" \x00▓\x00 \x00░\x00 ") - default: - // Default to non unicode characters. - bar.Format("[=> ]") - } - - // Start the progress bar. - if bar.Total > 0 { - bar.Start() - } - - // Copy for future - pgbar.ProgressBar = bar - - // Return new progress bar here. - return &pgbar -} - -// Set caption. -func (p *progressBar) SetCaption(caption string) *progressBar { - caption = fixateBarCaption(caption, getFixedWidth(p.ProgressBar.GetWidth(), 18)) - p.ProgressBar.Prefix(caption) - return p -} - -func (p *progressBar) Set64(length int64) *progressBar { - p.ProgressBar = p.ProgressBar.Set64(length) - return p -} - -func (p *progressBar) SetTotal(total int64) *progressBar { - p.ProgressBar.Total = total - return p -} - -// fixateBarCaption - fancify bar caption based on the terminal width. -func fixateBarCaption(caption string, width int) string { - switch { - case len(caption) > width: - // Trim caption to fit within the screen - trimSize := len(caption) - width + 3 - if trimSize < len(caption) { - caption = "..." + caption[trimSize:] - } - case len(caption) < width: - caption += strings.Repeat(" ", width-len(caption)) - } - return caption -} - -// getFixedWidth - get a fixed width based for a given percentage. -func getFixedWidth(width, percent int) int { - return width * percent / 100 -} diff --git a/cli/ui.go b/cli/ui.go new file mode 100644 index 00000000..c35c95d1 --- /dev/null +++ b/cli/ui.go @@ -0,0 +1,285 @@ +/* + * Warp (C) 2019-2024 MinIO, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +package cli + +import ( + "context" + "fmt" + "os" + "runtime/pprof" + "sync/atomic" + "time" + + "github.com/charmbracelet/bubbles/progress" + tea "github.com/charmbracelet/bubbletea" + "github.com/charmbracelet/lipgloss" + "github.com/minio/pkg/v3/console" + "github.com/minio/warp/pkg/aggregate" + "github.com/minio/warp/pkg/bench" + "github.com/muesli/termenv" +) + +type ui struct { + progress progress.Model + pct atomic.Pointer[float64] + phase atomic.Pointer[string] + phaseTxt atomic.Pointer[string] + updates atomic.Pointer[chan<- aggregate.UpdateReq] + start, end atomic.Pointer[time.Time] + pause atomic.Bool + quitPls atomic.Bool + showProgress bool + cancelFn atomic.Pointer[context.CancelFunc] + quitCh chan struct{} +} + +type tickMsg time.Time + +func tickCmd() tea.Cmd { + return tea.Tick(time.Second/2, func(t time.Time) tea.Msg { + return tickMsg(t) + }) +} + +func (u *ui) Init() tea.Cmd { + u.progress = progress.New(progress.WithScaledGradient("#c72e49", "#edf7f7"), progress.WithSolidFill("#c72e49")) + u.quitCh = make(chan struct{}) + return tea.Batch(tickCmd()) +} + +func (u *ui) Run() { + p := tea.NewProgram(u) + + if _, err := p.Run(); err != nil { + fmt.Printf("UI: %v", err) + } + close(u.quitCh) + if c := u.cancelFn.Load(); c != nil { + cancel := *c + cancel() + } + u.quitPls.Store(true) +} + +func (u *ui) Wait() { + if u.quitCh != nil { + <-u.quitCh + } +} + +const ( + padding = 2 + maxWidth = 80 +) + +func (u *ui) Update(msg tea.Msg) (tea.Model, tea.Cmd) { + if u.quitPls.Load() { + return u, tea.Quit + } + switch msg := msg.(type) { + case tea.KeyMsg: + switch msg.String() { + case "esc": + pprof.Lookup("goroutine").WriteTo(os.Stdout, 1) + case "ctrl+c", "q": + return u, tea.Quit + } + case tea.QuitMsg: + u.quitPls.Store(true) + return u, tea.Quit + case tea.WindowSizeMsg: + u.progress.Width = msg.Width - 4 + if u.progress.Width > maxWidth-padding { + u.progress.Width = maxWidth - padding + } + case tickMsg: + batch := []tea.Cmd{tickCmd()} + u.showProgress = false + if p := u.pct.Load(); p != nil { + u.showProgress = true + batch = append(batch, u.progress.SetPercent(*p)) + } else if start := u.start.Load(); start != nil { + u.showProgress = true + end := u.end.Load() + now := time.Now() + switch { + case now.Before(*start): + batch = append(batch, u.progress.SetPercent(0)) + case now.After(*end): + batch = append(batch, u.progress.SetPercent(1)) + default: + a, b := start.UnixNano(), end.UnixNano() + pct := float64(now.UnixNano()-a) / float64(b-a) + batch = append(batch, u.progress.SetPercent(pct)) + u.progress.SetPercent(pct) + } + } + if u.quitPls.Load() { + batch = append(batch, tea.Quit) + } + + return u, tea.Batch(batch...) + + case progress.FrameMsg: + // FrameMsg is sent when the progress bar wants to animate itself + progressModel, cmd := u.progress.Update(msg) + u.progress = progressModel.(progress.Model) + return u, cmd + } + + return u, nil +} + +func (u *ui) View() string { + res := titleStyle.Render("WARP S3 Benchmark Tool by MinIO") + res += "\n" + + if ph := u.phase.Load(); ph != nil { + status := "\n" + *ph + if ph := u.phaseTxt.Load(); ph != nil { + status += ": " + *ph + } + status += "...\n\n" + res += statusStyle.Render(status) + } + + res += defaultStyle.Render("\r λ ") + if u.showProgress { + res += u.progress.View() + "\n" + } else { + res += "\n" + } + if up := u.updates.Load(); up != nil { + reqCh := *up + respCh := make(chan *aggregate.Realtime, 1) + reqCh <- aggregate.UpdateReq{C: respCh} + var resp *aggregate.Realtime + select { + case resp = <-respCh: + case <-time.After(time.Second): + } + if resp != nil { + stats := fmt.Sprintf("\nReqs: %d, Errs:%d, Objs:%d, Bytes: %d\n", resp.Total.TotalRequests, resp.Total.TotalErrors, resp.Total.TotalObjects, resp.Total.TotalBytes) + ops := stringKeysSorted(resp.ByOpType) + for _, op := range ops { + tp := resp.ByOpType[op].Throughput + segs := tp.Segmented + if segs == nil || len(segs.Segments) == 0 { + continue + } + stats += fmt.Sprintf(" -%10s Average: %.0f Obj/s, %s; ", op, tp.ObjectsPS(), tp.BytesPS().String()) + lastOps := segs.Segments[len(segs.Segments)-1] + stats += fmt.Sprintf("Current %.0f Obj/s, %s", lastOps.OPS, bench.Throughput(lastOps.BPS)) + if len(resp.ByOpType[op].Requests) == 0 { + stats += ".\n" + continue + } + for _, reqs := range resp.ByOpType[op].Requests { + if len(reqs) > 0 { + reqs := reqs[len(reqs)-1] + if reqs.Single != nil { + stats += fmt.Sprintf(", %.1f ms/req", reqs.Single.DurAvgMillis) + if reqs.Single.FirstByte != nil { + stats += fmt.Sprintf(", TTFB: %.1fms", reqs.Single.FirstByte.AverageMillis) + } + } + stats += ".\n" + } else { + stats += "\n" + } + if len(resp.ByOpType[op].Requests) > 1 { + // Maybe handle more clients better... + break + } + } + } + res += statsStyle.Render(stats) + } + } + return res + "\n" +} + +func (u *ui) SetSubText(caption string) { + if u.quitPls.Load() == true { + u.Wait() + console.Printf("\r%-80s", caption) + return + } + u.phaseTxt.Store(&caption) +} + +func (u *ui) SetPhase(caption string) { + if u.quitPls.Load() == true { + u.Wait() + console.Println("\n" + caption) + return + } + u.phase.Store(&caption) + u.phaseTxt.Store(nil) +} + +func (u *ui) StartPrepare(caption string, progress <-chan float64, ur chan<- aggregate.UpdateReq) { + u.phase.Store(&caption) + u.phaseTxt.Store(nil) + if ur != nil { + u.updates.Store(&ur) + } + if progress != nil { + go func() { + for p := range progress { + u.pct.Store(&p) + } + u.pct.Store(nil) + }() + } else { + u.pct.Store(nil) + } +} + +func (u *ui) StartBenchmark(caption string, start, end time.Time, ur chan<- aggregate.UpdateReq) { + u.phase.Store(&caption) + u.phaseTxt.Store(nil) + u.end.Store(&end) + u.start.Store(&start) + if ur != nil { + u.updates.Store(&ur) + } +} + +func (u *ui) Pause(b bool) { + u.pause.Store(b) +} + +const borderCol = lipgloss.Color("#c72e49") + +var ( + titleStyle = func() lipgloss.Style { + b := lipgloss.RoundedBorder() + return lipgloss.NewStyle().BorderStyle(b).Padding(0, 1).Foreground(lipgloss.ANSIColor(termenv.ANSIBrightWhite)).BorderForeground(borderCol) + }() + + defaultStyle = func() lipgloss.Style { + return lipgloss.NewStyle().Foreground(lipgloss.ANSIColor(termenv.ANSIWhite)) + }() + statusStyle = func() lipgloss.Style { + return lipgloss.NewStyle().Foreground(lipgloss.ANSIColor(termenv.ANSIBrightBlue)) + }() + statsStyle = func() lipgloss.Style { + return lipgloss.NewStyle().Foreground(lipgloss.ANSIColor(termenv.ANSIWhite)) + }() +) diff --git a/go.mod b/go.mod index c55fb34b..b1e0849b 100644 --- a/go.mod +++ b/go.mod @@ -4,6 +4,9 @@ go 1.23 require ( github.com/bygui86/multi-profile/v2 v2.1.0 + github.com/charmbracelet/bubbles v0.20.0 + github.com/charmbracelet/bubbletea v1.2.4 + github.com/charmbracelet/lipgloss v1.0.0 github.com/cheggaaa/pb v1.0.29 github.com/dustin/go-humanize v1.0.1 github.com/fatih/color v1.18.0 @@ -17,6 +20,7 @@ require ( github.com/minio/minio-go/v7 v7.0.83 github.com/minio/pkg/v3 v3.0.26-0.20250106155027-2becdc33e233 github.com/minio/websocket v1.6.0 + github.com/muesli/termenv v0.15.2 github.com/posener/complete v1.2.3 golang.org/x/net v0.33.0 golang.org/x/time v0.9.0 @@ -25,7 +29,12 @@ require ( require ( github.com/apapsch/go-jsonmerge/v2 v2.0.0 // indirect + github.com/aymanbagabas/go-osc52/v2 v2.0.1 // indirect + github.com/charmbracelet/harmonica v0.2.0 // indirect + github.com/charmbracelet/x/ansi v0.6.0 // indirect + github.com/charmbracelet/x/term v0.2.1 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect + github.com/erikgeiser/coninput v0.0.0-20211004153227-1c3628e74d0f // indirect github.com/go-ini/ini v1.67.0 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/goccy/go-json v0.10.4 // indirect @@ -42,13 +51,17 @@ require ( github.com/lestrrat-go/iter v1.0.2 // indirect github.com/lestrrat-go/jwx/v2 v2.1.3 // indirect github.com/lestrrat-go/option v1.0.1 // indirect + github.com/lucasb-eyer/go-colorful v1.2.0 // indirect github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-isatty v0.0.20 // indirect + github.com/mattn/go-localereader v0.0.1 // indirect github.com/mattn/go-runewidth v0.0.16 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect + github.com/muesli/ansi v0.0.0-20230316100256-276c6243b2f6 // indirect + github.com/muesli/cancelreader v0.2.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect - github.com/oapi-codegen/runtime v1.1.1 // indirect + github.com/oapi-codegen/runtime v1.0.0 // indirect github.com/philhofer/fwd v1.1.3-0.20240916144458-20a13a1f6b7c // indirect github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 // indirect github.com/prometheus/client_model v0.6.1 // indirect diff --git a/go.sum b/go.sum index 8986788a..311262de 100644 --- a/go.sum +++ b/go.sum @@ -2,9 +2,23 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/RaveNoX/go-jsoncommentstrip v1.0.0/go.mod h1:78ihd09MekBnJnxpICcwzCMzGrKSKYe4AqU6PDYYpjk= github.com/apapsch/go-jsonmerge/v2 v2.0.0 h1:axGnT1gRIfimI7gJifB699GoE/oq+F2MU7Dml6nw9rQ= github.com/apapsch/go-jsonmerge/v2 v2.0.0/go.mod h1:lvDnEdqiQrp0O42VQGgmlKpxL1AP2+08jFMw88y4klk= +github.com/aymanbagabas/go-osc52/v2 v2.0.1 h1:HwpRHbFMcZLEVr42D4p7XBqjyuxQH5SMiErDT4WkJ2k= +github.com/aymanbagabas/go-osc52/v2 v2.0.1/go.mod h1:uYgXzlJ7ZpABp8OJ+exZzJJhRNQ2ASbcXHWsFqH8hp8= github.com/bmatcuk/doublestar v1.1.1/go.mod h1:UD6OnuiIn0yFxxA2le/rnRU1G4RaI4UvFv1sNto9p6w= github.com/bygui86/multi-profile/v2 v2.1.0 h1:x/jPqeL/6hJqLXoDI/H5zLPsSFbDR6IEbrBbFpkWQdw= github.com/bygui86/multi-profile/v2 v2.1.0/go.mod h1:f4qCZiQo1nnJdwbPoADUtdDXg3hhnpfgZ9iq3/kW4BA= +github.com/charmbracelet/bubbles v0.20.0 h1:jSZu6qD8cRQ6k9OMfR1WlM+ruM8fkPWkHvQWD9LIutE= +github.com/charmbracelet/bubbles v0.20.0/go.mod h1:39slydyswPy+uVOHZ5x/GjwVAFkCsV8IIVy+4MhzwwU= +github.com/charmbracelet/bubbletea v1.2.4 h1:KN8aCViA0eps9SCOThb2/XPIlea3ANJLUkv3KnQRNCE= +github.com/charmbracelet/bubbletea v1.2.4/go.mod h1:Qr6fVQw+wX7JkWWkVyXYk/ZUQ92a6XNekLXa3rR18MM= +github.com/charmbracelet/harmonica v0.2.0 h1:8NxJWRWg/bzKqqEaaeFNipOu77YR5t8aSwG4pgaUBiQ= +github.com/charmbracelet/harmonica v0.2.0/go.mod h1:KSri/1RMQOZLbw7AHqgcBycp8pgJnQMYYT8QZRqZ1Ao= +github.com/charmbracelet/lipgloss v1.0.0 h1:O7VkGDvqEdGi93X+DeqsQ7PKHDgtQfF8j8/O2qFMQNg= +github.com/charmbracelet/lipgloss v1.0.0/go.mod h1:U5fy9Z+C38obMs+T+tJqst9VGzlOYGj4ri9reL3qUlo= +github.com/charmbracelet/x/ansi v0.6.0 h1:qOznutrb93gx9oMiGf7caF7bqqubh6YIM0SWKyA08pA= +github.com/charmbracelet/x/ansi v0.6.0/go.mod h1:KBUFw1la39nl0dLl10l5ORDAqGXaeurTQmwyyVKse/Q= +github.com/charmbracelet/x/term v0.2.1 h1:AQeHeLZ1OqSXhrAWpYUtZyX1T3zVxfpZuEQMIQaGIAQ= +github.com/charmbracelet/x/term v0.2.1/go.mod h1:oQ4enTYFV7QN4m0i9mzHrViD7TQKvNEEkHUMCmsxdUg= github.com/cheggaaa/pb v1.0.29 h1:FckUN5ngEk2LpvuG0fw1GEFx6LtyY2pWI/Z2QgCnEYo= github.com/cheggaaa/pb v1.0.29/go.mod h1:W40334L7FMC5JKWldsTWbdGjLo0RxUKK73K+TuPxX30= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -14,6 +28,8 @@ github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 h1:rpfIENRNNilwHwZeG5+P150SMrnN github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= +github.com/erikgeiser/coninput v0.0.0-20211004153227-1c3628e74d0f h1:Y/CXytFA4m6baUTXGLOoWe4PQhGxaX0KpnayAqC48p4= +github.com/erikgeiser/coninput v0.0.0-20211004153227-1c3628e74d0f/go.mod h1:vw97MGsxSvLiUE2X8qFplwetxpGLQrlU1Q9AUEIzCaM= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= @@ -67,6 +83,8 @@ github.com/lestrrat-go/jwx/v2 v2.1.3 h1:Ud4lb2QuxRClYAmRleF50KrbKIoM1TddXgBrneT5 github.com/lestrrat-go/jwx/v2 v2.1.3/go.mod h1:q6uFgbgZfEmQrfJfrCo90QcQOcXFMfbI/fO0NqRtvZo= github.com/lestrrat-go/option v1.0.1 h1:oAzP2fvZGQKWkvHa1/SAcFolBEca1oN+mQ7eooNBEYU= github.com/lestrrat-go/option v1.0.1/go.mod h1:5ZHFbivi4xwXxhxY9XHDe2FHo6/Z7WWmtT7T5nBBp3I= +github.com/lucasb-eyer/go-colorful v1.2.0 h1:1nnpGOrhyZZuNyfu1QjKiUICQ74+3FNCN69Aj6K7nkY= +github.com/lucasb-eyer/go-colorful v1.2.0/go.mod h1:R4dSotOR9KMtayYi1e77YzuveK+i7ruzyGqttikkLy0= github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 h1:7UMa6KCCMjZEMDtTVdcGu0B1GmmC7QJKiCCjyTAWQy0= github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683/go.mod h1:ilwx/Dta8jXAgpFYFvSWEMwxmbWXyiUHkd5FwyKhb5k= github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= @@ -77,6 +95,8 @@ github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOA github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/go-localereader v0.0.1 h1:ygSAOl7ZXTx4RdPYinUpg6W99U8jWvWi9Ye2JC/oIi4= +github.com/mattn/go-localereader v0.0.1/go.mod h1:8fBrzywKY7BI3czFoHkuzRoWE9C+EiG4R1k4Cjx5p88= github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= @@ -98,10 +118,16 @@ github.com/minio/pkg/v3 v3.0.26-0.20250106155027-2becdc33e233 h1:SR5q/92Xqkj2Zg3 github.com/minio/pkg/v3 v3.0.26-0.20250106155027-2becdc33e233/go.mod h1:mIaN552nu0D2jiSk5BQC8LB25f44ytbOBJCuLtksX7Q= github.com/minio/websocket v1.6.0 h1:CPvnQvNvlVaQmvw5gtJNyYQhg4+xRmrPNhBbv8BdpAE= github.com/minio/websocket v1.6.0/go.mod h1:COH1CePZfHT9Ec1O7vZjTlX5uEPpyYnrifPNbu665DM= +github.com/muesli/ansi v0.0.0-20230316100256-276c6243b2f6 h1:ZK8zHtRHOkbHy6Mmr5D264iyp3TiX5OmNcI5cIARiQI= +github.com/muesli/ansi v0.0.0-20230316100256-276c6243b2f6/go.mod h1:CJlz5H+gyd6CUWT45Oy4q24RdLyn7Md9Vj2/ldJBSIo= +github.com/muesli/cancelreader v0.2.2 h1:3I4Kt4BQjOR54NavqnDogx/MIoWBFa0StPA8ELUXHmA= +github.com/muesli/cancelreader v0.2.2/go.mod h1:3XuTXfFS2VjM+HTLZY9Ak0l6eUKfijIfMUZ4EgX0QYo= +github.com/muesli/termenv v0.15.2 h1:GohcuySI0QmI3wN8Ok9PtKGkgkFIk7y6Vpb5PvrY+Wo= +github.com/muesli/termenv v0.15.2/go.mod h1:Epx+iuz8sNs7mNKhxzH4fWXGNpZwUaJKRS1noLXviQ8= github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= -github.com/oapi-codegen/runtime v1.1.1 h1:EXLHh0DXIJnWhdRPN2w4MXAzFyE4CskzhNLUmtpMYro= -github.com/oapi-codegen/runtime v1.1.1/go.mod h1:SK9X900oXmPWilYR5/WKPzt3Kqxn/uS/+lbpREv+eCg= +github.com/oapi-codegen/runtime v1.0.0 h1:P4rqFX5fMFWqRzY9M/3YF9+aPSPPB06IzP2P7oOxrWo= +github.com/oapi-codegen/runtime v1.0.0/go.mod h1:LmCUMQuPB4M/nLXilQXhHw+BLZdDb18B34OO356yJ/A= github.com/philhofer/fwd v1.1.3-0.20240916144458-20a13a1f6b7c h1:dAMKvw0MlJT1GshSTtih8C2gDs04w8dReiOGXrGLNoY= github.com/philhofer/fwd v1.1.3-0.20240916144458-20a13a1f6b7c/go.mod h1:RqIHx9QI14HlwKwm98g9Re5prTQ6LdeRQn+gXJFxsJM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= @@ -176,6 +202,7 @@ golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= diff --git a/main.go b/main.go index 493656fe..66918c73 100644 --- a/main.go +++ b/main.go @@ -23,6 +23,15 @@ import ( "github.com/minio/warp/cli" ) +var ( + version string + commit string + date string +) + func main() { + cli.GlobalVersion = version + cli.GlobalCommit = commit + cli.GlobalDate = date cli.Main(os.Args) } diff --git a/pkg/aggregate/aggregate.go b/pkg/aggregate/aggregate.go index 7bf96e8c..d62f2c28 100644 --- a/pkg/aggregate/aggregate.go +++ b/pkg/aggregate/aggregate.go @@ -133,7 +133,7 @@ func Aggregate(o bench.Operations, opts Options) Aggregated { a.MixedServerStats.Segmented = &ThroughputSegmented{ SegmentDurationMillis: durToMillis(segmentDur), } - a.MixedServerStats.Segmented.fill(segs, total) + a.MixedServerStats.Segmented.fill(segs, total.TotalBytes) } eps := o.SortSplitByEndpoint() @@ -224,7 +224,7 @@ func Aggregate(o bench.Operations, opts Options) Aggregated { a.Throughput.Segmented = &ThroughputSegmented{ SegmentDurationMillis: durToMillis(segmentDur), } - a.Throughput.Segmented.fill(segs, total) + a.Throughput.Segmented.fill(segs, total.TotalBytes) a.ObjectsPerOperation = ops.FirstObjPerOp() a.Concurrency = ops.Threads() a.Clients = ops.Clients() @@ -274,7 +274,7 @@ func Aggregate(o bench.Operations, opts Options) Aggregated { host.Segmented = &ThroughputSegmented{ SegmentDurationMillis: durToMillis(segmentDur), } - host.Segmented.fill(segs, total) + host.Segmented.fill(segs, total.TotalBytes) } epMu.Lock() a.ThroughputByHost[ep] = host diff --git a/pkg/aggregate/collector.go b/pkg/aggregate/collector.go new file mode 100644 index 00000000..93b83d96 --- /dev/null +++ b/pkg/aggregate/collector.go @@ -0,0 +1,175 @@ +/* + * Warp (C) 2019-2024 MinIO, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +package aggregate + +import ( + "context" + "math" + "sync" + "time" + + "github.com/minio/pkg/v3/console" + "github.com/minio/warp/pkg/bench" +) + +// LiveCollector return a collector, and a channel that will return the +// current aggregate on the channel whenever it is requested. +func LiveCollector(ctx context.Context, updates chan UpdateReq, clientID string) bench.Collector { + c := collector{ + rcv: make(chan bench.Operation, 1000), + } + // Unbuffered, so we can send on unblock. + if updates == nil { + updates = make(chan UpdateReq, 1000) + } + c.updates = updates + go func() { + final := Live(c.rcv, updates, clientID) + for { + select { + case <-ctx.Done(): + return + case req := <-updates: + select { + case req.C <- final: + default: + } + } + } + }() + return &c +} + +// UpdateReq is a request for an update. +// The latest will be sent on the provided channel, or nil if none is available yet. +// If the provided channel blocks no update will be sent. +type UpdateReq struct { + C chan<- *Realtime `json:"-"` + Reset bool `json:"reset"` + Final bool `json:"final"` +} + +type collector struct { + mu sync.Mutex + rcv chan bench.Operation + extra []chan<- bench.Operation + updates chan<- UpdateReq + doneFn []context.CancelFunc +} + +func (c *collector) AutoTerm(ctx context.Context, op string, threshold float64, wantSamples, _ int, minDur time.Duration) context.Context { + return AutoTerm(ctx, op, threshold, wantSamples, minDur, c.updates) +} + +// AutoTerm allows to set auto-termination on a context. +func AutoTerm(ctx context.Context, op string, threshold float64, wantSamples int, minDur time.Duration, updates chan<- UpdateReq) context.Context { + if updates == nil { + return ctx + } + ctx, cancel := context.WithCancel(ctx) + go func() { + defer cancel() + ticker := time.NewTicker(time.Second) + + checkloop: + for { + select { + case <-ctx.Done(): + ticker.Stop() + return + case <-ticker.C: + } + respCh := make(chan *Realtime, 1) + req := UpdateReq{C: respCh, Reset: false, Final: false} + updates <- req + resp := <-respCh + if resp == nil { + continue + } + + ops := resp.ByOpType[op] + if op == "" { + ops = &resp.Total + } + if ops == nil || ops.Throughput.Segmented == nil { + continue + } + start, end := ops.StartTime, ops.EndTime + if end.Sub(start) <= minDur { + // We don't have enough. + continue + } + if len(ops.Throughput.Segmented.Segments) < wantSamples { + continue + } + segs := ops.Throughput.Segmented.Segments + // Use last segment as our base. + lastSeg := segs[len(segs)-1] + mb, objs := lastSeg.BPS, lastSeg.OPS + // Only use the segments we are interested in. + segs = segs[len(segs)-wantSamples : len(segs)-1] + for _, seg := range segs { + segMB, segObjs := seg.BPS, seg.OPS + if mb > 0 { + if math.Abs(mb-segMB) > threshold*mb { + continue checkloop + } + continue + } + if math.Abs(objs-segObjs) > threshold*objs { + continue checkloop + } + } + // All checks passed. + if mb > 0 { + console.Eraseline() + console.Printf("\rThroughput %0.01fMiB/s within %f%% for %v. Assuming stability. Terminating benchmark.\n", + mb, threshold*100, + time.Duration(ops.Throughput.Segmented.SegmentDurationMillis*(len(segs)+1))*time.Millisecond) + } else { + console.Eraseline() + console.Printf("\rThroughput %0.01f objects/s within %f%% for %v. Assuming stability. Terminating benchmark.\n", + objs, threshold*100, + time.Duration(ops.Throughput.Segmented.SegmentDurationMillis*(len(segs)+1))*time.Millisecond) + } + return + } + }() + return ctx +} + +func (c *collector) Receiver() chan<- bench.Operation { + return c.rcv +} + +func (c *collector) AddOutput(operations ...chan<- bench.Operation) { + c.extra = append(c.extra, operations...) +} + +func (c *collector) Close() { + c.mu.Lock() + defer c.mu.Unlock() + if c.rcv != nil { + close(c.rcv) + c.rcv = nil + } + for _, cancel := range c.doneFn { + cancel() + } + c.doneFn = nil +} diff --git a/pkg/aggregate/live.go b/pkg/aggregate/live.go new file mode 100644 index 00000000..7ff6014e --- /dev/null +++ b/pkg/aggregate/live.go @@ -0,0 +1,949 @@ +/* + * Warp (C) 2019-2025 MinIO, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +package aggregate + +import ( + "bytes" + "fmt" + "io" + "math/bits" + "slices" + "sort" + "sync" + "sync/atomic" + "time" + + "github.com/fatih/color" + "github.com/minio/pkg/v3/console" + "github.com/minio/warp/pkg/bench" +) + +const currentVersion = 1 + +// Realtime is a collection of realtime aggregated data. +type Realtime struct { + DataVersion int `json:"v"` + Commandline string `json:"commandline"` + Final bool `json:"final"` + WarpVersion string `json:"warp_version,omitempty"` + WarpCommit string `json:"warp_commit,omitempty"` + WarpDate string `json:"warp_date,omitempty"` + + Total LiveAggregate `json:"total"` + ByOpType map[string]*LiveAggregate `json:"by_op_type,omitempty"` + // These are really not used. + ByHost map[string]*LiveAggregate `json:"by_host,omitempty"` + ByObjLog2Size map[int]*LiveAggregate `json:"by_obj_log_2_size,omitempty"` + ByClient map[string]*LiveAggregate `json:"by_client,omitempty"` + ByCategory map[bench.Category]*LiveAggregate `json:"by_category,omitempty"` +} + +type LiveAggregate struct { + Title string + // Total requests + TotalRequests int `json:"total_requests"` + // Total objects + TotalObjects int `json:"total_objects"` + // Total errors + TotalErrors int `json:"total_errors"` + // Total bytes + TotalBytes int64 `json:"total_bytes"` + // Concurrency is the number of threads seen. + Concurrency int `json:"concurrency"` + + // Unfiltered start time of this operation segment. + StartTime time.Time `json:"start_time"` + // Unfiltered end time of this operation segment. + EndTime time.Time `json:"end_time"` + + // Subset of errors. + FirstErrors []string `json:"first_errors"` + // Numbers of hosts + Hosts MapAsSlice `json:"hosts"` + // Number of warp clients. + Clients MapAsSlice `json:"clients"` + + throughput liveThroughput + + // Throughput information. + Throughput Throughput `json:"throughput"` + + // ThroughputByHost information. Without segments. + ThroughputByHost map[string]Throughput `json:"throughput_by_host"` + + // ThroughputByClient information. Without segments. + ThroughputByClient map[string]Throughput `json:"throughput_by_client"` + + // Requests segmented. + // Indexed by client. + Requests map[string]RequestSegments `json:"requests_by_client"` + + requests map[string]liveRequests + + threadIDs map[uint16]struct{} +} + +type RequestSegment struct { + StartTime time.Time `json:"start_time"` + EndTime time.Time `json:"end_time"` + + // Populated if requests are of difference object sizes. + Multi *MultiSizedRequests `json:"multi_sized_requests,omitempty"` + + // Populated if requests are all of same object size. + Single *SingleSizedRequests `json:"single_sized_requests,omitempty"` +} + +type RequestSegments []RequestSegment + +const maxFirstErrors = 10 + +// Add operation to aggregate. +func (l *LiveAggregate) Add(o bench.Operation) { + l.TotalRequests++ + l.TotalObjects += o.ObjPerOp + l.TotalBytes += o.Size + if o.ClientID != "" { + l.Clients.Add(o.ClientID) + } + if l.threadIDs == nil { + l.threadIDs = make(map[uint16]struct{}) + } + if _, ok := l.threadIDs[o.Thread]; !ok { + l.threadIDs[o.Thread] = struct{}{} + } + l.Hosts.Add(o.Endpoint) + if l.StartTime.IsZero() || l.StartTime.After(o.Start) { + l.StartTime = o.Start + } + if l.EndTime.Before(o.End) { + l.EndTime = o.End + } + if o.Err != "" { + if len(l.FirstErrors) < maxFirstErrors { + l.FirstErrors = append(l.FirstErrors, o.Err) + } + l.TotalErrors++ + } + if l.ThroughputByHost == nil { + l.ThroughputByHost = make(map[string]Throughput) + } + if l.ThroughputByClient == nil { + l.ThroughputByClient = make(map[string]Throughput) + } + l.ThroughputByHost[o.Endpoint] = l.ThroughputByHost[o.Endpoint].Add(o) + l.ThroughputByClient[o.ClientID] = l.ThroughputByClient[o.ClientID].Add(o) + l.throughput.Add(o) + if l.requests == nil { + l.requests = make(map[string]liveRequests, 10) + } + req := l.requests[o.ClientID] + req.Add(o) + l.requests[o.ClientID] = req +} + +// Merge l2 into l. +func (l *LiveAggregate) Merge(l2 LiveAggregate) { + l.Throughput.Merge(l2.Throughput) + if l.Requests == nil { + l.Requests = make(map[string]RequestSegments) + } + if len(l2.Requests) > 0 { + for k, v := range l2.Requests { + l.Requests[k] = v + } + } + l.Concurrency += l2.Concurrency + l.TotalBytes += l2.TotalBytes + l.TotalObjects += l2.TotalObjects + l.TotalErrors += l2.TotalErrors + l.FirstErrors = append(l.FirstErrors, l2.FirstErrors...) + if len(l.FirstErrors) > maxFirstErrors { + l.FirstErrors = l.FirstErrors[:maxFirstErrors] + } + l.Clients.AddMap(l2.Clients) + l.Hosts.AddMap(l2.Hosts) + l.TotalRequests += l2.TotalRequests + if l.StartTime.IsZero() || l2.StartTime.Before(l.StartTime) { + l.StartTime = l2.StartTime + } + if l2.EndTime.After(l.EndTime) { + l.EndTime = l2.EndTime + } + if l.ThroughputByHost == nil && len(l2.ThroughputByHost) != 0 { + l.ThroughputByHost = l2.ThroughputByHost + } else { + for k, v := range l2.ThroughputByHost { + v0 := l.ThroughputByHost[k] + v0.Merge(v) + l.ThroughputByHost[k] = v0 + } + } + if l.ThroughputByClient == nil && len(l2.ThroughputByClient) != 0 { + l.ThroughputByClient = l2.ThroughputByClient + } else { + for k, v := range l2.ThroughputByClient { + v0 := l.ThroughputByClient[k] + v0.Merge(v) + l.ThroughputByClient[k] = v0 + } + } + if l.Title == "" && l2.Title != "" { + l.Title = l2.Title + } +} + +// Update returns a temporary update without finalizing. +// The update will have no references to the live version. +func (l LiveAggregate) Update() LiveAggregate { + dst := l + dst.Throughput = dst.throughput.asThroughput() + // Clear maps... + dst.ThroughputByHost = nil + dst.ThroughputByClient = nil + dst.throughput = liveThroughput{} + + // TODO: PROBABLY NOT NEEDED AND FASTER TO REMOVE... + dst.Requests = make(map[string]RequestSegments, len(l.requests)) + for clientID, req := range l.requests { + startTime := time.Unix(req.firstSeg, 0) + if req.client != "" { + reqs := make([]RequestSegment, 0, len(req.single)+len(req.multi)) + for i := range req.multi { + reqs = append(reqs, RequestSegment{StartTime: startTime, EndTime: startTime.Add(time.Second * requestSegmentsDur), Multi: &req.multi[i]}) + startTime = startTime.Add(time.Second * requestSegmentsDur) + } + for i := range req.single { + reqs = append(reqs, RequestSegment{StartTime: startTime, EndTime: startTime.Add(time.Second * requestSegmentsDur), Single: &req.single[i]}) + startTime = startTime.Add(time.Second * requestSegmentsDur) + } + dst.Requests[clientID] = reqs + } + } + + dst.requests = nil + dst.Clients = l.Clients.Clone() + dst.Hosts = l.Hosts.Clone() + dst.FirstErrors = slices.Clone(l.FirstErrors) + dst.Title += " (update)" + + return dst +} + +func (l *LiveAggregate) Finalize() { + l.Throughput = l.throughput.asThroughput() + for clientID, reqs := range l.requests { + reqs.cycle() + startTime := time.Unix(reqs.firstSeg, 0) + dst := l.Requests[clientID] + l.Concurrency = len(l.threadIDs) + for i := range reqs.multi { + dst = append(dst, RequestSegment{StartTime: startTime, EndTime: startTime.Add(time.Second * requestSegmentsDur), Multi: &reqs.multi[i]}) + startTime = startTime.Add(time.Second * requestSegmentsDur) + } + for i := range reqs.single { + dst = append(dst, RequestSegment{StartTime: startTime, EndTime: startTime.Add(time.Second * requestSegmentsDur), Single: &reqs.single[i]}) + startTime = startTime.Add(time.Second * requestSegmentsDur) + } + if l.Requests == nil { + l.Requests = make(map[string]RequestSegments, 1) + } + l.Requests[clientID] = dst + } + l.Title += " (Final)" +} + +// ReportOptions provides options to report generation. +type ReportOptions struct { + Details bool + Color bool + SkipReqs bool +} + +func (o ReportOptions) printfColor(dst io.Writer) func(ca color.Attribute, format string, args ...interface{}) { + return func(ca color.Attribute, format string, args ...interface{}) { + if !o.Color { + fmt.Fprintf(dst, format, args...) + return + } + color.New(ca).Fprintf(dst, format, args...) + } +} + +func (o ReportOptions) printf(ca color.Attribute, format string, args ...interface{}) string { + if !o.Color { + return fmt.Sprintf(format, args...) + } + return color.New(ca).Sprintf(format, args...) +} + +func (o ReportOptions) col(ca color.Attribute, s string) string { + if !o.Color { + return s + } + return color.New(ca).Sprint(s) +} + +func (l LiveAggregate) Report(op string, o ReportOptions) string { + dst := bytes.NewBuffer(make([]byte, 0, 1024)) + printfColor := o.printfColor(dst) + col := o.col + details := o.Details + data := l + + if data.Throughput.Segmented == nil || len(data.Throughput.Segmented.Segments) < 2 { + printfColor(color.FgHiYellow, "Skipping %s too few samples. Longer benchmark run required for reliable results.\n\n", op) + if data.TotalErrors > 0 { + printfColor(color.FgHiRed, "Errors: %d\n", data.TotalErrors) + if details { + console.SetColor("Print", color.New(color.FgWhite)) + printfColor(color.FgWhite, "- First Errors:\n") + for _, err := range data.FirstErrors { + printfColor(color.FgWhite, " * %s\n", err) + } + } + dst.WriteByte('\n') + } + return dst.String() + } + + opCol := o.printf(color.FgHiYellow, "%s", op) + if details { + hostsString := "" + if len(data.Hosts) > 1 { + hostsString = fmt.Sprintf(" Hosts: %d.", len(data.Hosts)) + } + if len(data.Clients) > 1 { + hostsString = fmt.Sprintf("%s Warp Instances: %d.", hostsString, len(data.Clients)) + } + sz := "" + if data.TotalBytes > 0 { + sz = fmt.Sprintf("Size: %d bytes. ", data.TotalBytes/int64(data.TotalObjects)) + } + printfColor(color.FgWhite, "Report: %v (%d reqs). Ran %v\n", opCol, data.TotalRequests, data.Throughput.StringDuration()) + printfColor(color.FgWhite, " * Objects per request: %d. %vConcurrency: %d.%s\n", + data.TotalObjects/data.TotalRequests, sz, + data.Concurrency, hostsString) + } else { + printfColor(color.FgHiWhite, "Report: %s. Concurrency: %d. Ran: %v\n", opCol, data.Concurrency, time.Duration(data.Throughput.MeasureDurationMillis)*time.Millisecond) + } + printfColor(color.FgWhite, " * Average: %v\n", col(color.FgWhite, data.Throughput.StringDetails(details))) + if data.TotalErrors > 0 { + printfColor(color.FgHiRed, " * Errors: %d\n", data.TotalErrors) + if details { + console.SetColor("Print", color.New(color.FgWhite)) + printfColor(color.FgWhite, " - First Errors:\n") + for _, err := range data.FirstErrors { + printfColor(color.FgWhite, " * %s\n", err) + } + } + } + + if !o.SkipReqs { + ss, ms := mergeRequests(data.Requests) + if ss.MergedEntries > 0 { + printfColor(color.FgWhite, " * Reqs: %s\n", ss.StringByN()) + if ss.FirstByte != nil { + printfColor(color.FgWhite, " * TTFB: %v\n", ss.FirstByte.StringByN(ss.MergedEntries)) + } + } + if ms.MergedEntries > 0 { + ms.BySize.SortbySize() + for _, s := range ms.BySize { + printfColor(color.FgWhite, "\nRequest size %s -> %s . Requests: %d\n", s.MinSizeString, s.MaxSizeString, s.Requests) + printfColor(color.FgWhite, " * Reqs: %s \n", s.StringByN()) + if s.FirstByte != nil { + printfColor(color.FgWhite, " * TTFB: %s\n", s.FirstByte.StringByN(s.MergedEntries)) + } + } + } + } + dst.WriteByte('\n') + + if len(data.Hosts) > 1 { + printfColor(color.FgHiWhite, "Throughput by host:\n") + for _, ep := range data.Hosts.Slice() { + tp := data.ThroughputByHost[ep] + printfColor(color.FgWhite, " * %s:", ep) + printfColor(color.FgHiWhite, " Avg: %v", tp.StringDetails(details)) + if tp.Errors > 0 { + printfColor(color.FgHiRed, " - Errors: %d", data.TotalErrors) + } + dst.WriteByte('\n') + } + dst.WriteByte('\n') + } + + if len(data.Clients) > 1 { + printfColor(color.FgHiWhite, "Throughput by client:\n") + for i, client := range data.Clients.Slice() { + tp := data.ThroughputByClient[client] + printfColor(color.FgWhite, "Client %d throughput: ", i+1) + printfColor(color.FgHiWhite, "%s\n", tp.StringDetails(o.Details)) + if o.SkipReqs || !o.Details { + continue + } + var ss SingleSizedRequests + var ms MultiSizedRequests + for _, seg := range data.Requests[client] { + if seg.Single != nil { + ss.add(*seg.Single) + } + if seg.Multi != nil { + ms.add(*seg.Multi) + } + } + if ss.MergedEntries > 0 { + printfColor(color.FgWhite, " * Reqs: %s", ss.StringByN()) + if ss.FirstByte != nil { + printfColor(color.FgWhite, " * TTFB: %v\n", ss.FirstByte.StringByN(ss.MergedEntries)) + } else { + dst.WriteByte('\n') + } + } + if ms.MergedEntries > 0 { + ms.BySize.SortbySize() + for _, s := range ms.BySize { + printfColor(color.FgWhite, "\nRequest size %s -> %s . Requests: %d\n", s.MinSizeString, s.MaxSizeString, s.Requests) + printfColor(color.FgWhite, " * Reqs: %s", s.StringByN()) + if s.FirstByte != nil { + printfColor(color.FgWhite, ", TTFB: %s\n", s.FirstByte.StringByN(s.MergedEntries)) + } else { + dst.WriteByte('\n') + } + } + } + dst.WriteByte('\n') + } + } + + if segs := data.Throughput.Segmented; segs != nil { + dur := time.Millisecond * time.Duration(segs.SegmentDurationMillis) + printfColor(color.FgHiWhite, "Throughput, split into %d x 1s:\n", len(segs.Segments)) + printfColor(color.FgWhite, " * Fastest: %v\n", SegmentSmall{BPS: segs.FastestBPS, OPS: segs.FastestOPS, Start: segs.FastestStart}.StringLong(dur, details)) + printfColor(color.FgWhite, " * 50%% Median: %v\n", SegmentSmall{BPS: segs.MedianBPS, OPS: segs.MedianOPS, Start: segs.MedianStart}.StringLong(dur, details)) + printfColor(color.FgWhite, " * Slowest: %v\n", SegmentSmall{BPS: segs.SlowestBPS, OPS: segs.SlowestOPS, Start: segs.SlowestStart}.StringLong(dur, details)) + } + return dst.String() +} + +func (r *Realtime) Report(o ReportOptions) *bytes.Buffer { + dst := bytes.NewBuffer(make([]byte, 0, 1024)) + printfColor := o.printfColor(dst) + + wroteOps := 0 + allOps := stringKeysSorted(r.ByOpType) + for _, op := range allOps { + data := r.ByOpType[op] + if wroteOps > 0 { + printfColor(color.FgHiBlue, "\n──────────────────────────────────\n\n") + } + dst.WriteString(data.Report(op, o)) + wroteOps++ + } + dst.WriteByte('\n') + if len(allOps) > 1 && !r.overLappingOps() { + if len(allOps) > 1 { + if wroteOps > 0 { + printfColor(color.FgHiBlue, "\n──────────────────────────────────\n\n") + } + } + o.SkipReqs = len(allOps) > 1 + dst.WriteString(r.Total.Report("Total", o)) + o.SkipReqs = false + } + + return dst +} + +func finalizeValues[K comparable](m map[K]*LiveAggregate) { + for _, v := range m { + if v != nil { + v.Finalize() + } + } +} + +func mergeValues[K comparable](toP *map[K]*LiveAggregate, from map[K]*LiveAggregate) { + to := *toP + if to == nil { + to = make(map[K]*LiveAggregate, len(from)) + } + for k, v := range from { + if v != nil { + dst := to[k] + if dst == nil { + dst = &LiveAggregate{Title: v.Title} + } + dst.Merge(*v) + to[k] = dst + } + } + *toP = to +} + +func (r *Realtime) Finalize() { + finalizeValues(r.ByOpType) + finalizeValues(r.ByHost) + finalizeValues(r.ByObjLog2Size) + finalizeValues(r.ByClient) + finalizeValues(r.ByCategory) + r.Total.Finalize() + r.Final = true +} + +func (r *Realtime) Merge(other *Realtime) { + if other == nil { + return + } + mergeValues(&r.ByOpType, other.ByOpType) + mergeValues(&r.ByHost, other.ByHost) + mergeValues(&r.ByObjLog2Size, other.ByObjLog2Size) + mergeValues(&r.ByClient, other.ByClient) + mergeValues(&r.ByCategory, other.ByCategory) + r.Total.Merge(other.Total) + r.Final = other.Final && (r.Final || r.Total.TotalRequests == 0) + setIfEmpty := func(dst *string, alt string) { + if dst == nil { + return + } + if *dst == "" { + *dst = alt + } + } + setIfEmpty(&r.WarpCommit, other.WarpCommit) + setIfEmpty(&r.WarpDate, other.WarpDate) + setIfEmpty(&r.WarpVersion, other.WarpVersion) + setIfEmpty(&r.Commandline, other.Commandline) + if r.DataVersion == 0 { + r.DataVersion = other.DataVersion + } +} + +func newRealTime() Realtime { + return Realtime{ + DataVersion: currentVersion, + Total: LiveAggregate{Title: "Total"}, + ByOpType: make(map[string]*LiveAggregate), + ByHost: make(map[string]*LiveAggregate), + ByObjLog2Size: make(map[int]*LiveAggregate), + ByClient: make(map[string]*LiveAggregate), + ByCategory: make(map[bench.Category]*LiveAggregate), + } +} + +// Live collects operations and update requests. +func Live(ops <-chan bench.Operation, updates chan UpdateReq, clientID string) *Realtime { + a := newRealTime() + var reset atomic.Bool + var update atomic.Pointer[Realtime] + if updates != nil { + done := make(chan struct{}) + defer close(done) + go func() { + var finalQ []UpdateReq + defer func() { + if len(finalQ) > 0 { + go func() { + for _, r := range finalQ { + updates <- r + } + }() + } + }() + t := time.NewTicker(time.Second / 4) + defer t.Stop() + for { + select { + case r := <-updates: + if r.Reset { + reset.Store(true) + update.Store(nil) + continue + } + if r.Final { + finalQ = append(finalQ, r) + continue + } + select { + case r.C <- update.Load(): + default: + fmt.Println("could not send") + } + case <-done: + return + case <-t.C: // reload value + } + } + }() + } + lastUpdate := time.Now() + for op := range ops { + if reset.CompareAndSwap(true, false) { + a = newRealTime() + } + if clientID != "" { + op.ClientID = clientID + } + var wg sync.WaitGroup + wg.Add(6) + // 1 + go func() { + defer wg.Done() + byOp := a.ByOpType[op.OpType] + if byOp == nil { + byOp = &LiveAggregate{Title: "Operation: " + op.OpType} + a.ByOpType[op.OpType] = byOp + } + byOp.Add(op) + }() + // 2 + go func() { + defer wg.Done() + byHost := a.ByHost[op.Endpoint] + if byHost == nil { + byHost = &LiveAggregate{Title: "Host: " + op.Endpoint} + a.ByHost[op.Endpoint] = byHost + } + byHost.Add(op) + }() + // 3 + go func() { + defer wg.Done() + if op.ClientID != "" { + byClient := a.ByClient[op.ClientID] + if byClient == nil { + byClient = &LiveAggregate{Title: "Client: " + op.ClientID} + a.ByClient[op.ClientID] = byClient + } + byClient.Add(op) + } + }() + // 4 + go func() { + defer wg.Done() + if op.Size == 0 { + return + } + l2Size := bits.Len64(uint64(op.Size)) + bySize := a.ByObjLog2Size[l2Size] + if bySize == nil { + start := 0 + if l2Size > 0 { + start = 1 >> (l2Size - 1) + } + bySize = &LiveAggregate{Title: fmt.Sprintf("Size: %d->%d", start, (1< time.Second { + u := Realtime{Total: a.Total.Update(), ByOpType: make(map[string]*LiveAggregate, len(a.ByOpType))} + for k, v := range a.ByOpType { + if v != nil { + clone := v.Update() + u.ByOpType[k] = &clone + } + } + update.Store(&u) + lastUpdate = time.Now() + } + } + a.Finalize() + return &a +} + +type liveThroughput struct { + // Unit time (seconds) of first segment + segmentsStart int64 + + segments []liveSegments +} + +type liveSegments struct { + // Spread across segments from start to end + ops float64 + objs float64 + bytes float64 + opsStarted int + opsEnded int + fullOps int + partialOps int + + // For requests that started in this segment. + errors int + reqDur time.Duration + ttfb time.Duration +} + +func (l *liveThroughput) Add(o bench.Operation) { + startUnixNano := o.Start.UnixNano() + startUnix := startUnixNano / int64(time.Second) + if len(l.segments) == 0 { + l.segments = make([]liveSegments, 0, 100) + l.segmentsStart = startUnix + } + if startUnixNano < l.segmentsStart { + // Drop... + return + } + endUnixNano := o.End.UnixNano() + endUnix := endUnixNano / int64(time.Second) + durNanos := endUnixNano - startUnixNano + + startSeg := int(startUnix - l.segmentsStart) + endSeg := int(endUnix - l.segmentsStart + 1) + if endSeg > len(l.segments) { + // Append empty segments + l.segments = append(l.segments, make([]liveSegments, endSeg-len(l.segments))...) + } + segs := l.segments[startSeg:endSeg] + for i := range segs { + seg := &l.segments[startSeg+i] + if i == 0 { + seg.opsStarted++ + seg.reqDur += o.Duration() + if len(o.Err) > 0 { + seg.errors++ + } + seg.ttfb += o.TTFB() + } + if i == len(segs)-1 { + seg.opsEnded++ + } + // Happy path - doesn't cross segments + if len(segs) == 1 { + seg.fullOps++ + seg.ops++ + seg.objs += float64(o.ObjPerOp) + seg.bytes += float64(o.Size) + continue + } + + seg.partialOps++ + segStartNano := (startUnix + int64(i)) * int64(time.Second) + segEndNano := (startUnix + int64(i) + 1) * int64(time.Second) + nanosInSeg := int64(time.Second) + if startUnixNano >= segStartNano { + nanosInSeg = segEndNano - startUnixNano + } + if endUnixNano <= segEndNano { + nanosInSeg = endUnixNano - segStartNano + } + if nanosInSeg > 0 { + fraction := float64(nanosInSeg) / float64(durNanos) + seg.objs += float64(o.ObjPerOp) * fraction + seg.bytes += float64(o.Size) * fraction + seg.ops += fraction + } + } +} + +func (l liveThroughput) asThroughput() Throughput { + var t Throughput + t.StartTime = time.Unix(l.segmentsStart, 0) + t.EndTime = time.Unix(l.segmentsStart+int64(len(l.segments)), 0) + segs := l.segments + // Remove first and last... + const removeN = 2 + if len(segs) <= removeN*2 { + return t + } + segs = segs[removeN : len(segs)-removeN] + t.StartTime = time.Unix(l.segmentsStart, 0).Add(removeN * time.Second) + t.EndTime = t.StartTime.Add(time.Duration(len(l.segments)) * time.Second) + + var ts ThroughputSegmented + segments := make(bench.Segments, 0, len(segs)) + for i, seg := range segs { + t.Errors += seg.errors + t.Bytes += seg.bytes + t.Objects += seg.objs + t.Operations += seg.opsStarted + segments = append(segments, bench.Segment{ + Start: time.Unix(l.segmentsStart+int64(i), 0), + EndsBefore: time.Unix(l.segmentsStart+int64(i+1), -1), + OpType: "", + Host: "", + OpsStarted: seg.opsStarted, + PartialOps: seg.partialOps, + FullOps: seg.fullOps, + OpsEnded: seg.opsEnded, + Objects: seg.ops, + Errors: seg.errors, + ReqAvg: float64(seg.reqDur) / float64(seg.opsStarted), // TODO: CHECK 0 + TotalBytes: int64(seg.bytes), + ObjsPerOp: int(seg.objs / seg.ops), + }) + } + + if len(segs) > 0 { + ts.fill(segments, int64(t.BytesPS())) + ts.SegmentDurationMillis = 1000 + t.MeasureDurationMillis = len(segs) * 1000 + } + t.Segmented = &ts + + return t +} + +// requestSegmentsDur is the request segment duration. +const requestSegmentsDur = 10 + +type liveRequests struct { + // Unit time (seconds) of first segment + currStart int64 + firstSeg int64 + ops bench.Operations + isMulti *bool + client string + + single []SingleSizedRequests + multi []MultiSizedRequests +} + +func (l *liveRequests) Add(o bench.Operation) { + logTime := o.End.Unix() + if l.firstSeg == 0 { + l.firstSeg = logTime + l.currStart = logTime + } + if logTime < l.currStart { + // If slightly out of order, add to current + logTime = l.currStart + } + if logTime-l.currStart < requestSegmentsDur { + l.ops = append(l.ops, o) + return + } + if l.client == "" { + l.client = o.ClientID + } + l.cycle() + l.currStart += requestSegmentsDur + l.ops = l.ops[:0] + for l.currStart+requestSegmentsDur < logTime { + if *l.isMulti { + l.multi = append(l.multi, MultiSizedRequests{Skipped: true}) + } else { + l.single = append(l.single, SingleSizedRequests{Skipped: true}) + } + l.currStart += requestSegmentsDur + } + l.ops = append(l.ops, o) +} + +func (l *liveRequests) cycle() { + if len(l.ops) == 0 { + return + } + if l.isMulti == nil { + im := l.ops.MultipleSizes() + l.isMulti = &im + } + if *l.isMulti { + var tmp MultiSizedRequests + tmp.fill(l.ops, false) + // Remove some bonus fields... + for k := range tmp.BySize { + tmp.BySize[k].BpsPct = nil + if tmp.BySize[k].FirstByte != nil { + tmp.BySize[k].FirstByte.PercentilesMillis = nil + } + } + tmp.ByHost = nil + l.multi = append(l.multi, tmp) + } else { + var tmp SingleSizedRequests + tmp.fill(l.ops) + // Remove some fields that are excessive... + tmp.ByHost = nil + tmp.DurPct = nil + if tmp.FirstByte != nil { + tmp.FirstByte.PercentilesMillis = nil + } + l.single = append(l.single, tmp) + } + l.ops = l.ops[:0] +} + +// stringKeysSorted returns the keys as a sorted string slice. +func stringKeysSorted[K string, V any](m map[K]V) []string { + keys := make([]string, 0, len(m)) + for k := range m { + keys = append(keys, string(k)) + } + sort.Strings(keys) + return keys +} + +func (r Realtime) overLappingOps() bool { + byOp := r.ByOpType + if len(byOp) <= 1 { + return false + } + for opA, dA := range byOp { + for opB, dB := range byOp { + if opA == opB { + continue + } + if dA.StartTime.After(dB.EndTime) || dA.EndTime.Before(dB.StartTime) { + return true + } + } + } + return false +} + +func mergeRequests(data map[string]RequestSegments) (ss SingleSizedRequests, ms MultiSizedRequests) { + for _, reqs := range data { + for _, seg := range reqs { + if seg.Single != nil { + ss.add(*seg.Single) + } + if seg.Multi != nil { + ms.add(*seg.Multi) + } + } + } + return ss, ms +} diff --git a/pkg/aggregate/mapasslice.go b/pkg/aggregate/mapasslice.go new file mode 100644 index 00000000..816a5872 --- /dev/null +++ b/pkg/aggregate/mapasslice.go @@ -0,0 +1,129 @@ +/* + * Warp (C) 2019-2025 MinIO, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +package aggregate + +import ( + "bytes" + "encoding/json" + "sort" +) + +// MapAsSlice is a key-only map that is serialized as an array. +type MapAsSlice map[string]struct{} + +// Add value +func (m *MapAsSlice) Add(k string) { + if *m == nil { + *m = make(MapAsSlice) + } + mp := *m + mp[k] = struct{}{} +} + +// AddMap adds another map. +func (m *MapAsSlice) AddMap(other MapAsSlice) { + if *m == nil { + *m = make(MapAsSlice, len(other)) + } + mp := *m + for k := range other { + mp[k] = struct{}{} + } +} + +// AddSlice adds a slice. +func (m *MapAsSlice) AddSlice(other []string) { + if *m == nil { + *m = make(MapAsSlice, len(other)) + } + mp := *m + for _, k := range other { + mp[k] = struct{}{} + } +} + +// SetSlice replaces the value with the content of a slice. +func (m *MapAsSlice) SetSlice(v []string) { + *m = make(MapAsSlice, len(v)) + mp := *m + for _, k := range v { + mp[k] = struct{}{} + } +} + +// Clone returns a clone. +func (m *MapAsSlice) Clone() MapAsSlice { + if m == nil { + return MapAsSlice(nil) + } + mm := make(MapAsSlice, len(*m)) + for k, v := range *m { + mm[k] = v + } + return mm +} + +// MarshalJSON provides output as JSON. +func (m MapAsSlice) MarshalJSON() ([]byte, error) { + if m == nil { + return []byte("null"), nil + } + var dst bytes.Buffer + dst.WriteByte('[') + x := m.Slice() + for i, k := range x { + dst.WriteByte('"') + json.HTMLEscape(&dst, []byte(k)) + dst.WriteByte('"') + if i < len(x)-1 { + dst.WriteByte(',') + } + } + dst.WriteByte(']') + + return dst.Bytes(), nil +} + +// UnmarshalJSON reads an array of strings and sets them as keys in the map. +func (m *MapAsSlice) UnmarshalJSON(b []byte) error { + var tmp []string + err := json.Unmarshal(b, &tmp) + if err != nil { + return err + } + if tmp == nil { + *m = nil + return nil + } + dst := make(MapAsSlice, len(tmp)) + for _, v := range tmp { + dst[v] = struct{}{} + } + *m = dst + return nil +} + +// Slice returns the keys as a sorted slice. +func (m MapAsSlice) Slice() []string { + x := make([]string, 0, len(m)) + for k := range m { + x = append(x, k) + } + sort.Strings(x) + return x +} diff --git a/pkg/aggregate/requests.go b/pkg/aggregate/requests.go index 45549dc5..f383af39 100644 --- a/pkg/aggregate/requests.go +++ b/pkg/aggregate/requests.go @@ -18,6 +18,8 @@ package aggregate import ( + "fmt" + "sort" "sync" "time" @@ -27,42 +29,45 @@ import ( // SingleSizedRequests contains statistics when all objects have the same size. type SingleSizedRequests struct { // Request times by host. - ByHost map[string]SingleSizedRequests `json:"by_host,omitempty"` - LastAccess *SingleSizedRequests `json:"last_access,omitempty"` + ByHost map[string]SingleSizedRequests `json:"by_host,omitempty"` // FirstAccess is filled if the same object is accessed multiple times. - // This records the first touch of the object. + // This records the first access of the object. FirstAccess *SingleSizedRequests `json:"first_access,omitempty"` + // FirstAccess is filled if the same object is accessed multiple times. + // This records the last access of the object. + LastAccess *SingleSizedRequests `json:"last_access,omitempty"` + // Time to first byte if applicable. FirstByte *TTFB `json:"first_byte,omitempty"` // Host names, sorted. - HostNames []string + HostNames MapAsSlice `json:"host_names,omitempty"` - // DurPct is duration percentiles. - DurPct [101]int `json:"dur_percentiles_millis"` + // DurPct is duration percentiles (milliseconds). + DurPct *[101]float64 `json:"dur_percentiles_millis,omitempty"` // Median request duration. - DurMedianMillis int `json:"dur_median_millis"` + DurMedianMillis float64 `json:"dur_median_millis"` // Fastest request time. - FastestMillis int `json:"fastest_millis"` + FastestMillis float64 `json:"fastest_millis"` // Slowest request time. - SlowestMillis int `json:"slowest_millis"` + SlowestMillis float64 `json:"slowest_millis"` // StdDev is the standard deviation of requests. - StdDev int `json:"std_dev_millis"` + StdDev float64 `json:"std_dev_millis"` // 99% request time. - Dur99Millis int `json:"dur_99_millis"` + Dur99Millis float64 `json:"dur_99_millis"` // 90% request time. - Dur90Millis int `json:"dur_90_millis"` + Dur90Millis float64 `json:"dur_90_millis"` // Average request duration. - DurAvgMillis int `json:"dur_avg_millis"` + DurAvgMillis float64 `json:"dur_avg_millis"` // Total number of requests. Requests int `json:"requests"` @@ -71,7 +76,91 @@ type SingleSizedRequests struct { ObjSize int64 `json:"obj_size"` // Skipped if too little data. - Skipped bool `json:"skipped"` + Skipped bool `json:"skipped,omitempty"` + + // MergedEntries is a counter for the number of merged entries contained in this result. + MergedEntries int `json:"merged_entries"` +} + +func (a SingleSizedRequests) StringByN() string { + n := float64(a.MergedEntries) + if a.Requests == 0 || n == 0 { + return "" + } + invN := 1 / n + reqs := a + + fmtMillis := func(v float64) string { + if v*invN > float64(100*time.Millisecond) { + dur := time.Duration(v * float64(time.Millisecond) * invN).Round(time.Millisecond) + return dur.String() + } + return fmt.Sprintf("%.1fms", v*invN) + } + return fmt.Sprint( + "Avg: ", fmtMillis(reqs.DurAvgMillis), + ", 50%: ", fmtMillis(reqs.DurMedianMillis), + ", 90%: ", fmtMillis(reqs.Dur90Millis), + ", 99%: ", fmtMillis(reqs.Dur99Millis), + ", Fastest: ", fmtMillis(reqs.FastestMillis), + ", Slowest: ", fmtMillis(reqs.SlowestMillis), + ", StdDev: ", fmtMillis(reqs.StdDev), + ) +} + +func (a *SingleSizedRequests) add(b SingleSizedRequests) { + if b.Skipped { + return + } + a.Requests += b.Requests + a.ObjSize += b.ObjSize + a.DurAvgMillis += b.DurAvgMillis + a.DurMedianMillis += b.DurMedianMillis + if a.MergedEntries == 0 { + a.FastestMillis = b.FastestMillis + } else { + a.FastestMillis = min(a.FastestMillis, b.FastestMillis) + } + a.SlowestMillis = max(a.SlowestMillis, b.SlowestMillis) + a.Dur99Millis += b.Dur99Millis + a.Dur90Millis += b.Dur90Millis + a.StdDev += b.StdDev + a.MergedEntries += b.MergedEntries + a.HostNames.AddMap(b.HostNames) + if a.ByHost == nil && len(b.ByHost) > 0 { + a.ByHost = make(map[string]SingleSizedRequests, len(b.ByHost)) + } + for k, v := range b.ByHost { + x := a.ByHost[k] + x.add(v) + a.ByHost[k] = x + } + if a.DurPct == nil && b.DurPct == nil { + a.DurPct = &[101]float64{} + } + if b.DurPct != nil { + for i := range b.DurPct { + a.DurPct[i] += b.DurPct[i] + } + } + if b.FirstAccess != nil { + if a.FirstAccess == nil { + a.FirstAccess = &SingleSizedRequests{} + } + a.FirstAccess.add(*b.FirstAccess) + } + if b.LastAccess != nil { + if a.LastAccess == nil { + a.LastAccess = &SingleSizedRequests{} + } + a.LastAccess.add(*b.LastAccess) + } + if b.FirstByte != nil { + if a.FirstByte == nil { + a.FirstByte = &TTFB{} + } + a.FirstByte.add(*b.FirstByte) + } } func (a *SingleSizedRequests) fill(ops bench.Operations) { @@ -79,16 +168,18 @@ func (a *SingleSizedRequests) fill(ops bench.Operations) { ops.SortByDuration() a.Requests = len(ops) a.ObjSize = ops.FirstObjSize() - a.DurAvgMillis = durToMillis(ops.AvgDuration()) - a.StdDev = durToMillis(ops.StdDev()) - a.DurMedianMillis = durToMillis(ops.Median(0.5).Duration()) - a.Dur90Millis = durToMillis(ops.Median(0.9).Duration()) - a.Dur99Millis = durToMillis(ops.Median(0.99).Duration()) - a.SlowestMillis = durToMillis(ops.Median(1).Duration()) - a.FastestMillis = durToMillis(ops.Median(0).Duration()) + a.DurAvgMillis = durToMillisF(ops.AvgDuration()) + a.StdDev = durToMillisF(ops.StdDev()) + a.DurMedianMillis = durToMillisF(ops.Median(0.5).Duration()) + a.Dur90Millis = durToMillisF(ops.Median(0.9).Duration()) + a.Dur99Millis = durToMillisF(ops.Median(0.99).Duration()) + a.SlowestMillis = durToMillisF(ops.Median(1).Duration()) + a.FastestMillis = durToMillisF(ops.Median(0).Duration()) a.FirstByte = TtfbFromBench(ops.TTFB(start, end)) + a.DurPct = &[101]float64{} + a.MergedEntries = 1 for i := range a.DurPct[:] { - a.DurPct[i] = durToMillis(ops.Median(float64(i) / 100).Duration()) + a.DurPct[i] = durToMillisF(ops.Median(float64(i) / 100).Duration()) } } @@ -117,7 +208,7 @@ type RequestSizeRange struct { MaxSizeString string `json:"max_size_string"` // BpsPct is BPS percentiles. - BpsPct [101]float64 `json:"bps_percentiles"` + BpsPct *[101]float64 `json:"bps_percentiles,omitempty"` BpsMedian float64 `json:"bps_median"` AvgDurationMillis int `json:"avg_duration_millis"` @@ -137,69 +228,182 @@ type RequestSizeRange struct { MaxSize int `json:"max_size"` // Minimum size in request size range. MinSize int `json:"min_size"` + + // MergedEntries is a counter for the number of merged entries contained in this result. + MergedEntries int `json:"merged_entries"` +} + +func (s RequestSizeRange) String() string { + if s.MergedEntries <= 0 || s.Requests == 0 { + return "" + } + return fmt.Sprint("Average: ", bench.Throughput(s.BpsAverage), + ", 50%: ", bench.Throughput(s.BpsMedian), + ", 90%: ", bench.Throughput(s.Bps90), + ", 99%: ", bench.Throughput(s.Bps99), + ", Fastest: ", bench.Throughput(s.BpsFastest), + ", Slowest: ", bench.Throughput(s.BpsSlowest), + ) +} + +func (s RequestSizeRange) StringByN() string { + if s.MergedEntries <= 0 || s.Requests == 0 { + return "" + } + mul := 1 / float64(s.MergedEntries) + return fmt.Sprint("Avg: ", bench.Throughput(s.BpsAverage*mul), + ", 50%: ", bench.Throughput(s.BpsMedian*mul), + ", 90%: ", bench.Throughput(s.Bps90*mul), + ", 99%: ", bench.Throughput(s.Bps99*mul), + ", Fastest: ", bench.Throughput(s.BpsFastest*mul), + ", Slowest: ", bench.Throughput(s.BpsSlowest*mul), + ) } -func (r *RequestSizeRange) fill(s bench.SizeSegment) { - r.Requests = len(s.Ops) - r.MinSize = int(s.Smallest) - r.MaxSize = int(s.Biggest) - r.MinSizeString, r.MaxSizeString = s.SizesString() - r.AvgObjSize = int(s.Ops.AvgSize()) - r.AvgDurationMillis = durToMillis(s.Ops.AvgDuration()) - s.Ops.SortByThroughput() - r.BpsAverage = s.Ops.OpThroughput().Float() - r.BpsMedian = s.Ops.Median(0.5).BytesPerSec().Float() - r.Bps90 = s.Ops.Median(0.9).BytesPerSec().Float() - r.Bps99 = s.Ops.Median(0.99).BytesPerSec().Float() - r.BpsFastest = s.Ops.Median(0.0).BytesPerSec().Float() - r.BpsSlowest = s.Ops.Median(1).BytesPerSec().Float() - for i := range r.BpsPct[:] { - r.BpsPct[i] = s.Ops.Median(float64(i) / 100).BytesPerSec().Float() +func (s *RequestSizeRange) fill(ss bench.SizeSegment) { + ops := ss.Ops.SortByThroughputNonZero() + if len(ops) == 0 { + return + } + s.Requests = len(ops) + s.MinSize = int(ss.Smallest) + s.MaxSize = int(ss.Biggest) + s.MinSizeString, s.MaxSizeString = ss.SizesString() + s.AvgObjSize = int(ops.AvgSize()) + s.AvgDurationMillis = durToMillis(ops.AvgDuration()) + s.BpsAverage = ops.OpThroughput().Float() + s.BpsMedian = ops.Median(0.5).BytesPerSec().Float() + s.Bps90 = ops.Median(0.9).BytesPerSec().Float() + s.Bps99 = ops.Median(0.99).BytesPerSec().Float() + s.BpsFastest = ops.Median(0.0).BytesPerSec().Float() + s.BpsSlowest = ops.Median(1).BytesPerSec().Float() + s.BpsPct = &[101]float64{} + for i := range s.BpsPct[:] { + s.BpsPct[i] = ops.Median(float64(i) / 100).BytesPerSec().Float() } + s.MergedEntries = 1 } -func (r *RequestSizeRange) fillFirst(s bench.SizeSegment) { - if !s.Ops.IsMultiTouch() { +func (s *RequestSizeRange) add(b RequestSizeRange) { + s.Requests += b.Requests + if b.FirstByte != nil { + if s.FirstByte == nil { + s.FirstByte = &TTFB{} + } + s.FirstByte.add(*b.FirstByte) + } + // Min/Max should be set + s.AvgObjSize += b.AvgObjSize + s.AvgDurationMillis += b.AvgDurationMillis + s.BpsAverage += b.BpsAverage + s.BpsMedian += b.BpsMedian + s.BpsFastest += b.BpsFastest + s.BpsSlowest += b.BpsSlowest + if b.BpsPct != nil { + if s.BpsPct == nil { + s.BpsPct = &[101]float64{} + } + for i, v := range b.BpsPct[:] { + s.BpsPct[i] = v + } + } + s.MergedEntries += b.MergedEntries +} + +func (s *RequestSizeRange) fillFirstAccess(ss bench.SizeSegment) { + if !ss.Ops.IsMultiTouch() { return } - s.Ops = s.Ops.FilterFirst() + ss.Ops = ss.Ops.FilterFirst() a := RequestSizeRange{} - a.fill(s) - a.FirstByte = TtfbFromBench(s.Ops.TTFB(s.Ops.TimeRange())) + a.fill(ss) + s.FirstAccess = &a +} + +// RequestSizeRanges is an array of RequestSizeRange +type RequestSizeRanges []RequestSizeRange + +// SortbySize will sort the ranges by size. +func (r RequestSizeRanges) SortbySize() { + sort.Slice(r, func(i, j int) bool { + return r[i].MinSize < r[j].MinSize + }) +} - r.FirstAccess = &a +// FindMatching will find a matching range, or create a new range. +// New entries will not me added to r. +func (r RequestSizeRanges) FindMatching(want RequestSizeRange) (v *RequestSizeRange, found bool) { + for i := range r { + if want.MinSize >= r[i].MinSize && want.MaxSize <= r[i].MaxSize { + return &r[i], true + } + } + return &RequestSizeRange{ + MinSizeString: want.MinSizeString, + MaxSizeString: want.MaxSizeString, + MaxSize: want.MaxSize, + MinSize: want.MinSize, + }, false } // MultiSizedRequests contains statistics when objects have the same different size. type MultiSizedRequests struct { // ByHost contains request information by host. + // This data is not segmented. ByHost map[string]RequestSizeRange `json:"by_host,omitempty"` // BySize contains request times separated by sizes - BySize []RequestSizeRange `json:"by_size"` - - // HostNames are the host names, sorted. - HostNames []string + BySize RequestSizeRanges `json:"by_size"` // Total number of requests. Requests int `json:"requests"` + // Average object size AvgObjSize int64 `json:"avg_obj_size"` // Skipped if too little data. - Skipped bool `json:"skipped"` + Skipped bool `json:"skipped,omitempty"` + + // MergedEntries is a counter for the number of merged entries contained in this result. + MergedEntries int `json:"merged_entries"` +} + +func (a *MultiSizedRequests) add(b MultiSizedRequests) { + if b.Skipped { + return + } + if a.ByHost == nil { + a.ByHost = make(map[string]RequestSizeRange) + } + for ep, v := range b.ByHost { + av := a.ByHost[ep] + av.add(v) + a.ByHost[ep] = av + } + a.Requests += b.Requests + a.AvgObjSize += b.AvgObjSize + for _, toMerge := range b.BySize { + dst, found := a.BySize.FindMatching(toMerge) + dst.add(toMerge) + if !found { + a.BySize = append(a.BySize, toMerge) + a.BySize.SortbySize() + } + } + a.MergedEntries += b.MergedEntries } -func (a *MultiSizedRequests) fill(ops bench.Operations) { +func (a *MultiSizedRequests) fill(ops bench.Operations, fillFirstAccess bool) { start, end := ops.TimeRange() a.Requests = len(ops) - if len(ops) == 0 { + if len(ops) == 0 || end.Sub(start) < 100*time.Millisecond { a.Skipped = true return } a.AvgObjSize = ops.AvgSize() sizes := ops.SplitSizes(0.05) a.BySize = make([]RequestSizeRange, len(sizes)) + a.MergedEntries = 1 var wg sync.WaitGroup wg.Add(len(sizes)) for i := range sizes { @@ -208,7 +412,9 @@ func (a *MultiSizedRequests) fill(ops bench.Operations) { s := sizes[i] var r RequestSizeRange r.fill(s) - r.fillFirst(s) + if fillFirstAccess { + r.fillFirstAccess(s) + } r.FirstByte = TtfbFromBench(s.Ops.TTFB(start, end)) // Store a.BySize[i] = r @@ -229,12 +435,13 @@ func RequestAnalysisSingleSized(o bench.Operations, allThreads bool) *SingleSize res.Skipped = true return &res } + res.MergedEntries = 1 res.fill(active) res.fillFirstLast(o) - res.HostNames = o.Endpoints() + res.HostNames.SetSlice(o.Endpoints()) res.ByHost = RequestAnalysisHostsSingleSized(o) if len(res.HostNames) != len(res.ByHost) { - res.HostNames = o.ClientIDs(clientAsHostPrefix) + res.HostNames.SetSlice(o.ClientIDs(clientAsHostPrefix)) } return &res } @@ -284,12 +491,9 @@ func RequestAnalysisMultiSized(o bench.Operations, allThreads bool) *MultiSizedR res.Skipped = true return &res } - res.fill(active) + res.fill(active, true) res.ByHost = RequestAnalysisHostsMultiSized(active) - res.HostNames = active.Endpoints() - if len(res.HostNames) != len(res.ByHost) && len(res.ByHost) > 0 { - res.HostNames = o.ClientIDs(clientAsHostPrefix) - } + res.MergedEntries = 1 return &res } @@ -334,3 +538,7 @@ func RequestAnalysisHostsMultiSized(o bench.Operations) map[string]RequestSizeRa func durToMillis(d time.Duration) int { return int(d.Round(time.Millisecond) / time.Millisecond) } + +func durToMillisF(d time.Duration) float64 { + return float64(d) / float64(time.Millisecond) +} diff --git a/pkg/aggregate/throughput.go b/pkg/aggregate/throughput.go index a9a91119..89001ae8 100644 --- a/pkg/aggregate/throughput.go +++ b/pkg/aggregate/throughput.go @@ -20,6 +20,7 @@ package aggregate import ( "fmt" "math" + "sort" "time" "github.com/minio/warp/pkg/bench" @@ -37,12 +38,63 @@ type Throughput struct { Errors int `json:"errors"` // Time period of the throughput measurement. MeasureDurationMillis int `json:"measure_duration_millis"` - // Average bytes per second. Can be 0. - AverageBPS float64 `json:"average_bps"` - // Average operations per second. - AverageOPS float64 `json:"average_ops"` + // Total bytes. + Bytes float64 `json:"bytes"` + // Total objects + Objects float64 `json:"objects"` // Number of full operations - Operations int `json:"operations"` + Operations int `json:"ops"` +} + +func (t Throughput) Add(o bench.Operation) Throughput { + if t.StartTime.IsZero() || t.StartTime.After(o.Start) { + t.StartTime = o.Start + } + if t.EndTime.IsZero() || t.EndTime.Before(o.End) { + t.EndTime = o.End + } + t.MeasureDurationMillis = int(t.EndTime.Sub(t.StartTime).Milliseconds()) + t.Operations++ + if o.Err != "" { + t.Errors++ + } + t.Bytes += float64(o.Size) + t.Objects += float64(o.ObjPerOp) + return t +} + +// BytesPS returns the bytes per second throughput for the time segment. +func (t Throughput) BytesPS() bench.Throughput { + return bench.Throughput(1000 * t.Bytes / float64(t.MeasureDurationMillis)) +} + +// ObjectsPS returns the objects per second for the segment. +func (t Throughput) ObjectsPS() float64 { + return 1000 * float64(t.Objects) / float64(t.MeasureDurationMillis) +} + +// Merge currently running measurements. +func (t *Throughput) Merge(other Throughput) { + if other.Operations == 0 { + return + } + t.Errors += other.Errors + t.Bytes += other.Bytes + t.Objects += other.Objects + t.Operations += other.Operations + if t.StartTime.IsZero() || other.StartTime.Before(t.StartTime) { + t.StartTime = other.StartTime + } + if other.EndTime.After(t.EndTime) { + t.EndTime = other.EndTime + } + t.MeasureDurationMillis = int(t.EndTime.Sub(t.StartTime).Milliseconds()) + if t.Segmented == nil && other.Segmented != nil { + t.Segmented = &ThroughputSegmented{} + } + if other.Segmented != nil { + t.Segmented.Merge(*other.Segmented) + } } // String returns a string representation of the segment @@ -56,28 +108,34 @@ func (t Throughput) StringDuration() string { } // StringDetails returns a detailed string representation of the segment -func (t Throughput) StringDetails(_ bool) string { +func (t Throughput) StringDetails(details bool) string { + if t.Bytes == 0 && t.Objects == 0 { + return "" + } speed := "" - if t.AverageBPS > 0 { - speed = fmt.Sprintf("%.02f MiB/s, ", t.AverageBPS/(1<<20)) + if t.Bytes > 0 { + speed = fmt.Sprintf("%.02f MiB/s, ", t.BytesPS()/(1<<20)) } errs := "" if t.Errors > 0 { errs = fmt.Sprintf(", %d errors", t.Errors) } - return fmt.Sprintf("%s%.02f obj/s%s", - speed, t.AverageOPS, errs) + dur := "" + if details { + dur = fmt.Sprintf(" (%vs)", (t.MeasureDurationMillis+500)/1000) + } + return fmt.Sprintf("%s%.02f obj/s%s%s", + speed, t.ObjectsPS(), errs, dur) } func (t *Throughput) fill(total bench.Segment) { - mib, _, objs := total.SpeedPerSec() *t = Throughput{ Operations: total.FullOps, MeasureDurationMillis: durToMillis(total.EndsBefore.Sub(total.Start)), StartTime: total.Start, EndTime: total.EndsBefore, - AverageBPS: math.Round(mib*(1<<20)*10) / 10, - AverageOPS: math.Round(objs*100) / 100, + Bytes: float64(total.TotalBytes), + Objects: total.Objects, Errors: total.Errors, } } @@ -95,13 +153,14 @@ type ThroughputSegmented struct { SortedBy string `json:"sorted_by"` // All segments, sorted - Segments []SegmentSmall `json:"segments"` + Segments SegmentsSmall `json:"segments"` // Time of each segment. SegmentDurationMillis int `json:"segment_duration_millis"` - // Fastest segment bytes per second. Can be 0. In that case segments are sorted by operations per second. + + // Fastest segment bytes per second. Can be 0. In that case segments are sorted by objects per second. FastestBPS float64 `json:"fastest_bps"` - // Fastest segment in terms of operations per second. + // Fastest segment in terms of objects per second. FastestOPS float64 `json:"fastest_ops"` MedianBPS float64 `json:"median_bps"` MedianOPS float64 `json:"median_ops"` @@ -109,6 +168,106 @@ type ThroughputSegmented struct { SlowestOPS float64 `json:"slowest_ops"` } +type SegmentsSmall []SegmentSmall + +// SortByThroughput sorts the segments by throughput. +// Slowest first. +func (s SegmentsSmall) SortByThroughput() { + sort.Slice(s, func(i, j int) bool { + return s[i].BPS < s[j].BPS + }) +} + +// SortByObjsPerSec sorts the segments by the number of objects processed in the segment. +// Lowest first. +func (s SegmentsSmall) SortByObjsPerSec() { + sort.Slice(s, func(i, j int) bool { + return s[i].OPS < s[j].OPS + }) +} + +// SortByStartTime sorts the segments by the start time. +// Earliest first. +func (s SegmentsSmall) SortByStartTime() { + sort.Slice(s, func(i, j int) bool { + return s[i].Start.Before(s[j].Start) + }) +} + +// Median returns the m part median. +// m is clamped to the range 0 -> 1. +func (s SegmentsSmall) Median(m float64) SegmentSmall { + if len(s) == 0 { + return SegmentSmall{} + } + m = math.Round(float64(len(s)) * m) + m = math.Max(m, 0) + m = math.Min(m, float64(len(s)-1)) + return s[int(m)] +} + +// Merge 'other' into 't'. +// Will mutate (re-sort) both segments. +// Segments must have same time alignment. +func (s *SegmentsSmall) Merge(other SegmentsSmall) { + if len(other) == 0 { + return + } + a := *s + if len(a) == 0 { + a = append(a, other...) + *s = a + return + } + merged := make(SegmentsSmall, 0, len(a)) + a.SortByStartTime() + other.SortByStartTime() + // Add empty segments to a, so all in other are present + for len(a) > 0 && len(other) > 0 { + if len(other) == 0 { + merged = append(merged, a...) + break + } + if len(a) == 0 { + merged = append(merged, other...) + break + } + toMerge := other[0] + idx := -1 + for i := range a { + if a[i].Start.After(toMerge.Start) { + // Store in previous index. + break + } + idx = i + } + if idx == -1 { + merged = append(merged, toMerge) + other = other[1:] + } + if idx > 0 { + merged = append(merged, a[:idx]...) + a = a[idx:] + } + merged = append(merged, a[0].add(toMerge)) + other = other[1:] + } + merged.SortByStartTime() + *s = merged +} + +func (t *ThroughputSegmented) Merge(other ThroughputSegmented) { + t.Segments.SortByStartTime() + if len(other.Segments) == 0 { + return + } + if t.SegmentDurationMillis == 0 { + t.SegmentDurationMillis = other.SegmentDurationMillis + } + t.Segments.Merge(other.Segments) + t.fillFromSegs() +} + // BPSorOPS returns bytes per second if non zero otherwise operations per second as human readable string. func BPSorOPS(bps, ops float64) string { if bps > 0 { @@ -125,7 +284,7 @@ type SegmentSmall struct { // Bytes per second during the time segment. BPS float64 `json:"bytes_per_sec"` - // Operations per second during the time segment. + // Objects per second during the time segment. OPS float64 `json:"obj_per_sec"` // Errors logged during the time segment. @@ -147,6 +306,13 @@ func cloneBenchSegments(s bench.Segments) []SegmentSmall { return res } +func (s *SegmentSmall) add(other SegmentSmall) SegmentSmall { + s.Errors += other.Errors + s.OPS += other.OPS + s.BPS += other.BPS + return *s +} + // StringLong returns a long string representation of the segment. func (s SegmentSmall) StringLong(d time.Duration, details bool) string { speed := "" @@ -161,18 +327,18 @@ func (s SegmentSmall) StringLong(d time.Duration, details bool) string { speed, s.OPS, detail) } -func (a *ThroughputSegmented) fill(segs bench.Segments, total bench.Segment) { +func (t *ThroughputSegmented) fill(segs bench.Segments, totalBytes int64) { // Copy by time. segs.SortByTime() smallSegs := cloneBenchSegments(segs) // Sort to get correct medians. - if total.TotalBytes > 0 { + if totalBytes > 0 { segs.SortByThroughput() - a.SortedBy = "bps" + t.SortedBy = "bps" } else { segs.SortByObjsPerSec() - a.SortedBy = "ops" + t.SortedBy = "ops" } fast := segs.Median(1) @@ -188,10 +354,10 @@ func (a *ThroughputSegmented) fill(segs bench.Segments, total bench.Segment) { return math.Round(objs*100) / 100 } - *a = ThroughputSegmented{ + *t = ThroughputSegmented{ Segments: smallSegs, - SortedBy: a.SortedBy, - SegmentDurationMillis: a.SegmentDurationMillis, + SortedBy: t.SortedBy, + SegmentDurationMillis: t.SegmentDurationMillis, FastestStart: fast.Start, FastestBPS: bps(fast), FastestOPS: ops(fast), @@ -203,3 +369,42 @@ func (a *ThroughputSegmented) fill(segs bench.Segments, total bench.Segment) { SlowestOPS: ops(slow), } } + +func (t *ThroughputSegmented) fillFromSegs() { + // Copy by time. + segs := t.Segments + var byBPS bool + for _, seg := range segs { + if seg.BPS > 0 { + byBPS = true + break + } + } + // Sort to get correct medians. + if byBPS { + segs.SortByThroughput() + t.SortedBy = "bps" + } else { + segs.SortByObjsPerSec() + t.SortedBy = "ops" + } + + fast := segs.Median(1) + med := segs.Median(0.5) + slow := segs.Median(0) + + *t = ThroughputSegmented{ + Segments: segs, + SortedBy: t.SortedBy, + SegmentDurationMillis: t.SegmentDurationMillis, + FastestStart: fast.Start, + FastestBPS: fast.BPS, + FastestOPS: fast.OPS, + MedianStart: med.Start, + MedianBPS: med.BPS, + MedianOPS: med.OPS, + SlowestStart: slow.Start, + SlowestBPS: slow.BPS, + SlowestOPS: slow.OPS, + } +} diff --git a/pkg/aggregate/ttfb.go b/pkg/aggregate/ttfb.go index fd6ca9d4..e79ed162 100644 --- a/pkg/aggregate/ttfb.go +++ b/pkg/aggregate/ttfb.go @@ -26,16 +26,16 @@ import ( // TTFB contains times to first byte if applicable. type TTFB struct { - AverageMillis int `json:"average_millis"` - FastestMillis int `json:"fastest_millis"` - P25Millis int `json:"p25_millis"` - MedianMillis int `json:"median_millis"` - P75Millis int `json:"p75_millis"` - P90Millis int `json:"p90_millis"` - P99Millis int `json:"p99_millis"` - SlowestMillis int `json:"slowest_millis"` - StdDevMillis int `json:"std_dev_millis"` - PercentilesMillis [101]int `json:"percentiles_millis"` + AverageMillis float64 `json:"average_millis"` + FastestMillis float64 `json:"fastest_millis"` + P25Millis float64 `json:"p25_millis"` + MedianMillis float64 `json:"median_millis"` + P75Millis float64 `json:"p75_millis"` + P90Millis float64 `json:"p90_millis"` + P99Millis float64 `json:"p99_millis"` + SlowestMillis float64 `json:"slowest_millis"` + StdDevMillis float64 `json:"std_dev_millis"` + PercentilesMillis *[101]float64 `json:"percentiles_millis,omitempty"` } // String returns a human printable version of the time to first byte. @@ -44,15 +44,51 @@ func (t TTFB) String() string { return "" } return fmt.Sprintf("Avg: %v, Best: %v, 25th: %v, Median: %v, 75th: %v, 90th: %v, 99th: %v, Worst: %v StdDev: %v", - time.Duration(t.AverageMillis)*time.Millisecond, + time.Duration(t.AverageMillis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.FastestMillis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.P25Millis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.MedianMillis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.P75Millis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.P90Millis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.P99Millis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.SlowestMillis)*time.Millisecond.Round(time.Millisecond), + time.Duration(t.StdDevMillis)*time.Millisecond.Round(time.Millisecond)) +} + +func (t *TTFB) add(other TTFB) { + t.AverageMillis += other.AverageMillis + t.MedianMillis += other.MedianMillis + if other.FastestMillis != 0 { + // Deal with 0 value being the best always. + t.FastestMillis = min(t.FastestMillis, other.FastestMillis) + if t.FastestMillis == 0 { + t.FastestMillis = other.FastestMillis + } + } + t.SlowestMillis = max(t.SlowestMillis, other.SlowestMillis) + t.P25Millis += other.P25Millis + t.P75Millis += other.P75Millis + t.P90Millis += other.P90Millis + t.P99Millis += other.P99Millis + t.StdDevMillis += other.StdDevMillis +} + +func (t TTFB) StringByN(n int) string { + if t.AverageMillis == 0 || n == 0 { + return "" + } + // rounder... + hN := n / 2 + return fmt.Sprintf("Avg: %v, Best: %v, 25th: %v, Median: %v, 75th: %v, 90th: %v, 99th: %v, Worst: %v StdDev: %v", + time.Duration((hN+int(t.AverageMillis))/n)*time.Millisecond, time.Duration(t.FastestMillis)*time.Millisecond, - time.Duration(t.P25Millis)*time.Millisecond, - time.Duration(t.MedianMillis)*time.Millisecond, - time.Duration(t.P75Millis)*time.Millisecond, - time.Duration(t.P90Millis)*time.Millisecond, - time.Duration(t.P99Millis)*time.Millisecond, + time.Duration((hN+int(t.P25Millis))/n)*time.Millisecond, + time.Duration((hN+int(t.MedianMillis))/n)*time.Millisecond, + time.Duration((hN+int(t.P75Millis))/n)*time.Millisecond, + time.Duration((hN+int(t.P90Millis))/n)*time.Millisecond, + time.Duration((hN+int(t.P99Millis))/n)*time.Millisecond, time.Duration(t.SlowestMillis)*time.Millisecond, - time.Duration(t.StdDevMillis)*time.Millisecond) + time.Duration((hN+int(t.StdDevMillis))/n)*time.Millisecond) } // TtfbFromBench converts from bench.TTFB @@ -61,18 +97,19 @@ func TtfbFromBench(t bench.TTFB) *TTFB { return nil } t2 := TTFB{ - AverageMillis: durToMillis(t.Average), - SlowestMillis: durToMillis(t.Worst), - P25Millis: durToMillis(t.P25), - MedianMillis: durToMillis(t.Median), - P75Millis: durToMillis(t.P75), - P90Millis: durToMillis(t.P90), - P99Millis: durToMillis(t.P99), - StdDevMillis: durToMillis(t.StdDev), - FastestMillis: durToMillis(t.Best), + AverageMillis: durToMillisF(t.Average), + SlowestMillis: durToMillisF(t.Worst), + P25Millis: durToMillisF(t.P25), + MedianMillis: durToMillisF(t.Median), + P75Millis: durToMillisF(t.P75), + P90Millis: durToMillisF(t.P90), + P99Millis: durToMillisF(t.P99), + StdDevMillis: durToMillisF(t.StdDev), + FastestMillis: durToMillisF(t.Best), } + t2.PercentilesMillis = &[101]float64{} for i, v := range t.Percentiles[:] { - t2.PercentilesMillis[i] = durToMillis(v) + t2.PercentilesMillis[i] = durToMillisF(v) } return &t2 } diff --git a/pkg/bench/benchmark.go b/pkg/bench/benchmark.go index ad0ef4fe..ad2eec96 100644 --- a/pkg/bench/benchmark.go +++ b/pkg/bench/benchmark.go @@ -27,7 +27,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" "golang.org/x/time/rate" @@ -39,12 +38,12 @@ type Benchmark interface { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. - Start(ctx context.Context, wait chan struct{}) (Operations, error) + Start(ctx context.Context, wait chan struct{}) error - // Clean up after the benchmark run. + // Cleanup up after the benchmark run. Cleanup(ctx context.Context) - // Common returns the common parameters. + // GetCommon returns the common parameters. GetCommon() *Common } @@ -60,15 +59,19 @@ type Common struct { // ExtraFlags contains extra flags to add to remote clients. ExtraFlags map[string]string - Source func() generator.Source - ExtraOut []chan<- Operation + + // A source is a source that can be used for a single goroutine. + // It will have a shared prefix, if configured so. + Source func() generator.Source + + ExtraOut []chan<- Operation // Error should log an error similar to fmt.Print(data...) Error func(data ...interface{}) Client func() (cl *minio.Client, done func()) - Collector *Collector + Collector Collector Location string Bucket string @@ -102,6 +105,9 @@ type Common struct { // Transport used. Transport http.RoundTripper + + // UpdateStatus + UpdateStatus func(s string) } const ( @@ -142,8 +148,7 @@ func (c *Common) createEmptyBucket(ctx context.Context) error { if bvc, err := cl.GetBucketVersioning(ctx, c.Bucket); err == nil { c.Versioned = bvc.Status == "Enabled" } - console.Eraseline() - console.Infof("\rClearing Bucket %q to enable locking...", c.Bucket) + c.UpdateStatus(fmt.Sprintf("Clearing Bucket %q to enable locking", c.Bucket)) c.deleteAllInBucket(ctx) err = cl.RemoveBucket(ctx, c.Bucket) if err != nil { @@ -155,8 +160,7 @@ func (c *Common) createEmptyBucket(ctx context.Context) error { } if !x { - console.Eraseline() - console.Infof("\rCreating Bucket %q...", c.Bucket) + c.UpdateStatus(fmt.Sprintf("Creating Bucket %q", c.Bucket)) err := cl.MakeBucket(ctx, c.Bucket, minio.MakeBucketOptions{ Region: c.Location, ObjectLocking: c.Locking, @@ -180,8 +184,7 @@ func (c *Common) createEmptyBucket(ctx context.Context) error { } if c.Clear { - console.Eraseline() - console.Infof("\rClearing Bucket %q...", c.Bucket) + c.UpdateStatus(fmt.Sprintf("Clearing Bucket %q", c.Bucket)) c.deleteAllInBucket(ctx) } return nil @@ -212,15 +215,19 @@ func (c *Common) deleteAllInBucket(ctx context.Context, prefixes ...string) { if prefix != "" { opts.Prefix = prefix + "/" } + removed := 0 + c.UpdateStatus(fmt.Sprintf("Clearing Prefix %q", strings.Join([]string{c.Bucket, opts.Prefix}, "/"))) for object := range cl.ListObjects(ctx, c.Bucket, opts) { if object.Err != nil { c.Error(object.Err) return } + removed++ objectsCh <- object + if removed%1000 == 0 { + c.UpdateStatus(fmt.Sprintf("Clearing Prefix %q. Deleted %d objects", strings.Join([]string{c.Bucket, opts.Prefix}, "/"), removed)) + } } - console.Eraseline() - console.Infof("\rClearing Prefix %q...", strings.Join([]string{c.Bucket, opts.Prefix}, "/")) } }() @@ -237,6 +244,7 @@ func (c *Common) deleteAllInBucket(ctx context.Context, prefixes ...string) { continue } } + c.UpdateStatus("Cleanup Done") } // prepareProgress updates preparation progress with the value 0->1. @@ -251,15 +259,6 @@ func (c *Common) prepareProgress(progress float64) { } } -func (c *Common) addCollector() { - if c.DiscardOutput { - c.Collector = NewNullCollector() - } else { - c.Collector = NewCollector() - } - c.Collector.extra = c.ExtraOut -} - func (c *Common) rpsLimit(ctx context.Context) error { if c.RpsLimiter == nil { return nil diff --git a/pkg/bench/category.go b/pkg/bench/category.go new file mode 100644 index 00000000..370cd365 --- /dev/null +++ b/pkg/bench/category.go @@ -0,0 +1,73 @@ +/* + * Warp (C) 2019-2024 MinIO, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +package bench + +import ( + "math/bits" + "strings" +) + +//go:generate stringer -type Category -trimprefix=Cat $GOFILE + +// A Category allows requests to be separated into different categories. +type Category uint8 + +const ( + // CatCacheMiss means that caching was detected, but the object missed the cache. + CatCacheMiss Category = iota + + // CatCacheHit means that caching was detected and the object was cached. + CatCacheHit + + catLength +) + +// Categories is a bitfield that represents potentially several categories. +type Categories uint64 + +func NewCategories(c ...Category) Categories { + var cs Categories + for _, cat := range c { + cs |= 1 << cat + } + return cs +} + +// Split returns the categories +func (c Categories) Split() []Category { + if c == 0 { + return nil + } + res := make([]Category, 0, bits.OnesCount64(uint64(c))) + for i := Category(0); c != 0 && i < catLength; i++ { + if c&1 == 1 { + res = append(res, i) + } + c >>= 1 + } + return res +} + +func (c Categories) String() string { + cs := c.Split() + res := make([]string, len(cs)) + for i, c := range cs { + res[i] = c.String() + } + return strings.Join(res, ",") +} diff --git a/pkg/bench/category_string.go b/pkg/bench/category_string.go new file mode 100644 index 00000000..668aa777 --- /dev/null +++ b/pkg/bench/category_string.go @@ -0,0 +1,25 @@ +// Code generated by "stringer -type Category -trimprefix=Cat category.go"; DO NOT EDIT. + +package bench + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[CatCacheMiss-0] + _ = x[CatCacheHit-1] + _ = x[catLength-2] +} + +const _Category_name = "CacheMissCacheHitcatLength" + +var _Category_index = [...]uint8{0, 9, 17, 26} + +func (i Category) String() string { + if i >= Category(len(_Category_index)-1) { + return "Category(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _Category_name[_Category_index[i]:_Category_index[i+1]] +} diff --git a/pkg/bench/collector.go b/pkg/bench/collector.go index 5b48a914..5b28af22 100644 --- a/pkg/bench/collector.go +++ b/pkg/bench/collector.go @@ -26,7 +26,30 @@ import ( "github.com/minio/pkg/v3/console" ) -type Collector struct { +type Collector interface { + // AutoTerm will check if throughput is within 'threshold' (0 -> ) for wantSamples, + // when the current operations are split into 'splitInto' segments. + // The minimum duration for the calculation can be set as well. + // Segment splitting may cause less than this duration to be used. + AutoTerm(ctx context.Context, op string, threshold float64, wantSamples, splitInto int, minDur time.Duration) context.Context + + // Receiver returns the receiver of input + Receiver() chan<- Operation + + // AddOutput allows to add additional inputs. + AddOutput(...chan<- Operation) + + // Close the collector + Close() +} + +type OpsCollector func() Operations + +func EmptyOpsCollector() Operations { + return Operations{} +} + +type collector struct { rcv chan Operation ops Operations rcvWg sync.WaitGroup @@ -36,8 +59,10 @@ type Collector struct { opsMu sync.Mutex } -func NewCollector() *Collector { - r := &Collector{ +// NewOpsCollector returns a collector that will collect all operations in memory. +// After calling Close the returned function can be used to retrieve the operations. +func NewOpsCollector() (Collector, OpsCollector) { + r := &collector{ ops: make(Operations, 0, 10000), rcv: make(chan Operation, 1000), } @@ -53,12 +78,15 @@ func NewCollector() *Collector { r.opsMu.Unlock() } }() - return r + return r, func() Operations { + r.Close() + return r.ops + } } // NewNullCollector collects operations, but discards them. -func NewNullCollector() *Collector { - r := &Collector{ +func NewNullCollector() Collector { + r := &collector{ ops: make(Operations, 0), rcv: make(chan Operation, 1000), } @@ -78,7 +106,7 @@ func NewNullCollector() *Collector { // when the current operations are split into 'splitInto' segments. // The minimum duration for the calculation can be set as well. // Segment splitting may cause less than this duration to be used. -func (c *Collector) AutoTerm(ctx context.Context, op string, threshold float64, wantSamples, splitInto int, minDur time.Duration) context.Context { +func (c *collector) AutoTerm(ctx context.Context, op string, threshold float64, wantSamples, splitInto int, minDur time.Duration) context.Context { if wantSamples >= splitInto { panic("wantSamples >= splitInto") } @@ -150,15 +178,23 @@ func (c *Collector) AutoTerm(ctx context.Context, op string, threshold float64, return ctx } -func (c *Collector) Receiver() chan<- Operation { +func (c *collector) Receiver() chan<- Operation { return c.rcv } -func (c *Collector) Close() Operations { - close(c.rcv) - c.rcvWg.Wait() - for _, ch := range c.extra { - close(ch) +func (c *collector) Close() { + if c.rcv != nil { + close(c.rcv) + c.rcvWg.Wait() + c.rcv = nil + for _, ch := range c.extra { + close(ch) + } + c.extra = nil } - return c.ops + return +} + +func (c *collector) AddOutput(x ...chan<- Operation) { + c.extra = append(c.extra, x...) } diff --git a/pkg/bench/delete.go b/pkg/bench/delete.go index 3e0313e8..0edd930e 100644 --- a/pkg/bench/delete.go +++ b/pkg/bench/delete.go @@ -26,7 +26,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -88,7 +87,6 @@ func (d *Delete) Prepare(ctx context.Context) error { return (fmt.Errorf("no objects found for bucket %s", d.Bucket)) } done() - d.Collector = NewCollector() // Shuffle objects. // Benchmark will pick from slice in order. @@ -103,11 +101,10 @@ func (d *Delete) Prepare(ctx context.Context) error { return err } src := d.Source() - console.Eraseline() - console.Info("\rUploading ", d.CreateObjects, " objects of ", src.String()) + + d.UpdateStatus(fmt.Sprint("\rUploading ", d.CreateObjects, " objects of ", src.String())) var wg sync.WaitGroup wg.Add(d.Concurrency) - d.addCollector() objs := splitObjs(d.CreateObjects, d.Concurrency) var mu sync.Mutex @@ -191,7 +188,7 @@ func (d *Delete) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (d *Delete) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (d *Delete) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(d.Concurrency) c := d.Collector @@ -275,7 +272,7 @@ func (d *Delete) Start(ctx context.Context, wait chan struct{}) (Operations, err }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/fanout.go b/pkg/bench/fanout.go index 53ac6492..fb40efa4 100644 --- a/pkg/bench/fanout.go +++ b/pkg/bench/fanout.go @@ -41,10 +41,9 @@ func (u *Fanout) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (u *Fanout) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (u *Fanout) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(u.Concurrency) - u.addCollector() c := u.Collector if u.AutoTermDur > 0 { ctx = c.AutoTerm(ctx, http.MethodPost, u.AutoTermScale, autoTermCheck, autoTermSamples, u.AutoTermDur) @@ -133,7 +132,7 @@ func (u *Fanout) Start(ctx context.Context, wait chan struct{}) (Operations, err }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/get.go b/pkg/bench/get.go index 7ad13b4c..05799bb4 100644 --- a/pkg/bench/get.go +++ b/pkg/bench/get.go @@ -27,7 +27,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -52,7 +51,6 @@ type Get struct { // and upload a number of objects. func (g *Get) Prepare(ctx context.Context) error { // prepare the bench by listing object from the bucket - g.addCollector() if g.ListExisting { cl, done := g.Client() @@ -131,18 +129,17 @@ func (g *Get) Prepare(ctx context.Context) error { } done() } - console.Eraseline() x := "" if g.Versions > 1 { x = fmt.Sprintf(" with %d versions each", g.Versions) } - console.Info("\rUploading ", g.CreateObjects, " objects", x) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects", x)) var wg sync.WaitGroup wg.Add(g.Concurrency) objs := splitObjs(g.CreateObjects, g.Concurrency) - rcv := g.Collector.rcv + rcv := g.Collector.Receiver() var groupErr error var mu sync.Mutex @@ -240,7 +237,7 @@ func (f *firstByteRecorder) Read(p []byte) (n int, err error) { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Get) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Get) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -333,7 +330,7 @@ func (g *Get) Start(ctx context.Context, wait chan struct{}) (Operations, error) }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/list.go b/pkg/bench/list.go index 7e9f4006..975ba1e5 100644 --- a/pkg/bench/list.go +++ b/pkg/bench/list.go @@ -27,7 +27,6 @@ import ( "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -61,19 +60,17 @@ func (d *List) Prepare(ctx context.Context) error { } objPerPrefix := (d.CreateObjects + d.Concurrency - 1) / d.Concurrency - console.Eraseline() x := "" if d.Versions > 1 { x = fmt.Sprintf(" with %d versions each", d.Versions) } if d.NoPrefix { - console.Info("\rUploading ", objPerPrefix*d.Concurrency, " objects", x) + d.UpdateStatus(fmt.Sprint("Uploading ", objPerPrefix*d.Concurrency, " objects", x)) } else { - console.Info("\rUploading ", objPerPrefix*d.Concurrency, " objects", x, " in ", d.Concurrency, " prefixes") + d.UpdateStatus(fmt.Sprint("Uploading ", objPerPrefix*d.Concurrency, " objects", x, " in ", d.Concurrency, " prefixes")) } var wg sync.WaitGroup wg.Add(d.Concurrency) - d.addCollector() d.objects = make([]generator.Objects, d.Concurrency) var mu sync.Mutex objsCreated := 0 @@ -173,7 +170,7 @@ func (d *List) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (d *List) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (d *List) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(d.Concurrency) c := d.Collector @@ -255,7 +252,7 @@ func (d *List) Start(ctx context.Context, wait chan struct{}) (Operations, error }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/mixed.go b/pkg/bench/mixed.go index 7c8c472a..797df02e 100644 --- a/pkg/bench/mixed.go +++ b/pkg/bench/mixed.go @@ -28,7 +28,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -157,11 +156,9 @@ func (g *Mixed) Prepare(ctx context.Context) error { return err } src := g.Source() - console.Eraseline() - console.Info("\rUploading ", g.CreateObjects, " objects of ", src.String()) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects of ", src.String())) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() var groupErr error objs := splitObjs(g.CreateObjects, g.Concurrency) @@ -223,7 +220,7 @@ func (g *Mixed) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Mixed) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Mixed) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -387,7 +384,7 @@ func (g *Mixed) Start(ctx context.Context, wait chan struct{}) (Operations, erro }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/multipart.go b/pkg/bench/multipart.go index f1d24211..2a89d0c0 100644 --- a/pkg/bench/multipart.go +++ b/pkg/bench/multipart.go @@ -29,7 +29,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -52,8 +51,7 @@ func (g *Multipart) InitOnce(ctx context.Context) error { if err := g.createEmptyBucket(ctx); err != nil { return err } - console.Eraseline() - console.Info("\rCreating Object...") + g.UpdateStatus(fmt.Sprint("Creating Object...")) cl, done := g.Client() c := minio.Core{Client: cl} @@ -73,19 +71,16 @@ func (g *Multipart) Prepare(ctx context.Context) error { if g.PartStart+g.CreateParts > 10001 { return errors.New("10000 part limit exceeded") } - console.Println("") - console.Eraseline() - console.Info("Uploading ", g.CreateParts, " object parts") + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateParts, " object parts")) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() obj := make(chan int, g.CreateParts) for i := 0; i < g.CreateParts; i++ { obj <- i + g.PartStart } close(obj) - rcv := g.Collector.rcv + rcv := g.Collector.Receiver() var groupErr error var mu sync.Mutex @@ -185,15 +180,14 @@ func (g *Multipart) AfterPrepare(ctx context.Context) error { parts = append(parts, minio.CompletePart{PartNumber: i, ETag: etag}) i++ } - console.Eraseline() - console.Infof("\rCompleting Object with %d parts...", len(parts)) + g.UpdateStatus(fmt.Sprintf("Completing Object with %d parts...", len(parts))) _, err := c.CompleteMultipartUpload(ctx, g.Bucket, g.ObjName, g.UploadID, parts, g.PutOpts) return err } // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Multipart) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Multipart) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -268,7 +262,7 @@ func (g *Multipart) Start(ctx context.Context, wait chan struct{}) (Operations, }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/ops.go b/pkg/bench/ops.go index 7e8141e9..34ae9a2f 100644 --- a/pkg/bench/ops.go +++ b/pkg/bench/ops.go @@ -26,26 +26,27 @@ import ( "sort" "strconv" "strings" + "sync" "time" "github.com/dustin/go-humanize" - "github.com/minio/pkg/v3/console" ) type Operations []Operation type Operation struct { - Start time.Time `json:"start"` - End time.Time `json:"end"` - FirstByte *time.Time `json:"first_byte"` - OpType string `json:"type"` - Err string `json:"err"` - File string `json:"file,omitempty"` - ClientID string `json:"client_id"` - Endpoint string `json:"endpoint"` - ObjPerOp int `json:"ops"` - Size int64 `json:"size"` - Thread uint16 `json:"thread"` + Start time.Time `json:"start"` + End time.Time `json:"end"` + FirstByte *time.Time `json:"first_byte"` + OpType string `json:"type"` + Err string `json:"err"` + File string `json:"file,omitempty"` + ClientID string `json:"client_id"` + Endpoint string `json:"endpoint"` + ObjPerOp int `json:"ops"` + Size int64 `json:"size"` + Thread uint16 `json:"thread"` + Categories Categories `json:"cat"` } // Duration returns the duration o.End-o.Start @@ -83,7 +84,7 @@ func (o Operation) BytesPerSec() Throughput { } d := o.Duration() if d <= 0 { - return Throughput(math.Inf(1)) + return 0 } return Throughput(o.Size*int64(time.Second)) / Throughput(d) } @@ -264,7 +265,7 @@ func (o Operations) SortByOpType() { // Fastest operations first. func (o Operations) SortByDuration() { sort.Slice(o, func(i, j int) bool { - a, b := o[i].End.Sub(o[i].Start), o[j].End.Sub(o[j].Start) + a, b := o[i].End.UnixNano()-o[i].Start.UnixNano(), o[j].End.UnixNano()-o[j].Start.UnixNano() return a < b }) } @@ -274,7 +275,7 @@ func (o Operations) SortByDuration() { func (o Operations) SortByThroughput() { sort.Slice(o, func(i, j int) bool { a, b := &o[i], &o[j] - aDur, bDur := a.End.Sub(a.Start), b.End.Sub(b.Start) + aDur, bDur := a.End.UnixNano()-a.Start.UnixNano(), b.End.UnixNano()-b.Start.UnixNano() if a.Size == 0 || b.Size == 0 { return aDur < bDur } @@ -282,6 +283,18 @@ func (o Operations) SortByThroughput() { }) } +// SortByThroughputNonZero will sort the operations by throughput. +// Fastest operations first. +func (o Operations) SortByThroughputNonZero() Operations { + o.SortByThroughput() + for i, op := range o { + if op.Duration() > 0 { + return o[i:] + } + } + return nil +} + // Median returns the m part median of the assumed sorted list of operations. // m is clamped to the range 0 -> 1. func (o Operations) Median(m float64) Operation { @@ -302,7 +315,7 @@ func (o Operations) SortByTTFB() { if a.FirstByte == nil || b.FirstByte == nil { return a.Start.Before(b.Start) } - return a.FirstByte.Sub(a.Start) < b.FirstByte.Sub(b.Start) + return a.FirstByte.UnixNano()-a.Start.UnixNano() < b.FirstByte.UnixNano()-b.Start.UnixNano() }) } @@ -695,6 +708,7 @@ func (o Operations) SplitSizes(minShare float64) []SizeSegment { if !o.MultipleSizes() { return []SizeSegment{o.SingleSizeSegment()} } + // FIXME: This allocs like crazy... var res []SizeSegment minSz, maxSz := o.MinMaxSize() if minSz == 0 { @@ -1030,18 +1044,13 @@ func (o Operations) FilterErrors() Operations { // The comment, if any, is written at the end of the file, each line prefixed with '# '. func (o Operations) CSV(w io.Writer, comment string) error { bw := bufio.NewWriter(w) - _, err := bw.WriteString("idx\tthread\top\tclient_id\tn_objects\tbytes\tendpoint\tfile\terror\tstart\tfirst_byte\tend\tduration_ns\n") + _, err := bw.WriteString("idx\tthread\top\tclient_id\tn_objects\tbytes\tendpoint\tfile\terror\tstart\tfirst_byte\tend\tduration_ns\tcat\n") if err != nil { return err } for i, op := range o { - var ttfb string - if op.FirstByte != nil { - ttfb = op.FirstByte.Format(time.RFC3339Nano) - } - _, err := fmt.Fprintf(bw, "%d\t%d\t%s\t%s\t%d\t%d\t%s\t%s\t%s\t%s\t%s\t%s\t%d\n", i, op.Thread, op.OpType, op.ClientID, op.ObjPerOp, op.Size, csvEscapeString(op.Endpoint), op.File, csvEscapeString(op.Err), op.Start.Format(time.RFC3339Nano), ttfb, op.End.Format(time.RFC3339Nano), op.End.Sub(op.Start)/time.Nanosecond) - if err != nil { + if err := op.WriteCSV(bw, i); err != nil { return err } } @@ -1058,16 +1067,44 @@ func (o Operations) CSV(w io.Writer, comment string) error { return bw.Flush() } +func (o Operation) WriteCSV(w io.Writer, i int) error { + var ttfb string + if o.FirstByte != nil { + ttfb = o.FirstByte.Format(time.RFC3339Nano) + } + _, err := fmt.Fprintf(w, "%d\t%d\t%s\t%s\t%d\t%d\t%s\t%s\t%s\t%s\t%s\t%s\t%d\t%d\n", i, o.Thread, o.OpType, o.ClientID, o.ObjPerOp, o.Size, csvEscapeString(o.Endpoint), o.File, csvEscapeString(o.Err), o.Start.Format(time.RFC3339Nano), ttfb, o.End.Format(time.RFC3339Nano), o.End.Sub(o.Start)/time.Nanosecond, o.Categories) + return err +} + // OperationsFromCSV will load operations from CSV. func OperationsFromCSV(r io.Reader, analyzeOnly bool, offset, limit int, log func(msg string, v ...interface{})) (Operations, error) { + opCh := make(chan Operation, 1000) var ops Operations + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + for op := range opCh { + ops = append(ops, op) + } + }() + if err := StreamOperationsFromCSV(r, analyzeOnly, offset, limit, log, opCh); err != nil { + return nil, err + } + wg.Wait() + return ops, nil +} + +// StreamOperationsFromCSV will load operations from CSV. +func StreamOperationsFromCSV(r io.Reader, analyzeOnly bool, offset, limit int, log func(msg string, v ...interface{}), out chan<- Operation) error { + defer close(out) cr := csv.NewReader(r) cr.Comma = '\t' cr.ReuseRecord = true cr.Comment = '#' header, err := cr.Read() if err != nil { - return nil, err + return err } fieldIdx := make(map[string]int) for i, s := range header { @@ -1102,13 +1139,14 @@ func OperationsFromCSV(r io.Reader, analyzeOnly bool, offset, limit int, log fun return strconv.Itoa(i) } } + n := 0 for { values, err := cr.Read() if err == io.EOF { break } if err != nil { - return nil, err + return err } if len(values) == 0 { continue @@ -1119,31 +1157,39 @@ func OperationsFromCSV(r io.Reader, analyzeOnly bool, offset, limit int, log fun } start, err := time.Parse(time.RFC3339Nano, values[fieldIdx["start"]]) if err != nil { - return nil, err + return err } var ttfb *time.Time if fb := values[fieldIdx["first_byte"]]; fb != "" { t, err := time.Parse(time.RFC3339Nano, fb) if err != nil { - return nil, err + return err } ttfb = &t } end, err := time.Parse(time.RFC3339Nano, values[fieldIdx["end"]]) if err != nil { - return nil, err + return err } size, err := strconv.ParseInt(values[fieldIdx["bytes"]], 10, 64) if err != nil { - return nil, err + return err } thread, err := strconv.ParseUint(values[fieldIdx["thread"]], 10, 16) if err != nil { - return nil, err + return err } objs, err := strconv.ParseInt(values[fieldIdx["n_objects"]], 10, 64) if err != nil { - return nil, err + return err + } + var cat Categories + if idx, ok := fieldIdx["cat"]; ok { + c, err := strconv.ParseUint(values[idx], 10, 64) + if err != nil { + return err + } + cat = Categories(c) } var endpoint, clientID string if idx, ok := fieldIdx["endpoint"]; ok { @@ -1154,30 +1200,30 @@ func OperationsFromCSV(r io.Reader, analyzeOnly bool, offset, limit int, log fun } file := fileMap(values[fieldIdx["file"]]) - ops = append(ops, Operation{ - OpType: values[fieldIdx["op"]], - ObjPerOp: int(objs), - Start: start, - FirstByte: ttfb, - End: end, - Err: values[fieldIdx["error"]], - Size: size, - File: file, - Thread: uint16(thread), - Endpoint: endpoint, - ClientID: getClient(clientID), - }) - if log != nil && len(ops)%1000000 == 0 { - console.Eraseline() - log("\r%d operations loaded...", len(ops)) - } - if limit > 0 && len(ops) >= limit { + out <- Operation{ + OpType: values[fieldIdx["op"]], + ObjPerOp: int(objs), + Start: start, + FirstByte: ttfb, + End: end, + Err: values[fieldIdx["error"]], + Size: size, + File: file, + Thread: uint16(thread), + Endpoint: endpoint, + ClientID: getClient(clientID), + Categories: cat, + } + n++ + if log != nil && n%100000 == 0 { + log("%d operations loaded. Timestamp: %v", n, start.Round(time.Second).Local()) + } + if limit > 0 && n >= limit { break } } if log != nil { - console.Eraseline() - log("\r%d operations loaded... Done!\n", len(ops)) + log("%d operations loaded... Done!", n) } - return ops, nil + return nil } diff --git a/pkg/bench/put.go b/pkg/bench/put.go index bfe09c78..3c2a95d6 100644 --- a/pkg/bench/put.go +++ b/pkg/bench/put.go @@ -51,10 +51,9 @@ func (u *Put) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (u *Put) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (u *Put) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(u.Concurrency) - u.addCollector() c := u.Collector if u.AutoTermDur > 0 { ctx = c.AutoTerm(ctx, http.MethodPut, u.AutoTermScale, autoTermCheck, autoTermSamples, u.AutoTermDur) @@ -143,7 +142,7 @@ func (u *Put) Start(ctx context.Context, wait chan struct{}) (Operations, error) }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/retention.go b/pkg/bench/retention.go index e6abf683..ef3ea89d 100644 --- a/pkg/bench/retention.go +++ b/pkg/bench/retention.go @@ -26,7 +26,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -56,11 +55,9 @@ func (g *Retention) Prepare(ctx context.Context) error { } src := g.Source() - console.Eraseline() - console.Info("\rUploading ", g.CreateObjects, " objects with ", g.Versions, " versions each of ", src.String()) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects with ", g.Versions, " versions each of ", src.String())) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() objs := splitObjs(g.CreateObjects, g.Concurrency) var groupErr error var mu sync.Mutex @@ -143,7 +140,7 @@ func (g *Retention) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Retention) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Retention) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -208,7 +205,7 @@ func (g *Retention) Start(ctx context.Context, wait chan struct{}) (Operations, }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/s3zip.go b/pkg/bench/s3zip.go index 4d7997c5..0dc0448e 100644 --- a/pkg/bench/s3zip.go +++ b/pkg/bench/s3zip.go @@ -29,7 +29,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -49,10 +48,8 @@ func (g *S3Zip) Prepare(ctx context.Context) error { return err } - g.addCollector() src := g.Source() - console.Eraseline() - console.Info("\rUploading", g.ZipObjName, "with ", g.CreateFiles, " files each of ", src.String()) + g.UpdateStatus(fmt.Sprint("Uploading", g.ZipObjName, "with ", g.CreateFiles, " files each of ", src.String())) client, cldone := g.Client() defer cldone() @@ -125,7 +122,7 @@ func (g *S3Zip) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *S3Zip) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *S3Zip) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -199,7 +196,7 @@ func (g *S3Zip) Start(ctx context.Context, wait chan struct{}) (Operations, erro }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/select.go b/pkg/bench/select.go index f1adc9ea..7c370958 100644 --- a/pkg/bench/select.go +++ b/pkg/bench/select.go @@ -27,7 +27,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -49,11 +48,9 @@ func (g *Select) Prepare(ctx context.Context) error { return err } src := g.Source() - console.Eraseline() - console.Info("\rUploading ", g.CreateObjects, " objects of ", src.String()) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects of ", src.String())) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() objs := splitObjs(g.CreateObjects, g.Concurrency) var groupErr error @@ -129,7 +126,7 @@ func (g *Select) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Select) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Select) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -198,7 +195,7 @@ func (g *Select) Start(ctx context.Context, wait chan struct{}) (Operations, err }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/snowball.go b/pkg/bench/snowball.go index 315b6df0..3eb65f1a 100644 --- a/pkg/bench/snowball.go +++ b/pkg/bench/snowball.go @@ -57,14 +57,13 @@ func (s *Snowball) Prepare(ctx context.Context) error { } } } - s.addCollector() s.prefixes = make(map[string]struct{}, s.Concurrency) return s.createEmptyBucket(ctx) } // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (s *Snowball) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (s *Snowball) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(s.Concurrency) c := s.Collector @@ -186,7 +185,7 @@ func (s *Snowball) Start(ctx context.Context, wait chan struct{}) (Operations, e }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/stat.go b/pkg/bench/stat.go index edab0fd3..f18dff2f 100644 --- a/pkg/bench/stat.go +++ b/pkg/bench/stat.go @@ -26,7 +26,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -50,7 +49,6 @@ type Stat struct { // and upload a number of objects. func (g *Stat) Prepare(ctx context.Context) error { // prepare the bench by listing object from the bucket - g.addCollector() if g.ListExisting { cl, done := g.Client() @@ -128,18 +126,16 @@ func (g *Stat) Prepare(ctx context.Context) error { } done() } - console.Eraseline() x := "" if g.Versions > 1 { x = fmt.Sprintf(" with %d versions each", g.Versions) } - console.Info("\rUploading ", g.CreateObjects, " objects", x) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects", x)) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() objs := splitObjs(g.CreateObjects, g.Concurrency) - rcv := g.Collector.rcv + rcv := g.Collector.Receiver() var groupErr error var mu sync.Mutex @@ -219,7 +215,7 @@ func (g *Stat) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Stat) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Stat) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -285,7 +281,7 @@ func (g *Stat) Start(ctx context.Context, wait chan struct{}) (Operations, error }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket. diff --git a/pkg/bench/versioned.go b/pkg/bench/versioned.go index ee928d5f..83100599 100644 --- a/pkg/bench/versioned.go +++ b/pkg/bench/versioned.go @@ -28,7 +28,6 @@ import ( "time" "github.com/minio/minio-go/v7" - "github.com/minio/pkg/v3/console" "github.com/minio/warp/pkg/generator" ) @@ -61,11 +60,9 @@ func (g *Versioned) Prepare(ctx context.Context) error { g.Versioned = true } src := g.Source() - console.Eraseline() - console.Info("\rUploading ", g.CreateObjects, " objects of ", src.String()) + g.UpdateStatus(fmt.Sprint("Uploading ", g.CreateObjects, " objects of ", src.String())) var wg sync.WaitGroup wg.Add(g.Concurrency) - g.addCollector() objs := splitObjs(g.CreateObjects, g.Concurrency) var groupErr error @@ -127,7 +124,7 @@ func (g *Versioned) Prepare(ctx context.Context) error { // Start will execute the main benchmark. // Operations should begin executing when the start channel is closed. -func (g *Versioned) Start(ctx context.Context, wait chan struct{}) (Operations, error) { +func (g *Versioned) Start(ctx context.Context, wait chan struct{}) error { var wg sync.WaitGroup wg.Add(g.Concurrency) c := g.Collector @@ -291,7 +288,7 @@ func (g *Versioned) Start(ctx context.Context, wait chan struct{}) (Operations, }(i) } wg.Wait() - return c.Close(), nil + return nil } // Cleanup deletes everything uploaded to the bucket.