From 68cb930c5e014c30c7bc75175539df5b28c0a392 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 18 Jun 2018 14:14:36 +0300 Subject: [PATCH 01/14] Pipe metric samples in real-time and emit them from setup/teardown This fixes https://github.com/loadimpact/k6/issues/614 and helps with https://github.com/loadimpact/k6/issues/670 --- api/v1/setup_teardown_routes.go | 8 ++- cmd/options.go | 2 + core/engine.go | 34 +++++++--- core/local/local.go | 105 ++++++++++++++--------------- js/common/state.go | 6 +- js/modules/k6/http/http_request.go | 26 +++---- js/modules/k6/k6.go | 23 +++---- js/modules/k6/metrics/metrics.go | 4 +- js/modules/k6/ws/ws.go | 28 ++++---- js/runner.go | 45 +++++++------ lib/executor.go | 2 +- lib/options.go | 6 ++ lib/runner.go | 44 ++++++------ 13 files changed, 171 insertions(+), 162 deletions(-) diff --git a/api/v1/setup_teardown_routes.go b/api/v1/setup_teardown_routes.go index 0af4f5f045d7..777efa2fa433 100644 --- a/api/v1/setup_teardown_routes.go +++ b/api/v1/setup_teardown_routes.go @@ -84,9 +84,10 @@ func HandleSetSetupData(rw http.ResponseWriter, r *http.Request, p httprouter.Pa // HandleRunSetup executes the runner's Setup() method and returns the result func HandleRunSetup(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { - runner := common.GetEngine(r.Context()).Executor.GetRunner() + engine := common.GetEngine(r.Context()) + runner := engine.Executor.GetRunner() - if err := runner.Setup(r.Context()); err != nil { + if err := runner.Setup(r.Context(), engine.Samples); err != nil { apiError(rw, "Error executing setup", err.Error(), http.StatusInternalServerError) return } @@ -96,9 +97,10 @@ func HandleRunSetup(rw http.ResponseWriter, r *http.Request, p httprouter.Params // HandleRunTeardown executes the runner's Teardown() method func HandleRunTeardown(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { + engine := common.GetEngine(r.Context()) runner := common.GetEngine(r.Context()).Executor.GetRunner() - if err := runner.Teardown(r.Context()); err != nil { + if err := runner.Teardown(r.Context(), engine.Samples); err != nil { apiError(rw, "Error executing teardown", err.Error(), http.StatusInternalServerError) } } diff --git a/cmd/options.go b/cmd/options.go index d7a7338192d2..b79511a7c205 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -89,6 +89,8 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { // TODO: find a saner and more dev-friendly and error-proof way to handle options SetupTimeout: types.NullDuration{Duration: types.Duration(10 * time.Second), Valid: false}, TeardownTimeout: types.NullDuration{Duration: types.Duration(10 * time.Second), Valid: false}, + + MetricSamplesBufferSize: null.NewInt(1000, false), } stageStrings, err := flags.GetStringSlice("stage") diff --git a/core/engine.go b/core/engine.go index 55d0a5ace1d2..5ad11163941c 100644 --- a/core/engine.go +++ b/core/engine.go @@ -37,7 +37,7 @@ import ( const ( TickRate = 1 * time.Millisecond MetricsRate = 1 * time.Second - CollectRate = 10 * time.Millisecond + CollectRate = 50 * time.Millisecond ThresholdsRate = 2 * time.Second ShutdownTimeout = 10 * time.Second @@ -59,6 +59,8 @@ type Engine struct { Metrics map[string]*stats.Metric MetricsLock sync.Mutex + Samples chan stats.SampleContainer + // Assigned to metrics upon first received sample. thresholds map[string]stats.Thresholds submetrics map[string][]*stats.Submetric @@ -76,6 +78,7 @@ func NewEngine(ex lib.Executor, o lib.Options) (*Engine, error) { Executor: ex, Options: o, Metrics: make(map[string]*stats.Metric), + Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), } e.SetLogger(log.StandardLogger()) @@ -173,11 +176,10 @@ func (e *Engine) Run(ctx context.Context) error { } // Run the executor. - out := make(chan []stats.SampleContainer) errC := make(chan error) subwg.Add(1) go func() { - errC <- e.Executor.Run(subctx, out) + errC <- e.Executor.Run(subctx, e.Samples) e.logger.Debug("Engine: Executor terminated") subwg.Done() }() @@ -194,11 +196,14 @@ func (e *Engine) Run(ctx context.Context) error { errC = nil } subwg.Wait() - close(out) + close(e.Samples) }() - for sampleContainers := range out { - e.processSamples(sampleContainers...) + + sampleContainers := []stats.SampleContainer{} + for sc := range e.Samples { + sampleContainers = append(sampleContainers, sc) } + e.processSamples(sampleContainers) // Emit final metrics. e.emitMetrics() @@ -213,10 +218,17 @@ func (e *Engine) Run(ctx context.Context) error { collectorwg.Wait() }() + ticker := time.NewTicker(CollectRate) + sampleContainers := []stats.SampleContainer{} for { select { - case sampleContainers := <-out: - e.processSamples(sampleContainers...) + case <-ticker.C: + if len(sampleContainers) > 0 { + e.processSamples(sampleContainers) + sampleContainers = []stats.SampleContainer{} + } + case sc := <-e.Samples: + sampleContainers = append(sampleContainers, sc) case err := <-errC: errC = nil if err != nil { @@ -262,7 +274,7 @@ func (e *Engine) runMetricsEmission(ctx context.Context) { func (e *Engine) emitMetrics() { t := time.Now() - e.processSamples(stats.ConnectedSamples{ + e.processSamples([]stats.SampleContainer{stats.ConnectedSamples{ Samples: []stats.Sample{ { Time: t, @@ -278,7 +290,7 @@ func (e *Engine) emitMetrics() { }, Tags: e.Options.RunTags, Time: t, - }) + }}) } func (e *Engine) runThresholds(ctx context.Context, abort func()) { @@ -330,7 +342,7 @@ func (e *Engine) processThresholds(abort func()) { } } -func (e *Engine) processSamples(sampleCointainers ...stats.SampleContainer) { +func (e *Engine) processSamples(sampleCointainers []stats.SampleContainer) { if len(sampleCointainers) == 0 { return } diff --git a/core/local/local.go b/core/local/local.go index cb0431aad577..6285f5cfa7c1 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -36,6 +36,12 @@ import ( null "gopkg.in/guregu/null.v3" ) +// TODO: totally rewrite this! +// This is an overcomplicated and probably buggy piece of code that is a major PITA to refactor... +// It does a ton of stuff in a very convoluted way, has a and uses a very incomprihensible mix +// of all possible Go synchronization mechanisms (channels, mutexes, rwmutexes, atomics, +// and waitgroups) and has a bunch of contexts and tickers on top... + var _ lib.Executor = &Executor{} type vuHandle struct { @@ -45,7 +51,7 @@ type vuHandle struct { cancel context.CancelFunc } -func (h *vuHandle) run(logger *log.Logger, flow <-chan int64, out chan<- []stats.SampleContainer) { +func (h *vuHandle) run(logger *log.Logger, flow <-chan int64, iterDone chan<- struct{}) { h.RLock() ctx := h.ctx h.RUnlock() @@ -60,21 +66,12 @@ func (h *vuHandle) run(logger *log.Logger, flow <-chan int64, out chan<- []stats return } - var samples []stats.SampleContainer if h.vu != nil { - // TODO: Refactor this if we want better aggregation... - // As seen below, at the moment test scripts will emit their - // metric samples only after they've finished executing. This - // means that long-running scripts will emit samples with - // times long in the past, long after their respective time buckets - // have been sent to the cloud. If we instead pass on the out - // SampleContainer channel directly to the VU, we could Collect() - // those samples as soon as they are emitted. - - runSamples, err := h.vu.RunOnce(ctx) + err := h.vu.RunOnce(ctx) if err != nil { select { case <-ctx.Done(): + // Don't log errors from cancelled iterations default: if s, ok := err.(fmt.Stringer); ok { logger.Error(s.String()) @@ -83,9 +80,8 @@ func (h *vuHandle) run(logger *log.Logger, flow <-chan int64, out chan<- []stats } } } - samples = runSamples } - out <- samples + iterDone <- struct{}{} } } @@ -123,8 +119,11 @@ type Executor struct { // Current context, nil if a test isn't running right now. ctx context.Context - // Engineward output channel for samples. - out chan<- []stats.SampleContainer + // Output channel to which VUs send samples. + vuOut chan stats.SampleContainer + + // Channel on which VUs sigal that iterations are completed + iterDone chan struct{} // Flow control for VUs; iterations are run only after reading from this channel. flow chan int64 @@ -138,33 +137,34 @@ func New(r lib.Runner) *Executor { runTeardown: true, endIters: -1, endTime: -1, + vuOut: make(chan stats.SampleContainer, r.GetOptions().MetricSamplesBufferSize.Int64), + iterDone: make(chan struct{}), } } -func (e *Executor) Run(parent context.Context, out chan<- []stats.SampleContainer) (reterr error) { +func (e *Executor) Run(parent context.Context, engineOut chan<- stats.SampleContainer) (reterr error) { e.runLock.Lock() defer e.runLock.Unlock() if e.Runner != nil && e.runSetup { - if err := e.Runner.Setup(parent); err != nil { + if err := e.Runner.Setup(parent, engineOut); err != nil { return err } } ctx, cancel := context.WithCancel(parent) - vuOut := make(chan []stats.SampleContainer) vuFlow := make(chan int64) - e.lock.Lock() + vuOut := e.vuOut + iterDone := e.iterDone e.ctx = ctx - e.out = vuOut e.flow = vuFlow e.lock.Unlock() var cutoff time.Time defer func() { if e.Runner != nil && e.runTeardown { - err := e.Runner.Teardown(parent) + err := e.Runner.Teardown(parent, engineOut) if reterr == nil { reterr = err } else if err != nil { @@ -177,7 +177,7 @@ func (e *Executor) Run(parent context.Context, out chan<- []stats.SampleContaine e.lock.Lock() e.ctx = nil - e.out = nil + e.vuOut = nil e.flow = nil e.lock.Unlock() @@ -187,22 +187,17 @@ func (e *Executor) Run(parent context.Context, out chan<- []stats.SampleContaine close(wait) }() - var samples []stats.SampleContainer for { select { - case newSampleContainers := <-vuOut: + case newSampleContainer := <-vuOut: if cutoff.IsZero() { - samples = append(samples, newSampleContainers...) + engineOut <- newSampleContainer + } else if csc, ok := newSampleContainer.(stats.ConnectedSampleContainer); ok && csc.GetTime().Before(cutoff) { + engineOut <- newSampleContainer } else { - for _, nsc := range newSampleContainers { - if csc, ok := nsc.(stats.ConnectedSampleContainer); ok && csc.GetTime().Before(cutoff) { - samples = append(samples, nsc) - } else if nsc != nil { - for _, s := range nsc.GetSamples() { - if s.Time.Before(cutoff) { - samples = append(samples, s) - } - } + for _, s := range newSampleContainer.GetSamples() { + if s.Time.Before(cutoff) { + engineOut <- s } } } @@ -211,9 +206,6 @@ func (e *Executor) Run(parent context.Context, out chan<- []stats.SampleContaine select { case <-wait: close(vuOut) - if out != nil && len(samples) > 0 { - out <- samples - } return default: } @@ -291,21 +283,20 @@ func (e *Executor) Run(parent context.Context, out chan<- []stats.SampleContaine } } } - case samples := <-vuOut: - // Every iteration ends with a write to vuOut. Check if we've hit the end point. + case sampleContainer := <-vuOut: + engineOut <- sampleContainer + case <-iterDone: + // Every iteration ends with a write to iterDone. Check if we've hit the end point. // If not, make sure to include an Iterations bump in the list! - if out != nil { - var tags *stats.SampleTags - if e.Runner != nil { - tags = e.Runner.GetOptions().RunTags - } - samples = append(samples, stats.Sample{ - Time: time.Now(), - Metric: metrics.Iterations, - Value: 1, - Tags: tags, - }) - out <- samples + var tags *stats.SampleTags + if e.Runner != nil { + tags = e.Runner.GetOptions().RunTags + } + engineOut <- stats.Sample{ + Time: time.Now(), + Metric: metrics.Iterations, + Value: 1, + Tags: tags, } end := atomic.LoadInt64(&e.endIters) @@ -332,7 +323,7 @@ func (e *Executor) scale(ctx context.Context, num int64) error { e.lock.RLock() flow := e.flow - out := e.out + iterDone := e.iterDone e.lock.RUnlock() for i, handle := range e.vus { @@ -357,7 +348,7 @@ func (e *Executor) scale(ctx context.Context, num int64) error { e.wg.Add(1) go func() { - handle.run(e.Logger, flow, out) + handle.run(e.Logger, flow, iterDone) e.wg.Done() }() } @@ -514,6 +505,10 @@ func (e *Executor) SetVUsMax(max int64) error { return nil } + e.lock.RLock() + vuOut := e.vuOut + e.lock.RUnlock() + e.vusLock.Lock() defer e.vusLock.Unlock() @@ -521,7 +516,7 @@ func (e *Executor) SetVUsMax(max int64) error { for i := numVUsMax; i < max; i++ { var handle vuHandle if e.Runner != nil { - vu, err := e.Runner.NewVU() + vu, err := e.Runner.NewVU(vuOut) if err != nil { return err } diff --git a/js/common/state.go b/js/common/state.go index 4cf65ac14fb3..358b4f4f8ee0 100644 --- a/js/common/state.go +++ b/js/common/state.go @@ -33,7 +33,7 @@ import ( "golang.org/x/time/rate" ) -// Provides volatile state for a VU. +// State provides the volatile state for a VU. type State struct { // Global options. Options lib.Options @@ -53,8 +53,8 @@ type State struct { // Rate limits. RPSLimit *rate.Limiter - // Sample buffer, emitted at the end of the iteration. - Samples []stats.SampleContainer + // Sample channel, possibly buffered + Samples chan<- stats.SampleContainer // Buffer pool; use instead of allocating fresh buffers when possible. BPool *bpool.BufferPool diff --git a/js/modules/k6/http/http_request.go b/js/modules/k6/http/http_request.go index c358603b9675..113230a08be1 100644 --- a/js/modules/k6/http/http_request.go +++ b/js/modules/k6/http/http_request.go @@ -110,10 +110,7 @@ func (h *HTTP) Request(ctx context.Context, method string, url goja.Value, args return nil, err } - state := common.GetState(ctx) - res, samples, err := h.request(ctx, state, req) - state.Samples = append(state.Samples, samples...) - return res, err + return h.request(ctx, req) } type parsedHTTPRequest struct { @@ -346,7 +343,8 @@ func (h *HTTP) parseRequest(ctx context.Context, method string, reqURL URL, body // request() shouldn't mess with the goja runtime or other thread-unsafe // things because it's called concurrently by Batch() -func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTTPRequest) (*HTTPResponse, []stats.SampleContainer, error) { +func (h *HTTP) request(ctx context.Context, preq *parsedHTTPRequest) (*HTTPResponse, error) { + state := common.GetState(ctx) respReq := &HTTPRequest{ Method: preq.req.Method, @@ -387,7 +385,7 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT // Check rate limit *after* we've prepared a request; no need to wait with that part. if rpsLimit := state.RPSLimit; rpsLimit != nil { if err := rpsLimit.Wait(ctx); err != nil { - return nil, nil, err + return nil, err } } @@ -424,7 +422,6 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT }, } - statsSamples := []stats.SampleContainer{} // if digest authentication option is passed, make an initial request to get the authentication params to compute the authorization header if preq.auth == "digest" { username := preq.url.URL.User.Username() @@ -446,11 +443,11 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT } if preq.throw { - return nil, nil, err + return nil, err } resp.Error = err.Error() - return resp, statsSamples, nil + return resp, nil } if res.StatusCode == http.StatusUnauthorized { @@ -473,7 +470,7 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT } trail.SaveSamples(stats.NewSampleTags(tags)) delete(tags, "ip") - statsSamples = append(statsSamples, trail) + state.Samples <- trail } if preq.auth == "ntlm" { @@ -592,7 +589,7 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT } if preq.throw { - return nil, nil, resErr + return nil, resErr } } @@ -602,8 +599,8 @@ func (h *HTTP) request(ctx context.Context, state *common.State, preq *parsedHTT } } trail.SaveSamples(stats.IntoSampleTags(&tags)) - statsSamples = append(statsSamples, trail) - return resp, statsSamples, nil + state.Samples <- trail + return resp, nil } func (h *HTTP) Batch(ctx context.Context, reqsV goja.Value) (goja.Value, error) { @@ -704,7 +701,7 @@ func (h *HTTP) Batch(ctx context.Context, reqsV goja.Value) (goja.Value, error) defer hl.End() } - res, samples, err := h.request(ctx, state, parsedReq) + res, err := h.request(ctx, parsedReq) if err != nil { errs <- err return @@ -712,7 +709,6 @@ func (h *HTTP) Batch(ctx context.Context, reqsV goja.Value) (goja.Value, error) mutex.Lock() _ = retval.Set(key, res) - state.Samples = append(state.Samples, samples...) mutex.Unlock() errs <- nil diff --git a/js/modules/k6/k6.go b/js/modules/k6/k6.go index 437d7454a0ae..a385ce697426 100644 --- a/js/modules/k6/k6.go +++ b/js/modules/k6/k6.go @@ -79,14 +79,12 @@ func (*K6) Group(ctx context.Context, name string, fn goja.Callable) (goja.Value tags["iter"] = strconv.FormatInt(state.Iteration, 10) } - state.Samples = append(state.Samples, - stats.Sample{ - Time: t, - Metric: metrics.GroupDuration, - Tags: stats.IntoSampleTags(&tags), - Value: stats.D(t.Sub(startTime)), - }, - ) + state.Samples <- stats.Sample{ + Time: t, + Metric: metrics.GroupDuration, + Tags: stats.IntoSampleTags(&tags), + Value: stats.D(t.Sub(startTime)), + } return ret, err } @@ -150,15 +148,10 @@ func (*K6) Check(ctx context.Context, arg0, checks goja.Value, extras ...goja.Va default: if val.ToBoolean() { atomic.AddInt64(&check.Passes, 1) - state.Samples = append(state.Samples, - stats.Sample{Time: t, Metric: metrics.Checks, Tags: sampleTags, Value: 1}, - ) + state.Samples <- stats.Sample{Time: t, Metric: metrics.Checks, Tags: sampleTags, Value: 1} } else { atomic.AddInt64(&check.Fails, 1) - state.Samples = append(state.Samples, - stats.Sample{Time: t, Metric: metrics.Checks, Tags: sampleTags, Value: 0}, - ) - + state.Samples <- stats.Sample{Time: t, Metric: metrics.Checks, Tags: sampleTags, Value: 0} // A single failure makes the return value false. succ = false } diff --git a/js/modules/k6/metrics/metrics.go b/js/modules/k6/metrics/metrics.go index c42006de3c58..8b77a51f22bb 100644 --- a/js/modules/k6/metrics/metrics.go +++ b/js/modules/k6/metrics/metrics.go @@ -67,9 +67,7 @@ func (m Metric) Add(ctx context.Context, v goja.Value, addTags ...map[string]str vfloat = 1.0 } - state.Samples = append(state.Samples, - stats.Sample{Time: time.Now(), Metric: m.metric, Value: vfloat, Tags: stats.IntoSampleTags(&tags)}, - ) + state.Samples <- stats.Sample{Time: time.Now(), Metric: m.metric, Value: vfloat, Tags: stats.IntoSampleTags(&tags)} } type Metrics struct{} diff --git a/js/modules/k6/ws/ws.go b/js/modules/k6/ws/ws.go index f269962b5476..62c3d0087c4f 100644 --- a/js/modules/k6/ws/ws.go +++ b/js/modules/k6/ws/ws.go @@ -275,45 +275,41 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP sampleTags := stats.IntoSampleTags(&tags) - samples := []stats.SampleContainer{ - stats.ConnectedSamples{ - []stats.Sample{ - {Metric: metrics.WSSessions, Time: start, Tags: sampleTags, Value: 1}, - {Metric: metrics.WSConnecting, Time: start, Tags: sampleTags, Value: connectionDuration}, - {Metric: metrics.WSSessionDuration, Time: start, Tags: sampleTags, Value: sessionDuration}, - }, sampleTags, start, - }, + state.Samples <- stats.ConnectedSamples{ + []stats.Sample{ + {Metric: metrics.WSSessions, Time: start, Tags: sampleTags, Value: 1}, + {Metric: metrics.WSConnecting, Time: start, Tags: sampleTags, Value: connectionDuration}, + {Metric: metrics.WSSessionDuration, Time: start, Tags: sampleTags, Value: sessionDuration}, + }, sampleTags, start, } for _, msgSentTimestamp := range socket.msgSentTimestamps { - samples = append(samples, stats.Sample{ + state.Samples <- stats.Sample{ Metric: metrics.WSMessagesSent, Time: msgSentTimestamp, Tags: sampleTags, Value: 1, - }) + } } for _, msgReceivedTimestamp := range socket.msgReceivedTimestamps { - samples = append(samples, stats.Sample{ + state.Samples <- stats.Sample{ Metric: metrics.WSMessagesReceived, Time: msgReceivedTimestamp, Tags: sampleTags, Value: 1, - }) + } } for _, pingDelta := range socket.pingTimestamps { - samples = append(samples, stats.Sample{ + state.Samples <- stats.Sample{ Metric: metrics.WSPing, Time: pingDelta.pong, Tags: sampleTags, Value: stats.D(pingDelta.pong.Sub(pingDelta.ping)), - }) + } } - state.Samples = append(state.Samples, samples...) - return wsResponse, nil } } diff --git a/js/runner.go b/js/runner.go index cc27a4ba073c..0d3073f74949 100644 --- a/js/runner.go +++ b/js/runner.go @@ -102,15 +102,15 @@ func (r *Runner) MakeArchive() *lib.Archive { return r.Bundle.MakeArchive() } -func (r *Runner) NewVU() (lib.VU, error) { - vu, err := r.newVU() +func (r *Runner) NewVU(samplesOut chan<- stats.SampleContainer) (lib.VU, error) { + vu, err := r.newVU(samplesOut) if err != nil { return nil, err } return lib.VU(vu), nil } -func (r *Runner) newVU() (*VU, error) { +func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { // Instantiate a new bundle, make a VU out of it. bi, err := r.Bundle.Instantiate() if err != nil { @@ -172,6 +172,7 @@ func (r *Runner) newVU() (*VU, error) { TLSConfig: tlsConfig, Console: NewConsole(), BPool: bpool.NewBufferPool(100), + Samples: samplesOut, } vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) common.BindToGlobal(vu.Runtime, map[string]interface{}{ @@ -188,14 +189,14 @@ func (r *Runner) newVU() (*VU, error) { return vu, nil } -func (r *Runner) Setup(ctx context.Context) error { +func (r *Runner) Setup(ctx context.Context, out chan<- stats.SampleContainer) error { setupCtx, setupCancel := context.WithTimeout( ctx, time.Duration(r.Bundle.Options.SetupTimeout.Duration), ) defer setupCancel() - v, err := r.runPart(setupCtx, "setup", nil) + v, err := r.runPart(setupCtx, out, "setup", nil) if err != nil { return errors.Wrap(err, "setup") } @@ -216,14 +217,14 @@ func (r *Runner) SetSetupData(data interface{}) { r.setupData = data } -func (r *Runner) Teardown(ctx context.Context) error { +func (r *Runner) Teardown(ctx context.Context, out chan<- stats.SampleContainer) error { teardownCtx, teardownCancel := context.WithTimeout( ctx, time.Duration(r.Bundle.Options.TeardownTimeout.Duration), ) defer teardownCancel() - _, err := r.runPart(teardownCtx, "teardown", r.setupData) + _, err := r.runPart(teardownCtx, out, "teardown", r.setupData) return err } @@ -246,8 +247,8 @@ func (r *Runner) SetOptions(opts lib.Options) { // Runs an exported function in its own temporary VU, optionally with an argument. Execution is // interrupted if the context expires. No error is returned if the part does not exist. -func (r *Runner) runPart(ctx context.Context, name string, arg interface{}) (goja.Value, error) { - vu, err := r.newVU() +func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, name string, arg interface{}) (goja.Value, error) { + vu, err := r.newVU(out) if err != nil { return goja.Undefined(), err } @@ -265,7 +266,13 @@ func (r *Runner) runPart(ctx context.Context, name string, arg interface{}) (goj <-ctx.Done() vu.Runtime.Interrupt(errInterrupt) }() - v, _, err := vu.runFn(ctx, fn, vu.Runtime.ToValue(arg)) + + group, err := lib.NewGroup(name, r.GetDefaultGroup()) + if err != nil { + return goja.Undefined(), err + } + + v, _, err := vu.runFn(ctx, group, fn, vu.Runtime.ToValue(arg)) cancel() return v, err } @@ -283,6 +290,8 @@ type VU struct { Console *Console BPool *bpool.BufferPool + Samples chan<- stats.SampleContainer + setupData goja.Value // A VU will track the last context it was called with for cancellation. @@ -306,7 +315,7 @@ func (u *VU) Reconfigure(id int64) error { return nil } -func (u *VU) RunOnce(ctx context.Context) ([]stats.SampleContainer, error) { +func (u *VU) RunOnce(ctx context.Context) error { // Track the context and interrupt JS execution if it's cancelled. if u.interruptTrackedCtx != ctx { interCtx, interCancel := context.WithCancel(context.Background()) @@ -333,14 +342,11 @@ func (u *VU) RunOnce(ctx context.Context) ([]stats.SampleContainer, error) { } // Call the default function. - _, state, err := u.runFn(ctx, u.Default, u.setupData) - if err != nil { - return nil, err - } - return state.Samples, nil + _, _, err := u.runFn(ctx, u.Runner.defaultGroup, u.Default, u.setupData) + return err } -func (u *VU) runFn(ctx context.Context, fn goja.Callable, args ...goja.Value) (goja.Value, *common.State, error) { +func (u *VU) runFn(ctx context.Context, group *lib.Group, fn goja.Callable, args ...goja.Value) (goja.Value, *common.State, error) { cookieJar, err := cookiejar.New(nil) if err != nil { return goja.Undefined(), nil, err @@ -349,7 +355,7 @@ func (u *VU) runFn(ctx context.Context, fn goja.Callable, args ...goja.Value) (g state := &common.State{ Logger: u.Runner.Logger, Options: u.Runner.Bundle.Options, - Group: u.Runner.defaultGroup, + Group: group, HTTPTransport: u.HTTPTransport, Dialer: u.Dialer, TLSConfig: u.TLSConfig, @@ -357,6 +363,7 @@ func (u *VU) runFn(ctx context.Context, fn goja.Callable, args ...goja.Value) (g RPSLimit: u.Runner.RPSLimit, BPool: u.BPool, Vu: u.ID, + Samples: u.Samples, Iteration: u.Iteration, } @@ -385,7 +392,7 @@ func (u *VU) runFn(ctx context.Context, fn goja.Callable, args ...goja.Value) (g u.HTTPTransport.CloseIdleConnections() } - state.Samples = append(state.Samples, u.Dialer.GetTrail(startTime, endTime, sampleTags)) + state.Samples <- u.Dialer.GetTrail(startTime, endTime, sampleTags) return v, state, err } diff --git a/lib/executor.go b/lib/executor.go index 258ccbe380ab..9c955189d002 100644 --- a/lib/executor.go +++ b/lib/executor.go @@ -37,7 +37,7 @@ import ( // implemented to control a test running on a cluster or in the cloud. type Executor interface { // Run the Executor, funneling generated samples through the out channel. - Run(ctx context.Context, out chan<- []stats.SampleContainer) error + Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error // Is the executor currently running? IsRunning() bool diff --git a/lib/options.go b/lib/options.go index d29a24adc7a9..9493b02ab922 100644 --- a/lib/options.go +++ b/lib/options.go @@ -254,6 +254,9 @@ type Options struct { // Tags to be applied to all samples for this running RunTags *stats.SampleTags `json:"tags" envconfig:"tags"` + + // Buffer size of the channel for metric samples; 0 means unbuffered + MetricSamplesBufferSize null.Int `json:"metricSamplesBufferSize" envconfig:"metric_samples_buffer_size"` } // Returns the result of overwriting any fields with any that are set on the argument. @@ -351,5 +354,8 @@ func (o Options) Apply(opts Options) Options { if !opts.RunTags.IsEmpty() { o.RunTags = opts.RunTags } + if opts.MetricSamplesBufferSize.Valid { + o.MetricSamplesBufferSize = opts.MetricSamplesBufferSize + } return o } diff --git a/lib/runner.go b/lib/runner.go index c59689f880ec..4f75e2bf2d0b 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -26,8 +26,9 @@ import ( "github.com/loadimpact/k6/stats" ) -// Ensure MiniRunner conforms to Runner. +// Ensure mock implementations conform to the interfaces. var _ Runner = &MiniRunner{} +var _ VU = &MiniRunnerVU{} // A Runner is a factory for VUs. It should precompute as much as possible upon creation (parse // ASTs, load files into memory, etc.), so that spawning VUs becomes as fast as possible. @@ -43,10 +44,10 @@ type Runner interface { // Spawns a new VU. It's fine to make this function rather heavy, if it means a performance // improvement at runtime. Remember, this is called once per VU and normally only at the start // of a test - RunOnce() may be called hundreds of thousands of times, and must be fast. - NewVU() (VU, error) + NewVU(out chan<- stats.SampleContainer) (VU, error) // Runs pre-test setup, if applicable. - Setup(ctx context.Context) error + Setup(ctx context.Context, out chan<- stats.SampleContainer) error // Returns the setup data if setup() is specified and run, nil otherwise GetSetupData() interface{} @@ -55,7 +56,7 @@ type Runner interface { SetSetupData(interface{}) // Runs post-test teardown, if applicable. - Teardown(ctx context.Context) error + Teardown(ctx context.Context, out chan<- stats.SampleContainer) error // Returns the default (root) Group. GetDefaultGroup() *Group @@ -71,7 +72,7 @@ type Runner interface { type VU interface { // Runs the VU once. The VU is responsible for handling the Halting Problem, eg. making sure // that execution actually stops when the context is cancelled. - RunOnce(ctx context.Context) ([]stats.SampleContainer, error) + RunOnce(ctx context.Context) error // Assign the VU a new ID. Called by the Executor upon creation, but may be called multiple // times if the VU is recycled because the test was scaled down and then back up. @@ -80,9 +81,9 @@ type VU interface { // MiniRunner wraps a function in a runner whose VUs will simply call that function. type MiniRunner struct { - Fn func(ctx context.Context) ([]stats.SampleContainer, error) - SetupFn func(ctx context.Context) (interface{}, error) - TeardownFn func(ctx context.Context) error + Fn func(ctx context.Context, out chan<- stats.SampleContainer) error + SetupFn func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) + TeardownFn func(ctx context.Context, out chan<- stats.SampleContainer) error setupData interface{} @@ -90,21 +91,21 @@ type MiniRunner struct { Options Options } -func (r MiniRunner) VU() *MiniRunnerVU { - return &MiniRunnerVU{R: r} +func (r MiniRunner) VU(out chan<- stats.SampleContainer) *MiniRunnerVU { + return &MiniRunnerVU{R: r, Out: out} } func (r MiniRunner) MakeArchive() *Archive { return nil } -func (r MiniRunner) NewVU() (VU, error) { - return r.VU(), nil +func (r MiniRunner) NewVU(out chan<- stats.SampleContainer) (VU, error) { + return r.VU(out), nil } -func (r *MiniRunner) Setup(ctx context.Context) (err error) { +func (r *MiniRunner) Setup(ctx context.Context, out chan<- stats.SampleContainer) (err error) { if fn := r.SetupFn; fn != nil { - r.setupData, err = fn(ctx) + r.setupData, err = fn(ctx, out) } return } @@ -119,9 +120,9 @@ func (r *MiniRunner) SetSetupData(data interface{}) { r.setupData = data } -func (r MiniRunner) Teardown(ctx context.Context) error { +func (r MiniRunner) Teardown(ctx context.Context, out chan<- stats.SampleContainer) error { if fn := r.TeardownFn; fn != nil { - return fn(ctx) + return fn(ctx, out) } return nil } @@ -143,15 +144,16 @@ func (r *MiniRunner) SetOptions(opts Options) { // A VU spawned by a MiniRunner. type MiniRunnerVU struct { - R MiniRunner - ID int64 + R MiniRunner + Out chan<- stats.SampleContainer + ID int64 } -func (vu MiniRunnerVU) RunOnce(ctx context.Context) ([]stats.SampleContainer, error) { +func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { if vu.R.Fn == nil { - return []stats.SampleContainer{}, nil + return nil } - return vu.R.Fn(ctx) + return vu.R.Fn(ctx, vu.Out) } func (vu *MiniRunnerVU) Reconfigure(id int64) error { From 44328673ffd146985f31d10f99a18201bf26287d Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 19 Jun 2018 10:35:31 +0300 Subject: [PATCH 02/14] Fix deadlock on test cancellation (ex. via Ctrl+C) --- core/local/local.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/local/local.go b/core/local/local.go index 6285f5cfa7c1..dd361e13ddde 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -189,6 +189,8 @@ func (e *Executor) Run(parent context.Context, engineOut chan<- stats.SampleCont for { select { + case <-iterDone: + // Spool through all remaining iterations, do not emit stats since the Run() is over case newSampleContainer := <-vuOut: if cutoff.IsZero() { engineOut <- newSampleContainer From d2a7cb9c5c53c04ee9c5e4b4d576bce95dab3af0 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 20 Jun 2018 11:34:49 +0300 Subject: [PATCH 03/14] Fix a bug that caused the last sample slice to be discarded --- core/engine.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/engine.go b/core/engine.go index 5ad11163941c..a3441a42cc23 100644 --- a/core/engine.go +++ b/core/engine.go @@ -184,6 +184,7 @@ func (e *Engine) Run(ctx context.Context) error { subwg.Done() }() + sampleContainers := []stats.SampleContainer{} defer func() { // Shut down subsystems. subcancel() @@ -199,11 +200,12 @@ func (e *Engine) Run(ctx context.Context) error { close(e.Samples) }() - sampleContainers := []stats.SampleContainer{} for sc := range e.Samples { sampleContainers = append(sampleContainers, sc) } - e.processSamples(sampleContainers) + if len(sampleContainers) > 0 { + e.processSamples(sampleContainers) + } // Emit final metrics. e.emitMetrics() @@ -219,7 +221,6 @@ func (e *Engine) Run(ctx context.Context) error { }() ticker := time.NewTicker(CollectRate) - sampleContainers := []stats.SampleContainer{} for { select { case <-ticker.C: From e138874f1fb7b4dceefc3e1b4bedbb3f529d1231 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 20 Jun 2018 14:21:47 +0300 Subject: [PATCH 04/14] Fix all tests to use the new samples API --- api/v1/setup_teardown_routes_test.go | 8 +- core/engine_test.go | 34 ++--- core/local/local.go | 7 +- core/local/local_test.go | 105 ++++++++------ js/console_test.go | 6 +- js/initcontext_test.go | 4 +- js/modules/k6/http/http_request_test.go | 182 ++++++++++-------------- js/modules/k6/http/response_test.go | 38 ++--- js/modules/k6/k6_test.go | 53 ++++--- js/modules/k6/metrics/metrics_test.go | 14 +- js/modules/k6/ws/ws_test.go | 61 ++++---- js/runner_test.go | 86 +++++------ stats/stats.go | 13 ++ 13 files changed, 321 insertions(+), 290 deletions(-) diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index fbae3a192bd4..3479b00e6cb2 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -79,6 +79,10 @@ func TestSetupData(t *testing.T) { engine, err := core.NewEngine(executor, runner.GetOptions()) require.NoError(t, err) + ctx, cancel := context.WithCancel(context.Background()) + errC := make(chan error) + go func() { errC <- engine.Run(ctx) }() + handler := NewHandler() checkSetup := func(method, body, expResult string) { @@ -102,10 +106,6 @@ func TestSetupData(t *testing.T) { checkSetup("PUT", `{"v":2, "test":"mest"}`, `{"data": {"v":2, "test":"mest"}}`) checkSetup("GET", "", `{"data": {"v":2, "test":"mest"}}`) - ctx, cancel := context.WithCancel(context.Background()) - errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() - engine.Executor.SetPaused(false) select { diff --git a/core/engine_test.go b/core/engine_test.go index 1a8893b63bb5..c461994a57ee 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -56,6 +56,9 @@ func applyNullLogger(e *Engine) *logtest.Hook { // Wrapper around newEngine that applies a null logger. func newTestEngine(ex lib.Executor, opts lib.Options) (*Engine, error, *logtest.Hook) { + if !opts.MetricSamplesBufferSize.Valid { + opts.MetricSamplesBufferSize = null.IntFrom(200) + } e, err := NewEngine(ex, opts) if err != nil { return e, err, nil @@ -64,12 +67,8 @@ func newTestEngine(ex lib.Executor, opts lib.Options) (*Engine, error, *logtest. return e, nil, hook } -func L(r lib.Runner) lib.Executor { - return local.New(r) -} - -func LF(fn func(ctx context.Context) ([]stats.SampleContainer, error)) lib.Executor { - return L(&lib.MiniRunner{Fn: fn}) +func LF(fn func(ctx context.Context, out chan<- stats.SampleContainer) error) lib.Executor { + return local.New(&lib.MiniRunner{Fn: fn}) } func TestNewEngine(t *testing.T) { @@ -250,8 +249,8 @@ func TestEngineRun(t *testing.T) { signalChan := make(chan interface{}) var e *Engine - e, err, _ := newTestEngine(LF(func(ctx context.Context) (samples []stats.SampleContainer, err error) { - samples = append(samples, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1}) + e, err, _ := newTestEngine(LF(func(ctx context.Context, samples chan<- stats.SampleContainer) error { + samples <- stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1} close(signalChan) <-ctx.Done() @@ -263,8 +262,8 @@ func TestEngineRun(t *testing.T) { // 2. Sometimes the `case samples := <-vuOut` gets selected before the `<-ctx.Done()` in // core/local/local.go:Run() causing all samples from this mocked "RunOnce()" function to be accepted. time.Sleep(time.Millisecond * 10) - samples = append(samples, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 2}) - return samples, err + samples <- stats.Sample{Metric: testMetric, Time: time.Now(), Value: 2} + return nil }), lib.Options{ VUs: null.IntFrom(1), VUsMax: null.IntFrom(1), @@ -308,8 +307,9 @@ func TestEngineAtTime(t *testing.T) { func TestEngineCollector(t *testing.T) { testMetric := stats.New("test_metric", stats.Trend) - e, err, _ := newTestEngine(LF(func(ctx context.Context) ([]stats.SampleContainer, error) { - return []stats.SampleContainer{stats.Sample{Metric: testMetric}}, nil + e, err, _ := newTestEngine(LF(func(ctx context.Context, out chan<- stats.SampleContainer) error { + out <- stats.Sample{Metric: testMetric} + return nil }), lib.Options{VUs: null.IntFrom(1), VUsMax: null.IntFrom(1), Iterations: null.IntFrom(1)}) assert.NoError(t, err) @@ -343,7 +343,7 @@ func TestEngine_processSamples(t *testing.T) { assert.NoError(t, err) e.processSamples( - stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}, + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, ) assert.IsType(t, &stats.GaugeSink{}, e.Metrics["my_metric"].Sink) @@ -365,7 +365,7 @@ func TestEngine_processSamples(t *testing.T) { assert.EqualValues(t, map[string]string{"a": "1"}, sms[0].Tags.CloneTags()) e.processSamples( - stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1", "b": "2"})}, + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1", "b": "2"})}}, ) assert.IsType(t, &stats.GaugeSink{}, e.Metrics["my_metric"].Sink) @@ -387,7 +387,7 @@ func TestEngine_runThresholds(t *testing.T) { assert.NoError(t, err) e.processSamples( - stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}, + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, ) ctx, cancel := context.WithCancel(context.Background()) @@ -410,7 +410,7 @@ func TestEngine_runThresholds(t *testing.T) { assert.NoError(t, err) e.processSamples( - stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}, + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, ) ctx, cancel := context.WithCancel(context.Background()) @@ -463,7 +463,7 @@ func TestEngine_processThresholds(t *testing.T) { assert.NoError(t, err) e.processSamples( - stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}, + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, ) abortCalled := false diff --git a/core/local/local.go b/core/local/local.go index dd361e13ddde..911212587222 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -130,6 +130,11 @@ type Executor struct { } func New(r lib.Runner) *Executor { + var bufferSize int64 + if r != nil { + bufferSize = r.GetOptions().MetricSamplesBufferSize.Int64 + } + return &Executor{ Runner: r, Logger: log.StandardLogger(), @@ -137,7 +142,7 @@ func New(r lib.Runner) *Executor { runTeardown: true, endIters: -1, endTime: -1, - vuOut: make(chan stats.SampleContainer, r.GetOptions().MetricSamplesBufferSize.Int64), + vuOut: make(chan stats.SampleContainer, bufferSize), iterDone: make(chan struct{}), } } diff --git a/core/local/local_test.go b/core/local/local_test.go index 2567a89802a3..9475f66a9fd6 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -54,11 +54,11 @@ func TestExecutorSetupTeardownRun(t *testing.T) { setupC := make(chan struct{}) teardownC := make(chan struct{}) e := New(&lib.MiniRunner{ - SetupFn: func(ctx context.Context) (interface{}, error) { + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) { close(setupC) return nil, nil }, - TeardownFn: func(ctx context.Context) error { + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { close(teardownC) return nil }, @@ -66,7 +66,7 @@ func TestExecutorSetupTeardownRun(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) err := make(chan error, 1) - go func() { err <- e.Run(ctx, nil) }() + go func() { err <- e.Run(ctx, make(chan stats.SampleContainer, 100)) }() cancel() <-setupC <-teardownC @@ -74,21 +74,21 @@ func TestExecutorSetupTeardownRun(t *testing.T) { }) t.Run("Setup Error", func(t *testing.T) { e := New(&lib.MiniRunner{ - SetupFn: func(ctx context.Context) (interface{}, error) { + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) { return nil, errors.New("setup error") }, - TeardownFn: func(ctx context.Context) error { + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, }) - assert.EqualError(t, e.Run(context.Background(), nil), "setup error") + assert.EqualError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100)), "setup error") t.Run("Don't Run Setup", func(t *testing.T) { e := New(&lib.MiniRunner{ - SetupFn: func(ctx context.Context) (interface{}, error) { + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) { return nil, errors.New("setup error") }, - TeardownFn: func(ctx context.Context) error { + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, }) @@ -96,29 +96,29 @@ func TestExecutorSetupTeardownRun(t *testing.T) { e.SetEndIterations(null.IntFrom(1)) assert.NoError(t, e.SetVUsMax(1)) assert.NoError(t, e.SetVUs(1)) - assert.EqualError(t, e.Run(context.Background(), nil), "teardown error") + assert.EqualError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100)), "teardown error") }) }) t.Run("Teardown Error", func(t *testing.T) { e := New(&lib.MiniRunner{ - SetupFn: func(ctx context.Context) (interface{}, error) { + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) { return nil, nil }, - TeardownFn: func(ctx context.Context) error { + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, }) e.SetEndIterations(null.IntFrom(1)) assert.NoError(t, e.SetVUsMax(1)) assert.NoError(t, e.SetVUs(1)) - assert.EqualError(t, e.Run(context.Background(), nil), "teardown error") + assert.EqualError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100)), "teardown error") t.Run("Don't Run Teardown", func(t *testing.T) { e := New(&lib.MiniRunner{ - SetupFn: func(ctx context.Context) (interface{}, error) { + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) (interface{}, error) { return nil, nil }, - TeardownFn: func(ctx context.Context) error { + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, }) @@ -126,7 +126,7 @@ func TestExecutorSetupTeardownRun(t *testing.T) { e.SetEndIterations(null.IntFrom(1)) assert.NoError(t, e.SetVUsMax(1)) assert.NoError(t, e.SetVUs(1)) - assert.NoError(t, e.Run(context.Background(), nil)) + assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100))) }) }) } @@ -164,30 +164,48 @@ func TestExecutorStages(t *testing.T) { } for name, data := range testdata { t.Run(name, func(t *testing.T) { - e := New(nil) + e := New(&lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(100 * time.Millisecond) + return nil + }, + Options: lib.Options{ + MetricSamplesBufferSize: null.IntFrom(500), + }, + }) assert.NoError(t, e.SetVUsMax(10)) e.SetStages(data.Stages) - assert.NoError(t, e.Run(context.Background(), nil)) + assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 500))) assert.True(t, e.GetTime() >= data.Duration) }) } } func TestExecutorEndTime(t *testing.T) { - e := New(nil) + e := New(&lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(100 * time.Millisecond) + return nil + }, + Options: lib.Options{MetricSamplesBufferSize: null.IntFrom(200)}, + }) assert.NoError(t, e.SetVUsMax(10)) assert.NoError(t, e.SetVUs(10)) e.SetEndTime(types.NullDurationFrom(1 * time.Second)) assert.Equal(t, types.NullDurationFrom(1*time.Second), e.GetEndTime()) startTime := time.Now() - assert.NoError(t, e.Run(context.Background(), nil)) + assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 200))) assert.True(t, time.Now().After(startTime.Add(1*time.Second)), "test did not take 1s") t.Run("Runtime Errors", func(t *testing.T) { - e := New(&lib.MiniRunner{Fn: func(ctx context.Context) ([]stats.SampleContainer, error) { - return nil, errors.New("hi") - }}) + e := New(&lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(10 * time.Millisecond) + return errors.New("hi") + }, + Options: lib.Options{MetricSamplesBufferSize: null.IntFrom(200)}, + }) assert.NoError(t, e.SetVUsMax(10)) assert.NoError(t, e.SetVUs(10)) e.SetEndTime(types.NullDurationFrom(100 * time.Millisecond)) @@ -197,7 +215,7 @@ func TestExecutorEndTime(t *testing.T) { e.SetLogger(l) startTime := time.Now() - assert.NoError(t, e.Run(context.Background(), nil)) + assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 200))) assert.True(t, time.Now().After(startTime.Add(100*time.Millisecond)), "test did not take 100ms") assert.NotEmpty(t, hook.Entries) @@ -207,10 +225,13 @@ func TestExecutorEndTime(t *testing.T) { }) t.Run("End Errors", func(t *testing.T) { - e := New(&lib.MiniRunner{Fn: func(ctx context.Context) ([]stats.SampleContainer, error) { - <-ctx.Done() - return nil, errors.New("hi") - }}) + e := New(&lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + <-ctx.Done() + return errors.New("hi") + }, + Options: lib.Options{MetricSamplesBufferSize: null.IntFrom(200)}, + }) assert.NoError(t, e.SetVUsMax(10)) assert.NoError(t, e.SetVUs(10)) e.SetEndTime(types.NullDurationFrom(100 * time.Millisecond)) @@ -220,7 +241,7 @@ func TestExecutorEndTime(t *testing.T) { e.SetLogger(l) startTime := time.Now() - assert.NoError(t, e.Run(context.Background(), nil)) + assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 200))) assert.True(t, time.Now().After(startTime.Add(100*time.Millisecond)), "test did not take 100ms") assert.Empty(t, hook.Entries) @@ -231,32 +252,34 @@ func TestExecutorEndIterations(t *testing.T) { metric := &stats.Metric{Name: "test_metric"} var i int64 - e := New(&lib.MiniRunner{Fn: func(ctx context.Context) ([]stats.SampleContainer, error) { + e := New(&lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { select { case <-ctx.Done(): default: atomic.AddInt64(&i, 1) } - return []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.0}}, nil + out <- stats.Sample{Metric: metric, Value: 1.0} + return nil }}) assert.NoError(t, e.SetVUsMax(1)) assert.NoError(t, e.SetVUs(1)) e.SetEndIterations(null.IntFrom(100)) assert.Equal(t, null.IntFrom(100), e.GetEndIterations()) - samples := make(chan []stats.SampleContainer, 101) + samples := make(chan stats.SampleContainer, 201) assert.NoError(t, e.Run(context.Background(), samples)) assert.Equal(t, int64(100), e.GetIterations()) assert.Equal(t, int64(100), i) for i := 0; i < 100; i++ { - samples := <-samples - if assert.Len(t, samples, 2) { - assert.Equal(t, stats.Sample{Metric: metric, Value: 1.0}, samples[0]) - sample, ok := (samples[1]).(stats.Sample) - require.True(t, ok) - assert.Equal(t, metrics.Iterations, sample.Metric) - assert.Equal(t, float64(1), sample.Value) - } + mySample, ok := <-samples + require.True(t, ok) + assert.Equal(t, stats.Sample{Metric: metric, Value: 1.0}, mySample) + sample, ok := <-samples + require.True(t, ok) + iterSample, ok := (sample).(stats.Sample) + require.True(t, ok) + assert.Equal(t, metrics.Iterations, iterSample.Metric) + assert.Equal(t, float64(1), iterSample.Value) } } @@ -318,8 +341,8 @@ func TestExecutorSetVUs(t *testing.T) { }) t.Run("Raise", func(t *testing.T) { - e := New(&lib.MiniRunner{Fn: func(ctx context.Context) ([]stats.SampleContainer, error) { - return nil, nil + e := New(&lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + return nil }}) e.ctx = context.Background() diff --git a/js/console_test.go b/js/console_test.go index fada5c7da9cb..8ad55b85d86c 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -28,6 +28,7 @@ import ( "github.com/dop251/goja" "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" log "github.com/sirupsen/logrus" logtest "github.com/sirupsen/logrus/hooks/test" "github.com/spf13/afero" @@ -94,14 +95,15 @@ func TestConsole(t *testing.T) { }, afero.NewMemMapFs(), lib.RuntimeOptions{}) assert.NoError(t, err) - vu, err := r.newVU() + samples := make(chan stats.SampleContainer, 100) + vu, err := r.newVU(samples) assert.NoError(t, err) logger, hook := logtest.NewNullLogger() logger.Level = log.DebugLevel vu.Console.Logger = logger - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) entry := hook.LastEntry() diff --git a/js/initcontext_test.go b/js/initcontext_test.go index 3746b86cef5d..a51f8bbc0fe6 100644 --- a/js/initcontext_test.go +++ b/js/initcontext_test.go @@ -35,6 +35,7 @@ import ( "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/netext" + "github.com/loadimpact/k6/stats" "github.com/oxtoacart/bpool" log "github.com/sirupsen/logrus" "github.com/spf13/afero" @@ -372,7 +373,8 @@ func TestRequestWithBinaryFile(t *testing.T) { DualStack: true, })).DialContext, }, - BPool: bpool.NewBufferPool(1), + BPool: bpool.NewBufferPool(1), + Samples: make(chan stats.SampleContainer, 500), } ctx := context.Background() diff --git a/js/modules/k6/http/http_request_test.go b/js/modules/k6/http/http_request_test.go index d845717f5ae4..2d09da693130 100644 --- a/js/modules/k6/http/http_request_test.go +++ b/js/modules/k6/http/http_request_test.go @@ -98,7 +98,7 @@ func assertRequestMetricsEmitted(t *testing.T, sampleContainers []stats.SampleCo assert.True(t, seenReceiving, "url %s didn't emit Receiving", url) } -func newRuntime(t *testing.T) (*testutils.HTTPMultiBin, *common.State, *goja.Runtime, *context.Context) { +func newRuntime(t *testing.T) (*testutils.HTTPMultiBin, *common.State, chan stats.SampleContainer, *goja.Runtime, *context.Context) { tb := testutils.NewHTTPMultiBin(t) root, err := lib.NewGroup("", nil) @@ -110,6 +110,8 @@ func newRuntime(t *testing.T) (*testutils.HTTPMultiBin, *common.State, *goja.Run rt := goja.New() rt.SetFieldNameMapper(common.FieldNameMapper{}) + samples := make(chan stats.SampleContainer, 1000) + state := &common.State{ Options: lib.Options{ MaxRedirects: null.IntFrom(10), @@ -123,6 +125,7 @@ func newRuntime(t *testing.T) (*testutils.HTTPMultiBin, *common.State, *goja.Run TLSConfig: tb.TLSClientConfig, HTTPTransport: netext.NewHTTPTransport(tb.HTTPTransport), BPool: bpool.NewBufferPool(1), + Samples: samples, } ctx := new(context.Context) @@ -131,11 +134,11 @@ func newRuntime(t *testing.T) (*testutils.HTTPMultiBin, *common.State, *goja.Run *ctx = common.WithRuntime(*ctx, rt) rt.Set("http", common.Bind(rt, New(), ctx)) - return tb, state, rt, ctx + return tb, state, samples, rt, ctx } func TestRequestAndBatch(t *testing.T) { - tb, state, rt, ctx := newRuntime(t) + tb, state, samples, rt, ctx := newRuntime(t) defer tb.Cleanup() sr := tb.Replacer.Replace @@ -310,15 +313,18 @@ func TestRequestAndBatch(t *testing.T) { assert.Nil(t, hook.LastEntry()) }) t.Run("HTTP/2", func(t *testing.T) { - state.Samples = nil + stats.GetBufferedSamples(samples) // Clean up buffered samples from previous tests + _, err := common.RunString(rt, ` let res = http.request("GET", "https://http2.akamai.com/demo"); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.proto != "HTTP/2.0") { throw new Error("wrong proto: " + res.proto) } `) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", "https://http2.akamai.com/demo", "", 200, "") - for _, sampleC := range state.Samples { + + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", "https://http2.akamai.com/demo", "", 200, "") + for _, sampleC := range bufSamples { for _, sample := range sampleC.GetSamples() { proto, ok := sample.Tags.Get("proto") assert.True(t, ok) @@ -345,7 +351,7 @@ func TestRequestAndBatch(t *testing.T) { if (res.tls_version != %s) { throw new Error("wrong TLS version: " + res.tls_version); } `, versionTest.URL, versionTest.Version)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", versionTest.URL, "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", versionTest.URL, "", 200, "") }) } tlsCipherSuiteTests := []struct { @@ -361,17 +367,16 @@ func TestRequestAndBatch(t *testing.T) { if (res.tls_cipher_suite != "%s") { throw new Error("wrong TLS cipher suite: " + res.tls_cipher_suite); } `, cipherSuiteTest.URL, cipherSuiteTest.CipherSuite)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", cipherSuiteTest.URL, "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", cipherSuiteTest.URL, "", 200, "") }) } t.Run("ocsp_stapled_good", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let res = http.request("GET", "https://stackoverflow.com/"); if (res.ocsp.status != http.OCSP_STATUS_GOOD) { throw new Error("wrong ocsp stapled response status: " + res.ocsp.status); } `) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", "https://stackoverflow.com/", "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", "https://stackoverflow.com/", "", 200, "") }) }) t.Run("Invalid", func(t *testing.T) { @@ -414,13 +419,12 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Params", func(t *testing.T) { for _, literal := range []string{`undefined`, `null`} { t.Run(literal, func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, %s); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `), literal)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/headers"), "", 200, "") }) } @@ -429,20 +433,18 @@ func TestRequestAndBatch(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/cookies/set?key=value", null, { redirects: 0 }); if (res.cookies.key[0].value != "value") { throw new Error("wrong cookie value: " + res.cookies.key[0].value); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies/set?key=value"), "", 302, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies/set?key=value"), "", 302, "") }) t.Run("vuJar", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); @@ -454,14 +456,13 @@ func TestRequestAndBatch(t *testing.T) { if (jarCookies.key2 != undefined) { throw new Error("unexpected cookie in jar"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("requestScope", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key: "value" } }); if (res.json().key != "value") { throw new Error("wrong cookie value: " + res.json().key); } @@ -470,14 +471,13 @@ func TestRequestAndBatch(t *testing.T) { if (jarCookies.key != undefined) { throw new Error("unexpected cookie in jar"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("requestScopeReplace", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); @@ -487,14 +487,13 @@ func TestRequestAndBatch(t *testing.T) { if (jarCookies.key[0] != "value") { throw new Error("wrong cookie value in jar"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("redirect", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` http.cookieJar().set("HTTPBIN_URL/cookies", "key", "value"); let res = http.request("GET", "HTTPBIN_URL/cookies/set?key2=value2"); @@ -504,7 +503,7 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) assertRequestMetricsEmitted( t, - state.Samples, + stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), sr("HTTPBIN_URL/cookies/set?key2=value2"), @@ -517,7 +516,6 @@ func TestRequestAndBatch(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { domain: "HTTPBIN_DOMAIN" }); @@ -535,14 +533,13 @@ func TestRequestAndBatch(t *testing.T) { } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("path", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { path: "/cookies" }); @@ -560,14 +557,13 @@ func TestRequestAndBatch(t *testing.T) { } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("expires", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { expires: "Sun, 24 Jul 1983 17:01:02 GMT" }); @@ -582,14 +578,13 @@ func TestRequestAndBatch(t *testing.T) { } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) t.Run("secure", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = http.cookieJar(); jar.set("HTTPSBIN_IP_URL/cookies", "key", "value", { secure: true }); @@ -599,14 +594,13 @@ func TestRequestAndBatch(t *testing.T) { } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPSBIN_IP_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPSBIN_IP_URL/cookies"), "", 200, "") }) t.Run("localJar", func(t *testing.T) { cookieJar, err := cookiejar.New(nil) assert.NoError(t, err) state.CookieJar = cookieJar - state.Samples = nil _, err = common.RunString(rt, sr(` let jar = new http.CookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); @@ -618,13 +612,12 @@ func TestRequestAndBatch(t *testing.T) { if (jarCookies.key2 != undefined) { throw new Error("unexpected cookie in jar"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/cookies"), "", 200, "") }) }) t.Run("auth", func(t *testing.T) { t.Run("basic", func(t *testing.T) { - state.Samples = nil url := sr("http://bob:pass@HTTPBIN_IP:HTTPBIN_PORT/basic-auth/bob/pass") _, err := common.RunString(rt, fmt.Sprintf(` @@ -632,11 +625,10 @@ func TestRequestAndBatch(t *testing.T) { if (res.status != 200) { throw new Error("wrong status: " + res.status); } `, url)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", url, "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", url, "", 200, "") }) t.Run("digest", func(t *testing.T) { t.Run("success", func(t *testing.T) { - state.Samples = nil url := sr("http://bob:pass@HTTPBIN_IP:HTTPBIN_PORT/digest-auth/auth/bob/pass") _, err := common.RunString(rt, fmt.Sprintf(` @@ -644,10 +636,9 @@ func TestRequestAndBatch(t *testing.T) { if (res.status != 200) { throw new Error("wrong status: " + res.status); } `, url)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/digest-auth/auth/bob/pass"), url, 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_IP_URL/digest-auth/auth/bob/pass"), url, 200, "") }) t.Run("failure", func(t *testing.T) { - state.Samples = nil url := sr("http://bob:pass@HTTPBIN_IP:HTTPBIN_PORT/digest-auth/failure") _, err := common.RunString(rt, fmt.Sprintf(` @@ -658,43 +649,39 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("ntlm", func(t *testing.T) { t.Run("success auth", func(t *testing.T) { - state.Samples = nil url := strings.Replace(tb.ServerHTTP.URL+"/ntlm", "http://", "http://bob:pass@", -1) _, err := common.RunString(rt, fmt.Sprintf(` let res = http.request("GET", "%s", null, { auth: "ntlm" }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `, url)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", url, url, 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", url, url, 200, "") }) t.Run("failed auth", func(t *testing.T) { - state.Samples = nil url := strings.Replace(tb.ServerHTTP.URL+"/ntlm", "http://", "http://other:otherpass@", -1) _, err := common.RunString(rt, fmt.Sprintf(` let res = http.request("GET", "%s", null, { auth: "ntlm" }); if (res.status != 401) { throw new Error("wrong status: " + res.status); } `, url)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", url, url, 401, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", url, url, 401, "") }) }) }) t.Run("headers", func(t *testing.T) { for _, literal := range []string{`null`, `undefined`} { - state.Samples = nil t.Run(literal, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { headers: %s }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `), literal)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/headers"), "", 200, "") }) } t.Run("object", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { headers: { "X-My-Header": "value" }, @@ -703,11 +690,10 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().headers["X-My-Header"] != "value") { throw new Error("wrong X-My-Header: " + res.json().headers["X-My-Header"]); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/headers"), "", 200, "") }) t.Run("Host", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { headers: { "Host": "HTTPBIN_DOMAIN" }, @@ -716,32 +702,31 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().headers["Host"] != "HTTPBIN_DOMAIN") { throw new Error("wrong Host: " + res.json().headers["Host"]); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/headers"), "", 200, "") }) }) t.Run("tags", func(t *testing.T) { for _, literal := range []string{`null`, `undefined`} { t.Run(literal, func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: %s }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `), literal)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/headers"), "", 200, "") }) } t.Run("object", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: { tag: "value" } }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") - for _, sampleC := range state.Samples { + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/headers"), "", 200, "") + for _, sampleC := range bufSamples { for _, sample := range sampleC.GetSamples() { tagValue, ok := sample.Tags.Get("tag") assert.True(t, ok) @@ -755,14 +740,15 @@ func TestRequestAndBatch(t *testing.T) { defer func() { state.Options = oldOpts }() state.Options.RunTags = stats.IntoSampleTags(&map[string]string{"runtag1": "val1", "runtag2": "val2"}) - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: { method: "test", name: "myName", runtag1: "fromreq" } }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/headers"), "myName", 200, "") - for _, sampleC := range state.Samples { + + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/headers"), "myName", 200, "") + for _, sampleC := range bufSamples { for _, sample := range sampleC.GetSamples() { tagValue, ok := sample.Tags.Get("method") assert.True(t, ok) @@ -786,7 +772,6 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("GET", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.get("HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -794,10 +779,9 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().args.b != "2") { throw new Error("wrong ?b: " + res.json().args.b); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let a = "1"; let b = "2"; @@ -807,11 +791,10 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().args.b != b) { throw new Error("wrong ?b: " + res.json().args.b); } `) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?a=1&b=2"), sr("HTTPBIN_URL/get?a=${}&b=${}"), 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/get?a=1&b=2"), sr("HTTPBIN_URL/get?a=${}&b=${}"), 200, "") }) }) t.Run("HEAD", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.head("HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -819,18 +802,17 @@ func TestRequestAndBatch(t *testing.T) { if (!res.headers["Content-Length"]) { throw new Error("Missing or invalid Content-Length header!"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "HEAD", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "HEAD", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") }) t.Run("OPTIONS", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.options("HTTPBIN_URL/?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (!res.headers["Access-Control-Allow-Methods"]) { throw new Error("Missing Access-Control-Allow-Methods header!"); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "OPTIONS", sr("HTTPBIN_URL/?a=1&b=2"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "OPTIONS", sr("HTTPBIN_URL/?a=1&b=2"), "", 200, "") }) // DELETE HTTP requests shouldn't usually send a request body, they should use url parameters instead; references: @@ -838,14 +820,13 @@ func TestRequestAndBatch(t *testing.T) { // https://stackoverflow.com/questions/299628/is-an-entity-body-allowed-for-an-http-delete-request // https://tools.ietf.org/html/rfc7231#section-4.3.5 t.Run("DELETE", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.del("HTTPBIN_URL/delete?test=mest"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().args.test != "mest") { throw new Error("wrong args: " + JSON.stringify(res.json().args)); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "DELETE", sr("HTTPBIN_URL/delete?test=mest"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "DELETE", sr("HTTPBIN_URL/delete?test=mest"), "", 200, "") }) postMethods := map[string]string{ @@ -855,7 +836,6 @@ func TestRequestAndBatch(t *testing.T) { } for method, fn := range postMethods { t.Run(method, func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.%s("HTTPBIN_URL/%s", "data"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -863,10 +843,9 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().headers["Content-Type"]) { throw new Error("content type set: " + res.json().headers["Content-Type"]); } `), fn, strings.ToLower(method))) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") t.Run("object", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.%s("HTTPBIN_URL/%s", {a: "a", b: 2}); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -875,9 +854,8 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().headers["Content-Type"] != "application/x-www-form-urlencoded") { throw new Error("wrong content type: " + res.json().headers["Content-Type"]); } `), fn, strings.ToLower(method))) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") t.Run("Content-Type", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(sr(` let res = http.%s("HTTPBIN_URL/%s", {a: "a", b: 2}, {headers: {"Content-Type": "application/x-www-form-urlencoded; charset=utf-8"}}); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -886,7 +864,7 @@ func TestRequestAndBatch(t *testing.T) { if (res.json().headers["Content-Type"] != "application/x-www-form-urlencoded; charset=utf-8") { throw new Error("wrong content type: " + res.json().headers["Content-Type"]); } `), fn, strings.ToLower(method))) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") }) }) }) @@ -894,7 +872,6 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Batch", func(t *testing.T) { t.Run("GET", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let reqs = [ ["GET", "HTTPBIN_URL/"], @@ -906,11 +883,11 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].url != reqs[key][1]) { throw new Error("wrong url: " + res[key].url); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let fragment = "get"; let reqs = [ @@ -923,12 +900,12 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].url != reqs[key][1].url) { throw new Error("wrong url: " + key + ": " + res[key].url + " != " + reqs[key][1].url); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get"), sr("HTTPBIN_URL/${}"), 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get"), sr("HTTPBIN_URL/${}"), 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") }) t.Run("Shorthand", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let reqs = [ "HTTPBIN_URL/", @@ -940,11 +917,11 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].url != reqs[key]) { throw new Error("wrong url: " + key + ": " + res[key].url); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let fragment = "get"; let reqs = [ @@ -957,13 +934,13 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].url != reqs[key].url) { throw new Error("wrong url: " + key + ": " + res[key].url + " != " + reqs[key].url); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get"), sr("HTTPBIN_URL/${}"), 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get"), sr("HTTPBIN_URL/${}"), 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") }) }) t.Run("ObjectForm", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let reqs = [ { method: "GET", url: "HTTPBIN_URL/" }, @@ -975,12 +952,12 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].url != reqs[key].url) { throw new Error("wrong url: " + key + ": " + res[key].url + " != " + reqs[key].url); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") }) t.Run("ObjectKeys", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let reqs = { shorthand: "HTTPBIN_URL/get?r=shorthand", @@ -994,15 +971,14 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].json().args.r != key) { throw new Error("wrong request id: " + key); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?r=shorthand"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?r=arr"), "arr", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?r=obj1"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?r=obj2"), "obj2", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get?r=shorthand"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get?r=arr"), "arr", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get?r=obj1"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_URL/get?r=obj2"), "obj2", 200, "") }) t.Run("BodyAndParams", func(t *testing.T) { - state.Samples = nil - testStr := "testbody" rt.Set("someStrFile", testStr) rt.Set("someBinFile", []byte(testStr)) @@ -1040,12 +1016,12 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].json().data != "testbody" && res[key].json().form.hello != "world!") { throw new Error("wrong response for " + key + ": " + res[key].body); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_IP_URL/post"), "myname", 200, "") + bufSamples := stats.GetBufferedSamples(samples) + assertRequestMetricsEmitted(t, bufSamples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, bufSamples, "POST", sr("HTTPBIN_IP_URL/post"), "myname", 200, "") }) }) t.Run("POST", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.batch([ ["POST", "HTTPBIN_URL/post", { key: "value" }] ]); for (var key in res) { @@ -1053,10 +1029,9 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].json().form.key != "value") { throw new Error("wrong form: " + key + ": " + JSON.stringify(res[key].json().form)); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "POST", sr("HTTPBIN_URL/post"), "", 200, "") }) t.Run("PUT", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.batch([ ["PUT", "HTTPBIN_URL/put", { key: "value" }] ]); for (var key in res) { @@ -1064,7 +1039,7 @@ func TestRequestAndBatch(t *testing.T) { if (res[key].json().form.key != "value") { throw new Error("wrong form: " + key + ": " + JSON.stringify(res[key].json().form)); } }`)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "PUT", sr("HTTPBIN_URL/put"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "PUT", sr("HTTPBIN_URL/put"), "", 200, "") }) }) @@ -1102,7 +1077,7 @@ func TestRequestAndBatch(t *testing.T) { }) } func TestSystemTags(t *testing.T) { - tb, state, rt, _ := newRuntime(t) + tb, state, samples, rt, _ := newRuntime(t) defer tb.Cleanup() // Handple paths with custom logic @@ -1142,11 +1117,12 @@ func TestSystemTags(t *testing.T) { for num, tc := range testedSystemTags { t.Run(fmt.Sprintf("TC %d with only %s", num, tc.tag), func(t *testing.T) { state.Options.SystemTags = lib.GetTagSet(tc.tag) - state.Samples = nil _, err := common.RunString(rt, tc.code) assert.NoError(t, err) - assert.NotEmpty(t, state.Samples) - for _, sampleC := range state.Samples { + + bufSamples := stats.GetBufferedSamples(samples) + assert.NotEmpty(t, bufSamples) + for _, sampleC := range bufSamples { for _, sample := range sampleC.GetSamples() { assert.NotEmpty(t, sample.Tags) for emittedTag, emittedVal := range sample.Tags.CloneTags() { diff --git a/js/modules/k6/http/response_test.go b/js/modules/k6/http/response_test.go index 848e9e9dde23..12e0cbe40f25 100644 --- a/js/modules/k6/http/response_test.go +++ b/js/modules/k6/http/response_test.go @@ -28,6 +28,7 @@ import ( "testing" "github.com/loadimpact/k6/js/common" + "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" ) @@ -77,7 +78,7 @@ func myFormHandler(w http.ResponseWriter, r *http.Request) { } func TestResponse(t *testing.T) { - tb, state, rt, _ := newRuntime(t) + tb, state, samples, rt, _ := newRuntime(t) defer tb.Cleanup() root := state.Group sr := tb.Replacer.Replace @@ -85,14 +86,13 @@ func TestResponse(t *testing.T) { tb.Mux.HandleFunc("/myforms/get", myFormHandler) t.Run("Html", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/html"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.body.indexOf("Herman Melville - Moby-Dick") == -1) { throw new Error("wrong body: " + res.body); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/html"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/html"), "", 200, "") t.Run("html", func(t *testing.T) { _, err := common.RunString(rt, ` @@ -123,18 +123,16 @@ func TestResponse(t *testing.T) { defer func() { state.Group = old }() } - state.Samples = nil _, err = common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/html"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.body.indexOf("Herman Melville - Moby-Dick") == -1) { throw new Error("wrong body: " + res.body); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/html"), "", 200, "::my group") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/html"), "", 200, "::my group") }) }) t.Run("Json", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -142,7 +140,7 @@ func TestResponse(t *testing.T) { if (res.json().args.b != "2") { throw new Error("wrong ?b: " + res.json().args.b); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") t.Run("Invalid", func(t *testing.T) { _, err := common.RunString(rt, sr(`http.request("GET", "HTTPBIN_URL/html").json();`)) @@ -152,7 +150,6 @@ func TestResponse(t *testing.T) { t.Run("SubmitForm", func(t *testing.T) { t.Run("withoutArgs", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -168,11 +165,10 @@ func TestResponse(t *testing.T) { ) { throw new Error("incorrect body: " + JSON.stringify(data, null, 4) ); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "POST", sr("HTTPBIN_URL/post"), "", 200, "") }) t.Run("withFields", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -188,11 +184,10 @@ func TestResponse(t *testing.T) { ) { throw new Error("incorrect body: " + JSON.stringify(data, null, 4) ); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "POST", sr("HTTPBIN_URL/post"), "", 200, "") }) t.Run("withRequestParams", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -202,11 +197,10 @@ func TestResponse(t *testing.T) { if (headers["My-Fancy-Header"][0] !== "SomeValue" ) { throw new Error("incorrect headers: " + JSON.stringify(headers)); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "POST", sr("HTTPBIN_URL/post"), "", 200, "") }) t.Run("withFormSelector", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -222,11 +216,10 @@ func TestResponse(t *testing.T) { ) { throw new Error("incorrect body: " + JSON.stringify(data, null, 4) ); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "POST", sr("HTTPBIN_URL/post"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "POST", sr("HTTPBIN_URL/post"), "", 200, "") }) t.Run("withNonExistentForm", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -236,7 +229,6 @@ func TestResponse(t *testing.T) { }) t.Run("withGetMethod", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/myforms/get"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -251,13 +243,12 @@ func TestResponse(t *testing.T) { ) { throw new Error("incorrect body: " + JSON.stringify(data, null, 4) ); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/myforms/get"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/myforms/get"), "", 200, "") }) }) t.Run("ClickLink", func(t *testing.T) { t.Run("withoutArgs", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -265,11 +256,10 @@ func TestResponse(t *testing.T) { if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/links/10/1"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/links/10/1"), "", 200, "") }) t.Run("withSelector", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -277,11 +267,10 @@ func TestResponse(t *testing.T) { if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/links/10/4"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/links/10/4"), "", 200, "") }) t.Run("withNonExistentLink", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -291,7 +280,6 @@ func TestResponse(t *testing.T) { }) t.Run("withRequestParams", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -301,7 +289,7 @@ func TestResponse(t *testing.T) { if (headers["My-Fancy-Header"][0] !== "SomeValue" ) { throw new Error("incorrect headers: " + JSON.stringify(headers)); } `)) assert.NoError(t, err) - assertRequestMetricsEmitted(t, state.Samples, "GET", sr("HTTPBIN_URL/get"), "", 200, "") + assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/get"), "", 200, "") }) }) } diff --git a/js/modules/k6/k6_test.go b/js/modules/k6/k6_test.go index 5add9b10bcee..66f0fc3afd68 100644 --- a/js/modules/k6/k6_test.go +++ b/js/modules/k6/k6_test.go @@ -89,7 +89,7 @@ func TestGroup(t *testing.T) { assert.NoError(t, err) rt := goja.New() - state := &common.State{Group: root} + state := &common.State{Group: root, Samples: make(chan stats.SampleContainer, 1000)} ctx := context.Background() ctx = common.WithState(ctx, state) @@ -112,7 +112,6 @@ func TestGroup(t *testing.T) { assert.EqualError(t, err, "GoError: group and check names may not contain '::'") }) } - func TestCheck(t *testing.T) { rt := goja.New() @@ -125,23 +124,26 @@ func TestCheck(t *testing.T) { *ctx = baseCtx rt.Set("k6", common.Bind(rt, New(), ctx)) - getState := func() *common.State { + getState := func() (*common.State, chan stats.SampleContainer) { + samples := make(chan stats.SampleContainer, 1000) return &common.State{ Group: root, Options: lib.Options{ SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), }, - } + Samples: samples, + }, samples } t.Run("Object", func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) _, err := common.RunString(rt, `k6.check(null, { "check": true })`) assert.NoError(t, err) - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) @@ -154,15 +156,16 @@ func TestCheck(t *testing.T) { } t.Run("Multiple", func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) _, err := common.RunString(rt, `k6.check(null, { "a": true, "b": false })`) assert.NoError(t, err) - assert.Len(t, state.Samples, 2) + bufSamples := stats.GetBufferedSamples(samples) + assert.Len(t, bufSamples, 2) var foundA, foundB bool - for _, sampleC := range state.Samples { + for _, sampleC := range bufSamples { for _, sample := range sampleC.GetSamples() { name, ok := sample.Tags.Get("check") assert.True(t, ok) @@ -189,14 +192,15 @@ func TestCheck(t *testing.T) { }) t.Run("Array", func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) _, err := common.RunString(rt, `k6.check(null, [ true ])`) assert.NoError(t, err) - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) @@ -210,15 +214,18 @@ func TestCheck(t *testing.T) { }) t.Run("Literal", func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) _, err := common.RunString(rt, `k6.check(null, 12345)`) assert.NoError(t, err) - assert.Len(t, state.Samples, 0) + assert.Len(t, stats.GetBufferedSamples(samples), 0) }) t.Run("Throws", func(t *testing.T) { + state, _ := getState() + *ctx = common.WithState(baseCtx, state) + _, err := common.RunString(rt, ` k6.check(null, { "a": function() { throw new Error("error A") }, @@ -250,7 +257,7 @@ func TestCheck(t *testing.T) { t.Run(name, func(t *testing.T) { for value, succ := range testdata { t.Run(value, func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) v, err := common.RunString(rt, fmt.Sprintf(tpl, value)) @@ -258,8 +265,9 @@ func TestCheck(t *testing.T) { assert.Equal(t, succ, v.Export()) } - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) @@ -283,7 +291,7 @@ func TestCheck(t *testing.T) { root, err := lib.NewGroup("", nil) assert.NoError(t, err) - state := &common.State{Group: root} + state := &common.State{Group: root, Samples: make(chan stats.SampleContainer, 1000)} ctx2, cancel := context.WithCancel(common.WithState(baseCtx, state)) *ctx = ctx2 @@ -309,7 +317,7 @@ func TestCheck(t *testing.T) { }) t.Run("Tags", func(t *testing.T) { - state := getState() + state, samples := getState() *ctx = common.WithState(baseCtx, state) v, err := common.RunString(rt, `k6.check(null, {"check": true}, {a: 1, b: "2"})`) @@ -317,8 +325,9 @@ func TestCheck(t *testing.T) { assert.Equal(t, true, v.Export()) } - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) diff --git a/js/modules/k6/metrics/metrics_test.go b/js/modules/k6/metrics/metrics_test.go index 583651f2ff3a..6ce732f95d62 100644 --- a/js/modules/k6/metrics/metrics_test.go +++ b/js/modules/k6/metrics/metrics_test.go @@ -67,9 +67,11 @@ func TestMetrics(t *testing.T) { root, _ := lib.NewGroup("", nil) child, _ := root.Group("child") + samples := make(chan stats.SampleContainer, 1000) state := &common.State{ Options: lib.Options{SystemTags: lib.GetTagSet("group")}, Group: root, + Samples: samples, } isTimeString := "" @@ -99,11 +101,11 @@ func TestMetrics(t *testing.T) { for name, val := range values { t.Run(name, func(t *testing.T) { t.Run("Simple", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(`m.add(%v)`, val.JS)) assert.NoError(t, err) - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) @@ -117,11 +119,11 @@ func TestMetrics(t *testing.T) { } }) t.Run("Tags", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, fmt.Sprintf(`m.add(%v, {a:1})`, val.JS)) assert.NoError(t, err) - if assert.Len(t, state.Samples, 1) { - sample, ok := state.Samples[0].(stats.Sample) + bufSamples := stats.GetBufferedSamples(samples) + if assert.Len(t, bufSamples, 1) { + sample, ok := bufSamples[0].(stats.Sample) require.True(t, ok) assert.NotZero(t, sample.Time) diff --git a/js/modules/k6/ws/ws_test.go b/js/modules/k6/ws/ws_test.go index 24fa42f7ca02..dbf59346fc18 100644 --- a/js/modules/k6/ws/ws_test.go +++ b/js/modules/k6/ws/ws_test.go @@ -90,6 +90,8 @@ func makeWsProto(s string) string { } func TestSession(t *testing.T) { + //TODO: split and paralelize tests + root, err := lib.NewGroup("", nil) assert.NoError(t, err) @@ -100,12 +102,14 @@ func TestSession(t *testing.T) { KeepAlive: 60 * time.Second, DualStack: true, }) + samples := make(chan stats.SampleContainer, 1000) state := &common.State{ Group: root, Dialer: dialer, Options: lib.Options{ SystemTags: lib.GetTagSet("url", "proto", "status", "subproto"), }, + Samples: samples, } ctx := context.Background() @@ -123,7 +127,7 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") t.Run("connect_wss", func(t *testing.T) { _, err := common.RunString(rt, ` @@ -134,10 +138,9 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "wss://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "wss://demos.kaazing.com/echo", 101, "") t.Run("open", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let opened = false; let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ @@ -150,10 +153,9 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") t.Run("send_receive", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ socket.on("open", function() { @@ -169,12 +171,13 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") - assertMetricEmitted(t, metrics.WSMessagesSent, state.Samples, "ws://demos.kaazing.com/echo") - assertMetricEmitted(t, metrics.WSMessagesReceived, state.Samples, "ws://demos.kaazing.com/echo") + + samplesBuf := stats.GetBufferedSamples(samples) + assertSessionMetricsEmitted(t, samplesBuf, "", "ws://demos.kaazing.com/echo", 101, "") + assertMetricEmitted(t, metrics.WSMessagesSent, samplesBuf, "ws://demos.kaazing.com/echo") + assertMetricEmitted(t, metrics.WSMessagesReceived, samplesBuf, "ws://demos.kaazing.com/echo") t.Run("interval", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let counter = 0; let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ @@ -187,10 +190,9 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") t.Run("timeout", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let start = new Date().getTime(); let ellapsed = new Date().getTime() - start; @@ -206,10 +208,9 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") t.Run("ping", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let pongReceived = false; let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ @@ -228,11 +229,12 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") - assertMetricEmitted(t, metrics.WSPing, state.Samples, "ws://demos.kaazing.com/echo") + + samplesBuf = stats.GetBufferedSamples(samples) + assertSessionMetricsEmitted(t, samplesBuf, "", "ws://demos.kaazing.com/echo", 101, "") + assertMetricEmitted(t, metrics.WSPing, samplesBuf, "ws://demos.kaazing.com/echo") t.Run("multiple_handlers", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let pongReceived = false; let otherPongReceived = false; @@ -261,11 +263,12 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") - assertMetricEmitted(t, metrics.WSPing, state.Samples, "ws://demos.kaazing.com/echo") + + samplesBuf = stats.GetBufferedSamples(samples) + assertSessionMetricsEmitted(t, samplesBuf, "", "ws://demos.kaazing.com/echo", 101, "") + assertMetricEmitted(t, metrics.WSPing, samplesBuf, "ws://demos.kaazing.com/echo") t.Run("close", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let closed = false; let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ @@ -280,7 +283,7 @@ func TestSession(t *testing.T) { `) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") } func TestErrors(t *testing.T) { @@ -294,12 +297,14 @@ func TestErrors(t *testing.T) { KeepAlive: 60 * time.Second, DualStack: true, }) + samples := make(chan stats.SampleContainer, 1000) state := &common.State{ Group: root, Dialer: dialer, Options: lib.Options{ SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), }, + Samples: samples, } ctx := context.Background() @@ -309,7 +314,6 @@ func TestErrors(t *testing.T) { rt.Set("ws", common.Bind(rt, New(), &ctx)) t.Run("invalid_url", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let res = ws.connect("INVALID", function(socket){ socket.on("open", function() { @@ -321,7 +325,6 @@ func TestErrors(t *testing.T) { }) t.Run("send_after_close", func(t *testing.T) { - state.Samples = nil _, err := common.RunString(rt, ` let hasError = false; let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ @@ -339,7 +342,7 @@ func TestErrors(t *testing.T) { } `) assert.NoError(t, err) - assertSessionMetricsEmitted(t, state.Samples, "", "ws://demos.kaazing.com/echo", 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", "ws://demos.kaazing.com/echo", 101, "") }) } @@ -358,10 +361,13 @@ func TestSystemTags(t *testing.T) { //TODO: test for actual tag values after removing the dependency on the // external service demos.kaazing.com (https://github.com/loadimpact/k6/issues/537) testedSystemTags := []string{"group", "status", "subproto", "url", "ip"} + + samples := make(chan stats.SampleContainer, 1000) state := &common.State{ Group: root, Dialer: dialer, Options: lib.Options{SystemTags: lib.GetTagSet(testedSystemTags...)}, + Samples: samples, } ctx := context.Background() @@ -375,7 +381,6 @@ func TestSystemTags(t *testing.T) { state.Options.SystemTags = map[string]bool{ expectedTag: true, } - state.Samples = nil _, err := common.RunString(rt, ` let res = ws.connect("ws://demos.kaazing.com/echo", function(socket){ socket.on("open", function() { @@ -391,7 +396,7 @@ func TestSystemTags(t *testing.T) { `) assert.NoError(t, err) - for _, sampleContainer := range state.Samples { + for _, sampleContainer := range stats.GetBufferedSamples(samples) { for _, sample := range sampleContainer.GetSamples() { for emittedTag := range sample.Tags.CloneTags() { assert.Equal(t, expectedTag, emittedTag) @@ -413,12 +418,14 @@ func TestTLSConfig(t *testing.T) { KeepAlive: 60 * time.Second, DualStack: true, }) + samples := make(chan stats.SampleContainer, 1000) state := &common.State{ Group: root, Dialer: dialer, Options: lib.Options{ SystemTags: lib.GetTagSet("url", "proto", "status", "subproto", "ip"), }, + Samples: samples, } ctx := context.Background() @@ -445,7 +452,7 @@ func TestTLSConfig(t *testing.T) { `, url)) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", url, 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", url, 101, "") t.Run("custom certificates", func(t *testing.T) { state.TLSConfig = tb.TLSClientConfig @@ -458,5 +465,5 @@ func TestTLSConfig(t *testing.T) { `, url)) assert.NoError(t, err) }) - assertSessionMetricsEmitted(t, state.Samples, "", url, 101, "") + assertSessionMetricsEmitted(t, stats.GetBufferedSamples(samples), "", url, 101, "") } diff --git a/js/runner_test.go b/js/runner_test.go index 2c7bae20ccfd..3b4858cbce58 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -55,15 +55,16 @@ func TestRunnerNew(t *testing.T) { assert.NoError(t, err) t.Run("NewVU", func(t *testing.T) { - vu_, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) assert.NoError(t, err) - vu := vu_.(*VU) - assert.Equal(t, int64(0), vu.Runtime.Get("counter").Export()) + vuc, ok := vu.(*VU) + assert.True(t, ok) + assert.Equal(t, int64(0), vuc.Runtime.Get("counter").Export()) t.Run("RunOnce", func(t *testing.T) { - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) - assert.Equal(t, int64(1), vu.Runtime.Get("counter").Export()) + assert.Equal(t, int64(1), vuc.Runtime.Get("counter").Export()) }) }) }) @@ -156,18 +157,19 @@ func TestSetupTeardown(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { + samples := make(chan stats.SampleContainer, 100) t.Run(name, func(t *testing.T) { - if !assert.NoError(t, r.Setup(context.Background())) { + if !assert.NoError(t, r.Setup(context.Background(), samples)) { return } - vu, err := r.NewVU() + vu, err := r.NewVU(samples) if assert.NoError(t, err) { - _, err := vu.RunOnce(context.Background()) + err := vu.RunOnce(context.Background()) assert.NoError(t, err) } - assert.NoError(t, r.Teardown(context.Background())) + assert.NoError(t, r.Teardown(context.Background(), samples)) }) } } @@ -227,11 +229,11 @@ func TestRunnerIntegrationImports(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) }) } @@ -261,7 +263,7 @@ func TestVURunContext(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { t.Run(name, func(t *testing.T) { - vu, err := r.newVU() + vu, err := r.newVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } @@ -281,7 +283,7 @@ func TestVURunContext(t *testing.T) { assert.Equal(t, vu.HTTPTransport, state.HTTPTransport) } }) - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) assert.True(t, fnCalled, "fn() not called") }) @@ -308,13 +310,13 @@ func TestVURunInterrupt(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { t.Run(name, func(t *testing.T) { - vu, err := r.newVU() + vu, err := r.newVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) defer cancel() - _, err = vu.RunOnce(ctx) + err = vu.RunOnce(ctx) assert.EqualError(t, err, "context cancelled at /script.js:1:1(1)") }) } @@ -348,7 +350,7 @@ func TestVUIntegrationGroups(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { t.Run(name, func(t *testing.T) { - vu, err := r.newVU() + vu, err := r.newVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } @@ -373,7 +375,7 @@ func TestVUIntegrationGroups(t *testing.T) { assert.Equal(t, "my group", g.Parent.Name) assert.Equal(t, r.GetDefaultGroup(), g.Parent.Parent) }) - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) assert.True(t, fnOuterCalled, "fnOuter() not called") assert.True(t, fnInnerCalled, "fnInner() not called") @@ -404,15 +406,16 @@ func TestVUIntegrationMetrics(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { t.Run(name, func(t *testing.T) { - vu, err := r.newVU() + samples := make(chan stats.SampleContainer, 100) + vu, err := r.newVU(samples) if !assert.NoError(t, err) { return } - samples, err := vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) sampleCount := 0 - for i, sampleC := range samples { + for i, sampleC := range stats.GetBufferedSamples(samples) { for j, s := range sampleC.GetSamples() { sampleCount++ switch i + j { @@ -478,11 +481,11 @@ func TestVUIntegrationInsecureRequests(t *testing.T) { t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) if data.errMsg != "" { assert.EqualError(t, err, data.errMsg) } else { @@ -523,11 +526,11 @@ func TestVUIntegrationBlacklist(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.EqualError(t, err, "GoError: Get http://10.1.2.3/: IP (10.1.2.3) is in a blacklisted range (10.0.0.0/8)") }) } @@ -576,12 +579,12 @@ func TestVUIntegrationHosts(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) if !assert.NoError(t, err) { return } @@ -643,11 +646,11 @@ func TestVUIntegrationTLSConfig(t *testing.T) { t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) if data.errMsg != "" { assert.EqualError(t, err, data.errMsg) } else { @@ -687,15 +690,16 @@ func TestVUIntegrationHTTP2(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + samples := make(chan stats.SampleContainer, 100) + vu, err := r.NewVU(samples) if !assert.NoError(t, err) { return } - samples, err := vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) protoFound := false - for _, sampleC := range samples { + for _, sampleC := range stats.GetBufferedSamples(samples) { for _, sample := range sampleC.GetSamples() { if proto, ok := sample.Tags.Get("proto"); ok { protoFound = true @@ -717,9 +721,9 @@ func TestVUIntegrationOpenFunctionError(t *testing.T) { }, afero.NewMemMapFs(), lib.RuntimeOptions{}) assert.NoError(t, err) - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) assert.NoError(t, err) - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.EqualError(t, err, "GoError: \"open\" function is only available to the init code (aka global scope), see https://docs.k6.io/docs/test-life-cycle for more information") } @@ -764,12 +768,12 @@ func TestVUIntegrationCookies(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } for i := 0; i < 2; i++ { - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) } }) @@ -798,12 +802,12 @@ func TestVUIntegrationVUID(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } assert.NoError(t, vu.Reconfigure(1234)) - _, err = vu.RunOnce(context.Background()) + err = vu.RunOnce(context.Background()) assert.NoError(t, err) }) } @@ -898,9 +902,9 @@ func TestVUIntegrationClientCerts(t *testing.T) { for name, r := range runners { t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - _, err := vu.RunOnce(context.Background()) + err := vu.RunOnce(context.Background()) assert.EqualError(t, err, fmt.Sprintf("GoError: Get https://%s: remote error: tls: bad certificate", listener.Addr().String())) } }) @@ -942,9 +946,9 @@ func TestVUIntegrationClientCerts(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { t.Run(name, func(t *testing.T) { - vu, err := r.NewVU() + vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - _, err := vu.RunOnce(context.Background()) + err := vu.RunOnce(context.Background()) assert.NoError(t, err) } }) diff --git a/stats/stats.go b/stats/stats.go index 67a8b782b7f3..de4fc9c2582c 100644 --- a/stats/stats.go +++ b/stats/stats.go @@ -370,6 +370,19 @@ var _ SampleContainer = Samples{} var _ ConnectedSampleContainer = Sample{} var _ ConnectedSampleContainer = ConnectedSamples{} +// GetBufferedSamples will read all present (i.e. buffered or currently being pushed) +// values in the input channel and return them as a slice. +func GetBufferedSamples(input <-chan SampleContainer) (result []SampleContainer) { + for { + select { + case val := <-input: + result = append(result, val) + default: + return + } + } +} + // A Metric defines the shape of a set of data. type Metric struct { Name string `json:"name"` From 15d1c866aab0c78c7be24d7846cff02b7b378ab4 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 20 Jun 2018 14:30:44 +0300 Subject: [PATCH 05/14] Fix a comment typo --- core/local/local.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/local/local.go b/core/local/local.go index 911212587222..5f8f4ea88882 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -38,7 +38,7 @@ import ( // TODO: totally rewrite this! // This is an overcomplicated and probably buggy piece of code that is a major PITA to refactor... -// It does a ton of stuff in a very convoluted way, has a and uses a very incomprihensible mix +// It does a ton of stuff in a very convoluted way, has a and uses a very incomprehensible mix // of all possible Go synchronization mechanisms (channels, mutexes, rwmutexes, atomics, // and waitgroups) and has a bunch of contexts and tickers on top... From caa707fd7559859078d8bd07f72524ed2af7f986 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 20 Jun 2018 15:44:48 +0300 Subject: [PATCH 06/14] Update the release notes --- release notes/upcoming.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/release notes/upcoming.md b/release notes/upcoming.md index 879269443dc6..374ecebe0ccc 100644 --- a/release notes/upcoming.md +++ b/release notes/upcoming.md @@ -8,6 +8,12 @@ Description of feature. **Docs**: [Title](http://k6.readme.io/docs/TODO) +## Internals + +### Real-time metrics (#678) + +Previously most metrics were emitted only when a script iteration ended. With these changes, metrics would be continuously pushed in real-time, even in the middle of a script iteration. This should slightly decrease memory usage and help a lot with the aggregation efficiency of the cloud collector. + ## Bugs fixed! -* Category: description of bug. (#PR) \ No newline at end of file +* Metrics emitted by `setup()` and `teardown()` are not discarded anymore. They are emitted and have the implicit root `group` tag values of `setup` and `teardown` respectively (#678) \ No newline at end of file From f1536066394b3580191bb1d4d897e507b4f5c106 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 21 Jun 2018 15:24:44 +0300 Subject: [PATCH 07/14] Fix group tag for netext.Trail samples and add tests --- core/local/local_test.go | 158 +++++++++++++++++++++++++++++++++++++++ js/runner.go | 6 +- 2 files changed, 162 insertions(+), 2 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 9475f66a9fd6..c8ebe2dcc27e 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -22,16 +22,21 @@ package local import ( "context" + "net" "sync/atomic" "testing" "time" + "github.com/loadimpact/k6/lib/netext" + + "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/pkg/errors" logtest "github.com/sirupsen/logrus/hooks/test" + "github.com/spf13/afero" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" @@ -424,3 +429,156 @@ func TestExecutorSetVUs(t *testing.T) { }) }) } + +func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { + t.Parallel() + script := []byte(` + import { Counter } from "k6/metrics"; + import { sleep } from "k6"; + + var counter = new Counter("test_counter"); + + export function setup() { + console.log("setup(), sleeping for 1 second"); + counter.add(1, { place: "setupBeforeSleep" }); + sleep(1); + console.log("setup sleep is done"); + counter.add(2, { place: "setupAfterSleep" }); + return { "some": ["data"], "v": 1 }; + } + + export function teardown(data) { + console.log("teardown(" + JSON.stringify(data) + "), sleeping for 1 second"); + counter.add(3, { place: "teardownBeforeSleep" }); + sleep(1); + if (!data || data.v != 1) { + throw new Error("incorrect data: " + JSON.stringify(data)); + } + console.log("teardown sleep is done"); + counter.add(4, { place: "teardownAfterSleep" }); + } + + export default function (data) { + console.log("default(" + JSON.stringify(data) + ") with ENV=" + JSON.stringify(__ENV) + " for in ITER " + __ITER + " and VU " + __VU); + counter.add(5, { place: "defaultBeforeSleep" }); + if (!data || data.v != 1) { + throw new Error("incorrect data: " + JSON.stringify(data)); + } + sleep(1); + console.log("default() for in ITER " + __ITER + " and VU " + __VU + " done!"); + counter.add(6, { place: "defaultAfterSleep" }); + }`) + + runner, err := js.New( + &lib.SourceData{Filename: "/script.js", Data: script}, + afero.NewMemMapFs(), + lib.RuntimeOptions{}, + ) + require.NoError(t, err) + + options := lib.Options{ + SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), + SetupTimeout: types.NullDurationFrom(4 * time.Second), + TeardownTimeout: types.NullDurationFrom(4 * time.Second), + } + runner.SetOptions(options) + + executor := New(runner) + executor.SetEndIterations(null.IntFrom(2)) + executor.SetVUsMax(1) + executor.SetVUs(1) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + done := make(chan struct{}) + sampleContainers := make(chan stats.SampleContainer) + go func() { + assert.NoError(t, executor.Run(ctx, sampleContainers)) + close(done) + }() + + expectIn := func(from, to time.Duration, expected stats.SampleContainer) { + start := time.Now() + from = from * time.Millisecond + to = to * time.Millisecond + for { + select { + case sampleContainer := <-sampleContainers: + now := time.Now() + elapsed := now.Sub(start) + if elapsed < from { + t.Errorf("Received sample earlier (%s) than expected (%s)", elapsed, from) + return + } + assert.IsType(t, expected, sampleContainer) + expSamples := expected.GetSamples() + gotSamples := sampleContainer.GetSamples() + if assert.Len(t, gotSamples, len(expSamples)) { + for i, s := range gotSamples { + expS := expSamples[i] + if s.Metric != metrics.IterationDuration { + assert.Equal(t, expS.Value, s.Value) + } + assert.Equal(t, expS.Metric.Name, s.Metric.Name) + assert.Equal(t, expS.Tags.CloneTags(), s.Tags.CloneTags()) + assert.InDelta(t, 0, now.Sub(s.Time), float64(50*time.Millisecond)) + } + } + return + case <-time.After(to): + t.Errorf("Did not receive sample in the maximum alotted time (%s)", to) + return + } + } + } + + getTags := func(args ...string) *stats.SampleTags { + tags := map[string]string{} + for i := 0; i < len(args)-1; i += 2 { + tags[args[i]] = args[i+1] + } + return stats.IntoSampleTags(&tags) + } + testCounter := stats.New("test_counter", stats.Counter) + getSample := func(expValue float64, expMetric *stats.Metric, expTags ...string) stats.SampleContainer { + return stats.Sample{ + Metric: expMetric, + Time: time.Now(), + Tags: getTags(expTags...), + Value: expValue, + } + } + getDummyTrail := func(group string) stats.SampleContainer { + return netext.NewDialer(net.Dialer{}).GetTrail(time.Now(), time.Now(), getTags("group", group)) + } + + // Initially give a long time (5s) for the executor to start + expectIn(0, 5000, getSample(1, testCounter, "group", "::setup", "place", "setupBeforeSleep")) + expectIn(900, 1100, getSample(2, testCounter, "group", "::setup", "place", "setupAfterSleep")) + expectIn(0, 100, getDummyTrail("::setup")) + + expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep")) + expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep")) + expectIn(0, 100, getDummyTrail("")) + expectIn(0, 100, getSample(1, metrics.Iterations)) + + expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep")) + expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep")) + expectIn(0, 100, getDummyTrail("")) + expectIn(0, 100, getSample(1, metrics.Iterations)) + + expectIn(0, 1000, getSample(3, testCounter, "group", "::teardown", "place", "teardownBeforeSleep")) + expectIn(900, 1100, getSample(4, testCounter, "group", "::teardown", "place", "teardownAfterSleep")) + expectIn(0, 100, getDummyTrail("::teardown")) + + for { + select { + case s := <-sampleContainers: + t.Fatalf("Did not expect anything in the sample channel bug got %#v", s) + case <-time.After(3 * time.Second): + t.Fatalf("Local executor took way to long to finish") + case <-done: + return // Exit normally + } + } +} diff --git a/js/runner.go b/js/runner.go index 0d3073f74949..77de66574895 100644 --- a/js/runner.go +++ b/js/runner.go @@ -386,13 +386,15 @@ func (u *VU) runFn(ctx context.Context, group *lib.Group, fn goja.Callable, args if state.Options.SystemTags["iter"] { tags["iter"] = strconv.FormatInt(iter, 10) } - sampleTags := stats.IntoSampleTags(&tags) + if state.Options.SystemTags["group"] { + tags["group"] = group.Path + } if u.Runner.Bundle.Options.NoConnectionReuse.Bool { u.HTTPTransport.CloseIdleConnections() } - state.Samples <- u.Dialer.GetTrail(startTime, endTime, sampleTags) + state.Samples <- u.Dialer.GetTrail(startTime, endTime, stats.IntoSampleTags(&tags)) return v, state, err } From ae4076b8bb037390bc6d1ca16a3a3807ce2b1737 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 22 Jun 2018 16:05:17 +0300 Subject: [PATCH 08/14] Add test for correct treshold handling with setup and teardown This addresses the first part of https://github.com/loadimpact/k6/issues/680 --- core/engine_test.go | 67 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/core/engine_test.go b/core/engine_test.go index c461994a57ee..3b066282fc55 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -749,3 +749,70 @@ func TestRunTags(t *testing.T) { } } } + +func TestSetupTeardownThresholds(t *testing.T) { + t.Parallel() + tb := testutils.NewHTTPMultiBin(t) + defer tb.Cleanup() + + script := []byte(tb.Replacer.Replace(` + import http from "k6/http"; + import { check } from "k6"; + import { Counter } from "k6/metrics"; + + let statusCheck = { "status is 200": (r) => r.status === 200 } + let myCounter = new Counter("setup_teardown"); + + export let options = { + iterations: 5, + thresholds: { + "setup_teardown": ["count == 2"], + "iterations": ["count == 5"], + "http_reqs": ["count == 7"], + }, + }; + + export function setup() { + check(http.get("HTTPBIN_IP_URL"), statusCheck) && myCounter.add(1); + }; + + export default function () { + check(http.get("HTTPBIN_IP_URL"), statusCheck); + }; + + export function teardown() { + check(http.get("HTTPBIN_IP_URL"), statusCheck) && myCounter.add(1); + }; + `)) + + runner, err := js.New( + &lib.SourceData{Filename: "/script.js", Data: script}, + afero.NewMemMapFs(), + lib.RuntimeOptions{}, + ) + require.NoError(t, err) + runner.SetOptions(runner.GetOptions().Apply(lib.Options{ + SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), + SetupTimeout: types.NullDurationFrom(3 * time.Second), + TeardownTimeout: types.NullDurationFrom(3 * time.Second), + VUs: null.IntFrom(3), + VUsMax: null.IntFrom(3), + })) + + engine, err := NewEngine(local.New(runner), runner.GetOptions()) + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + errC := make(chan error) + go func() { errC <- engine.Run(ctx) }() + + select { + case <-time.After(10 * time.Second): + cancel() + t.Fatal("Test timed out") + case err := <-errC: + cancel() + require.NoError(t, err) + require.False(t, engine.IsTainted()) + } +} From f4241d782bd171105d3cd24f5d2ad9b92cd2da0e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 22 Jun 2018 16:10:13 +0300 Subject: [PATCH 09/14] Fix a typo --- core/local/local_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index c8ebe2dcc27e..130db0741184 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -526,7 +526,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { } return case <-time.After(to): - t.Errorf("Did not receive sample in the maximum alotted time (%s)", to) + t.Errorf("Did not receive sample in the maximum allotted time (%s)", to) return } } From 241a5fdfbeffc179ec31fdc407f7220a9efc6dc1 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 25 Jun 2018 11:15:10 +0300 Subject: [PATCH 10/14] Add type safety and missing run and result statuses --- core/engine.go | 2 +- lib/collector.go | 25 ++++++++++++++++++------- stats/cloud/api.go | 25 ++++++++++++++++--------- stats/cloud/collector.go | 4 ++-- stats/dummy/collector.go | 9 +++++---- stats/influxdb/collector.go | 3 ++- stats/json/collector.go | 2 +- stats/kafka/collector.go | 3 ++- 8 files changed, 47 insertions(+), 26 deletions(-) diff --git a/core/engine.go b/core/engine.go index 55d0a5ace1d2..f4fd3f03f786 100644 --- a/core/engine.go +++ b/core/engine.go @@ -104,7 +104,7 @@ func NewEngine(ex lib.Executor, o lib.Options) (*Engine, error) { return e, nil } -func (e *Engine) setRunStatus(status int) { +func (e *Engine) setRunStatus(status lib.RunStatus) { if len(e.Collectors) == 0 { return } diff --git a/lib/collector.go b/lib/collector.go index 40d315161f79..e3b77837b027 100644 --- a/lib/collector.go +++ b/lib/collector.go @@ -26,13 +26,24 @@ import ( "github.com/loadimpact/k6/stats" ) -// Run Status used by cloud collector +// RunStatus values can be used by k6 to denote how a script run ends +// and by the cloud executor and collector so that k6 knows the current +// status of a particular script run. +type RunStatus int + +// Possible run status values; iota isn't used intentionally const ( - RunStatusFinished = 3 - RunStatusAbortedUser = 5 - RunStatusAbortedSystem = 6 - RunStatusAbortedScriptError = 7 - RunStatusAbortedThreshold = 8 + RunStatusCreated RunStatus = -2 + RunStatusValidated RunStatus = -1 + RunStatusQueued RunStatus = 0 + RunStatusInitializing RunStatus = 1 + RunStatusRunning RunStatus = 2 + RunStatusFinished RunStatus = 3 + RunStatusTimedOut RunStatus = 4 + RunStatusAbortedUser RunStatus = 5 + RunStatusAbortedSystem RunStatus = 6 + RunStatusAbortedScriptError RunStatus = 7 + RunStatusAbortedThreshold RunStatus = 8 ) // A Collector abstracts the process of funneling samples to an external storage backend, @@ -57,5 +68,5 @@ type Collector interface { GetRequiredSystemTags() TagSet // Set run status - SetRunStatus(status int) + SetRunStatus(status RunStatus) } diff --git a/stats/cloud/api.go b/stats/cloud/api.go index f78945e4b1fd..dd51da19429b 100644 --- a/stats/cloud/api.go +++ b/stats/cloud/api.go @@ -33,6 +33,13 @@ import ( "github.com/pkg/errors" ) +type ResultStatus int + +const ( + ResultStatusPassed ResultStatus = 0 + ResultStatusFailed ResultStatus = 1 +) + type ThresholdResult map[string]map[string]bool type TestRun struct { @@ -50,10 +57,10 @@ type CreateTestRunResponse struct { } type TestProgressResponse struct { - RunStatusText string `json:"run_status_text"` - RunStatus int `json:"run_status"` - ResultStatus int `json:"result_status"` - Progress float64 `json:"progress"` + RunStatusText string `json:"run_status_text"` + RunStatus lib.RunStatus `json:"run_status"` + ResultStatus ResultStatus `json:"result_status"` + Progress float64 `json:"progress"` } type LoginResponse struct { @@ -156,17 +163,17 @@ func (c *Client) StartCloudTestRun(name string, projectID int64, arc *lib.Archiv return ctrr.ReferenceID, nil } -func (c *Client) TestFinished(referenceID string, thresholds ThresholdResult, tained bool, runStatus int) error { +func (c *Client) TestFinished(referenceID string, thresholds ThresholdResult, tained bool, runStatus lib.RunStatus) error { url := fmt.Sprintf("%s/tests/%s", c.baseURL, referenceID) - resultStatus := 0 + resultStatus := ResultStatusPassed if tained { - resultStatus = 1 + resultStatus = ResultStatusFailed } data := struct { - ResultStatus int `json:"result_status"` - RunStatus int `json:"run_status"` + ResultStatus ResultStatus `json:"result_status"` + RunStatus lib.RunStatus `json:"run_status"` Thresholds ThresholdResult `json:"thresholds"` }{ resultStatus, diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index 16e20261e728..eb2a351aacc2 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -51,7 +51,7 @@ type Collector struct { client *Client anonymous bool - runStatus int + runStatus lib.RunStatus bufferMutex sync.Mutex bufferHTTPTrails []*netext.Trail @@ -457,6 +457,6 @@ func (c *Collector) GetRequiredSystemTags() lib.TagSet { return lib.GetTagSet("name", "method", "status", "error", "check", "group") } -func (c *Collector) SetRunStatus(status int) { +func (c *Collector) SetRunStatus(status lib.RunStatus) { c.runStatus = status } diff --git a/stats/dummy/collector.go b/stats/dummy/collector.go index 21524af7a8b4..f5ad2f9a8aee 100644 --- a/stats/dummy/collector.go +++ b/stats/dummy/collector.go @@ -30,7 +30,7 @@ import ( // Collector implements the lib.Collector interface and should be used only for testing type Collector struct { - runStatus int + RunStatus lib.RunStatus SampleContainers []stats.SampleContainer Samples []stats.Sample @@ -48,7 +48,7 @@ func (c *Collector) MakeConfig() interface{} { return nil } // Run just blocks until the context is done func (c *Collector) Run(ctx context.Context) { <-ctx.Done() - log.Debugf("finished status: %s", c.runStatus) + log.Debugf("finished status: %s", c.RunStatus) } // Collect just appends all of the samples passed to it to the internal sample slice. @@ -74,6 +74,7 @@ func (c *Collector) GetRequiredSystemTags() lib.TagSet { return lib.TagSet{} // There are no required tags for this collector } -func (c *Collector) SetRunStatus(status int) { - c.runStatus = status +// SetRunStatus just saves the passed status for later inspection +func (c *Collector) SetRunStatus(status lib.RunStatus) { + c.RunStatus = status } diff --git a/stats/influxdb/collector.go b/stats/influxdb/collector.go index 75ed4eee8194..67ad808ab554 100644 --- a/stats/influxdb/collector.go +++ b/stats/influxdb/collector.go @@ -192,4 +192,5 @@ func (c *Collector) GetRequiredSystemTags() lib.TagSet { return lib.TagSet{} // There are no required tags for this collector } -func (c *Collector) SetRunStatus(status int) {} +// SetRunStatus does nothing in the InfluxDB collector +func (c *Collector) SetRunStatus(status lib.RunStatus) {} diff --git a/stats/json/collector.go b/stats/json/collector.go index 9f8d8ce012c5..d0a9b1a0fa60 100644 --- a/stats/json/collector.go +++ b/stats/json/collector.go @@ -72,7 +72,7 @@ func (c *Collector) Init() error { return nil } -func (c *Collector) SetRunStatus(status int) {} +func (c *Collector) SetRunStatus(status lib.RunStatus) {} func (c *Collector) Run(ctx context.Context) { log.WithField("filename", c.fname).Debug("JSON: Writing JSON metrics") diff --git a/stats/kafka/collector.go b/stats/kafka/collector.go index 91194c8d7a93..1de508ead47c 100644 --- a/stats/kafka/collector.go +++ b/stats/kafka/collector.go @@ -103,7 +103,8 @@ func (c *Collector) GetRequiredSystemTags() lib.TagSet { return lib.TagSet{} // There are no required tags for this collector } -func (c *Collector) SetRunStatus(status int) {} +// SetRunStatus does nothing in the Kafka collector +func (c *Collector) SetRunStatus(status lib.RunStatus) {} func (c *Collector) formatSamples(samples stats.Samples) ([]string, error) { var metrics []string From 7888f9b268f847e8d29ef71ca3f8a9f8a9bd1539 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 25 Jun 2018 11:16:16 +0300 Subject: [PATCH 11/14] Fix nil pointer and improve code readability --- cmd/cloud.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 4fa2d26959a6..0f4e3ecfe55f 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -30,6 +30,7 @@ import ( "time" "github.com/kelseyhightower/envconfig" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats/cloud" "github.com/loadimpact/k6/ui" "github.com/pkg/errors" @@ -168,7 +169,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud case <-ticker.C: testProgress, progressErr = client.GetTestProgress(refID) if progressErr == nil { - if (testProgress.RunStatus > 2) || (exitOnRunning && testProgress.RunStatus == 2) { + if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } progress.Progress = testProgress.Progress @@ -180,17 +181,22 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud break runningLoop } case sig := <-sigC: - log.WithField("sig", sig).Debug("Exiting in response to signal") + log.WithField("sig", sig).Print("Exiting in response to signal...") err := client.StopCloudTestRun(refID) if err != nil { log.WithError(err).Error("Stop cloud test error") } - shouldExitLoop = true + shouldExitLoop = true // Exit after the next GetTestProgress call } } + + if testProgress == nil { + return ExitCode{errors.New("Test progress error"), 98} + } + fmt.Fprintf(stdout, " test status: %s\n", ui.ValueColor.Sprint(testProgress.RunStatusText)) - if testProgress.ResultStatus == 1 { + if testProgress.ResultStatus == cloud.ResultStatusFailed { return ExitCode{errors.New("The test has failed"), 99} } From 41177756e724350ab0dfe693afd53e672af002a3 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 25 Jun 2018 11:39:32 +0300 Subject: [PATCH 12/14] Update the release notes --- release notes/upcoming.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release notes/upcoming.md b/release notes/upcoming.md index 9cd704535191..1ddc88b657d3 100644 --- a/release notes/upcoming.md +++ b/release notes/upcoming.md @@ -24,7 +24,7 @@ Description of feature. ## Bugs fixed! -* Category: description of bug. (#PR) +* Fixed a potential `nil` pointer error when the `k6 cloud` command is interrupted. (#682) ## Breaking Changes * The `--no-connection-reuse` option has been re-purposed and now disables keep-alive connections globally. The newly added `--no-vu-connection-reuse` option does what was previously done by `--no-connection-reuse` - it closes any open connections between iterations of a VU, but allows for reusing them inside of a single iteration. (#676) From f7ff8007c50c83cd583605fd2035ad27c2a7c055 Mon Sep 17 00:00:00 2001 From: Luiz Filho Date: Mon, 25 Jun 2018 13:24:39 -0300 Subject: [PATCH 13/14] Refactor influx config struct to use null types (#679) closes #586 --- cmd/login_influxdb.go | 14 ++----- lib/types/types.go | 55 ++++++++++++++++++++++++ lib/types/types_test.go | 71 +++++++++++++++++++++++++++++++ stats/influxdb/collector.go | 2 +- stats/influxdb/config.go | 79 +++++++++++++++++++++-------------- stats/influxdb/config_test.go | 26 ++++++------ stats/influxdb/util.go | 31 +++++++------- stats/influxdb/util_test.go | 3 +- 8 files changed, 210 insertions(+), 71 deletions(-) diff --git a/cmd/login_influxdb.go b/cmd/login_influxdb.go index 0836469ce62f..b384814d12f5 100644 --- a/cmd/login_influxdb.go +++ b/cmd/login_influxdb.go @@ -54,34 +54,28 @@ This will set the default server used when just "-o influxdb" is passed.`, } conf = conf.Apply(urlConf) } - if conf.Addr == "" { - conf.Addr = "http://localhost:8086" - } - if conf.DB == "" { - conf.DB = "k6" - } form := ui.Form{ Fields: []ui.Field{ ui.StringField{ Key: "Addr", Label: "Address", - Default: conf.Addr, + Default: conf.Addr.String, }, ui.StringField{ Key: "DB", Label: "Database", - Default: conf.DB, + Default: conf.DB.String, }, ui.StringField{ Key: "Username", Label: "Username", - Default: conf.Username, + Default: conf.Username.String, }, ui.StringField{ Key: "Password", Label: "Password", - Default: conf.Password, + Default: conf.Password.String, }, }, } diff --git a/lib/types/types.go b/lib/types/types.go index 451068aaf236..a6e6305837fe 100644 --- a/lib/types/types.go +++ b/lib/types/types.go @@ -23,9 +23,64 @@ package types import ( "bytes" "encoding/json" + "fmt" + "reflect" "time" + + null "gopkg.in/guregu/null.v3" ) +func NullDecoder(f reflect.Type, t reflect.Type, data interface{}) (interface{}, error) { + typeFrom := f.String() + typeTo := t.String() + + expectedType := "" + switch typeTo { + case "null.String": + if typeFrom == reflect.String.String() { + return null.StringFrom(data.(string)), nil + } + expectedType = reflect.String.String() + case "null.Bool": + if typeFrom == reflect.Bool.String() { + return null.BoolFrom(data.(bool)), nil + } + expectedType = reflect.Bool.String() + case "null.Int": + if typeFrom == reflect.Int.String() { + return null.IntFrom(int64(data.(int))), nil + } + if typeFrom == reflect.Int32.String() { + return null.IntFrom(int64(data.(int32))), nil + } + if typeFrom == reflect.Int64.String() { + return null.IntFrom(data.(int64)), nil + } + expectedType = reflect.Int.String() + case "null.Float": + if typeFrom == reflect.Float32.String() { + return null.FloatFrom(float64(data.(float32))), nil + } + if typeFrom == reflect.Float64.String() { + return null.FloatFrom(data.(float64)), nil + } + expectedType = reflect.Float32.String() + " or " + reflect.Float64.String() + case "types.NullDuration": + if typeFrom == reflect.String.String() { + var d NullDuration + err := d.UnmarshalText([]byte(data.(string))) + return d, err + } + expectedType = reflect.String.String() + } + + if expectedType != "" { + return data, fmt.Errorf("expected '%s', got '%s'", expectedType, typeFrom) + } + + return data, nil +} + // Duration is an alias for time.Duration that de/serialises to JSON as human-readable strings. type Duration time.Duration diff --git a/lib/types/types_test.go b/lib/types/types_test.go index 3cf3da1dcd2d..d98db6473f2d 100644 --- a/lib/types/types_test.go +++ b/lib/types/types_test.go @@ -22,12 +22,83 @@ package types import ( "encoding/json" + "fmt" "testing" "time" + "github.com/mitchellh/mapstructure" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" ) +func TestNullDecoder(t *testing.T) { + type foo struct { + Strs []string + Str null.String + Boolean null.Bool + Integer null.Int + Integer32 null.Int + Integer64 null.Int + Float32 null.Float + Float64 null.Float + Dur NullDuration + } + f := foo{} + dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: NullDecoder, + Result: &f, + }) + require.NoError(t, err) + + conf := map[string]interface{}{ + "strs": []string{"fake"}, + "str": "bar", + "boolean": true, + "integer": 42, + "integer32": int32(42), + "integer64": int64(42), + "float32": float32(3.14), + "float64": float64(3.14), + "dur": "1m", + } + + err = dec.Decode(conf) + require.NoError(t, err) + + require.Equal(t, foo{ + Strs: []string{"fake"}, + Str: null.StringFrom("bar"), + Boolean: null.BoolFrom(true), + Integer: null.IntFrom(42), + Integer32: null.IntFrom(42), + Integer64: null.IntFrom(42), + Float32: null.FloatFrom(3.140000104904175), + Float64: null.FloatFrom(3.14), + Dur: NewNullDuration(1*time.Minute, true), + }, f) + + input := map[string][]interface{}{ + "Str": {true, "string", "bool"}, + "Boolean": {"invalid", "bool", "string"}, + "Integer": {"invalid", "int", "string"}, + "Integer32": {true, "int", "bool"}, + "Integer64": {"invalid", "int", "string"}, + "Float32": {true, "float32 or float64", "bool"}, + "Float64": {"invalid", "float32 or float64", "string"}, + "Dur": {10, "string", "int"}, + } + + for k, v := range input { + t.Run("Error Message/"+k, func(t *testing.T) { + err = dec.Decode(map[string]interface{}{ + k: v[0], + }) + assert.EqualError(t, err, fmt.Sprintf("1 error(s) decoding:\n\n* error decoding '%s': expected '%s', got '%s'", k, v[1], v[2])) + }) + } +} + func TestDuration(t *testing.T) { t.Run("String", func(t *testing.T) { assert.Equal(t, "1m15s", Duration(75*time.Second).String()) diff --git a/stats/influxdb/collector.go b/stats/influxdb/collector.go index 75ed4eee8194..008364b62c85 100644 --- a/stats/influxdb/collector.go +++ b/stats/influxdb/collector.go @@ -94,7 +94,7 @@ func (c *Collector) Collect(scs []stats.SampleContainer) { } func (c *Collector) Link() string { - return c.Config.Addr + return c.Config.Addr.String } func (c *Collector) commit() { diff --git a/stats/influxdb/config.go b/stats/influxdb/config.go index c9b443abeb5b..52298b7290b4 100644 --- a/stats/influxdb/config.go +++ b/stats/influxdb/config.go @@ -26,58 +26,63 @@ import ( "strings" "github.com/kubernetes/helm/pkg/strvals" + "github.com/loadimpact/k6/lib/types" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + null "gopkg.in/guregu/null.v3" ) type Config struct { // Connection. - Addr string `json:"addr" envconfig:"INFLUXDB_ADDR"` - Username string `json:"username,omitempty" envconfig:"INFLUXDB_USERNAME"` - Password string `json:"password,omitempty" envconfig:"INFLUXDB_PASSWORD"` - Insecure bool `json:"insecure,omitempty" envconfig:"INFLUXDB_INSECURE"` - PayloadSize int `json:"payloadSize,omitempty" envconfig:"INFLUXDB_PAYLOAD_SIZE"` + Addr null.String `json:"addr" envconfig:"INFLUXDB_ADDR"` + Username null.String `json:"username,omitempty" envconfig:"INFLUXDB_USERNAME"` + Password null.String `json:"password,omitempty" envconfig:"INFLUXDB_PASSWORD"` + Insecure null.Bool `json:"insecure,omitempty" envconfig:"INFLUXDB_INSECURE"` + PayloadSize null.Int `json:"payloadSize,omitempty" envconfig:"INFLUXDB_PAYLOAD_SIZE"` // Samples. - DB string `json:"db" envconfig:"INFLUXDB_DB"` - Precision string `json:"precision,omitempty" envconfig:"INFLUXDB_PRECISION"` - Retention string `json:"retention,omitempty" envconfig:"INFLUXDB_RETENTION"` - Consistency string `json:"consistency,omitempty" envconfig:"INFLUXDB_CONSISTENCY"` - TagsAsFields []string `json:"tagsAsFields,omitempty" envconfig:"INFLUXDB_TAGS_AS_FIELDS"` + DB null.String `json:"db" envconfig:"INFLUXDB_DB"` + Precision null.String `json:"precision,omitempty" envconfig:"INFLUXDB_PRECISION"` + Retention null.String `json:"retention,omitempty" envconfig:"INFLUXDB_RETENTION"` + Consistency null.String `json:"consistency,omitempty" envconfig:"INFLUXDB_CONSISTENCY"` + TagsAsFields []string `json:"tagsAsFields,omitempty" envconfig:"INFLUXDB_TAGS_AS_FIELDS"` } func NewConfig() *Config { - c := &Config{TagsAsFields: []string{"vu", "iter", "url"}} + c := &Config{ + Addr: null.NewString("http://localhost:8086", false), + DB: null.NewString("k6", false), + TagsAsFields: []string{"vu", "iter", "url"}, + } return c } func (c Config) Apply(cfg Config) Config { - //TODO: fix this, use nullable values like all other configs... - if cfg.Addr != "" { + if cfg.Addr.Valid { c.Addr = cfg.Addr } - if cfg.Username != "" { + if cfg.Username.Valid { c.Username = cfg.Username } - if cfg.Password != "" { + if cfg.Password.Valid { c.Password = cfg.Password } - if cfg.Insecure { + if cfg.Insecure.Valid { c.Insecure = cfg.Insecure } - if cfg.PayloadSize > 0 { + if cfg.PayloadSize.Valid && cfg.PayloadSize.Int64 > 0 { c.PayloadSize = cfg.PayloadSize } - if cfg.DB != "" { + if cfg.DB.Valid { c.DB = cfg.DB } - if cfg.Precision != "" { + if cfg.Precision.Valid { c.Precision = cfg.Precision } - if cfg.Retention != "" { + if cfg.Retention.Valid { c.Retention = cfg.Retention } - if cfg.Consistency != "" { + if cfg.Consistency.Valid { c.Consistency = cfg.Consistency } if len(cfg.TagsAsFields) > 0 { @@ -105,8 +110,15 @@ func ParseMap(m map[string]interface{}) (Config, error) { if v, ok := m["tagsAsFields"].(string); ok { m["tagsAsFields"] = []string{v} } + dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{ + DecodeHook: types.NullDecoder, + Result: &c, + }) + if err != nil { + return c, err + } - err := mapstructure.Decode(m, &c) + err = dec.Decode(m) return c, err } @@ -117,14 +129,15 @@ func ParseURL(text string) (Config, error) { return c, err } if u.Host != "" { - c.Addr = u.Scheme + "://" + u.Host + c.Addr = null.StringFrom(u.Scheme + "://" + u.Host) } if db := strings.TrimPrefix(u.Path, "/"); db != "" { - c.DB = db + c.DB = null.StringFrom(db) } if u.User != nil { - c.Username = u.User.Username() - c.Password, _ = u.User.Password() + c.Username = null.StringFrom(u.User.Username()) + pass, _ := u.User.Password() + c.Password = null.StringFrom(pass) } for k, vs := range u.Query() { switch k { @@ -132,20 +145,22 @@ func ParseURL(text string) (Config, error) { switch vs[0] { case "": case "false": - c.Insecure = false + c.Insecure = null.BoolFrom(false) case "true": - c.Insecure = true + c.Insecure = null.BoolFrom(true) default: return c, errors.Errorf("insecure must be true or false, not %s", vs[0]) } case "payload_size": - c.PayloadSize, err = strconv.Atoi(vs[0]) + var size int + size, err = strconv.Atoi(vs[0]) + c.PayloadSize = null.IntFrom(int64(size)) case "precision": - c.Precision = vs[0] + c.Precision = null.StringFrom(vs[0]) case "retention": - c.Retention = vs[0] + c.Retention = null.StringFrom(vs[0]) case "consistency": - c.Consistency = vs[0] + c.Consistency = null.StringFrom(vs[0]) case "tagsAsFields": c.TagsAsFields = vs default: diff --git a/stats/influxdb/config_test.go b/stats/influxdb/config_test.go index 414875c1bda1..d3db7021c92e 100644 --- a/stats/influxdb/config_test.go +++ b/stats/influxdb/config_test.go @@ -24,15 +24,17 @@ import ( "testing" "github.com/stretchr/testify/assert" + null "gopkg.in/guregu/null.v3" ) func TestParseArg(t *testing.T) { testdata := map[string]Config{ - "": {}, - "db=dbname": {DB: "dbname"}, - "addr=http://localhost:8086": {Addr: "http://localhost:8086"}, - "addr=http://localhost:8086,db=dbname": {Addr: "http://localhost:8086", DB: "dbname"}, - "addr=http://localhost:8086,db=dbname,insecure=false,payloadSize=69": {Addr: "http://localhost:8086", DB: "dbname", Insecure: false, PayloadSize: 69}, + "": {}, + "db=dbname": {DB: null.StringFrom("dbname")}, + "addr=http://localhost:8086": {Addr: null.StringFrom("http://localhost:8086")}, + "addr=http://localhost:8086,db=dbname": {Addr: null.StringFrom("http://localhost:8086"), DB: null.StringFrom("dbname")}, + "addr=http://localhost:8086,db=dbname,insecure=false,payloadSize=69,": {Addr: null.StringFrom("http://localhost:8086"), DB: null.StringFrom("dbname"), Insecure: null.BoolFrom(false), PayloadSize: null.IntFrom(69)}, + "addr=http://localhost:8086,db=dbname,insecure=false,payloadSize=69,tagsAsFields={fake}": {Addr: null.StringFrom("http://localhost:8086"), DB: null.StringFrom("dbname"), Insecure: null.BoolFrom(false), PayloadSize: null.IntFrom(69), TagsAsFields: []string{"fake"}}, } for str, expConfig := range testdata { @@ -48,20 +50,20 @@ func TestParseArg(t *testing.T) { func TestParseURL(t *testing.T) { testdata := map[string]Config{ "": {}, - "dbname": {DB: "dbname"}, - "/dbname": {DB: "dbname"}, - "http://localhost:8086": {Addr: "http://localhost:8086"}, - "http://localhost:8086/dbname": {Addr: "http://localhost:8086", DB: "dbname"}, + "dbname": {DB: null.StringFrom("dbname")}, + "/dbname": {DB: null.StringFrom("dbname")}, + "http://localhost:8086": {Addr: null.StringFrom("http://localhost:8086")}, + "http://localhost:8086/dbname": {Addr: null.StringFrom("http://localhost:8086"), DB: null.StringFrom("dbname")}, } queries := map[string]struct { Config Config Err string }{ "?": {Config{}, ""}, - "?insecure=false": {Config{Insecure: false}, ""}, - "?insecure=true": {Config{Insecure: true}, ""}, + "?insecure=false": {Config{Insecure: null.BoolFrom(false)}, ""}, + "?insecure=true": {Config{Insecure: null.BoolFrom(true)}, ""}, "?insecure=ture": {Config{}, "insecure must be true or false, not ture"}, - "?payload_size=69": {Config{PayloadSize: 69}, ""}, + "?payload_size=69": {Config{PayloadSize: null.IntFrom(69)}, ""}, "?payload_size=a": {Config{}, "strconv.Atoi: parsing \"a\": invalid syntax"}, } for str, data := range testdata { diff --git a/stats/influxdb/util.go b/stats/influxdb/util.go index c2f4b074c561..4f88975836ae 100644 --- a/stats/influxdb/util.go +++ b/stats/influxdb/util.go @@ -24,35 +24,36 @@ import ( "strings" client "github.com/influxdata/influxdb/client/v2" + null "gopkg.in/guregu/null.v3" ) func MakeClient(conf Config) (client.Client, error) { - if strings.HasPrefix(conf.Addr, "udp://") { + if strings.HasPrefix(conf.Addr.String, "udp://") { return client.NewUDPClient(client.UDPConfig{ - Addr: strings.TrimPrefix(conf.Addr, "udp://"), - PayloadSize: conf.PayloadSize, + Addr: strings.TrimPrefix(conf.Addr.String, "udp://"), + PayloadSize: int(conf.PayloadSize.Int64), }) } - if conf.Addr == "" { - conf.Addr = "http://localhost:8086" + if conf.Addr.String == "" { + conf.Addr = null.StringFrom("http://localhost:8086") } return client.NewHTTPClient(client.HTTPConfig{ - Addr: conf.Addr, - Username: conf.Username, - Password: conf.Password, + Addr: conf.Addr.String, + Username: conf.Username.String, + Password: conf.Password.String, UserAgent: "k6", - InsecureSkipVerify: conf.Insecure, + InsecureSkipVerify: conf.Insecure.Bool, }) } func MakeBatchConfig(conf Config) client.BatchPointsConfig { - if conf.DB == "" { - conf.DB = "k6" + if !conf.DB.Valid || conf.DB.String == "" { + conf.DB = null.StringFrom("k6") } return client.BatchPointsConfig{ - Precision: conf.Precision, - Database: conf.DB, - RetentionPolicy: conf.Retention, - WriteConsistency: conf.Consistency, + Precision: conf.Precision.String, + Database: conf.DB.String, + RetentionPolicy: conf.Retention.String, + WriteConsistency: conf.Consistency.String, } } diff --git a/stats/influxdb/util_test.go b/stats/influxdb/util_test.go index d793b12a4568..2ad746d10824 100644 --- a/stats/influxdb/util_test.go +++ b/stats/influxdb/util_test.go @@ -25,6 +25,7 @@ import ( client "github.com/influxdata/influxdb/client/v2" "github.com/stretchr/testify/assert" + null "gopkg.in/guregu/null.v3" ) func TestMakeBatchConfig(t *testing.T) { @@ -37,7 +38,7 @@ func TestMakeBatchConfig(t *testing.T) { t.Run("DB Set", func(t *testing.T) { assert.Equal(t, client.BatchPointsConfig{Database: "dbname"}, - MakeBatchConfig(Config{DB: "dbname"}), + MakeBatchConfig(Config{DB: null.StringFrom("dbname")}), ) }) } From 748892c0fdd7bf8a89c96e24b926eca7e0d62a3e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 28 Jun 2018 14:07:28 +0300 Subject: [PATCH 14/14] Future-proof GetBufferedSamples() Currently the samples channel is not closed, but this change would save us from headaches if we decide to change that in the future --- stats/stats.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/stats/stats.go b/stats/stats.go index de4fc9c2582c..32fdae1683af 100644 --- a/stats/stats.go +++ b/stats/stats.go @@ -375,7 +375,10 @@ var _ ConnectedSampleContainer = ConnectedSamples{} func GetBufferedSamples(input <-chan SampleContainer) (result []SampleContainer) { for { select { - case val := <-input: + case val, ok := <-input: + if !ok { + return + } result = append(result, val) default: return