From d7833003c256fa90b945658a49d143a883ac59f7 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:11:01 +0300 Subject: [PATCH 001/350] Move the scheduler interfaces and configmap in lib --- lib/helpers.go | 153 +++++++++++ lib/{scheduler => }/helpers_test.go | 19 +- lib/{scheduler/interfaces.go => lib.go} | 15 +- lib/scheduler/configmap.go | 128 --------- lib/schedulers.go | 331 ++++++++++++++++++++++++ 5 files changed, 490 insertions(+), 156 deletions(-) create mode 100644 lib/helpers.go rename lib/{scheduler => }/helpers_test.go (73%) rename lib/{scheduler/interfaces.go => lib.go} (60%) delete mode 100644 lib/scheduler/configmap.go create mode 100644 lib/schedulers.go diff --git a/lib/helpers.go b/lib/helpers.go new file mode 100644 index 00000000000..20659a5cc4a --- /dev/null +++ b/lib/helpers.go @@ -0,0 +1,153 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package lib + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "strings" + "time" +) + +// StrictJSONUnmarshal decodes a JSON in a strict manner, emitting an error if there +// are unknown fields or unexpected data +func StrictJSONUnmarshal(data []byte, v interface{}) error { + dec := json.NewDecoder(bytes.NewReader(data)) + dec.DisallowUnknownFields() + dec.UseNumber() + + if err := dec.Decode(&v); err != nil { + return err + } + if dec.More() { + //TODO: use a custom error? + return fmt.Errorf("unexpected data after the JSON object") + } + return nil +} + +// GetMaxPlannedVUs returns the maximum number of planned VUs at any stage of +// the scheduler execution plan. +func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { + for _, s := range steps { + stepMaxPlannedVUs := s.PlannedVUs + if stepMaxPlannedVUs > result { + result = stepMaxPlannedVUs + } + } + return result +} + +// GetMaxPossibleVUs returns the maximum number of planned + unplanned (i.e. +// initialized mid-test) VUs at any stage of the scheduler execution plan. +// Unplanned VUs are possible in some schedulers, like the arrival-rate ones, as +// a way to have a low number of pre-allocated VUs, but be able to initialize +// new ones in the middle of the test, if needed. For example, if the remote +// system starts responding very slowly and all of the pre-allocated VUs are +// waiting for it. +// +// IMPORTANT 1: Getting planned and unplanned VUs separately for the whole +// duration of a test can often lead to mistakes. That's why this function is +// called GetMaxPossibleVUs() and why there is no GetMaxUnplannedVUs() function. +// +// As an example, imagine that you have a scheduler with MaxPlannedVUs=20 and +// MaxUnaplannedVUs=0, followed immediately after by another scheduler with +// MaxPlannedVUs=10 and MaxUnaplannedVUs=10. If you MaxPlannedVUs for the whole +// test is 20, and MaxUnaplannedVUs, but since those schedulers won't run +// concurrently, MaxVUs for the whole test is not 30, rather it's 20, since 20 +// VUs will be sufficient to run the test. +// +// IMPORTANT 2: this has one very important exception. The manual execution +// scheduler doesn't use the MaxUnplannedVUs (i.e. this function will return 0), +// since their initialization and usage is directly controlled by the user and +// is effectively bounded only by the resources of the machine k6 is running on. +func GetMaxPossibleVUs(steps []ExecutionStep) (result uint64) { + for _, s := range steps { + stepMaxPossibleVUs := s.PlannedVUs + s.MaxUnplannedVUs + if stepMaxPossibleVUs > result { + result = stepMaxPossibleVUs + } + } + return result +} + +// GetEndOffset returns the time offset of the last step of the execution plan, +// and whether that step is a final one, i.e. whether the number of planned or +// unplanned +func GetEndOffset(steps []ExecutionStep) (lastStepOffset time.Duration, isFinal bool) { + if len(steps) == 0 { + return 0, true + } + lastStep := steps[len(steps)-1] + return lastStep.TimeOffset, (lastStep.PlannedVUs == 0 && lastStep.MaxUnplannedVUs == 0) +} + +// A helper function for joining error messages into a single string +func concatErrors(errors []error, separator string) string { + errStrings := make([]string, len(errors)) + for i, e := range errors { + errStrings[i] = e.Error() + } + return strings.Join(errStrings, separator) +} + +// StreamExecutionSteps launches a new goroutine and emits all execution steps +// at their appropriate time offsets over the returned unbuffered channel. If +// closeChanWhenDone is specified, it will close the channel after it sends the +// last step. If it isn't, or if the context is cancelled, the internal +// goroutine will be stopped, *but the channel will remain open*! +// +// As usual, steps in the supplied slice have to be sorted by their TimeOffset +// values in an ascending order. Of course, multiple events can have the same +// time offset (incl. 0). +func StreamExecutionSteps( + ctx context.Context, startTime time.Time, steps []ExecutionStep, closeChanWhenDone bool, +) <-chan ExecutionStep { + + ch := make(chan ExecutionStep) + go func() { + for _, step := range steps { + offsetDiff := step.TimeOffset - time.Since(startTime) + if offsetDiff > 0 { // wait until time of event arrives + select { + case <-ctx.Done(): + return // exit if context is cancelled + case <-time.After(offsetDiff): //TODO: reuse a timer? + // do nothing + } + } + select { + case <-ctx.Done(): + return // exit if context is cancelled + case ch <- step: // send the step + } + } + + // Close the channel only if all steps were sent successfully (i.e. the + // parent context didn't die) and we were instructed to do so. + if closeChanWhenDone { + close(ch) + } + }() + return ch +} diff --git a/lib/scheduler/helpers_test.go b/lib/helpers_test.go similarity index 73% rename from lib/scheduler/helpers_test.go rename to lib/helpers_test.go index 8165f922fd6..7bcdf2ea8df 100644 --- a/lib/scheduler/helpers_test.go +++ b/lib/helpers_test.go @@ -18,7 +18,7 @@ * */ -package scheduler +package lib import ( "fmt" @@ -28,19 +28,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestCheckPercentagesSum(t *testing.T) { - t.Parallel() - assert.NoError(t, checkPercentagesSum([]float64{100})) - assert.NoError(t, checkPercentagesSum([]float64{50, 50})) - assert.NoError(t, checkPercentagesSum([]float64{100.0 / 3, 100.0 / 3, 100.0 / 3})) - assert.NoError(t, checkPercentagesSum([]float64{33.33, 33.33, 33.34})) - - assert.Error(t, checkPercentagesSum([]float64{})) - assert.Error(t, checkPercentagesSum([]float64{100 / 3, 100 / 3, 100 / 3})) - assert.Error(t, checkPercentagesSum([]float64{33.33, 33.33, 33.33})) - assert.Error(t, checkPercentagesSum([]float64{40, 40, 40})) -} - func TestStrictJSONUnmarshal(t *testing.T) { t.Parallel() type someElement struct { @@ -67,7 +54,7 @@ func TestStrictJSONUnmarshal(t *testing.T) { for i, tc := range testCases { tc := tc t.Run(fmt.Sprintf("TestCase#%d", i), func(t *testing.T) { - err := strictJSONUnmarshal([]byte(tc.data), &tc.destination) + err := StrictJSONUnmarshal([]byte(tc.data), &tc.destination) if tc.expectedError { require.Error(t, err) return @@ -77,3 +64,5 @@ func TestStrictJSONUnmarshal(t *testing.T) { }) } } + +//TODO: test EventStream very thoroughly diff --git a/lib/scheduler/interfaces.go b/lib/lib.go similarity index 60% rename from lib/scheduler/interfaces.go rename to lib/lib.go index 764df42ae85..c1a664e7e5d 100644 --- a/lib/scheduler/interfaces.go +++ b/lib/lib.go @@ -18,16 +18,5 @@ * */ -package scheduler - -import "time" - -// Config is an interface that should be implemented by all scheduler config types -type Config interface { - GetBaseConfig() BaseConfig - Validate() []error - GetMaxVUs() int64 - GetMaxDuration() time.Duration // includes max timeouts, to allow us to share VUs between schedulers in the future - //TODO: Split(percentages []float64) ([]Config, error) - //TODO: String() method that could be used for priting descriptions of the currently running schedulers for the UI? -} +// Package lib contains most interfaces and base structs of k6. +package lib diff --git a/lib/scheduler/configmap.go b/lib/scheduler/configmap.go deleted file mode 100644 index 072f26f81e7..00000000000 --- a/lib/scheduler/configmap.go +++ /dev/null @@ -1,128 +0,0 @@ -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2019 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package scheduler - -import ( - "encoding/json" - "fmt" - "sync" -) - -// ConfigMap can contain mixed scheduler config types -type ConfigMap map[string]Config - -// ConfigConstructor is a simple function that returns a concrete Config instance -// with the specified name and all default values correctly initialized -type ConfigConstructor func(name string, rawJSON []byte) (Config, error) - -//nolint:gochecknoglobals -var ( - configTypesMutex sync.RWMutex - configConstructors = make(map[string]ConfigConstructor) -) - -// RegisterConfigType adds the supplied ConfigConstructor as the constructor for its -// type in the configConstructors map, in a thread-safe manner -func RegisterConfigType(configType string, constructor ConfigConstructor) { - configTypesMutex.Lock() - defer configTypesMutex.Unlock() - - if constructor == nil { - panic("scheduler configs: constructor is nil") - } - if _, configTypeExists := configConstructors[configType]; configTypeExists { - panic("scheduler configs: RegisterConfigType called twice for " + configType) - } - - configConstructors[configType] = constructor -} - -// GetParsedConfig returns a struct instance corresponding to the supplied -// config type. It will be fully initialized - with both the default values of -// the type, as well as with whatever the user had specified in the JSON -func GetParsedConfig(name, configType string, rawJSON []byte) (result Config, err error) { - configTypesMutex.Lock() - defer configTypesMutex.Unlock() - - constructor, exists := configConstructors[configType] - if !exists { - return nil, fmt.Errorf("unknown execution scheduler type '%s'", configType) - } - return constructor(name, rawJSON) -} - -// UnmarshalJSON implements the json.Unmarshaler interface in a two-step manner, -// creating the correct type of configs based on the `type` property. -func (scs *ConfigMap) UnmarshalJSON(data []byte) error { - if len(data) == 0 { - return nil - } - - if len(data) == 4 && string(data) == "null" { - return nil - } - - //TODO: use a more sophisticated combination of dec.Token() and dec.More(), - // which would allow us to support both arrays and maps for this config? - var protoConfigs map[string]protoConfig - if err := strictJSONUnmarshal(data, &protoConfigs); err != nil { - return err - } - - result := make(ConfigMap, len(protoConfigs)) - for k, v := range protoConfigs { - if v.Type == "" { - return fmt.Errorf("execution config '%s' doesn't have a type value", k) - } - config, err := GetParsedConfig(k, v.Type, v.rawJSON) - if err != nil { - return err - } - result[k] = config - } - - *scs = result - - return nil -} - -// Validate checks if all of the specified scheduler options make sense -func (scs ConfigMap) Validate() (errors []error) { - for name, scheduler := range scs { - if schedErr := scheduler.Validate(); len(schedErr) != 0 { - errors = append(errors, - fmt.Errorf("scheduler %s has errors: %s", name, concatErrors(schedErr, ", "))) - } - } - return errors -} - -type protoConfig struct { - BaseConfig - rawJSON json.RawMessage -} - -// UnmarshalJSON just reads unmarshals the base config (to get the type), but it also -// stores the unprocessed JSON so we can parse the full config in the next step -func (pc *protoConfig) UnmarshalJSON(b []byte) error { - *pc = protoConfig{BaseConfig{}, b} - return json.Unmarshal(b, &pc.BaseConfig) -} diff --git a/lib/schedulers.go b/lib/schedulers.go new file mode 100644 index 00000000000..d1e8514bfd4 --- /dev/null +++ b/lib/schedulers.go @@ -0,0 +1,331 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package lib + +import ( + "context" + "encoding/json" + "fmt" + "sort" + "strings" + "sync" + "time" + + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" +) + +//TODO: remove globals and use some type of explicit dependency injection? +//nolint:gochecknoglobals +var ( + schedulerConfigTypesMutex sync.RWMutex + schedulerConfigConstructors = make(map[string]SchedulerConfigConstructor) +) + +// ExecutionStep is used by different schedulers to specify the planned number +// of VUs they will need at a particular time. The times are relative to their +// StartTime, i.e. they don't take into account the specific starting time of +// the scheduler, as that will be considered by the external executor +// separately. +// +// A slice [{t1, v1}, {t2, v2}, {t3, v3}, ..., {tn, vn}] of execution steps +// means that a scheduler will need 0 VUs until t1, it will need v1 number of +// VUs from time t1 until t2, need v2 number of VUs from time t2 to t3, and so +// on. t1 is usually 0, tn is usually the same as GetMaxDuration() and vn is +// usually 0. +// +// Keep in mind that t(i) may be exactly equal to t(i+i), when there's an abrupt +// transition in the number of VUs required by a scheduler. For example, the +// variable-looping-vus scheduler may have 0-duration stages, or it may scale up +// VUs in its last stage right until the end. These +// +// []ExecutionStep is also used by the SchedulerConfigMap, to represent the +// amount of needed VUs among all schedulers, during the whole execution of a +// test script. In that context, each scheduler's StartTime is accounted for and +// included in the offsets. +type ExecutionStep struct { + TimeOffset time.Duration + PlannedVUs uint64 + MaxUnplannedVUs uint64 +} + +//TODO: make []ExecutionStep or []SchedulerConfig their own type? + +// SchedulerConfig is an interface that should be implemented by all scheduler config types +type SchedulerConfig interface { + Validate() []error + + GetName() string + GetType() string + GetStartTime() time.Duration + GetGracefulStop() time.Duration + + // This is used to validate whether a particular script can run in the cloud + // or, in the future, in the native k6 distributed execution. Currently only + // the manual-execution scheduler should return false. + IsDistributable() bool + + GetEnv() map[string]string + GetExec() null.String //TODO: use interface{} so plain http requests can be specified? + + // Calculates the VU requirements in different stages of the scheduler's + // execution, including any extensions caused by waiting for iterations to + // finish with graceful stops or ramp-downs. + GetExecutionRequirements(*ExecutionSegment) []ExecutionStep + + // Return a human-readable description of the scheduler + GetDescription(es *ExecutionSegment) string + + NewScheduler(*ExecutorState, *logrus.Entry) (Scheduler, error) +} + +// InitVUFunc is just a shorthand so we don't have to type the function +// signature every time. +type InitVUFunc func(context.Context, *logrus.Entry) (VU, error) + +// Scheduler is the interface all schedulers should implement +type Scheduler interface { + GetConfig() SchedulerConfig + GetProgress() *pb.ProgressBar + GetLogger() *logrus.Entry + + Init(ctx context.Context) error + Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error + + // Currently used only in the manual executor + IsPausable() bool // TODO: make a separate interfaces + LiveUpdate(pause bool, newConfig SchedulerConfig) error +} + +// SchedulerConfigConstructor is a simple function that returns a concrete +// Config instance with the specified name and all default values correctly +// initialized +type SchedulerConfigConstructor func(name string, rawJSON []byte) (SchedulerConfig, error) + +// RegisterSchedulerConfigType adds the supplied SchedulerConfigConstructor as +// the constructor for its type in the configConstructors map, in a thread-safe +// manner +func RegisterSchedulerConfigType(configType string, constructor SchedulerConfigConstructor) { + schedulerConfigTypesMutex.Lock() + defer schedulerConfigTypesMutex.Unlock() + + if constructor == nil { + panic("scheduler configs: constructor is nil") + } + if _, configTypeExists := schedulerConfigConstructors[configType]; configTypeExists { + panic("scheduler configs: lib.RegisterSchedulerConfigType called twice for " + configType) + } + + schedulerConfigConstructors[configType] = constructor +} + +// SchedulerConfigMap can contain mixed scheduler config types +type SchedulerConfigMap map[string]SchedulerConfig + +// UnmarshalJSON implements the json.Unmarshaler interface in a two-step manner, +// creating the correct type of configs based on the `type` property. +func (scs *SchedulerConfigMap) UnmarshalJSON(data []byte) error { + if len(data) == 0 { + return nil + } + + if len(data) == 4 && string(data) == "null" { + return nil + } + + //TODO: use a more sophisticated combination of dec.Token() and dec.More(), + // which would allow us to support both arrays and maps for this config? + var protoConfigs map[string]protoSchedulerConfig + if err := StrictJSONUnmarshal(data, &protoConfigs); err != nil { + return err + } + + result := make(SchedulerConfigMap, len(protoConfigs)) + for k, v := range protoConfigs { + if v.configType == "" { + return fmt.Errorf("execution config '%s' doesn't have a type value", k) + } + config, err := GetParsedSchedulerConfig(k, v.configType, v.rawJSON) + if err != nil { + return err + } + result[k] = config + } + + *scs = result + + return nil +} + +// Validate checks if all of the specified scheduler options make sense +func (scs SchedulerConfigMap) Validate() (errors []error) { + for name, scheduler := range scs { + if schedErr := scheduler.Validate(); len(schedErr) != 0 { + errors = append(errors, + fmt.Errorf("scheduler %s has errors: %s", name, concatErrors(schedErr, ", "))) + } + } + return errors +} + +// GetSortedSchedulerConfigs returns a slice with the scheduler configurations, +// sorted in a consistent and predictable manner. It is useful when we want or +// have to avoid using maps with string keys (and tons of string lookups in +// them) and avoid the unpredictable iterations over Go maps. Slices allow us +// constant-time lookups and ordered iterations. +// +// The configs in the returned slice will be sorted by their start times in an +// ascending order, and alphabetically by their names (which are unique) if +// there are ties. +func (scs SchedulerConfigMap) GetSortedSchedulerConfigs() []SchedulerConfig { + configs := make([]SchedulerConfig, len(scs)) + + // Populate the configs slice with sorted scheduler configs + i := 0 + for _, config := range scs { + configs[i] = config // populate the slice in an unordered manner + i++ + } + sort.Slice(configs, func(a, b int) bool { // sort by (start time, name) + switch { + case configs[a].GetStartTime() < configs[b].GetStartTime(): + return true + case configs[a].GetStartTime() == configs[b].GetStartTime(): + return strings.Compare(configs[a].GetName(), configs[b].GetName()) < 0 + default: + return false + } + }) + + return configs +} + +// GetFullExecutionRequirements combines the execution requirements from all of +// the configured schedulers. It takes into account their start times and their +// individual VU requirements and calculates the total VU requirements for each +// moment in the test execution. +func (scs SchedulerConfigMap) GetFullExecutionRequirements(executionSegment *ExecutionSegment) []ExecutionStep { + sortedConfigs := scs.GetSortedSchedulerConfigs() + + // Combine the steps and requirements from all different schedulers, and + // sort them by their time offset, counting the schedulers' startTimes as + // well. + type trackedStep struct { + ExecutionStep + configID int + } + trackedSteps := []trackedStep{} + for configID, config := range sortedConfigs { // orderly iteration over a slice + configStartTime := config.GetStartTime() + configSteps := config.GetExecutionRequirements(executionSegment) + for _, cs := range configSteps { + cs.TimeOffset += configStartTime // add the scheduler start time to the step time offset + trackedSteps = append(trackedSteps, trackedStep{cs, configID}) + } + } + // Sort by (time offset, config id). It's important that we use stable + // sorting algorithm, since there are could be steps with the same time from + // the same scheduler and their order is important. + sort.SliceStable(trackedSteps, func(a, b int) bool { + switch { + case trackedSteps[a].TimeOffset < trackedSteps[b].TimeOffset: + return true + case trackedSteps[a].TimeOffset == trackedSteps[b].TimeOffset: + return trackedSteps[a].configID < trackedSteps[b].configID + default: + return false + } + }) + + // Go through all of the sorted steps from all of the schedulers, and + // build a new list of execution steps that consolidates all of their + // requirements. If multiple schedulers have an execution step at exactly + // the same time offset, they will be combined into a single new execution + // step with the sum of the values from the previous ones. + currentTimeOffset := time.Duration(0) + currentPlannedVUs := make([]uint64, len(scs)) + currentMaxUnplannedVUs := make([]uint64, len(scs)) + sum := func(data []uint64) (result uint64) { // sigh... + for _, val := range data { + result += val + } + return result + } + consolidatedSteps := []ExecutionStep{} + addCurrentStepIfDifferent := func() { + newPlannedVUs := sum(currentPlannedVUs) + newMaxUnplannedVUs := sum(currentMaxUnplannedVUs) + stepsLen := len(consolidatedSteps) + if stepsLen == 0 || + consolidatedSteps[stepsLen-1].PlannedVUs != newPlannedVUs || + consolidatedSteps[stepsLen-1].MaxUnplannedVUs != newMaxUnplannedVUs { + + consolidatedSteps = append(consolidatedSteps, ExecutionStep{ + TimeOffset: currentTimeOffset, + PlannedVUs: newPlannedVUs, + MaxUnplannedVUs: newMaxUnplannedVUs, + }) + } + } + for _, step := range trackedSteps { + // If the time offset is different, create a new step with the current values + if step.TimeOffset != currentTimeOffset { + addCurrentStepIfDifferent() + currentTimeOffset = step.TimeOffset + } + currentPlannedVUs[step.configID] = step.PlannedVUs + currentMaxUnplannedVUs[step.configID] = step.MaxUnplannedVUs + } + addCurrentStepIfDifferent() // Add the last step + return consolidatedSteps +} + +// GetParsedSchedulerConfig returns a struct instance corresponding to the supplied +// config type. It will be fully initialized - with both the default values of +// the type, as well as with whatever the user had specified in the JSON +func GetParsedSchedulerConfig(name, configType string, rawJSON []byte) (result SchedulerConfig, err error) { + schedulerConfigTypesMutex.Lock() + defer schedulerConfigTypesMutex.Unlock() + + constructor, exists := schedulerConfigConstructors[configType] + if !exists { + return nil, fmt.Errorf("unknown execution scheduler type '%s'", configType) + } + return constructor(name, rawJSON) +} + +type protoSchedulerConfig struct { + configType string + rawJSON json.RawMessage +} + +// UnmarshalJSON just reads unmarshals the base config (to get the type), but it also +// stores the unprocessed JSON so we can parse the full config in the next step +func (pc *protoSchedulerConfig) UnmarshalJSON(b []byte) error { + var tmp struct { + ConfigType string `json:"type"` + } + err := json.Unmarshal(b, &tmp) + *pc = protoSchedulerConfig{tmp.ConfigType, b} + return err +} From 9d21575affe98998d8c39ce46f9ffc60b184739b Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:12:34 +0300 Subject: [PATCH 002/350] Add the base execution segment type --- lib/execution_segment.go | 207 ++++++++++++++++++++++++++++++++++ lib/execution_segment_test.go | 23 ++++ 2 files changed, 230 insertions(+) create mode 100644 lib/execution_segment.go create mode 100644 lib/execution_segment_test.go diff --git a/lib/execution_segment.go b/lib/execution_segment.go new file mode 100644 index 00000000000..2e8ad8be09e --- /dev/null +++ b/lib/execution_segment.go @@ -0,0 +1,207 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package lib + +import ( + "encoding" + "fmt" + "math/big" + "strings" +) + +// ExecutionSegment represents a (start, end] partition of the total execution +// work for a specific test. For example, if we want the split the execution of a +// test in 2 different parts, we can split it in two segments (0, 0.5] and (0,5, 1]. +// +// We use rational numbers so it's easier to verify the correctness and easier to +// reason about portions of indivisible things, like VUs. This way, we can easily +// split a test in thirds (i.e. (0, 1/3], (1/3, 2/3], (2/3, 1]), without fearing +// that we'll lose a VU along the way... +// +// The most important part is that if work is split between multiple k6 instances, +// each k6 instance can precisely and reproducably calculate its share of the work, +// just by knowing its own segment. There won't be a need to schedule the +// execution from a master node, or to even know how many other k6 instances are +// running! +type ExecutionSegment struct { + // 0 <= from < to <= 1 + from *big.Rat + to *big.Rat + + // derived, equals to-from, but pre-calculated here for speed + length *big.Rat +} + +// Ensure we implement those interfaces +var _ encoding.TextUnmarshaler = &ExecutionSegment{} +var _ fmt.Stringer = &ExecutionSegment{} + +// Helpful "constants" so we don't initialize them in every function call +var zeroRat, oneRat = big.NewRat(0, 1), big.NewRat(1, 1) //nolint:gochecknoglobals +var oneBigInt, twoBigInt = big.NewInt(1), big.NewInt(2) //nolint:gochecknoglobals + +// NewExecutionSegment validates the supplied arguments (basically, that 0 <= +// from < to <= 1) and either returns an error, or it returns a +// fully-initialized and usable execution segment. +func NewExecutionSegment(from, to *big.Rat) (*ExecutionSegment, error) { + if from.Cmp(zeroRat) < 0 { + return nil, fmt.Errorf("segment start value should be at least 0 but was %s", from.FloatString(2)) + } + if from.Cmp(to) >= 0 { + return nil, fmt.Errorf("segment start(%s) should be less than its end(%s)", from.FloatString(2), to.FloatString(2)) + } + if to.Cmp(oneRat) > 0 { + return nil, fmt.Errorf("segment end value shouldn't be more than 1 but was %s", to.FloatString(2)) + } + return &ExecutionSegment{ + from: from, + to: to, + length: new(big.Rat).Sub(to, from), + }, nil +} + +// stringToRat is a helper function that tries to convert a string to a rational +// number while allowing percentage, decimal, and fraction values. +func stringToRat(s string) (*big.Rat, error) { + if strings.HasSuffix(s, "%") { + num, ok := new(big.Int).SetString(strings.TrimSuffix(s, "%"), 10) + if !ok { + return nil, fmt.Errorf("'%s' is not a valid percentage", s) + } + return new(big.Rat).SetFrac(num, big.NewInt(100)), nil + } + rat, ok := new(big.Rat).SetString(s) + if !ok { + return nil, fmt.Errorf("'%s' is not a valid percentage, decimal, fraction or interval value", s) + } + return rat, nil +} + +// UnmarshalText implements the encoding.TextUnmarshaler interface, so that +// execution segments can be specified as CLI flags, environment variables, and +// JSON strings. +// +// We are able to parse both single percentage/float/fraction values, and actual +// (from; to] segments. For the single values, we just treat them as the +// beginning segment - thus the execution segment can be used as a shortcut for +// quickly running an arbitrarily scaled-down version of a test. +// +// The parsing logic is that values with a colon, i.e. ':', are full segments: +// `1/2:3/4`, `0.5:0.75`, `50%:75%`, and even `2/4:75%` should be (1/2, 3/4] +// And values without a hyphen are the end of a first segment: +// `20%`, `0.2`, and `1/5` should be converted to (0, 1/5] +// empty values should probably be treated as "1", i.e. the whole execution +func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { + from := zeroRat + toStr := string(text) + if strings.ContainsRune(toStr, ':') { + fromToStr := strings.SplitN(toStr, ":", 2) + toStr = fromToStr[1] + if from, err = stringToRat(fromToStr[0]); err != nil { + return err + } + } + + to, err := stringToRat(toStr) + if err != nil { + return err + } + + segment, err := NewExecutionSegment(from, to) + if err != nil { + return err + } + *es = *segment + return nil +} + +func (es *ExecutionSegment) String() string { + if es == nil { + return "0:1" + } + return es.from.RatString() + ":" + es.to.RatString() +} + +// FloatLength is a helper method for getting some more human-readable +// information about the execution segment. +func (es *ExecutionSegment) FloatLength() float64 { + if es == nil { + return 1.0 + } + res, _ := es.length.Float64() + return res +} + +//TODO: add a NewFromString() and Split() methods + +// helper function for rounding (up) of rational numbers to big.Int values +func roundUp(rat *big.Rat) *big.Int { + quo, rem := new(big.Int).QuoRem(rat.Num(), rat.Denom(), new(big.Int)) + + if rem.Mul(rem, twoBigInt).Cmp(rat.Denom()) >= 0 { + return quo.Add(quo, oneBigInt) + } + return quo +} + +// Scale proportionally scales the supplied value, according to the execution +// segment's position and size of the work. +func (es *ExecutionSegment) Scale(value int64) int64 { + if es == nil { // no execution segment, i.e. 100% + return value + } + // Instead of the first proposal that used remainders and floor: + // floor( (value * from) % 1 + value * length ) + // We're using an alternative approach with rounding that (hopefully) has + // the same properties, but it's simpler and has better precision: + // round( (value * from) - round(value * from) + (value * (to - from)) )? + // which reduces to: + // round( (value * to) - round(value * from) )? + + toValue := big.NewRat(value, 1) + toValue.Mul(toValue, es.to) + + fromValue := big.NewRat(value, 1) + fromValue.Mul(fromValue, es.from) + + toValue.Sub(toValue, new(big.Rat).SetFrac(roundUp(fromValue), oneBigInt)) + + return roundUp(toValue).Int64() +} + +// InPlaceScaleRat scales rational numbers in-place - it changes the passed +// argument (and also returns it, to allow for chaining, like many other big.Rat +// methods). +func (es *ExecutionSegment) InPlaceScaleRat(value *big.Rat) *big.Rat { + if es == nil { // no execution segment, i.e. 100% + return value + } + return value.Mul(value, es.length) +} + +// CopyScaleRat scales rational numbers without changing them - creates a new +// bit.Rat object and uses it for the calculation. +func (es *ExecutionSegment) CopyScaleRat(value *big.Rat) *big.Rat { + if es == nil { // no execution segment, i.e. 100% + return value + } + return new(big.Rat).Mul(value, es.length) +} diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go new file mode 100644 index 00000000000..df4f6251fa1 --- /dev/null +++ b/lib/execution_segment_test.go @@ -0,0 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package lib + +//TODO From c5b63c5aea79b6f207216c0aff3d0ee5c90289b8 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:22:20 +0300 Subject: [PATCH 003/350] Move execution building in lib.scheduler and remove warnings --- cmd/config.go | 114 ++-------------- cmd/config_consolidation_test.go | 58 ++++---- lib/scheduler/execution_config_shortcuts.go | 139 ++++++++++++++++++++ lib/testutils/logrus_hook.go | 20 +++ lib/testutils/test_output.go | 41 ++++++ 5 files changed, 234 insertions(+), 138 deletions(-) create mode 100644 lib/scheduler/execution_config_shortcuts.go create mode 100644 lib/testutils/test_output.go diff --git a/cmd/config.go b/cmd/config.go index 139b1e01286..439c0483de9 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -37,7 +37,6 @@ import ( "github.com/loadimpact/k6/stats/influxdb" "github.com/loadimpact/k6/stats/kafka" "github.com/loadimpact/k6/stats/statsd/common" - log "github.com/sirupsen/logrus" "github.com/spf13/afero" "github.com/spf13/pflag" null "gopkg.in/guregu/null.v3" @@ -73,6 +72,14 @@ type Config struct { } `json:"collectors"` } +// Validate checks if all of the specified options make sense +func (c Config) Validate() []error { + errors := c.Options.Validate() + //TODO: validate all of the other options... that we should have already been validating... + //TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation + return errors +} + func (c Config) Apply(cfg Config) Config { c.Options = c.Options.Apply(cfg.Options) if len(cfg.Out) > 0 { @@ -179,100 +186,6 @@ func readEnvConfig() (conf Config, err error) { return conf, nil } -type executionConflictConfigError string - -func (e executionConflictConfigError) Error() string { - return string(e) -} - -var _ error = executionConflictConfigError("") - -// This checks for conflicting options and turns any shortcut options (i.e. duration, iterations, -// stages) into the proper scheduler configuration -func buildExecutionConfig(conf Config) (Config, error) { - result := conf - switch { - case conf.Duration.Valid: - if conf.Iterations.Valid { - //TODO: make this an executionConflictConfigError in the next version - log.Warnf("Specifying both duration and iterations is deprecated and won't be supported in the future k6 versions") - } - - if len(conf.Stages) > 0 { // stages isn't nil (not set) and isn't explicitly set to empty - //TODO: make this an executionConflictConfigError in the next version - log.Warnf("Specifying both duration and stages is deprecated and won't be supported in the future k6 versions") - } - - if conf.Execution != nil { - return result, executionConflictConfigError("specifying both duration and execution is not supported") - } - - if conf.Duration.Duration <= 0 { - //TODO: make this an executionConflictConfigError in the next version - log.Warnf("Specifying infinite duration in this way is deprecated and won't be supported in the future k6 versions") - } else { - ds := scheduler.NewConstantLoopingVUsConfig(lib.DefaultSchedulerName) - ds.VUs = conf.VUs - ds.Duration = conf.Duration - ds.Interruptible = null.NewBool(true, false) // Preserve backwards compatibility - result.Execution = scheduler.ConfigMap{lib.DefaultSchedulerName: ds} - } - - case len(conf.Stages) > 0: // stages isn't nil (not set) and isn't explicitly set to empty - if conf.Iterations.Valid { - //TODO: make this an executionConflictConfigError in the next version - log.Warnf("Specifying both iterations and stages is deprecated and won't be supported in the future k6 versions") - } - - if conf.Execution != nil { - return conf, executionConflictConfigError("specifying both stages and execution is not supported") - } - - ds := scheduler.NewVariableLoopingVUsConfig(lib.DefaultSchedulerName) - ds.StartVUs = conf.VUs - for _, s := range conf.Stages { - if s.Duration.Valid { - ds.Stages = append(ds.Stages, scheduler.Stage{Duration: s.Duration, Target: s.Target}) - } - } - ds.Interruptible = null.NewBool(true, false) // Preserve backwards compatibility - result.Execution = scheduler.ConfigMap{lib.DefaultSchedulerName: ds} - - case conf.Iterations.Valid: - if conf.Execution != nil { - return conf, executionConflictConfigError("specifying both iterations and execution is not supported") - } - // TODO: maybe add a new flag that will be used as a shortcut to per-VU iterations? - - ds := scheduler.NewSharedIterationsConfig(lib.DefaultSchedulerName) - ds.VUs = conf.VUs - ds.Iterations = conf.Iterations - result.Execution = scheduler.ConfigMap{lib.DefaultSchedulerName: ds} - - default: - if conf.Execution != nil { // If someone set this, regardless if its empty - //TODO: remove this warning in the next version - log.Warnf("The execution settings are not functional in this k6 release, they will be ignored") - } - - if len(conf.Execution) == 0 { // If unset or set to empty - // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config - // with 1 VU and 1 iteration. We're choosing the per-VU config, since that one could also - // be executed both locally, and in the cloud. - result.Execution = scheduler.ConfigMap{ - lib.DefaultSchedulerName: scheduler.NewPerVUIterationsConfig(lib.DefaultSchedulerName), - } - } - } - - //TODO: validate the config; questions: - // - separately validate the duration, iterations and stages for better error messages? - // - or reuse the execution validation somehow, at the end? or something mixed? - // - here or in getConsolidatedConfig() or somewhere else? - - return result, nil -} - // Assemble the final consolidated configuration from all of the different sources: // - start with the CLI-provided options to get shadowed (non-Valid) defaults in there // - add the global file config options @@ -302,11 +215,11 @@ func getConsolidatedConfig(fs afero.Fs, cliConf Config, runner lib.Runner) (conf } conf = conf.Apply(envConf).Apply(cliConf) - return buildExecutionConfig(conf) + conf.Options, err = scheduler.BuildExecutionConfig(conf.Options) + + return conf, err } -//TODO: remove ↓ -//nolint:unparam func validateConfig(conf Config) error { errList := conf.Validate() if len(errList) == 0 { @@ -317,9 +230,6 @@ func validateConfig(conf Config) error { for _, err := range errList { errMsgParts = append(errMsgParts, fmt.Sprintf("\t- %s", err.Error())) } - errMsg := errors.New(strings.Join(errMsgParts, "\n")) - //TODO: actually return the error here instead of warning, so k6 aborts on config validation errors - log.Warn(errMsg) - return nil + return errors.New(strings.Join(errMsgParts, "\n")) } diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index bfcf726f77b..87b04f56d3d 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -21,7 +21,6 @@ package cmd import ( "fmt" - "io" "io/ioutil" "os" "strings" @@ -73,13 +72,12 @@ func setEnv(t *testing.T, newEnv []string) (restoreEnv func()) { } func verifyOneIterPerOneVU(t *testing.T, c Config) { - // No config anywhere should result in a 1 VU with a 1 uninterruptible iteration config + // No config anywhere should result in a 1 VU with a 1 iteration config sched := c.Execution[lib.DefaultSchedulerName] require.NotEmpty(t, sched) require.IsType(t, scheduler.PerVUIteationsConfig{}, sched) perVuIters, ok := sched.(scheduler.PerVUIteationsConfig) require.True(t, ok) - assert.Equal(t, null.NewBool(false, false), perVuIters.Interruptible) assert.Equal(t, null.NewInt(1, false), perVuIters.Iterations) assert.Equal(t, null.NewInt(1, false), perVuIters.VUs) } @@ -105,7 +103,6 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing require.IsType(t, scheduler.ConstantLoopingVUsConfig{}, sched) clvc, ok := sched.(scheduler.ConstantLoopingVUsConfig) require.True(t, ok) - assert.Equal(t, null.NewBool(true, false), clvc.Interruptible) assert.Equal(t, vus, clvc.VUs) assert.Equal(t, types.NullDurationFrom(duration), clvc.Duration) assert.Equal(t, vus, c.VUs) @@ -120,7 +117,6 @@ func verifyVarLoopingVUs(startVus null.Int, stages []scheduler.Stage) func(t *te require.IsType(t, scheduler.VariableLoopingVUsConfig{}, sched) clvc, ok := sched.(scheduler.VariableLoopingVUsConfig) require.True(t, ok) - assert.Equal(t, null.NewBool(true, false), clvc.Interruptible) assert.Equal(t, startVus, clvc.StartVUs) assert.Equal(t, startVus, c.VUs) assert.Equal(t, stages, clvc.Stages) @@ -208,21 +204,8 @@ func resetStickyGlobalVars() { exitOnRunning = false configFilePath = "" runType = "" - runNoSetup = false - runNoTeardown = false } -// Something that makes the test also be a valid io.Writer, useful for passing it -// as an output for logs and CLI flag help messages... -type testOutput struct{ *testing.T } - -func (to testOutput) Write(p []byte) (n int, err error) { - to.Logf("%s", p) - return len(p), nil -} - -var _ io.Writer = testOutput{} - // exp contains the different events or errors we expect our test case to trigger. // for space and clarity, we use the fact that by default, all of the struct values are false type exp struct { @@ -230,7 +213,7 @@ type exp struct { cliReadError bool consolidationError bool validationErrors bool - logWarning bool //TODO: remove in the next version? + logWarning bool } // A hell of a complicated test case, that still doesn't test things fully... @@ -272,17 +255,17 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { {opts{cli: []string{"-s", "10s:5", "-s", "10s:"}}, exp{validationErrors: true}, nil}, {opts{fs: defaultConfig(`{"stages": [{"duration": "20s"}], "vus": 10}`)}, exp{validationErrors: true}, nil}, // These should emit a warning - //TODO: in next version, those should be an error - {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{logWarning: true}, nil}, - {opts{cli: []string{"-u", "2", "-d", "10s", "-s", "10s:20"}}, exp{logWarning: true}, nil}, - {opts{cli: []string{"-u", "3", "-i", "5", "-s", "10s:20"}}, exp{logWarning: true}, nil}, - {opts{cli: []string{"-u", "3", "-d", "0"}}, exp{logWarning: true}, nil}, + //TODO: these should probably emit a validation error? + {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{consolidationError: true}, nil}, + {opts{cli: []string{"-u", "2", "-d", "10s", "-s", "10s:20"}}, exp{consolidationError: true}, nil}, + {opts{cli: []string{"-u", "3", "-i", "5", "-s", "10s:20"}}, exp{consolidationError: true}, nil}, + {opts{cli: []string{"-u", "3", "-d", "0"}}, exp{consolidationError: true}, nil}, { opts{runner: &lib.Options{ VUs: null.IntFrom(5), Duration: types.NullDurationFrom(44 * time.Second), Iterations: null.IntFrom(10), - }}, exp{logWarning: true}, nil, + }}, exp{consolidationError: true}, nil, }, {opts{fs: defaultConfig(`{"execution": {}}`)}, exp{logWarning: true}, verifyOneIterPerOneVU}, // Test if environment variable shortcuts are working as expected @@ -314,15 +297,14 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { env: []string{"K6_DURATION=15s"}, cli: []string{"--stage", ""}, }, - exp{}, verifyConstLoopingVUs(I(10), 15*time.Second), + exp{logWarning: true}, verifyOneIterPerOneVU, }, { opts{ runner: &lib.Options{VUs: null.IntFrom(5), Duration: types.NullDurationFrom(50 * time.Second)}, cli: []string{"--iterations", "5"}, }, - //TODO: this shouldn't be a warning in the next version, but the result will be different - exp{logWarning: true}, verifyConstLoopingVUs(I(5), 50*time.Second), + exp{}, verifySharedIters(I(5), I(5)), }, { opts{ @@ -330,16 +312,16 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { runner: &lib.Options{VUs: null.IntFrom(5)}, }, exp{}, - verifyVarLoopingVUs(null.NewInt(5, true), buildStages(20, 10)), + verifyVarLoopingVUs(I(5), buildStages(20, 10)), }, { opts{ fs: defaultConfig(`{"stages": [{"duration": "20s", "target": 10}]}`), runner: &lib.Options{VUs: null.IntFrom(5)}, - env: []string{"K6_VUS=15", "K6_ITERATIONS=15"}, + env: []string{"K6_VUS=15", "K6_ITERATIONS=17"}, }, - exp{logWarning: true}, //TODO: this won't be a warning in the next version, but the result will be different - verifyVarLoopingVUs(null.NewInt(15, true), buildStages(20, 10)), + exp{}, + verifySharedIters(I(15), I(17)), }, { opts{ @@ -357,8 +339,8 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { opts{ fs: defaultConfig(`{ "execution": { "someKey": { - "type": "constant-looping-vus", "vus": 10, "duration": "60s", "interruptible": false, - "iterationTimeout": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc" + "type": "constant-looping-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", + "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc" }}}`), env: []string{"K6_ITERATIONS=25"}, cli: []string{"--vus", "12"}, @@ -366,6 +348,9 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { exp{}, verifySharedIters(I(12), I(25)), }, + //TODO: test manual execution + //TODO: test execution-segment + // Just in case, verify that no options will result in the same 1 vu 1 iter config {opts{}, exp{}, verifyOneIterPerOneVU}, //TODO: test for differences between flagsets @@ -380,7 +365,8 @@ func runTestCase( logHook *testutils.SimpleLogrusHook, ) { t.Logf("Test with opts=%#v and exp=%#v\n", testCase.options, testCase.expected) - log.SetOutput(testOutput{t}) + output := testutils.NewTestOutput(t) + log.SetOutput(output) logHook.Drain() restoreEnv := setEnv(t, testCase.options.env) @@ -388,7 +374,7 @@ func runTestCase( flagSet := newFlagSet() defer resetStickyGlobalVars() - flagSet.SetOutput(testOutput{t}) + flagSet.SetOutput(output) //flagSet.PrintDefaults() cliErr := flagSet.Parse(testCase.options.cli) diff --git a/lib/scheduler/execution_config_shortcuts.go b/lib/scheduler/execution_config_shortcuts.go new file mode 100644 index 00000000000..2efb1eeb2f5 --- /dev/null +++ b/lib/scheduler/execution_config_shortcuts.go @@ -0,0 +1,139 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package scheduler + +import ( + "github.com/loadimpact/k6/lib" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" +) + +// ExecutionConflictError is a custom error type used for all of the errors in +// the BuildExecutionConfig() function. +type ExecutionConflictError string + +func (e ExecutionConflictError) Error() string { + return string(e) +} + +var _ error = ExecutionConflictError("") + +// BuildExecutionConfig checks for conflicting options and turns any shortcut +// options (i.e. duration, iterations, stages) into the proper long-form +// scheduler configuration in the execution property. +func BuildExecutionConfig(opts lib.Options) (lib.Options, error) { + result := opts + + switch { + case opts.Duration.Valid: + if opts.Iterations.Valid { + return result, ExecutionConflictError( + "using multiple execution config shortcuts (`duration` and `iterations`) simultaneously is not allowed", + ) + } + + if len(opts.Stages) > 0 { // stages isn't nil (not set) and isn't explicitly set to empty + return result, ExecutionConflictError( + "using multiple execution config shortcuts (`duration` and `stages`) simultaneously is not allowed", + ) + } + + if opts.Execution != nil { + return result, ExecutionConflictError( + "using an execution configuration shortcut (`duration`) and `execution` simultaneously is not allowed", + ) + } + + if opts.Duration.Duration <= 0 { + //TODO: move this validation to Validate()? + return result, ExecutionConflictError( + "`duration` should be more than 0, for infinite duration use the manual-execution scheduler", + ) + } + + ds := NewConstantLoopingVUsConfig(lib.DefaultSchedulerName) + ds.VUs = opts.VUs + ds.Duration = opts.Duration + result.Execution = lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + + case len(opts.Stages) > 0: // stages isn't nil (not set) and isn't explicitly set to empty + if opts.Iterations.Valid { + return result, ExecutionConflictError( + "using multiple execution config shortcuts (`stages` and `iterations`) simultaneously is not allowed", + ) + } + + if opts.Execution != nil { + return opts, ExecutionConflictError( + "using an execution configuration shortcut (`stages`) and `execution` simultaneously is not allowed", + ) + } + + ds := NewVariableLoopingVUsConfig(lib.DefaultSchedulerName) + ds.StartVUs = opts.VUs + for _, s := range opts.Stages { + if s.Duration.Valid { + ds.Stages = append(ds.Stages, Stage{Duration: s.Duration, Target: s.Target}) + } + } + result.Execution = lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + + case opts.Iterations.Valid: + if opts.Execution != nil { + return opts, ExecutionConflictError( + "using an execution configuration shortcut (`iterations`) and `execution` simultaneously is not allowed", + ) + } + // TODO: maybe add a new flag that will be used as a shortcut to per-VU iterations? + + ds := NewSharedIterationsConfig(lib.DefaultSchedulerName) + ds.VUs = opts.VUs + ds.Iterations = opts.Iterations + result.Execution = lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + + case len(opts.Execution) > 0: + // Do nothing, execution was explicitly specified + default: + // Check if we should emit some warnings + if opts.Stages != nil && len(opts.Stages) == 0 { + // No someone explicitly set stages to empty + logrus.Warnf("`stages` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") + } + if opts.Execution != nil && len(opts.Execution) == 0 { + // No shortcut, and someone explicitly set execution to empty + logrus.Warnf("`execution` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") + } + // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config + // with 1 VU and 1 iteration. We're choosing the per-VU config, since that one could also + // be executed both locally, and in the cloud. + result.Execution = lib.SchedulerConfigMap{ + lib.DefaultSchedulerName: NewPerVUIterationsConfig(lib.DefaultSchedulerName), + } + result.Iterations = null.NewInt(1, false) + } + + //TODO: validate the config; questions: + // - separately validate the duration, iterations and stages for better error messages? + // - or reuse the execution validation somehow, at the end? or something mixed? + // - here or in getConsolidatedConfig() or somewhere else? + + return result, nil +} diff --git a/lib/testutils/logrus_hook.go b/lib/testutils/logrus_hook.go index 4a355bf8ba0..63d4c0d1499 100644 --- a/lib/testutils/logrus_hook.go +++ b/lib/testutils/logrus_hook.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package testutils import ( diff --git a/lib/testutils/test_output.go b/lib/testutils/test_output.go new file mode 100644 index 00000000000..67a8ea284c3 --- /dev/null +++ b/lib/testutils/test_output.go @@ -0,0 +1,41 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package testutils + +import ( + "io" + "testing" +) + +// Something that makes the test also be a valid io.Writer, useful for passing it +// as an output for logs and CLI flag help messages... +type testOutput struct{ *testing.T } + +func (to testOutput) Write(p []byte) (n int, err error) { + to.Logf("%s", p) + return len(p), nil +} + +// NewTestOutput returns a simple io.Writer implementation that uses the test's +// logger as an output. +func NewTestOutput(t *testing.T) io.Writer { + return testOutput{t} +} From 1e884e01281e0ff16b630d80bf63613f00c8277c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:25:41 +0300 Subject: [PATCH 004/350] Add the new executor state and interface --- lib/executor.go | 550 ++++++++++++++++++++++++++++++++++++++----- lib/executor_test.go | 51 ++++ 2 files changed, 548 insertions(+), 53 deletions(-) create mode 100644 lib/executor_test.go diff --git a/lib/executor.go b/lib/executor.go index 9c955189d00..18b07aa65d2 100644 --- a/lib/executor.go +++ b/lib/executor.go @@ -1,7 +1,7 @@ /* * * k6 - a next-generation load testing tool - * Copyright (C) 2016 Load Impact + * Copyright (C) 2019 Load Impact * * This program is free software: you can redistribute it and/or modify * it under the terms of the GNU Affero General Public License as @@ -22,66 +22,510 @@ package lib import ( "context" + "fmt" + "sync" + "sync/atomic" "time" - "github.com/loadimpact/k6/lib/types" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/stats" - log "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "github.com/pkg/errors" ) -// An Executor is in charge of scheduling VUs created by a wrapped Runner, but decouples how you -// control a swarm of VUs from the details of how or even where they're scheduled. +// An Executor is in charge of initializing schedulers and using them to +// initialize and schedule VUs created by a wrapped Runner. It decouples how a +// swarm of VUs is controlled from the details of how or even where they're +// scheduled. // -// The core/local executor schedules VUs on the local machine, but the same interface may be -// implemented to control a test running on a cluster or in the cloud. +// The core/local executor schedules VUs on the local machine, but the same +// interface may be implemented to control a test running on a cluster or in the +// cloud. +//TODO: flesh out the interface after actually having more than one implementation... type Executor interface { - // Run the Executor, funneling generated samples through the out channel. + // Returns the wrapped runner. May return nil if not applicable, eg. + // if we're remote controlling a test running on another machine. + GetRunner() Runner + + // Return the ExecutorState instance from which different statistics for the + // current state of the runner could be retrieved. + GetState() *ExecutorState + + // Return the instances of the configured schedulers + GetSchedulers() []Scheduler + + // Init initializes all schedulers, including all of their needed VUs. + Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error + + // Run the Executor, funneling generated metric samples through the supplied + // out channel. Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error - // Is the executor currently running? - IsRunning() bool - // Returns the wrapped runner. May return nil if not applicable, eg. if we're remote - // controlling a test running on another machine. - GetRunner() Runner + // Pause a test, or start/resume it. To check if a test is paused, use + // GetState().IsPaused(). + // + // Currently, any test can be started in a paused state. This will cause k6 + // to initialize all needed VUs, but it won't actually start the test. + // Later, the test can be started for real be resuming/unpausing it from the + // REST API. + // + // After a test is actually started, it may become impossible to pause it + // again. That is denoted by having SetPaused(true) return an error. The + // likely cause is that some of the schedulers for the test don't support + // pausing after the test has been started. + // + // IMPORTANT: Currently only the manual scheduler can be paused and resumed + // multiple times in the middle of the test execution! Even then, "pausing" + // is a bit misleading, since k6 won't pause in the middle of the currently + // executing iterations. It will allow the currently in progress iterations + // to finish, and it just won't start any new ones nor will it increment + // the value returned by GetCurrentTestRunDuration(). + SetPaused(paused bool) error +} + +// MaxTimeToWaitForPlannedVU specifies the maximum allowable time for a +// scheduler to wait for a planned VU to be retrieved from the +// ExecutorState.PlannedVUs buffer. If it's exceeded, k6 will emit a warning log +// message, since it either means that there's a bug in the k6 scheduling code, +// or that the machine is overloaded and the scheduling code suffers from +// delays. +// +// Critically, exceeding this time *doesn't* result in an aborted test +// or any test errors, and the scheduler will continue to try and borrow the VU +// (potentially resulting in further warnings). We likely should emit a k6 +// metric about it in the future. +// TODO: emit a metric every time this is exceeded? +const MaxTimeToWaitForPlannedVU = 400 * time.Millisecond + +// MaxRetriesGetPlannedVU how many times we should wait for +// MaxTimeToWaitForPlannedVU before we actually return an error. +const MaxRetriesGetPlannedVU = 5 + +// ExecutorState contains a few different things: +// - Some convenience items, that are needed by all schedulers, like the +// execution segment and the unique VU ID generator. By keeping those here, +// we can just pass the ExecutorState to the different schedulers, instead of +// individually passing them each item. +// - Mutable counters that different schedulers modify and other parts of +// k6 can read, e.g. for the vus and vus_max metrics k6 emits every second. +// - Pausing controls and statistics. +// +// The counters and timestamps here are primarily meant to be used for +// information extraction and avoidance of ID collisions. Using many of the +// counters here for synchronization between VUs could result in HIDDEN data +// races, because the Go data race detector can't detect any data races +// involving atomics... +// +// The only functionality indended for synchronization is the one revolving +// around pausing, and uninitializedUnplannedVUs for restrictring the number of +// unplanned VUs being initialized. +type ExecutorState struct { + // A copy of the options, so the different schedulers have access to them. + // They will need to access things like the current execution segment, the + // per-run metrics tags, etc. + // + // Obviously, they are not meant to be changed... They should be a constant + // during the execution of a single test, but we can't easily enforce that + // via the Go type system... + Options Options + + // vus is the shared channel buffer that contains all of the VUs that have + // been initialized and aren't currently being used by a scheduler. + // + // It contains both pre-initialized (i.e. planned) VUs, as well as any + // unplanned VUs. Planned VUs are initialized before a test begins, while + // unplanned VUS can be initialized in the middle of the test run by a + // scheduler and have been relinquished after it has finished working with + // them. Usually, unplanned VUs are initialized by one of the arrival-rate + // schedulers, after they have exhausted their PreAllocatedVUs. After the + // scheduler is done with the VUs, it will put in this channel, so it could + // potentially be reused by other schedulers further along in the test. + // + // Different schedulers cooperatively borrow VUs from here when they are + // needed and return them when they are done with them. There's no central + // enforcement of correctness, i.e. that a scheduler takes more VUs from + // here than its execution plan has stipulated. The correctness guarantee + // lies with the actual schedulers - bugs in one can affect others. + // + // That's why the field is private and we force schedulers to use the + // GetPlannedVU(), GetUnplannedVU(), and ReturnVU() methods instead of work + // directly with the channel. These methods will emit a warning or can even + // return an error if retrieving a VU takes more than + // MaxTimeToWaitForPlannedVU. + vus chan VU + + // The current VU ID, used for the __VU execution context variable. Use the + // GetUniqueVUIdentifier() to get unique values for each VU, starting from 1 + // (for backwards compatibility...) + currentVUIdentifier *uint64 + + //TODO: add something similar, but for iterations? Currently, there isn't + // a straightforward way to get a unique sequential identifier per iteration + // in the context of a single k6 instance. Combining __VU and __ITER gives us + // a unique identifier, but it's unwieldy and somewhat cumbersome. + + // Total number of currently initialized VUs. Generally equal to + // currentVUIdentifier minus 1, since initializedVUs starts from 0 and is + // incremented only after a VU is initialized, while CurrentVUIdentifier is + // incremented before a VU is initialized. + initializedVUs *uint64 + + // Total number of unplanned VUs we haven't initialized yet. It starts + // being equal to GetMaxPossibleVUs(executionPlan)-GetMaxPlannedVUs(), and + // may stay that way if no unplanned VUs are initialized. Once it reaches 0, + // no more unplanned VUs can be initialized. + uninitializedUnplannedVUs *int64 + + // Injected when the executor's init function is called, used for + // initializing unplanned VUs. + initVUFunc InitVUFunc + + // The number of VUs that are currently executing the test script. This also + // includes any VUs that are in the process of gracefully winding down, + // either at the end of the test, or when VUs are ramping down. + activeVUs *uint64 + + // The total number of full (i.e uninterrupted) iterations that have been + // completed so far. + fullIterationsCount *uint64 + + // The total number of iterations that have been interrupted during their + // execution. The potential interruption causes vary - end of a specified + // script `duration`, scaling down of VUs via `stages`, a user hitting + // Ctrl+C, change of `vus` via the manual executor's REST API, etc. + partialIterationsCount *uint64 + + // A nanosecond UNIX timestamp that is set when the test is actually + // started. The default 0 value is used to denote that the test hasn't + // started yet... + startTime *int64 + + // A nanosecond UNIX timestamp that is set when the ends, either by an early + // context cancel or at its regularly scheduled time. The default 0 value is + // used to denote that the test hasn't ended yet... + endTime *int64 + + // Stuff related to pausing follows. Read the docs in Executor for more + // information regarding how pausing works in k6. + // + // When we pause the execution in the middle of the test, we save the + // current timestamp in currentPauseTime. When we resume the execution, we + // set currentPauseTime back to 0 and we add the (time.Now() - + // currentPauseTime) duration to totalPausedDuration (unless the test hasn't + // started yet). + // + // Thus, the algorithm for GetCurrentTestRunDuration() is very + // straightforward: + // - if the test hasn't started, return 0 + // - set endTime to: + // - the current pauseTime, if not zero + // - time.Now() otherwise + // - return (endTime - startTime - totalPausedDuration) + // + // Quickly checking for IsPaused() just means comparing the currentPauseTime + // with 0, a single atomic operation. + // + // But if we want to wait until a script resumes, or be notified of the + // start/resume event from a channel (as part of a select{}), we have to + // acquire the pauseStateLock, get the current resumeNotify instance, + // release the lock and wait to read from resumeNotify (when it's closed by + // Resume()). + currentPauseTime *int64 + pauseStateLock sync.RWMutex + totalPausedDuration time.Duration // only modified behind the lock + resumeNotify chan struct{} +} + +// NewExecutorState initializes all of the pointers in the ExecutorState +// with zeros. It also makes sure that the initial state is unpaused, by +// setting resumeNotify to an already closed channel. +func NewExecutorState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *ExecutorState { + resumeNotify := make(chan struct{}) + close(resumeNotify) // By default the ExecutorState starts unpaused + + maxUnplannedUninitializedVUs := int64(maxPossibleVUs - maxPlannedVUs) + + return &ExecutorState{ + Options: options, + vus: make(chan VU, maxPossibleVUs), + + currentVUIdentifier: new(uint64), + initializedVUs: new(uint64), + uninitializedUnplannedVUs: &maxUnplannedUninitializedVUs, + activeVUs: new(uint64), + fullIterationsCount: new(uint64), + partialIterationsCount: new(uint64), + startTime: new(int64), + endTime: new(int64), + currentPauseTime: new(int64), + pauseStateLock: sync.RWMutex{}, + totalPausedDuration: 0, // Accessed only behind the pauseStateLock + resumeNotify: resumeNotify, + } +} + +// GetUniqueVUIdentifier returns an auto-incrementing unique VU ID, used for __VU. +// It starts from 1 (for backwards compatibility...) +func (es *ExecutorState) GetUniqueVUIdentifier() uint64 { + return atomic.AddUint64(es.currentVUIdentifier, 1) +} + +// GetInitializedVUsCount returns the total number of currently initialized VUs. +// +// Important: this doesn't include any temporary/service VUs that are destroyed +// after they are used. These are created for the initial retrieval of the +// exported script options and for the execution of setup() and teardown() +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) GetInitializedVUsCount() uint64 { + return atomic.LoadUint64(es.initializedVUs) +} + +// GetCurrentlyActiveVUsCount returns the number of VUs that are currently +// executing the test script. This also includes any VUs that are in the process +// of gracefullt winding down. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) GetCurrentlyActiveVUsCount() uint64 { + return atomic.LoadUint64(es.activeVUs) +} + +// GetFullIterationCount returns the total of full (i.e uninterrupted) iterations +// that have been completed so far. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) GetFullIterationCount() uint64 { + return atomic.LoadUint64(es.fullIterationsCount) +} + +// AddFullIterations increments the number of full (i.e uninterrupted) iterations +// by the provided amount. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) AddFullIterations(count uint64) uint64 { + return atomic.AddUint64(es.fullIterationsCount, count) +} + +// GetPartialIterationCount returns the total of partial (i.e interrupted) +// iterations that have been completed so far. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) GetPartialIterationCount() uint64 { + return atomic.LoadUint64(es.partialIterationsCount) +} + +// AddPartialIterations increments the number of partial (i.e interrupted) +// iterations by the provided amount. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) AddPartialIterations(count uint64) uint64 { + return atomic.AddUint64(es.partialIterationsCount, count) +} + +// MarkStarted saves the current timestamp as the test start time. +// +// CAUTION: Calling MarkStarted() a second time for the same execution state will +// result in a panic! +func (es *ExecutorState) MarkStarted() { + if !atomic.CompareAndSwapInt64(es.startTime, 0, time.Now().UnixNano()) { + panic("the executor was started a second time") + } +} + +// MarkEnded saves the current timestamp as the test end time. +// +// CAUTION: Calling MarkEnded() a second time for the same execution state will +// result in a panic! +func (es *ExecutorState) MarkEnded() { + if !atomic.CompareAndSwapInt64(es.endTime, 0, time.Now().UnixNano()) { + panic("the executor was started a second time") + } +} + +// HasStarted returns true if the test has actually started executing. +// It will return false while a test is in the init phase, or if it has +// been initially paused. But if will return true if a test is paused +// midway through its execution (see above for details regarind the +// feasibility of that pausing for normal schedulers). +func (es *ExecutorState) HasStarted() bool { + return atomic.LoadInt64(es.startTime) != 0 +} + +// HasEnded returns true if the test has finished executing. It will return +// false until MarkEnded() is called. +func (es *ExecutorState) HasEnded() bool { + return atomic.LoadInt64(es.endTime) != 0 +} + +// IsPaused quickly returns whether the test is currently paused, by reading +// the atomic currentPauseTime timestamp +func (es *ExecutorState) IsPaused() bool { + return atomic.LoadInt64(es.currentPauseTime) != 0 +} + +// GetCurrentTestRunDuration returns the duration for which the test has already +// ran. If the test hasn't started yet, that's 0. If it has started, but has +// been paused midway through, it will return the time up until the pause time. +// And if it's currently running, it will return the time since the start time. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) GetCurrentTestRunDuration() time.Duration { + startTime := atomic.LoadInt64(es.startTime) + if startTime == 0 { + // The test hasn't started yet + return 0 + } + + es.pauseStateLock.RLock() + endTime := atomic.LoadInt64(es.endTime) + pausedDuration := es.totalPausedDuration + es.pauseStateLock.RUnlock() + + if endTime == 0 { + pauseTime := atomic.LoadInt64(es.currentPauseTime) + if pauseTime != 0 { + endTime = pauseTime + } else { + // The test isn't paused or finished, use the current time instead + endTime = time.Now().UnixNano() + } + } + + return time.Duration(endTime-startTime) - pausedDuration +} + +// Pause pauses the current execution. It acquires the lock, writes +// the current timestamp in currentPauseTime, and makes a new +// channel for resumeNotify. +// Pause can return an error if the test was already paused. +func (es *ExecutorState) Pause() error { + es.pauseStateLock.Lock() + defer es.pauseStateLock.Unlock() + + if !atomic.CompareAndSwapInt64(es.currentPauseTime, 0, time.Now().UnixNano()) { + return errors.New("test execution was already paused") + } + es.resumeNotify = make(chan struct{}) + return nil +} + +// Resume unpauses the test execution. Unless the test wasn't +// yet started, it calculates the duration between now and +// the old currentPauseTime and adds it to +// Resume will emit an error if the test wasn't paused. +func (es *ExecutorState) Resume() error { + es.pauseStateLock.Lock() + defer es.pauseStateLock.Unlock() + + currentPausedTime := atomic.SwapInt64(es.currentPauseTime, 0) + if currentPausedTime == 0 { + return errors.New("test execution wasn't paused") + } + + // Check that it's not the pause before execution actually starts + if atomic.LoadInt64(es.startTime) != 0 { + es.totalPausedDuration += time.Duration(time.Now().UnixNano() - currentPausedTime) + } + + close(es.resumeNotify) + + return nil +} + +// ResumeNotify returns a channel which will be closed (i.e. could +// be read from) as soon as the test execution is resumed. +// +// Since tests would likely be paused only rarely, unless you +// directly need to be notified via a channel that the test +// isn't paused or that it has resumed, it's probably a good +// idea to first use the IsPaused() method, since it will be much +// faster. +// +// And, since tests won't be paused most of the time, it's +// probably better to check for that like this: +// if executionState.IsPaused() { +// <-executionState.ResumeNotify() +// } +func (es *ExecutorState) ResumeNotify() <-chan struct{} { + es.pauseStateLock.RLock() + defer es.pauseStateLock.RUnlock() + return es.resumeNotify +} + +// GetPlannedVU tries to get a pre-initialized VU from the buffer channel. This +// shouldn't fail and should generally be an instantaneous action, but if it +// doesn't happen for MaxTimeToWaitForPlannedVU, a warning will be printed. If +// we reach that timeout more than MaxRetriesGetPlannedVU number of times, this +// function will return an error, since we either have a bug with some +// scheduler, or the machine is very, very overloaded. +func (es *ExecutorState) GetPlannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { + for i := 1; i <= MaxRetriesGetPlannedVU; i++ { + select { + case vu := <-es.vus: + atomic.AddUint64(es.activeVUs, 1) + //TODO: set environment and exec + return vu, nil + case <-ctx.Done(): + return nil, ctx.Err() + case <-time.After(MaxTimeToWaitForPlannedVU): + logger.Warnf("Could not get a VU from the buffer for %s", time.Duration(i)*MaxTimeToWaitForPlannedVU) + } + } + return nil, fmt.Errorf( + "could not get a VU from the buffer in %s", + MaxRetriesGetPlannedVU*MaxTimeToWaitForPlannedVU, + ) +} + +// SetInitVUFunc is called by the executor's init function, and it's used for +// setting the "constructor" function used for the initializing unplanned VUs. +// +// TODO: figure out a better dependency injection method? +func (es *ExecutorState) SetInitVUFunc(initVUFunc InitVUFunc) { + es.initVUFunc = initVUFunc +} + +// GetUnplannedVU checks if any unplanned VUs remain to be initialized, and if +// they do, it initializes one and returns it. If all unplanned VUs have already +// been initialized, it returns one from the global vus buffer. +// +// IMPORTANT: GetUnplannedVU() doesn't do any checking if the requesting +// scheduler is actually allowed to have the VU at this particular time. +// Schedulers are trusted to correctly declare their needs (via their +// GetExecutionRequirements() methods) and then to never ask for more VUs than +// they have specified in those requirements. +func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { + remVUs := atomic.AddInt64(es.uninitializedUnplannedVUs, -1) + if remVUs < 0 { + logger.Debug("Reusing a previously initialized unplanned VU") + atomic.AddInt64(es.uninitializedUnplannedVUs, 1) + return es.GetPlannedVU(ctx, logger) + } + if es.initVUFunc == nil { + return nil, fmt.Errorf("initVUFunc wasn't set in the executor state") + } + logger.Debug("Initializing an unplanned VU, this may affect test results") + newVU, err := es.initVUFunc(ctx, logger) + if err != nil { + return nil, err + } + atomic.AddUint64(es.activeVUs, 1) + atomic.AddUint64(es.initializedVUs, 1) + return newVU, err +} + +// AddInitializedVU is a helper function that adds VUs into the buffer and +// increases the initialized VUs counter. +func (es *ExecutorState) AddInitializedVU(vu VU) { + es.vus <- vu + atomic.AddUint64(es.initializedVUs, 1) +} - // Get and set the logger. This is propagated to the Runner. - GetLogger() *log.Logger - SetLogger(l *log.Logger) - - // Get and set the list of stages. - GetStages() []Stage - SetStages(s []Stage) - - // Get iterations executed so far, get and set how many to end the test after. - GetIterations() int64 - GetEndIterations() null.Int - SetEndIterations(i null.Int) - - // Get time elapsed so far, accounting for pauses, get and set at what point to end the test. - GetTime() time.Duration - GetEndTime() types.NullDuration - SetEndTime(t types.NullDuration) - - // Check whether the test is paused, or pause it. A paused won't start any new iterations (but - // will allow currently in progress ones to finish), and will not increment the value returned - // by GetTime(). - IsPaused() bool - SetPaused(paused bool) - - // Get and set the number of currently active VUs. - // It is an error to try to set this higher than MaxVUs. - GetVUs() int64 - SetVUs(vus int64) error - - // Get and set the number of allocated, available VUs. - // Please note that initialising new VUs is a very expensive operation, and doing it during a - // running test may skew metrics; if you're not sure how many you will need, it's generally - // speaking better to preallocate too many than too few. - GetVUsMax() int64 - SetVUsMax(max int64) error - - // Set whether or not to run setup/teardown phases. Default is to run all of them. - SetRunSetup(r bool) - SetRunTeardown(r bool) +// ReturnVU is a helper function that puts VUs back into the buffer and +// decreases the active VUs counter. +func (es *ExecutorState) ReturnVU(vu VU) { + es.vus <- vu + // From the official atomic.AddUint64() docs: "to subtract a signed positive + // constant value c from x, do AddUint64(&x, ^uint64(c-1))" + atomic.AddUint64(es.activeVUs, ^uint64(0)) } diff --git a/lib/executor_test.go b/lib/executor_test.go new file mode 100644 index 00000000000..dd3d6c10176 --- /dev/null +++ b/lib/executor_test.go @@ -0,0 +1,51 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package lib + +import ( + "math/rand" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestExecutorStateVUIDs(t *testing.T) { + es := NewExecutorState(Options{}, 0, 0) //TODO: fix + assert.Equal(t, uint64(1), es.GetUniqueVUIdentifier()) + assert.Equal(t, uint64(2), es.GetUniqueVUIdentifier()) + assert.Equal(t, uint64(3), es.GetUniqueVUIdentifier()) + wg := sync.WaitGroup{} + rand.Seed(time.Now().UnixNano()) + count := rand.Intn(50) + wg.Add(count) + for i := 0; i < count; i++ { + go func() { + es.GetUniqueVUIdentifier() + wg.Done() + }() + } + wg.Wait() + assert.Equal(t, uint64(4+count), es.GetUniqueVUIdentifier()) +} + +//TODO: way more tests... From 6d80f052af2af4e908d69f3556f868618e070619 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:27:18 +0300 Subject: [PATCH 005/350] Update the scheduler configs and add the new schedulers --- lib/scheduler/base_config.go | 115 +++-- lib/scheduler/base_scheduler.go | 89 ++++ lib/scheduler/constant_arrival_rate.go | 191 ++++++++- lib/scheduler/constant_looping_vus.go | 159 +++++-- lib/scheduler/helpers.go | 196 +++++++-- lib/scheduler/manual.go | 169 ++++++++ lib/scheduler/per_vu_iterations.go | 141 +++++- lib/scheduler/schedulers_test.go | 454 ++++++++++++++------ lib/scheduler/shared_iterations.go | 155 ++++++- lib/scheduler/variable_arrival_rate.go | 330 +++++++++++++- lib/scheduler/variable_looping_vus.go | 571 +++++++++++++++++++++++-- lib/scheduler/vu_handle.go | 167 ++++++++ 12 files changed, 2396 insertions(+), 341 deletions(-) create mode 100644 lib/scheduler/base_scheduler.go create mode 100644 lib/scheduler/manual.go create mode 100644 lib/scheduler/vu_handle.go diff --git a/lib/scheduler/base_config.go b/lib/scheduler/base_config.go index 73a4c22a4b4..4a18fb3a643 100644 --- a/lib/scheduler/base_config.go +++ b/lib/scheduler/base_config.go @@ -22,39 +22,34 @@ package scheduler import ( "fmt" + "regexp" + "strings" "time" "github.com/loadimpact/k6/lib/types" null "gopkg.in/guregu/null.v3" ) -const minPercentage = 0.01 - -// The maximum time k6 will wait after an iteration is supposed to be done -const maxIterationTimeout = 300 * time.Second +var schedulerNameWhitelist = regexp.MustCompile(`^[0-9a-zA-Z_-]+$`) //nolint:gochecknoglobals // BaseConfig contains the common config fields for all schedulers type BaseConfig struct { - Name string `json:"-"` // set via the JS object key - Type string `json:"type"` - StartTime types.NullDuration `json:"startTime"` - Interruptible null.Bool `json:"interruptible"` - IterationTimeout types.NullDuration `json:"iterationTimeout"` - Env map[string]string `json:"env"` - Exec null.String `json:"exec"` // function name, externally validated - Percentage float64 `json:"-"` // 100, unless Split() was called + Name string `json:"-"` // set via the JS object key + Type string `json:"type"` + StartTime types.NullDuration `json:"startTime"` + GracefulStop types.NullDuration `json:"gracefulStop"` + Env map[string]string `json:"env"` + Exec null.String `json:"exec"` // function name, externally validated //TODO: future extensions like tags, distribution, others? } // NewBaseConfig returns a default base config with the default values -func NewBaseConfig(name, configType string, interruptible bool) BaseConfig { +func NewBaseConfig(name, configType string) BaseConfig { return BaseConfig{ - Name: name, - Type: configType, - Interruptible: null.NewBool(interruptible, false), - IterationTimeout: types.NewNullDuration(30*time.Second, false), - Percentage: 100, + Name: name, + Type: configType, + GracefulStop: types.NewNullDuration(30*time.Second, false), } } @@ -65,39 +60,83 @@ func (bc BaseConfig) Validate() (errors []error) { if bc.Name == "" { errors = append(errors, fmt.Errorf("scheduler name shouldn't be empty")) } - if bc.Type == "" { - errors = append(errors, fmt.Errorf("missing or empty type field")) - } - if bc.Percentage < minPercentage || bc.Percentage > 100 { + if !schedulerNameWhitelist.MatchString(bc.Name) { errors = append(errors, fmt.Errorf( - "percentage should be between %f and 100, but is %f", minPercentage, bc.Percentage, + "the scheduler name should contain only numbers, latin letters, underscores, and dashes", )) } if bc.Exec.Valid && bc.Exec.String == "" { errors = append(errors, fmt.Errorf("exec value cannot be empty")) } + if bc.Type == "" { + errors = append(errors, fmt.Errorf("missing or empty type field")) + } // The actually reasonable checks: if bc.StartTime.Duration < 0 { - errors = append(errors, fmt.Errorf("scheduler start time can't be negative")) + errors = append(errors, fmt.Errorf("the startTime can't be negative")) } - iterTimeout := time.Duration(bc.IterationTimeout.Duration) - if iterTimeout < 0 || iterTimeout > maxIterationTimeout { - errors = append(errors, fmt.Errorf( - "the iteration timeout should be between 0 and %s, but is %s", maxIterationTimeout, iterTimeout, - )) + if bc.GracefulStop.Duration < 0 { + errors = append(errors, fmt.Errorf("the gracefulStop timeout can't be negative")) } return errors } -// GetBaseConfig just returns itself -func (bc BaseConfig) GetBaseConfig() BaseConfig { - return bc +// GetName returns the name of the scheduler. +func (bc BaseConfig) GetName() string { + return bc.Name +} + +// GetType returns the scheduler's type as a string ID. +func (bc BaseConfig) GetType() string { + return bc.Type +} + +// GetStartTime returns the starting time, relative to the beginning of the +// actual test, that this scheduler is supposed to execute. +func (bc BaseConfig) GetStartTime() time.Duration { + return time.Duration(bc.StartTime.Duration) } -// CopyWithPercentage is a helper function that just sets the percentage to -// the specified amount. -func (bc BaseConfig) CopyWithPercentage(percentage float64) *BaseConfig { - c := bc - c.Percentage = percentage - return &c +// GetGracefulStop returns how long k6 is supposed to wait for any still +// running iterations to finish executing at the end of the normal scheduler +// duration, before it actually kills them. +// +// Of course, that doesn't count when the user manually interrupts the test, +// then iterations are immediately stopped. +func (bc BaseConfig) GetGracefulStop() time.Duration { + return time.Duration(bc.GracefulStop.Duration) +} + +// GetEnv returns any specific environment key=value pairs that +// are configured for the scheduler. +func (bc BaseConfig) GetEnv() map[string]string { + return bc.Env +} + +// GetExec returns the configured custom exec value, if any. +func (bc BaseConfig) GetExec() null.String { + return bc.Exec +} + +// IsDistributable returns true since by default all schedulers could be run in +// a distributed manner. +func (bc BaseConfig) IsDistributable() bool { + return true +} + +// getBaseInfo is a helper method for the the "parent" String methods. +func (bc BaseConfig) getBaseInfo(facts ...string) string { + if bc.Exec.Valid { + facts = append(facts, fmt.Sprintf("exec: %s", bc.Exec.String)) + } + if bc.StartTime.Duration > 0 { + facts = append(facts, fmt.Sprintf("startTime: %s", bc.StartTime.Duration)) + } + if bc.GracefulStop.Duration > 0 { + facts = append(facts, fmt.Sprintf("gracefulStop: %s", bc.GracefulStop.Duration)) + } + if len(facts) == 0 { + return "" + } + return " (" + strings.Join(facts, ", ") + ")" } diff --git a/lib/scheduler/base_scheduler.go b/lib/scheduler/base_scheduler.go new file mode 100644 index 00000000000..4e5469210e1 --- /dev/null +++ b/lib/scheduler/base_scheduler.go @@ -0,0 +1,89 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package scheduler + +import ( + "context" + "fmt" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" +) + +// BaseScheduler is a helper struct that contains common properties and methods +// between most schedulers. It is intended to be used as an anonymous struct +// inside of most of the schedulers, for the purpose of reducing boilerplate +// code. +type BaseScheduler struct { + config lib.SchedulerConfig + executorState *lib.ExecutorState + logger *logrus.Entry + progress *pb.ProgressBar +} + +// NewBaseScheduler just returns an initialized BaseScheduler +func NewBaseScheduler(config lib.SchedulerConfig, es *lib.ExecutorState, logger *logrus.Entry) *BaseScheduler { + return &BaseScheduler{ + config: config, + executorState: es, + logger: logger, + progress: pb.New( + pb.WithLeft(config.GetName), + ), + } +} + +// Init doesn't do anything for most schedulers, since initialization of all +// planned VUs is handled by the executor. +func (bs *BaseScheduler) Init(_ context.Context) error { + return nil +} + +// GetConfig returns the configuration with which this scheduler was launched. +func (bs BaseScheduler) GetConfig() lib.SchedulerConfig { + return bs.config +} + +// GetLogger returns the scheduler logger entry. +func (bs BaseScheduler) GetLogger() *logrus.Entry { + return bs.logger +} + +// GetProgress just returns the progressbar pointer. +func (bs BaseScheduler) GetProgress() *pb.ProgressBar { + return bs.progress +} + +// IsPausable just returns false, since most schedulers are not pausable after +// they have been started... +func (BaseScheduler) IsPausable() bool { + return false +} + +// LiveUpdate just returns false, since most schedulers' configs cannot be +// updated in-flight. +func (bs BaseScheduler) LiveUpdate(_ bool, _ lib.SchedulerConfig) error { + return fmt.Errorf( + "%s scheduler '%s' doesn't support pausing or live configuration updates", + bs.config.GetType(), bs.config.GetName(), + ) +} diff --git a/lib/scheduler/constant_arrival_rate.go b/lib/scheduler/constant_arrival_rate.go index ed53297e924..504684dc9b5 100644 --- a/lib/scheduler/constant_arrival_rate.go +++ b/lib/scheduler/constant_arrival_rate.go @@ -21,21 +21,31 @@ package scheduler import ( + "context" "fmt" + "math" + "sync/atomic" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const constantArrivalRateType = "constant-arrival-rate" func init() { - RegisterConfigType(constantArrivalRateType, func(name string, rawJSON []byte) (Config, error) { - config := NewConstantArrivalRateConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) - return config, err - }) + lib.RegisterSchedulerConfigType( + constantArrivalRateType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := NewConstantArrivalRateConfig(name) + err := lib.StrictJSONUnmarshal(rawJSON, &config) + return config, err + }, + ) } // ConstantArrivalRateConfig stores config for the constant arrival-rate scheduler @@ -55,13 +65,39 @@ type ConstantArrivalRateConfig struct { // NewConstantArrivalRateConfig returns a ConstantArrivalRateConfig with default values func NewConstantArrivalRateConfig(name string) ConstantArrivalRateConfig { return ConstantArrivalRateConfig{ - BaseConfig: NewBaseConfig(name, constantArrivalRateType, false), + BaseConfig: NewBaseConfig(name, constantArrivalRateType), TimeUnit: types.NewNullDuration(1*time.Second, false), } } -// Make sure we implement the Config interface -var _ Config = &ConstantArrivalRateConfig{} +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &ConstantArrivalRateConfig{} + +// GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. +func (carc ConstantArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(carc.PreAllocatedVUs.Int64) +} + +// GetMaxVUs is just a helper method that returns the scaled max VUs. +func (carc ConstantArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(carc.MaxVUs.Int64) +} + +// GetDescription returns a human-readable description of the scheduler options +func (carc ConstantArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { + preAllocatedVUs, maxVUs := carc.GetPreAllocatedVUs(es), carc.GetMaxVUs(es) + maxVUsRange := fmt.Sprintf("maxVUs: %d", preAllocatedVUs) + if maxVUs > preAllocatedVUs { + maxVUsRange += fmt.Sprintf("-%d", maxVUs) + } + + timeUnit := time.Duration(carc.TimeUnit.Duration) + arrRate := getScaledArrivalRate(es, carc.Rate.Int64, timeUnit) + arrRatePerSec, _ := getArrivalRatePerSec(arrRate).Float64() + + return fmt.Sprintf("%.2f iterations/s for %s%s", arrRatePerSec, carc.Duration.Duration, + carc.getBaseInfo(maxVUsRange)) +} // Validate makes sure all options are configured and valid func (carc ConstantArrivalRateConfig) Validate() []error { @@ -99,17 +135,136 @@ func (carc ConstantArrivalRateConfig) Validate() []error { return errors } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (carc ConstantArrivalRateConfig) GetMaxVUs() int64 { - return carc.MaxVUs.Int64 +// GetExecutionRequirements just reserves the number of specified VUs for the +// whole duration of the scheduler, including the maximum waiting time for +// iterations to gracefully stop. +func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: uint64(es.Scale(carc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(es.Scale(carc.MaxVUs.Int64 - carc.PreAllocatedVUs.Int64)), + }, { + TimeOffset: time.Duration(carc.Duration.Duration + carc.GracefulStop.Duration), + PlannedVUs: 0, + MaxUnplannedVUs: 0, + }, + } +} + +// NewScheduler creates a new ConstantArrivalRate scheduler +func (carc ConstantArrivalRateConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + + return ConstantArrivalRate{ + BaseScheduler: NewBaseScheduler(carc, es, logger), + config: carc, + }, nil +} + +// ConstantArrivalRate tries to execute a specific number of iterations for a +// specific period. +type ConstantArrivalRate struct { + *BaseScheduler + config ConstantArrivalRateConfig } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (carc ConstantArrivalRateConfig) GetMaxDuration() time.Duration { - maxDuration := carc.Duration.Duration - if !carc.Interruptible.Bool { - maxDuration += carc.IterationTimeout.Duration +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &ConstantArrivalRate{} + +// Run executes a specific number of iterations with each confugured VU. +// +// TODO: Reuse the variable arrival rate method? +func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := car.executorState.Options.ExecutionSegment + gracefulStop := car.config.GetGracefulStop() + duration := time.Duration(car.config.Duration.Duration) + preAllocatedVUs := car.config.GetPreAllocatedVUs(segment) + maxVUs := car.config.GetMaxVUs(segment) + + arrivalRate := getScaledArrivalRate(segment, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) + tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) + arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + ticker := time.NewTicker(tickerPeriod) // the rate can't be 0 because of the validation + + // Make sure the log and the progress bar have accurate information + car.logger.WithFields(logrus.Fields{ + "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, + "tickerPeriod": tickerPeriod, "type": car.config.GetType(), + }).Debug("Starting scheduler run...") + + // Pre-allocate VUs, but reserve space in the buffer for up to MaxVUs + vus := make(chan lib.VU, maxVUs) + for i := int64(0); i < preAllocatedVUs; i++ { + vu, err := car.executorState.GetPlannedVU(ctx, car.logger) + if err != nil { + return err + } + vus <- vu + } + + initialisedVUs := new(uint64) + *initialisedVUs = uint64(preAllocatedVUs) + + vusFmt := pb.GetFixedLengthIntFormat(maxVUs) + fmtStr := pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 2) + + " iters/s, " + vusFmt + " out of " + vusFmt + " VUs active" + + progresFn := func() (float64, string) { + spent := time.Since(startTime) + currentInitialisedVUs := atomic.LoadUint64(initialisedVUs) + vusInBuffer := uint64(len(vus)) + return math.Min(1, float64(spent)/float64(duration)), fmt.Sprintf(fmtStr, + arrivalRatePerSec, currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, + ) + } + car.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, car, progresFn) + + regDurationDone := regDurationCtx.Done() + runIterationBasic := getIterationRunner(car.executorState, car.logger, out) + runIteration := func(vu lib.VU) { + runIterationBasic(maxDurationCtx, vu) + vus <- vu + } + + remainingUnplannedVUs := maxVUs - preAllocatedVUs + // Make sure we put back planned and unplanned VUs back in the global + // buffer, and as an extra incentive, this replaces a waitgroup. + defer func() { + unplannedVUs := maxVUs - remainingUnplannedVUs + for i := int64(0); i < unplannedVUs; i++ { + car.executorState.ReturnVU(<-vus) + } + }() + + for { + select { + case <-ticker.C: + select { + case vu := <-vus: + // ideally, we get the VU from the buffer without any issues + go runIteration(vu) + default: + if remainingUnplannedVUs == 0 { + //TODO: emit an error metric? + car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) + break + } + remainingUnplannedVUs-- + vu, err := car.executorState.GetUnplannedVU(maxDurationCtx, car.logger) + if err != nil { + remainingUnplannedVUs++ + return err + } + atomic.AddUint64(initialisedVUs, 1) + go runIteration(vu) + } + case <-regDurationDone: + return nil + } } - return time.Duration(maxDuration) } diff --git a/lib/scheduler/constant_looping_vus.go b/lib/scheduler/constant_looping_vus.go index a6293cb1b17..19b2610fe15 100644 --- a/lib/scheduler/constant_looping_vus.go +++ b/lib/scheduler/constant_looping_vus.go @@ -21,21 +21,30 @@ package scheduler import ( + "context" "fmt" + "sync" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const constantLoopingVUsType = "constant-looping-vus" func init() { - RegisterConfigType(constantLoopingVUsType, func(name string, rawJSON []byte) (Config, error) { - config := NewConstantLoopingVUsConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) - return config, err - }) + lib.RegisterSchedulerConfigType( + constantLoopingVUsType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := NewConstantLoopingVUsConfig(name) + err := lib.StrictJSONUnmarshal(rawJSON, &config) + return config, err + }, + ) } // The minimum duration we'll allow users to schedule. This doesn't affect the stages @@ -52,62 +61,134 @@ type ConstantLoopingVUsConfig struct { // NewConstantLoopingVUsConfig returns a ConstantLoopingVUsConfig with default values func NewConstantLoopingVUsConfig(name string) ConstantLoopingVUsConfig { return ConstantLoopingVUsConfig{ - BaseConfig: NewBaseConfig(name, constantLoopingVUsType, false), + BaseConfig: NewBaseConfig(name, constantLoopingVUsType), VUs: null.NewInt(1, false), } } -// Make sure we implement the Config interface -var _ Config = &ConstantLoopingVUsConfig{} +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &ConstantLoopingVUsConfig{} + +// GetVUs returns the scaled VUs for the scheduler. +func (clvc ConstantLoopingVUsConfig) GetVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(clvc.VUs.Int64) +} + +// GetDescription returns a human-readable description of the scheduler options +func (clvc ConstantLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { + return fmt.Sprintf("%d looping VUs for %s%s", + clvc.GetVUs(es), clvc.Duration.Duration, clvc.getBaseInfo()) +} // Validate makes sure all options are configured and valid -func (lcv ConstantLoopingVUsConfig) Validate() []error { - errors := lcv.BaseConfig.Validate() - if lcv.VUs.Int64 <= 0 { +func (clvc ConstantLoopingVUsConfig) Validate() []error { + errors := clvc.BaseConfig.Validate() + if clvc.VUs.Int64 <= 0 { errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) } - if !lcv.Duration.Valid { + if !clvc.Duration.Valid { errors = append(errors, fmt.Errorf("the duration is unspecified")) - } else if time.Duration(lcv.Duration.Duration) < minDuration { + } else if time.Duration(clvc.Duration.Duration) < minDuration { errors = append(errors, fmt.Errorf( - "the duration should be at least %s, but is %s", minDuration, lcv.Duration, + "the duration should be at least %s, but is %s", minDuration, clvc.Duration, )) } return errors } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (lcv ConstantLoopingVUsConfig) GetMaxVUs() int64 { - return lcv.VUs.Int64 +// GetExecutionRequirements just reserves the number of specified VUs for the +// whole duration of the scheduler, including the maximum waiting time for +// iterations to gracefully stop. +func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: uint64(clvc.GetVUs(es)), + }, + { + TimeOffset: time.Duration(clvc.Duration.Duration + clvc.GracefulStop.Duration), + PlannedVUs: 0, + }, + } +} + +// NewScheduler creates a new ConstantLoopingVUs scheduler +func (clvc ConstantLoopingVUsConfig) NewScheduler(es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + return ConstantLoopingVUs{ + BaseScheduler: NewBaseScheduler(clvc, es, logger), + config: clvc, + }, nil } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (lcv ConstantLoopingVUsConfig) GetMaxDuration() time.Duration { - maxDuration := lcv.Duration.Duration - if !lcv.Interruptible.Bool { - maxDuration += lcv.IterationTimeout.Duration - } - return time.Duration(maxDuration) +// ConstantLoopingVUs maintains a constant number of VUs running for the +// specified duration. +type ConstantLoopingVUs struct { + *BaseScheduler + config ConstantLoopingVUsConfig } -// Split divides the VUS as best it can, but keeps the same duration -func (lcv ConstantLoopingVUsConfig) Split(percentages []float64) ([]Config, error) { - if err := checkPercentagesSum(percentages); err != nil { - return nil, err +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &ConstantLoopingVUs{} + +// Run constantly loops through as many iterations as possible on a fixed number +// of VUs for the specified duration. +func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := clv.executorState.Options.ExecutionSegment + numVUs := clv.config.GetVUs(segment) + duration := time.Duration(clv.config.Duration.Duration) + gracefulStop := clv.config.GetGracefulStop() + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + + // Make sure the log and the progress bar have accurate information + clv.logger.WithFields( + logrus.Fields{"vus": numVUs, "duration": duration, "type": clv.config.GetType()}, + ).Debug("Starting scheduler run...") + + progresFn := func() (float64, string) { + spent := time.Since(startTime) + if spent > duration { + return 1, fmt.Sprintf("constant looping %d VUs for %s", numVUs, duration) + } + return float64(spent) / float64(duration), fmt.Sprintf( + "constant looping %d VUs, %s/%s", numVUs, pb.GetFixedLengthDuration(spent, duration), duration, + ) + } + clv.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, clv, progresFn) + + // Actually schedule the VUs and iterations... + wg := sync.WaitGroup{} + regDurationDone := regDurationCtx.Done() + runIteration := getIterationRunner(clv.executorState, clv.logger, out) + + handleVU := func(vu lib.VU) { + defer clv.executorState.ReturnVU(vu) + defer wg.Done() + + for { + select { + case <-regDurationDone: + return // don't make more iterations + default: + // continue looping + } + runIteration(maxDurationCtx, vu) + } } - configs := make([]Config, len(percentages)) - for i, p := range percentages { - //TODO: figure out a better approach for the proportional distribution - // of the VUs (which are indivisible items)... - // Some sort of "pick closest match to percentage and adjust remaining"? - configs[i] = &ConstantLoopingVUsConfig{ - BaseConfig: *lcv.BaseConfig.CopyWithPercentage(p), - VUs: null.IntFrom(int64(float64(lcv.VUs.Int64) / p)), - Duration: lcv.Duration, + + for i := int64(0); i < numVUs; i++ { + wg.Add(1) + vu, err := clv.executorState.GetPlannedVU(ctx, clv.logger) + if err != nil { + return err } + go handleVU(vu) } - return configs, nil + + wg.Wait() + return nil } diff --git a/lib/scheduler/helpers.go b/lib/scheduler/helpers.go index 31bf37bf22a..de0a0aff61f 100644 --- a/lib/scheduler/helpers.go +++ b/lib/scheduler/helpers.go @@ -21,47 +21,35 @@ package scheduler import ( - "bytes" - "encoding/json" + "context" "fmt" - "math" - "strings" -) + "math/big" + "time" -// A helper function to verify percentage distributions -func checkPercentagesSum(percentages []float64) error { - var sum float64 - for _, v := range percentages { - sum += v - } - if math.Abs(100-sum) >= minPercentage { - return fmt.Errorf("split percentage sum is %.2f while it should be 100", sum) - } - return nil -} + "github.com/loadimpact/k6/ui/pb" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" +) -// A helper function for joining error messages into a single string -func concatErrors(errors []error, separator string) string { - errStrings := make([]string, len(errors)) - for i, e := range errors { - errStrings[i] = e.Error() +func sumStagesDuration(stages []Stage) (result time.Duration) { + for _, s := range stages { + result += time.Duration(s.Duration.Duration) } - return strings.Join(errStrings, separator) + return } -// Decode a JSON in a strict manner, emitting an error if there are unknown fields -func strictJSONUnmarshal(data []byte, v interface{}) error { - dec := json.NewDecoder(bytes.NewReader(data)) - dec.DisallowUnknownFields() - dec.UseNumber() - - if err := dec.Decode(&v); err != nil { - return err - } - if dec.More() { - return fmt.Errorf("unexpected data after the JSON object") +func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 { + max := unscaledStartValue + for _, s := range stages { + if s.Target.Int64 > max { + max = s.Target.Int64 + } } - return nil + return max } // A helper function to avoid code duplication @@ -86,3 +74,143 @@ func validateStages(stages []Stage) []error { } return errors } + +// getIterationRunner is a helper function that returns an iteration executor +// closure. It takes care of updating metrics, executor stat statistics and +// warning messages. +func getIterationRunner(executorState *lib.ExecutorState, logger *logrus.Entry, out chan<- stats.SampleContainer, +) func(context.Context, lib.VU) { + + return func(ctx context.Context, vu lib.VU) { + err := vu.RunOnce(ctx) + + //TODO: track (non-ramp-down) errors from script iterations as a metric, + // and have a default threshold that will abort the script when the error + // rate exceeds a certain percentage + + select { + case <-ctx.Done(): + // Don't log errors or emit iterations metrics from cancelled iterations + executorState.AddPartialIterations(1) + default: + if err != nil { + if s, ok := err.(fmt.Stringer); ok { + logger.Error(s.String()) + } else { + logger.Error(err.Error()) + } + //TODO: investigate context cancelled errors + } + + out <- stats.Sample{ + Time: time.Now(), + Metric: metrics.Iterations, + Value: 1, + Tags: executorState.Options.RunTags, + } + executorState.AddFullIterations(1) + } + } +} + +// getDurationContexts is used to create sub-contexts that can restrict a +// scheduler to only run for its allotted time. +// +// If the scheduler doesn't have a graceful stop period for iterations, then +// both returned sub-contexts will be the same one, with a timeout equal to +// supplied regular scheduler duration. +// +// But if a graceful stop is enabled, then the first returned context (and the +// cancel func) will be for the "outer" sub-context. Its timeout will include +// both the regular duration and the specified graceful stop period. The second +// context will be a sub-context of the first one and its timeout will include +// only the regular duration. +// +// In either case, the usage of these contexts should be like this: +// - As long as the regDurationCtx isn't done, new iterations can be started. +// - After regDurationCtx is done, no new iterations should be started; every +// VU that finishes an iteration from now on can be returned to the buffer +// pool in the executor state struct. +// - After maxDurationCtx is done, any VUs with iterations will be +// interrupted by the context's closing and will be returned to the buffer. +// - If you want to interrupt the execution of all VUs prematurely (e.g. there +// was an error or something like that), trigger maxDurationCancel(). +// - If the whole test is aborted, the parent context will be cancelled, so +// that will also cancel these contexts, thus the "general abort" case is +// handled transparently. +func getDurationContexts(parentCtx context.Context, regularDuration, gracefulStop time.Duration) ( + startTime time.Time, maxDurationCtx, regDurationCtx context.Context, maxDurationCancel func(), +) { + startTime = time.Now() + maxEndTime := startTime.Add(regularDuration + gracefulStop) + + maxDurationCtx, maxDurationCancel = context.WithDeadline(parentCtx, maxEndTime) + if gracefulStop == 0 { + return startTime, maxDurationCtx, maxDurationCtx, maxDurationCancel + } + regDurationCtx, _ = context.WithDeadline(maxDurationCtx, startTime.Add(regularDuration)) //nolint:govet + return startTime, maxDurationCtx, regDurationCtx, maxDurationCancel +} + +// trackProgress is a helper function that monitors certain end-events in a +// scheduler and updates it's progressbar accordingly. +func trackProgress( + parentCtx, maxDurationCtx, regDurationCtx context.Context, + sched lib.Scheduler, snapshot func() (float64, string), +) { + progressBar := sched.GetProgress() + logger := sched.GetLogger() + + <-regDurationCtx.Done() // Wait for the regular context to be over + gracefulStop := sched.GetConfig().GetGracefulStop() + if parentCtx.Err() == nil && gracefulStop > 0 { + p, right := snapshot() + logger.WithField("gracefulStop", gracefulStop).Debug( + "Regular duration is done, waiting for iterations to gracefully finish", + ) + progressBar.Modify(pb.WithConstProgress(p, right+", gracefully stopping...")) + } + + <-maxDurationCtx.Done() + p, right := snapshot() + select { + case <-parentCtx.Done(): + progressBar.Modify(pb.WithConstProgress(p, right+" interrupted!")) + default: + progressBar.Modify(pb.WithConstProgress(p, right+" done!")) + } +} + +// getScaledArrivalRate returns a rational number containing the scaled value of +// the given rate over the given period. This should generally be the first +// function that's called, before we do any calculations with the users-supplied +// rates in the arrival-rate executors. +func getScaledArrivalRate(es *lib.ExecutionSegment, rate int64, period time.Duration) *big.Rat { + return es.InPlaceScaleRat(big.NewRat(rate, int64(period))) +} + +// just a cached value to avoid allocationg it every getTickerPeriod() call +var zero = big.NewInt(0) //nolint:gochecknoglobals + +// getTickerPeriod is just a helper function that returns the ticker interval* +// we need for given arrival-rate parameters. +// +// It's possible for this function to return a zero duration (i.e. valid=false) +// and 0 isn't a valid ticker period. This happens so we don't divide by 0 when +// the arrival-rate period is 0. This case has to be handled separately. +func getTickerPeriod(scaledArrivalRate *big.Rat) types.NullDuration { + if scaledArrivalRate.Num().Cmp(zero) == 0 { + return types.NewNullDuration(0, false) + } + // Basically, the ticker rate is time.Duration(1/arrivalRate). Considering + // that time.Duration is represented as int64 nanoseconds, no meaningful + // precision is likely to be lost here... + result, _ := new(big.Rat).SetFrac(scaledArrivalRate.Denom(), scaledArrivalRate.Num()).Float64() + return types.NewNullDuration(time.Duration(result), true) +} + +// getArrivalRatePerSec returns the iterations per second rate. +func getArrivalRatePerSec(scaledArrivalRate *big.Rat) *big.Rat { + perSecRate := big.NewRat(int64(time.Second), 1) + return perSecRate.Mul(perSecRate, scaledArrivalRate) +} diff --git a/lib/scheduler/manual.go b/lib/scheduler/manual.go new file mode 100644 index 00000000000..ef13b85a07e --- /dev/null +++ b/lib/scheduler/manual.go @@ -0,0 +1,169 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package scheduler + +import ( + "errors" + "fmt" + "time" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" +) + +const manualExecution = "manual-execution" + +// ManualExecutionConfig stores VUs and duration +type ManualExecutionConfig struct { + StartVUs null.Int + MaxVUs null.Int + Duration types.NullDuration +} + +// NewManualExecutionConfig returns a ManualExecutionConfig with default values +func NewManualExecutionConfig(startVUs, maxVUs null.Int, duration types.NullDuration) ManualExecutionConfig { + if !maxVUs.Valid { + maxVUs = startVUs + } + return ManualExecutionConfig{startVUs, maxVUs, duration} +} + +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &ManualExecutionConfig{} + +// GetDescription returns a human-readable description of the scheduler options +func (mec ManualExecutionConfig) GetDescription(_ *lib.ExecutionSegment) string { + duration := "" + if mec.Duration.Duration != 0 { + duration = fmt.Sprintf(" and duration %s", mec.Duration) + } + return fmt.Sprintf( + "Manual execution with %d starting and %d initialized VUs%s", + mec.StartVUs.Int64, mec.MaxVUs.Int64, duration, + ) +} + +// Validate makes sure all options are configured and valid +func (mec ManualExecutionConfig) Validate() []error { + var errors []error + if mec.StartVUs.Int64 <= 0 { + errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) + } + + if mec.MaxVUs.Int64 < mec.StartVUs.Int64 { + errors = append(errors, fmt.Errorf("the number of MaxVUs should more than or equal to the starting number of VUs")) + } + + if !mec.Duration.Valid { + errors = append(errors, fmt.Errorf("the duration should be specified, for infinite duration use 0")) + } else if time.Duration(mec.Duration.Duration) < 0 { + errors = append(errors, fmt.Errorf( + "the duration shouldn't be negative, for infinite duration use 0", + )) + } + + return errors +} + +// GetExecutionRequirements just reserves the number of starting VUs for the whole +// duration of the scheduler, so these VUs can be initialized in the beginning of the +// test. +// +// Importantly, if 0 (i.e. infinite) duration is configured, this scheduler doesn't +// emit the last step to relinquish these VUs. +// +// Also, the manual execution scheduler doesn't set MaxUnplannedVUs in the returned steps, +// since their initialization and usage is directly controlled by the user and is effectively +// bounded only by the resources of the machine k6 is running on. +// +// This is not a problem, because the MaxUnplannedVUs are mostly meant to be used for +// calculating the maximum possble number of initialized VUs at any point during a test +// run. That's used for sizing purposes and for user qouta checking in the cloud execution, +// where the manual scheduler isn't supported. +func (mec ManualExecutionConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + startVUs := lib.ExecutionStep{ + TimeOffset: 0, + PlannedVUs: uint64(es.Scale(mec.StartVUs.Int64)), + MaxUnplannedVUs: 0, // intentional, see function comment + } + + maxDuration := time.Duration(mec.Duration.Duration) + if maxDuration == 0 { + // Infinite duration, don't emit 0 VUs at the end since there's no planned end + return []lib.ExecutionStep{startVUs} + } + return []lib.ExecutionStep{startVUs, { + TimeOffset: maxDuration, + PlannedVUs: 0, + MaxUnplannedVUs: 0, // intentional, see function comment + }} +} + +// GetName always returns manual-execution, since this config can't be +// specified in the exported script options. +func (ManualExecutionConfig) GetName() string { + return manualExecution +} + +// GetType always returns manual-execution, since that's this special +// config's type... +func (ManualExecutionConfig) GetType() string { + return manualExecution +} + +// GetStartTime always returns 0, since the manual execution scheduler +// always starts in the beginning and is always the only scheduler. +func (ManualExecutionConfig) GetStartTime() time.Duration { + return 0 +} + +// GetGracefulStop always returns 0, since we still don't support graceful +// stops or ramp downs in the manual execution mode. +//TODO: implement? +func (ManualExecutionConfig) GetGracefulStop() time.Duration { + return 0 +} + +// GetEnv returns an empty map, since the manual executor doesn't support custom +// environment variables. +func (ManualExecutionConfig) GetEnv() map[string]string { + return nil +} + +// GetExec always returns nil, for now there's no way to execute custom funcions in +// the manual execution mode. +func (ManualExecutionConfig) GetExec() null.String { + return null.NewString("", false) +} + +// IsDistributable simply returns false because there's no way to reliably +// distribute the manual execution scheduler. +func (ManualExecutionConfig) IsDistributable() bool { + return false +} + +// NewScheduler creates a new ManualExecution "scheduler" +func (mec ManualExecutionConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + return nil, errors.New("not implemented 4") //TODO +} diff --git a/lib/scheduler/per_vu_iterations.go b/lib/scheduler/per_vu_iterations.go index 25501411ea4..8fe991643bc 100644 --- a/lib/scheduler/per_vu_iterations.go +++ b/lib/scheduler/per_vu_iterations.go @@ -21,19 +21,26 @@ package scheduler import ( + "context" "fmt" + "sync" + "sync/atomic" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const perVUIterationsType = "per-vu-iterations" func init() { - RegisterConfigType(perVUIterationsType, func(name string, rawJSON []byte) (Config, error) { + lib.RegisterSchedulerConfigType(perVUIterationsType, func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { config := NewPerVUIterationsConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) + err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err }) } @@ -49,15 +56,35 @@ type PerVUIteationsConfig struct { // NewPerVUIterationsConfig returns a PerVUIteationsConfig with default values func NewPerVUIterationsConfig(name string) PerVUIteationsConfig { return PerVUIteationsConfig{ - BaseConfig: NewBaseConfig(name, perVUIterationsType, false), + BaseConfig: NewBaseConfig(name, perVUIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), - MaxDuration: types.NewNullDuration(1*time.Hour, false), + MaxDuration: types.NewNullDuration(10*time.Minute, false), //TODO: shorten? } } -// Make sure we implement the Config interface -var _ Config = &PerVUIteationsConfig{} +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &PerVUIteationsConfig{} + +// GetVUs returns the scaled VUs for the scheduler. +func (pvic PerVUIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(pvic.VUs.Int64) +} + +// GetIterations returns the UNSCALED iteration count for the scheduler. It's +// important to note that scaling per-VU iteration scheduler affects only the +// number of VUs. If we also scaled the iterations, scaling would have quadratic +// effects instead of just linear. +func (pvic PerVUIteationsConfig) GetIterations() int64 { + return pvic.Iterations.Int64 +} + +// GetDescription returns a human-readable description of the scheduler options +func (pvic PerVUIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { + return fmt.Sprintf("%d iterations for each of %d VUs%s", + pvic.GetIterations(), pvic.GetVUs(es), + pvic.getBaseInfo(fmt.Sprintf("maxDuration: %s", pvic.MaxDuration.Duration))) +} // Validate makes sure all options are configured and valid func (pvic PerVUIteationsConfig) Validate() []error { @@ -79,17 +106,99 @@ func (pvic PerVUIteationsConfig) Validate() []error { return errors } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (pvic PerVUIteationsConfig) GetMaxVUs() int64 { - return pvic.VUs.Int64 +// GetExecutionRequirements just reserves the number of specified VUs for the +// whole duration of the scheduler, including the maximum waiting time for +// iterations to gracefully stop. +func (pvic PerVUIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: uint64(pvic.GetVUs(es)), + }, + { + TimeOffset: time.Duration(pvic.MaxDuration.Duration + pvic.GracefulStop.Duration), + PlannedVUs: 0, + }, + } +} + +// NewScheduler creates a new PerVUIteations scheduler +func (pvic PerVUIteationsConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + + return PerVUIteations{ + BaseScheduler: NewBaseScheduler(pvic, es, logger), + config: pvic, + }, nil +} + +// PerVUIteations executes a specific number of iterations with each VU. +type PerVUIteations struct { + *BaseScheduler + config PerVUIteationsConfig } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (pvic PerVUIteationsConfig) GetMaxDuration() time.Duration { - maxDuration := pvic.MaxDuration.Duration - if !pvic.Interruptible.Bool { - maxDuration += pvic.IterationTimeout.Duration +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &PerVUIteations{} + +// Run executes a specific number of iterations with each confugured VU. +func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := pvi.executorState.Options.ExecutionSegment + numVUs := pvi.config.GetVUs(segment) + iterations := pvi.config.GetIterations() + duration := time.Duration(pvi.config.MaxDuration.Duration) + gracefulStop := pvi.config.GetGracefulStop() + + _, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + + // Make sure the log and the progress bar have accurate information + pvi.logger.WithFields(logrus.Fields{ + "vus": numVUs, "iterations": iterations, "maxDuration": duration, "type": pvi.config.GetType(), + }).Debug("Starting scheduler run...") + + totalIters := uint64(numVUs * iterations) + doneIters := new(uint64) + fmtStr := pb.GetFixedLengthIntFormat(int64(totalIters)) + "/%d iters, %d from each of %d VUs" + progresFn := func() (float64, string) { + currentDoneIters := atomic.LoadUint64(doneIters) + return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( + fmtStr, currentDoneIters, totalIters, iterations, numVUs, + ) } - return time.Duration(maxDuration) + pvi.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, pvi, progresFn) + + // Actually schedule the VUs and iterations... + wg := sync.WaitGroup{} + regDurationDone := regDurationCtx.Done() + runIteration := getIterationRunner(pvi.executorState, pvi.logger, out) + + handleVU := func(vu lib.VU) { + defer pvi.executorState.ReturnVU(vu) + defer wg.Done() + + for i := int64(0); i < iterations; i++ { + select { + case <-regDurationDone: + return // don't make more iterations + default: + // continue looping + } + runIteration(maxDurationCtx, vu) + atomic.AddUint64(doneIters, 1) + } + } + + for i := int64(0); i < numVUs; i++ { + wg.Add(1) + vu, err := pvi.executorState.GetPlannedVU(ctx, pvi.logger) + if err != nil { + return err + } + go handleVU(vu) + } + + wg.Wait() + return nil } diff --git a/lib/scheduler/schedulers_test.go b/lib/scheduler/schedulers_test.go index 11c07453f64..7f503682f37 100644 --- a/lib/scheduler/schedulers_test.go +++ b/lib/scheduler/schedulers_test.go @@ -26,164 +26,246 @@ import ( "testing" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" ) +type exp struct { + parseError bool + validationError bool + custom func(t *testing.T, cm lib.SchedulerConfigMap) +} + type configMapTestCase struct { - rawJSON string - expectParseError bool - expectValidationError bool - customValidator func(t *testing.T, cm ConfigMap) + rawJSON string + expected exp } //nolint:lll,gochecknoglobals var configMapTestCases = []configMapTestCase{ - {"", true, false, nil}, - {"1234", true, false, nil}, - {"asdf", true, false, nil}, - {"'adsf'", true, false, nil}, - {"[]", true, false, nil}, - {"{}", false, false, func(t *testing.T, cm ConfigMap) { - assert.Equal(t, cm, ConfigMap{}) - }}, - {"{}asdf", true, false, nil}, - {"null", false, false, func(t *testing.T, cm ConfigMap) { + {"", exp{parseError: true}}, + {"1234", exp{parseError: true}}, + {"asdf", exp{parseError: true}}, + {"'adsf'", exp{parseError: true}}, + {"[]", exp{parseError: true}}, + {"{}", exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + assert.Equal(t, cm, lib.SchedulerConfigMap{}) + }}}, + {"{}asdf", exp{parseError: true}}, + {"null", exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { assert.Nil(t, cm) - }}, - {`{"someKey": {}}`, true, false, nil}, - {`{"someKey": {"type": "constant-blah-blah", "vus": 10, "duration": "60s"}}`, true, false, nil}, - {`{"someKey": {"type": "constant-looping-vus", "uknownField": "should_error"}}`, true, false, nil}, - {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", "env": 123}}`, true, false, nil}, + }}}, + {`{"someKey": {}}`, exp{parseError: true}}, + {`{"someKey": {"type": "constant-blah-blah", "vus": 10, "duration": "60s"}}`, exp{parseError: true}}, + {`{"someKey": {"type": "constant-looping-vus", "uknownField": "should_error"}}`, exp{parseError: true}}, + {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", "env": 123}}`, exp{parseError: true}}, // Validation errors for constant-looping-vus and the base config - {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", "interruptible": false, - "iterationTimeout": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", + "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewConstantLoopingVUsConfig("someKey") sched.VUs = null.IntFrom(10) sched.Duration = types.NullDurationFrom(1 * time.Minute) - sched.Interruptible = null.BoolFrom(false) - sched.IterationTimeout = types.NullDurationFrom(10 * time.Second) + sched.GracefulStop = types.NullDurationFrom(10 * time.Second) sched.StartTime = types.NullDurationFrom(70 * time.Second) sched.Exec = null.StringFrom("someFunc") sched.Env = map[string]string{"test": "mest"} - require.Equal(t, cm, ConfigMap{"someKey": sched}) - require.Equal(t, sched.BaseConfig, cm["someKey"].GetBaseConfig()) - assert.Equal(t, 70*time.Second, cm["someKey"].GetMaxDuration()) - assert.Equal(t, int64(10), cm["someKey"].GetMaxVUs()) + require.Equal(t, cm, lib.SchedulerConfigMap{"someKey": sched}) + require.Equal(t, sched.BaseConfig.Name, cm["someKey"].GetName()) + require.Equal(t, sched.BaseConfig.Type, cm["someKey"].GetType()) + require.Equal(t, sched.BaseConfig.GetGracefulStop(), cm["someKey"].GetGracefulStop()) + require.Equal(t, + sched.BaseConfig.StartTime.Duration, + types.Duration(cm["someKey"].GetStartTime()), + ) + require.Equal(t, sched.BaseConfig.Env, cm["someKey"].GetEnv()) + assert.Empty(t, cm["someKey"].Validate()) - }}, - {`{"aname": {"type": "constant-looping-vus", "duration": "60s"}}`, false, false, nil}, - {`{"": {"type": "constant-looping-vus", "vus": 10, "duration": "60s"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 0.5}}`, true, false, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 0, "duration": "60s"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": -1, "duration": "60s"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "0s"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "startTime": "-10s"}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "exec": ""}}`, false, true, nil}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "iterationTimeout": "-2s"}}`, false, true, nil}, + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "10 looping VUs for 1m0s (exec: someFunc, startTime: 1m10s, gracefulStop: 10s)", cm["someKey"].GetDescription(nil)) + + schedReqs := cm["someKey"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 70*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) + totalReqs := cm.GetFullExecutionRequirements(nil) + endOffset, isFinal = lib.GetEndOffset(totalReqs) + assert.Equal(t, 140*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) + + }}, + }, + {`{"aname": {"type": "constant-looping-vus", "duration": "60s"}}`, exp{}}, + {`{"": {"type": "constant-looping-vus", "vus": 10, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 0.5}}`, exp{parseError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 10}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 0, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": -1, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "0s"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "startTime": "-10s"}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "exec": ""}}`, exp{validationError: true}}, + {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "gracefulStop": "-2s"}}`, exp{validationError: true}}, // variable-looping-vus - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 20, "iterationTimeout": "15s", - "stages": [{"duration": "60s", "target": 30}, {"duration": "120s", "target": 10}]}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", + "startTime": "23s", "stages": [{"duration": "60s", "target": 30}, {"duration": "130s", "target": 10}]}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewVariableLoopingVUsConfig("varloops") - sched.IterationTimeout = types.NullDurationFrom(15 * time.Second) + sched.GracefulStop = types.NullDurationFrom(15 * time.Second) + sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) sched.StartVUs = null.IntFrom(20) + sched.StartTime = types.NullDurationFrom(23 * time.Second) sched.Stages = []Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(60 * time.Second)}, - {Target: null.IntFrom(10), Duration: types.NullDurationFrom(120 * time.Second)}, + {Target: null.IntFrom(10), Duration: types.NullDurationFrom(130 * time.Second)}, } - require.Equal(t, cm, ConfigMap{"varloops": sched}) - assert.Equal(t, int64(30), cm["varloops"].GetMaxVUs()) - assert.Equal(t, 195*time.Second, cm["varloops"].GetMaxDuration()) + require.Equal(t, cm, lib.SchedulerConfigMap{"varloops": sched}) + assert.Empty(t, cm["varloops"].Validate()) - }}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 0, "stages": [{"duration": "60s", "target": 0}]}}`, false, false, nil}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": -1, "stages": [{"duration": "60s", "target": 30}]}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "-60s", "target": 30}]}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "60s", "target": -30}]}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus", "stages": [{"duration": "60s"}]}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus", "stages": [{"target": 30}]}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus", "stages": []}}`, false, true, nil}, - {`{"varloops": {"type": "variable-looping-vus"}}`, false, true, nil}, + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "Up to 30 looping VUs for 3m10s over 2 stages (gracefulRampDown: 10s, startTime: 23s, gracefulStop: 15s)", cm["varloops"].GetDescription(nil)) + schedReqs := cm["varloops"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 205*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(30), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs := cm.GetFullExecutionRequirements(nil) + endOffset, isFinal = lib.GetEndOffset(totalReqs) + assert.Equal(t, 228*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(30), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) + }}, + }, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 0, "stages": [{"duration": "60s", "target": 0}]}}`, exp{}}, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": -1, "stages": [{"duration": "60s", "target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "-60s", "target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "60s", "target": -30}]}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus", "stages": [{"duration": "60s"}]}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus", "stages": [{"target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus", "stages": []}}`, exp{validationError: true}}, + {`{"varloops": {"type": "variable-looping-vus"}}`, exp{validationError: true}}, // shared-iterations - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"ishared": {"type": "shared-iterations", "iterations": 22, "vus": 12, "maxDuration": "100s"}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewSharedIterationsConfig("ishared") - sched.Iterations = null.IntFrom(20) - sched.VUs = null.IntFrom(10) - require.Equal(t, cm, ConfigMap{"ishared": sched}) - assert.Equal(t, int64(10), cm["ishared"].GetMaxVUs()) - assert.Equal(t, 3630*time.Second, cm["ishared"].GetMaxDuration()) + sched.Iterations = null.IntFrom(22) + sched.MaxDuration = types.NullDurationFrom(100 * time.Second) + sched.VUs = null.IntFrom(12) + assert.Empty(t, cm["ishared"].Validate()) - }}, - {`{"ishared": {"type": "shared-iterations"}}`, false, false, nil}, // Has 1 VU & 1 iter default values - {`{"ishared": {"type": "shared-iterations", "iterations": 20}}`, false, false, nil}, - {`{"ishared": {"type": "shared-iterations", "vus": 10}}`, false, true, nil}, // error because VUs are more than iters - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "30m"}}`, false, false, nil}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, false, true, nil}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, false, true, nil}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": -10}}`, false, true, nil}, - {`{"ishared": {"type": "shared-iterations", "iterations": -1, "vus": 1}}`, false, true, nil}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 30}}`, false, true, nil}, + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "22 iterations shared among 12 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(nil)) + schedReqs := cm["ishared"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 130*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(12), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(12), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, schedReqs, totalReqs) + }}, + }, + {`{"ishared": {"type": "shared-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values + {`{"ishared": {"type": "shared-iterations", "iterations": 20}}`, exp{}}, + {`{"ishared": {"type": "shared-iterations", "vus": 10}}`, exp{validationError: true}}, // error because VUs are more than iters + {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "30m"}}`, exp{}}, + {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, + {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, + {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, + {`{"ishared": {"type": "shared-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, + {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 30}}`, exp{validationError: true}}, // per-vu-iterations - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 23, "vus": 13, "gracefulStop": 0}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewPerVUIterationsConfig("ipervu") - sched.Iterations = null.IntFrom(20) - sched.VUs = null.IntFrom(10) - require.Equal(t, cm, ConfigMap{"ipervu": sched}) - assert.Equal(t, int64(10), cm["ipervu"].GetMaxVUs()) - assert.Equal(t, 3630*time.Second, cm["ipervu"].GetMaxDuration()) + sched.Iterations = null.IntFrom(23) + sched.GracefulStop = types.NullDurationFrom(0) + sched.VUs = null.IntFrom(13) + assert.Empty(t, cm["ipervu"].Validate()) + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "23 iterations for each of 13 VUs (maxDuration: 10m0s)", cm["ipervu"].GetDescription(nil)) + + schedReqs := cm["ipervu"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 600*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(13), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(13), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, schedReqs, totalReqs) }}, - {`{"ipervu": {"type": "per-vu-iterations"}}`, false, false, nil}, // Has 1 VU & 1 iter default values - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20}}`, false, false, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "vus": 10}}`, false, false, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10}}`, false, false, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, false, true, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, false, true, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": -10}}`, false, true, nil}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": -1, "vus": 1}}`, false, true, nil}, + }, + {`{"ipervu": {"type": "per-vu-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20}}`, exp{}}, + {`{"ipervu": {"type": "per-vu-iterations", "vus": 10}}`, exp{}}, + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10}}`, exp{}}, + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, + {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, + {`{"ipervu": {"type": "per-vu-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, // constant-arrival-rate - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"carrival": {"type": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewConstantArrivalRateConfig("carrival") - sched.Rate = null.IntFrom(10) + sched.Rate = null.IntFrom(30) sched.Duration = types.NullDurationFrom(10 * time.Minute) sched.TimeUnit = types.NullDurationFrom(1 * time.Minute) sched.PreAllocatedVUs = null.IntFrom(20) sched.MaxVUs = null.IntFrom(30) - require.Equal(t, cm, ConfigMap{"carrival": sched}) - assert.Equal(t, int64(30), cm["carrival"].GetMaxVUs()) - assert.Equal(t, 630*time.Second, cm["carrival"].GetMaxDuration()) + assert.Empty(t, cm["carrival"].Validate()) - }}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, false, false, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30, "timeUnit": "-1s"}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "maxVUs": 30}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "preAllocatedVUs": 20, "maxVUs": 30}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0m", "preAllocatedVUs": 20, "maxVUs": 30}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 0, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 15}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0s", "preAllocatedVUs": 20, "maxVUs": 25}}`, false, true, nil}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": -2, "maxVUs": 25}}`, false, true, nil}, + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "0.50 iterations/s for 10m0s (maxVUs: 20-30, gracefulStop: 30s)", cm["carrival"].GetDescription(nil)) + + schedReqs := cm["carrival"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 630*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(20), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) + totalReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, schedReqs, totalReqs) + }}, + }, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30, "timeUnit": "-1s"}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 0, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 15}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0s", "preAllocatedVUs": 20, "maxVUs": 25}}`, exp{validationError: true}}, + {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": -2, "maxVUs": 25}}`, exp{validationError: true}}, // variable-arrival-rate - {`{"varrival": {"type": "variable-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, "maxVUs": 50, - "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, - false, false, func(t *testing.T, cm ConfigMap) { + {`{"varrival": {"type": "variable-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, + "maxVUs": 50, "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, + exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { sched := NewVariableArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) sched.Stages = []Stage{ @@ -193,20 +275,34 @@ var configMapTestCases = []configMapTestCase{ sched.TimeUnit = types.NullDurationFrom(30 * time.Second) sched.PreAllocatedVUs = null.IntFrom(20) sched.MaxVUs = null.IntFrom(50) - require.Equal(t, cm, ConfigMap{"varrival": sched}) - assert.Equal(t, int64(50), cm["varrival"].GetMaxVUs()) - assert.Equal(t, 510*time.Second, cm["varrival"].GetMaxDuration()) + require.Equal(t, cm, lib.SchedulerConfigMap{"varrival": sched}) + assert.Empty(t, cm["varrival"].Validate()) + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "Up to 1.00 iterations/s for 8m0s over 2 stages (maxVUs: 20-50, gracefulStop: 30s)", cm["varrival"].GetDescription(nil)) + + schedReqs := cm["varrival"].GetExecutionRequirements(nil) + endOffset, isFinal := lib.GetEndOffset(schedReqs) + assert.Equal(t, 510*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(20), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(50), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, schedReqs, totalReqs) }}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, false, false, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": -20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "startRate": -1, "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}], "timeUnit": "-1s"}}`, false, true, nil}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 30, "maxVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, false, true, nil}, + }, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": -20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "startRate": -1, "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}], "timeUnit": "-1s"}}`, exp{validationError: true}}, + {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 30, "maxVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + //TODO: more tests of mixed schedulers and execution plans } func TestConfigMapParsingAndValidation(t *testing.T) { @@ -215,25 +311,121 @@ func TestConfigMapParsingAndValidation(t *testing.T) { tc := tc t.Run(fmt.Sprintf("TestCase#%d", i), func(t *testing.T) { t.Logf(tc.rawJSON) - var result ConfigMap + var result lib.SchedulerConfigMap err := json.Unmarshal([]byte(tc.rawJSON), &result) - if tc.expectParseError { + if tc.expected.parseError { require.Error(t, err) return } require.NoError(t, err) - validationErrors := result.Validate() - if tc.expectValidationError { - assert.NotEmpty(t, validationErrors) + parseErrors := result.Validate() + if tc.expected.validationError { + assert.NotEmpty(t, parseErrors) } else { - assert.Empty(t, validationErrors) + assert.Empty(t, parseErrors) } - if tc.customValidator != nil { - tc.customValidator(t, result) + if tc.expected.custom != nil { + tc.expected.custom(t, result) } }) } } -//TODO: check percentage split calculations +func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { + t.Parallel() + conf := NewVariableLoopingVUsConfig("test") + conf.StartVUs = null.IntFrom(4) + conf.Stages = []Stage{ + {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, + } + + expRawStepsNoZeroEnd := []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 3 * time.Second, PlannedVUs: 5}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 3}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 8 * time.Second, PlannedVUs: 2}, + {TimeOffset: 9 * time.Second, PlannedVUs: 3}, + {TimeOffset: 10 * time.Second, PlannedVUs: 4}, + {TimeOffset: 11 * time.Second, PlannedVUs: 5}, + {TimeOffset: 12 * time.Second, PlannedVUs: 4}, + {TimeOffset: 13 * time.Second, PlannedVUs: 3}, + {TimeOffset: 14 * time.Second, PlannedVUs: 2}, + {TimeOffset: 15 * time.Second, PlannedVUs: 1}, + {TimeOffset: 16 * time.Second, PlannedVUs: 2}, + {TimeOffset: 17 * time.Second, PlannedVUs: 3}, + {TimeOffset: 18 * time.Second, PlannedVUs: 4}, + {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + } + rawStepsNoZeroEnd := conf.getRawExecutionSteps(nil, false) + assert.Equal(t, expRawStepsNoZeroEnd, rawStepsNoZeroEnd) + endOffset, isFinal := lib.GetEndOffset(rawStepsNoZeroEnd) + assert.Equal(t, 20*time.Second, endOffset) + assert.Equal(t, false, isFinal) + + rawStepsZeroEnd := conf.getRawExecutionSteps(nil, true) + assert.Equal(t, + append(expRawStepsNoZeroEnd, lib.ExecutionStep{TimeOffset: 23 * time.Second, PlannedVUs: 0}), + rawStepsZeroEnd, + ) + endOffset, isFinal = lib.GetEndOffset(rawStepsZeroEnd) + assert.Equal(t, 23*time.Second, endOffset) + assert.Equal(t, true, isFinal) + + // GracefulStop and GracefulRampDown equal to the default 30 sec + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 50 * time.Second, PlannedVUs: 1}, + {TimeOffset: 53 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(nil)) + + // Try a longer GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(80 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 50 * time.Second, PlannedVUs: 1}, + {TimeOffset: 103 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(nil)) + + // Try a much shorter GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(3 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(nil)) + + // Try a zero GracefulStop + conf.GracefulStop = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 23 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(nil)) + + // Try a zero GracefulStop and GracefulRampDown, i.e. raw steps with 0 end cap + conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(nil)) +} diff --git a/lib/scheduler/shared_iterations.go b/lib/scheduler/shared_iterations.go index d6ef2e2bb92..110ba25899b 100644 --- a/lib/scheduler/shared_iterations.go +++ b/lib/scheduler/shared_iterations.go @@ -21,21 +21,31 @@ package scheduler import ( + "context" "fmt" + "sync" + "sync/atomic" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const sharedIterationsType = "shared-iterations" func init() { - RegisterConfigType(sharedIterationsType, func(name string, rawJSON []byte) (Config, error) { - config := NewSharedIterationsConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) - return config, err - }) + lib.RegisterSchedulerConfigType( + sharedIterationsType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := NewSharedIterationsConfig(name) + err := lib.StrictJSONUnmarshal(rawJSON, &config) + return config, err + }, + ) } // SharedIteationsConfig stores the number of VUs iterations, as well as maxDuration settings @@ -49,15 +59,32 @@ type SharedIteationsConfig struct { // NewSharedIterationsConfig returns a SharedIteationsConfig with default values func NewSharedIterationsConfig(name string) SharedIteationsConfig { return SharedIteationsConfig{ - BaseConfig: NewBaseConfig(name, sharedIterationsType, false), + BaseConfig: NewBaseConfig(name, sharedIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), - MaxDuration: types.NewNullDuration(1*time.Hour, false), + MaxDuration: types.NewNullDuration(10*time.Minute, false), //TODO: shorten? } } -// Make sure we implement the Config interface -var _ Config = &SharedIteationsConfig{} +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &SharedIteationsConfig{} + +// GetVUs returns the scaled VUs for the scheduler. +func (sic SharedIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(sic.VUs.Int64) +} + +// GetIterations returns the scaled iteration count for the scheduler. +func (sic SharedIteationsConfig) GetIterations(es *lib.ExecutionSegment) int64 { + return es.Scale(sic.Iterations.Int64) +} + +// GetDescription returns a human-readable description of the scheduler options +func (sic SharedIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { + return fmt.Sprintf("%d iterations shared among %d VUs%s", + sic.GetIterations(es), sic.GetVUs(es), + sic.getBaseInfo(fmt.Sprintf("maxDuration: %s", sic.MaxDuration.Duration))) +} // Validate makes sure all options are configured and valid func (sic SharedIteationsConfig) Validate() []error { @@ -82,17 +109,107 @@ func (sic SharedIteationsConfig) Validate() []error { return errors } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (sic SharedIteationsConfig) GetMaxVUs() int64 { - return sic.VUs.Int64 +// GetExecutionRequirements just reserves the number of specified VUs for the +// whole duration of the scheduler, including the maximum waiting time for +// iterations to gracefully stop. +func (sic SharedIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: uint64(sic.GetVUs(es)), + }, + { + TimeOffset: time.Duration(sic.MaxDuration.Duration + sic.GracefulStop.Duration), + PlannedVUs: 0, + }, + } +} + +// NewScheduler creates a new SharedIteations scheduler +func (sic SharedIteationsConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + + return SharedIteations{ + BaseScheduler: NewBaseScheduler(sic, es, logger), + config: sic, + }, nil +} + +// SharedIteations executes a specific total number of iterations, which are +// all shared by the configured VUs. +type SharedIteations struct { + *BaseScheduler + config SharedIteationsConfig } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (sic SharedIteationsConfig) GetMaxDuration() time.Duration { - maxDuration := sic.MaxDuration.Duration - if !sic.Interruptible.Bool { - maxDuration += sic.IterationTimeout.Duration +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &PerVUIteations{} + +// Run executes a specific total number of iterations, which are all shared by +// the configured VUs. +func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := si.executorState.Options.ExecutionSegment + numVUs := si.config.GetVUs(segment) + iterations := si.config.GetIterations(segment) + duration := time.Duration(si.config.MaxDuration.Duration) + gracefulStop := si.config.GetGracefulStop() + + _, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + + // Make sure the log and the progress bar have accurate information + si.logger.WithFields(logrus.Fields{ + "vus": numVUs, "iterations": iterations, "maxDuration": duration, "type": si.config.GetType(), + }).Debug("Starting scheduler run...") + + totalIters := uint64(iterations) + doneIters := new(uint64) + fmtStr := pb.GetFixedLengthIntFormat(int64(totalIters)) + "/%d shared iters among %d VUs" + progresFn := func() (float64, string) { + currentDoneIters := atomic.LoadUint64(doneIters) + return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( + fmtStr, currentDoneIters, totalIters, numVUs, + ) + } + si.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, si, progresFn) + + // Actually schedule the VUs and iterations... + wg := sync.WaitGroup{} + regDurationDone := regDurationCtx.Done() + runIteration := getIterationRunner(si.executorState, si.logger, out) + + attemptedIters := new(uint64) + handleVU := func(vu lib.VU) { + defer si.executorState.ReturnVU(vu) + defer wg.Done() + + for { + attemptedIterNumber := atomic.AddUint64(attemptedIters, 1) + if attemptedIterNumber > totalIters { + return + } + + runIteration(maxDurationCtx, vu) + atomic.AddUint64(doneIters, 1) + select { + case <-regDurationDone: + return // don't make more iterations + default: + // continue looping + } + } } - return time.Duration(maxDuration) + + for i := int64(0); i < numVUs; i++ { + wg.Add(1) + vu, err := si.executorState.GetPlannedVU(ctx, si.logger) + if err != nil { + return err + } + go handleVU(vu) + } + + wg.Wait() + return nil } diff --git a/lib/scheduler/variable_arrival_rate.go b/lib/scheduler/variable_arrival_rate.go index 76ba7a51dcb..df72ceb95f3 100644 --- a/lib/scheduler/variable_arrival_rate.go +++ b/lib/scheduler/variable_arrival_rate.go @@ -21,21 +21,36 @@ package scheduler import ( + "context" "fmt" + "math" + "math/big" + "sync/atomic" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const variableArrivalRateType = "variable-arrival-rate" +// How often we can make arrival rate adjustments when processing stages +// TODO: make configurable, in some bounds? +const minIntervalBetweenRateAdjustments = 250 * time.Millisecond + func init() { - RegisterConfigType(variableArrivalRateType, func(name string, rawJSON []byte) (Config, error) { - config := NewVariableArrivalRateConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) - return config, err - }) + lib.RegisterSchedulerConfigType( + variableArrivalRateType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := NewVariableArrivalRateConfig(name) + err := lib.StrictJSONUnmarshal(rawJSON, &config) + return config, err + }, + ) } // VariableArrivalRateConfig stores config for the variable arrival-rate scheduler @@ -55,13 +70,40 @@ type VariableArrivalRateConfig struct { // NewVariableArrivalRateConfig returns a VariableArrivalRateConfig with default values func NewVariableArrivalRateConfig(name string) VariableArrivalRateConfig { return VariableArrivalRateConfig{ - BaseConfig: NewBaseConfig(name, variableArrivalRateType, false), + BaseConfig: NewBaseConfig(name, variableArrivalRateType), TimeUnit: types.NewNullDuration(1*time.Second, false), } } -// Make sure we implement the Config interface -var _ Config = &VariableArrivalRateConfig{} +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &VariableArrivalRateConfig{} + +// GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. +func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(varc.PreAllocatedVUs.Int64) +} + +// GetMaxVUs is just a helper method that returns the scaled max VUs. +func (varc VariableArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(varc.MaxVUs.Int64) +} + +// GetDescription returns a human-readable description of the scheduler options +func (varc VariableArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { + //TODO: something better? always show iterations per second? + maxVUsRange := fmt.Sprintf("maxVUs: %d", es.Scale(varc.PreAllocatedVUs.Int64)) + if varc.MaxVUs.Int64 > varc.PreAllocatedVUs.Int64 { + maxVUsRange += fmt.Sprintf("-%d", es.Scale(varc.MaxVUs.Int64)) + } + maxUnscaledRate := getStagesUnscaledMaxTarget(varc.StartRate.Int64, varc.Stages) + maxArrRatePerSec, _ := getArrivalRatePerSec( + getScaledArrivalRate(es, maxUnscaledRate, time.Duration(varc.TimeUnit.Duration)), + ).Float64() + + return fmt.Sprintf("Up to %.2f iterations/s for %s over %d stages%s", + maxArrRatePerSec, sumStagesDuration(varc.Stages), + len(varc.Stages), varc.getBaseInfo(maxVUsRange)) +} // Validate makes sure all options are configured and valid func (varc VariableArrivalRateConfig) Validate() []error { @@ -92,20 +134,268 @@ func (varc VariableArrivalRateConfig) Validate() []error { return errors } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (varc VariableArrivalRateConfig) GetMaxVUs() int64 { - return varc.MaxVUs.Int64 +// GetExecutionRequirements just reserves the number of specified VUs for the +// whole duration of the scheduler, including the maximum waiting time for +// iterations to gracefully stop. +func (varc VariableArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: uint64(es.Scale(varc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(es.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), + }, + { + TimeOffset: sumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), + PlannedVUs: 0, + MaxUnplannedVUs: 0, + }, + } +} + +type rateChange struct { + // At what time should the rate below be applied. + timeOffset time.Duration + // Equals 1/rate: if rate was "1/5s", then this value, which is intended to + // be passed to time.NewTicker(), will be 5s. There's a special case when + // the rate is 0, for which we'll set Valid=false. That's because 0 isn't a + // valid ticker period and shouldn't be passed to time.NewTicker(). Instead, + // an empty or stopped ticker should be used. + tickerPeriod types.NullDuration +} + +// A helper method to generate the plan how the rate changes would happen. +func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.ExecutionSegment) []rateChange { + timeUnit := time.Duration(varc.TimeUnit.Duration) + // Important note for accuracy: we must work with and scale only the + // rational numbers, never the raw target values directly. It matters most + // for the accuracy of the intermediate rate change values, but it's + // important even here. + // + // Say we have a desired rate growth from 1/sec to 2/sec over 1 minute, and + // we split the test into two segments of 20% and 80%. If we used the whole + // numbers for scaling, then the instance executing the first segment won't + // ever do even a single request, since scale(20%, 1) would be 0, whereas + // the rational value for scale(20%, 1/sec) is 0.2/sec, or rather 1/5sec... + currentRate := getScaledArrivalRate(segment, varc.StartRate.Int64, timeUnit) + + rateChanges := []rateChange{} + timeFromStart := time.Duration(0) + + for _, stage := range varc.Stages { + stageTargetRate := getScaledArrivalRate(segment, stage.Target.Int64, timeUnit) + stageDuration := time.Duration(stage.Duration.Duration) + + if currentRate.Cmp(stageTargetRate) == 0 { + // We don't have to do anything but update the time offset + // if the rate wasn't changed in this stage + timeFromStart += stageDuration + continue + } + + // Handle 0-duration stages, i.e. instant rate jumps + if stageDuration == 0 { + rateChanges = append(rateChanges, rateChange{ + timeOffset: timeFromStart, + tickerPeriod: getTickerPeriod(stageTargetRate), + }) + currentRate = stageTargetRate + continue + } + // Basically, find out how many regular intervals with size of at least + // minIntervalBetweenRateAdjustments are in the stage's duration, and + // then use that number to calculate the actual step. All durations have + // nanosecond precision, so there isn't any actual loss of precision... + stepNumber := (stageDuration / minIntervalBetweenRateAdjustments) + if stepNumber > 1 { + stepInterval := stageDuration / stepNumber + for t := stepInterval; ; t += stepInterval { + if stageDuration-t < minIntervalBetweenRateAdjustments { + break + } + + rateDiff := new(big.Rat).Sub(stageTargetRate, currentRate) + tArrivalRate := new(big.Rat).Add( + currentRate, + rateDiff.Mul(rateDiff, big.NewRat(int64(t), int64(stageDuration))), + ) + + rateChanges = append(rateChanges, rateChange{ + timeOffset: timeFromStart + t, + tickerPeriod: getTickerPeriod(tArrivalRate), + }) + } + } + timeFromStart += stageDuration + rateChanges = append(rateChanges, rateChange{ + timeOffset: timeFromStart, + tickerPeriod: getTickerPeriod(stageTargetRate), + }) + currentRate = stageTargetRate + } + + return rateChanges +} + +// NewScheduler creates a new VariableArrivalRate scheduler +func (varc VariableArrivalRateConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + + return VariableArrivalRate{ + BaseScheduler: NewBaseScheduler(varc, es, logger), + config: varc, + plannedRateChanges: varc.getPlannedRateChanges(es.Options.ExecutionSegment), + }, nil +} + +// VariableArrivalRate tries to execute a specific number of iterations for a +// specific period. +//TODO: combine with the ConstantArrivalRate? +type VariableArrivalRate struct { + *BaseScheduler + config VariableArrivalRateConfig + plannedRateChanges []rateChange +} + +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &VariableArrivalRate{} + +// streamRateChanges is a helper method that emits rate change events at their +// proper time. +func (varr VariableArrivalRate) streamRateChanges(ctx context.Context, startTime time.Time) <-chan rateChange { + ch := make(chan rateChange) + go func() { + for _, step := range varr.plannedRateChanges { + offsetDiff := step.timeOffset - time.Since(startTime) + if offsetDiff > 0 { // wait until time of event arrives + select { + case <-ctx.Done(): + return // exit if context is cancelled + case <-time.After(offsetDiff): //TODO: reuse a timer? + // do nothing + } + } + select { + case <-ctx.Done(): + return // exit if context is cancelled + case ch <- step: // send the step + } + } + }() + return ch } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (varc VariableArrivalRateConfig) GetMaxDuration() time.Duration { - var maxDuration types.Duration - for _, s := range varc.Stages { - maxDuration += s.Duration.Duration +// Run executes a specific number of iterations with each confugured VU. +func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := varr.executorState.Options.ExecutionSegment + gracefulStop := varr.config.GetGracefulStop() + duration := sumStagesDuration(varr.config.Stages) + preAllocatedVUs := varr.config.GetPreAllocatedVUs(segment) + maxVUs := varr.config.GetMaxVUs(segment) + + timeUnit := time.Duration(varr.config.TimeUnit.Duration) + startArrivalRate := getScaledArrivalRate(segment, varr.config.StartRate.Int64, timeUnit) + + maxUnscaledRate := getStagesUnscaledMaxTarget(varr.config.StartRate.Int64, varr.config.Stages) + maxArrivalRatePerSec, _ := getArrivalRatePerSec(getScaledArrivalRate(segment, maxUnscaledRate, timeUnit)).Float64() + startTickerPeriod := getTickerPeriod(startArrivalRate) + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + ticker := &time.Ticker{} + if startTickerPeriod.Valid { + ticker = time.NewTicker(time.Duration(startTickerPeriod.Duration)) } - if !varc.Interruptible.Bool { - maxDuration += varc.IterationTimeout.Duration + + // Make sure the log and the progress bar have accurate information + varr.logger.WithFields(logrus.Fields{ + "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, "numStages": len(varr.config.Stages), + "startTickerPeriod": startTickerPeriod.Duration, "type": varr.config.GetType(), + }).Debug("Starting scheduler run...") + + // Pre-allocate VUs, but reserve space in the buffer for up to MaxVUs + vus := make(chan lib.VU, maxVUs) + for i := int64(0); i < preAllocatedVUs; i++ { + vu, err := varr.executorState.GetPlannedVU(ctx, varr.logger) + if err != nil { + return err + } + vus <- vu + } + + initialisedVUs := new(uint64) + *initialisedVUs = uint64(preAllocatedVUs) + + tickerPeriod := new(int64) + *tickerPeriod = int64(startTickerPeriod.Duration) + + fmtStr := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 2) + " iters/s, " + + pb.GetFixedLengthIntFormat(maxVUs) + " out of " + pb.GetFixedLengthIntFormat(maxVUs) + " VUs active" + progresFn := func() (float64, string) { + currentInitialisedVUs := atomic.LoadUint64(initialisedVUs) + currentTickerPeriod := atomic.LoadInt64(tickerPeriod) + vusInBuffer := uint64(len(vus)) + + itersPerSec := 0.0 + if currentTickerPeriod > 0 { + itersPerSec = float64(time.Second) / float64(currentTickerPeriod) + } + return math.Min(1, float64(time.Since(startTime))/float64(duration)), fmt.Sprintf(fmtStr, + itersPerSec, currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, + ) + } + varr.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) + + regDurationDone := regDurationCtx.Done() + runIterationBasic := getIterationRunner(varr.executorState, varr.logger, out) + runIteration := func(vu lib.VU) { + runIterationBasic(maxDurationCtx, vu) + vus <- vu + } + + remainingUnplannedVUs := maxVUs - preAllocatedVUs + // Make sure we put back planned and unplanned VUs back in the global + // buffer, and as an extra incentive, this replaces a waitgroup. + defer func() { + unplannedVUs := maxVUs - remainingUnplannedVUs + for i := int64(0); i < unplannedVUs; i++ { + varr.executorState.ReturnVU(<-vus) + } + }() + + rateChangesStream := varr.streamRateChanges(maxDurationCtx, startTime) + + for { + select { + case rateChange := <-rateChangesStream: + newPeriod := rateChange.tickerPeriod + ticker.Stop() + if newPeriod.Valid { + ticker = time.NewTicker(time.Duration(newPeriod.Duration)) + } + atomic.StoreInt64(tickerPeriod, int64(newPeriod.Duration)) + case <-ticker.C: + select { + case vu := <-vus: + // ideally, we get the VU from the buffer without any issues + go runIteration(vu) + default: + if remainingUnplannedVUs == 0 { + //TODO: emit an error metric? + varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) + break + } + remainingUnplannedVUs-- + vu, err := varr.executorState.GetUnplannedVU(maxDurationCtx, varr.logger) + if err != nil { + remainingUnplannedVUs++ + return err + } + atomic.AddUint64(initialisedVUs, 1) + go runIteration(vu) + } + case <-regDurationDone: + return nil + } } - return time.Duration(maxDuration) } diff --git a/lib/scheduler/variable_looping_vus.go b/lib/scheduler/variable_looping_vus.go index f1ab3de1769..4207d8018ef 100644 --- a/lib/scheduler/variable_looping_vus.go +++ b/lib/scheduler/variable_looping_vus.go @@ -21,43 +21,83 @@ package scheduler import ( + "context" "fmt" + "math" + "sync" + "sync/atomic" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) const variableLoopingVUsType = "variable-looping-vus" +// How often we can make VU adjustments when processing stages +// TODO: make configurable, in some bounds? +const minIntervalBetweenVUAdjustments = 100 * time.Millisecond + func init() { - RegisterConfigType(variableLoopingVUsType, func(name string, rawJSON []byte) (Config, error) { - config := NewVariableLoopingVUsConfig(name) - err := strictJSONUnmarshal(rawJSON, &config) - return config, err - }) + lib.RegisterSchedulerConfigType( + variableLoopingVUsType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := NewVariableLoopingVUsConfig(name) + err := lib.StrictJSONUnmarshal(rawJSON, &config) + return config, err + }, + ) } // Stage contains type Stage struct { Duration types.NullDuration `json:"duration"` Target null.Int `json:"target"` // TODO: maybe rename this to endVUs? something else? + //TODO: add a progression function? } // VariableLoopingVUsConfig stores the configuration for the stages scheduler type VariableLoopingVUsConfig struct { BaseConfig - StartVUs null.Int `json:"startVUs"` - Stages []Stage `json:"stages"` + StartVUs null.Int `json:"startVUs"` + Stages []Stage `json:"stages"` + GracefulRampDown types.NullDuration `json:"gracefulRampDown"` } // NewVariableLoopingVUsConfig returns a VariableLoopingVUsConfig with its default values func NewVariableLoopingVUsConfig(name string) VariableLoopingVUsConfig { - return VariableLoopingVUsConfig{BaseConfig: NewBaseConfig(name, variableLoopingVUsType, false)} + return VariableLoopingVUsConfig{ + BaseConfig: NewBaseConfig(name, variableLoopingVUsType), + StartVUs: null.NewInt(1, false), + GracefulRampDown: types.NewNullDuration(30*time.Second, false), + } +} + +// Make sure we implement the lib.SchedulerConfig interface +var _ lib.SchedulerConfig = &VariableLoopingVUsConfig{} + +// GetStartVUs is just a helper method that returns the scaled starting VUs. +func (vlvc VariableLoopingVUsConfig) GetStartVUs(es *lib.ExecutionSegment) int64 { + return es.Scale(vlvc.StartVUs.Int64) } -// Make sure we implement the Config interface -var _ Config = &VariableLoopingVUsConfig{} +// GetGracefulRampDown is just a helper method that returns the graceful +// ramp-down period as a standard Go time.Duration value... +func (vlvc VariableLoopingVUsConfig) GetGracefulRampDown() time.Duration { + return time.Duration(vlvc.GracefulRampDown.Duration) +} + +// GetDescription returns a human-readable description of the scheduler options +func (vlvc VariableLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { + maxVUs := es.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) + return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", + maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), + vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) +} // Validate makes sure all options are configured and valid func (vlvc VariableLoopingVUsConfig) Validate() []error { @@ -69,26 +109,505 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { return append(errors, validateStages(vlvc.Stages)...) } -// GetMaxVUs returns the absolute maximum number of possible concurrently running VUs -func (vlvc VariableLoopingVUsConfig) GetMaxVUs() int64 { - maxVUs := vlvc.StartVUs.Int64 - for _, s := range vlvc.Stages { - if s.Target.Int64 > maxVUs { - maxVUs = s.Target.Int64 +// getRawExecutionSteps calculates and returns as execution steps the number of +// actively running VUs the scheduler should have at every moment. +// +// It doesn't take into account graceful ramp-downs. It also doesn't deal with +// the end-of-scheduler drop to 0 VUs, whether graceful or not. These are +// handled by GetExecutionRequirements(), which internally uses this method and +// reserveVUsForGracefulRampDowns(). +// +// The zeroEnd argument tells the method if we should artificially add a step +// with 0 VUs at offset sum(stages.duration), i.e. when the scheduler is +// supposed to end. +// +// It's also important to note how scaling works. Say, we ramp up from 0 to 10 +// VUs over 10 seconds and then back to 0, and we want to split the execution in +// 2 equal segments (i.e. execution segments "0:0.5" and "0.5:1"). The original +// execution steps would look something like this: +// +// VUs ^ +// 10| * +// 9| *** +// 8| ***** +// 7| ******* +// 6| ********* +// 5| *********** +// 4| ************* +// 3| *************** +// 2| ***************** +// 1| ******************* +// 0------------------------> time(s) +// 01234567890123456789012 (t%10) +// 00000000001111111111222 (t/10) +// +// The chart for one of the execution segments would look like this: +// +// VUs ^ +// 5| XXX +// 4| XXXXXXX +// 3| XXXXXXXXXXX +// 2| XXXXXXXXXXXXXXX +// 1| XXXXXXXXXXXXXXXXXXX +// 0------------------------> time(s) +// 01234567890123456789012 (t%10) +// 00000000001111111111222 (t/10) +// +// And the chart for the other execution segment would look like this: +// +// VUs ^ +// 5| y +// 4| YYYYY +// 3| YYYYYYYYY +// 2| YYYYYYYYYYYYY +// 1| YYYYYYYYYYYYYYYYY +// 0------------------------> time(s) +// 01234567890123456789012 (t%10) +// 00000000001111111111222 (t/10) +// +// Notice the time offsets and the slower ramping up and down. All of that is +// because the sum of the two execution segments has to produce exactly the +// original shape, as if the test ran on a single machine: +// +// VUs ^ +// 10| Y +// 9| YYY +// 8| YYYYY +// 7| YYYYYYY +// 6| YYYYYYYYY +// 5| YYYYXXXYYYY +// 4| YYYXXXXXXXYYY +// 3| YYXXXXXXXXXXXYY +// 2| YXXXXXXXXXXXXXXXY +// 1| XXXXXXXXXXXXXXXXXXX +// 0------------------------> time(s) +// 01234567890123456789012 (t%10) +// 00000000001111111111222 (t/10) +// +// More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 +func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegment, zeroEnd bool) []lib.ExecutionStep { + // For accurate results, calculations are done with the unscaled values, and + // the values are scaled only before we add them to the steps result slice + fromVUs := vlvc.StartVUs.Int64 + + abs := func(n int64) int64 { // sigh... + if n < 0 { + return -n + } + return n + } + + // Reserve the scaled StartVUs at the beginning + prevScaledVUs := es.Scale(vlvc.StartVUs.Int64) + steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} + timeFromStart := time.Duration(0) + totalDuration := time.Duration(0) + + for _, stage := range vlvc.Stages { + stageEndVUs := stage.Target.Int64 + stageDuration := time.Duration(stage.Duration.Duration) + totalDuration += stageDuration + + stageVUAbsDiff := abs(stageEndVUs - fromVUs) + if stageVUAbsDiff == 0 { + // We don't have to do anything but update the time offset + // if the number of VUs wasn't changed in this stage + timeFromStart += stageDuration + continue + } + + // Handle 0-duration stages, i.e. instant VU jumps + if stageDuration == 0 { + fromVUs = stageEndVUs + prevScaledVUs = es.Scale(stageEndVUs) + steps = append(steps, lib.ExecutionStep{ + TimeOffset: timeFromStart, + PlannedVUs: uint64(prevScaledVUs), + }) + continue + } + + // For each stage, limit any VU adjustments between the previous + // number of VUs and the stage's target to happen at most once + // every minIntervalBetweenVUAdjustments. No floats or ratios, + // since nanoseconds should be good enough for anyone... :) + stepInterval := stageDuration / time.Duration(stageVUAbsDiff) + if stepInterval < minIntervalBetweenVUAdjustments { + stepInterval = minIntervalBetweenVUAdjustments + } + + // Loop through the potential steps, adding an item to the + // result only when there's a change in the number of VUs. + // + // IMPORTANT: we have to be very careful of rounding errors, + // both from the step duration and from the VUs. It's especially + // importatnt that the scaling via the execution segment should + // happen AFTER the rest of the calculations have been done and + // we've rounded the global "global" number of VUs. + for t := stepInterval; ; t += stepInterval { // Skip step the first step, since we've already added that + if time.Duration(abs(int64(stageDuration-t))) < minIntervalBetweenVUAdjustments { + // Skip the last step of the stage, add it below to correct any minor clock skew + break + } + stepGlobalVUs := fromVUs + int64( + math.Round((float64(t)*float64(stageEndVUs-fromVUs))/float64(stageDuration)), + ) + stepScaledVus := es.Scale(stepGlobalVUs) + + if stepScaledVus == prevScaledVUs { + // only add steps when there's a change in the number of VUs + continue + } + + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + + steps = append(steps, lib.ExecutionStep{ + TimeOffset: timeFromStart + t, + PlannedVUs: uint64(stepScaledVus), + }) + prevScaledVUs = stepScaledVus + } + + fromVUs = stageEndVUs + prevScaledVUs = es.Scale(stageEndVUs) + timeFromStart += stageDuration + steps = append(steps, lib.ExecutionStep{ + TimeOffset: timeFromStart, + PlannedVUs: uint64(prevScaledVUs), + }) + } + + if zeroEnd && steps[len(steps)-1].PlannedVUs != 0 { + // If the last PlannedVUs value wasn't 0, add a last step with 0 + steps = append(steps, lib.ExecutionStep{TimeOffset: totalDuration, PlannedVUs: 0}) + } + return steps +} + +// If the graceful ramp-downs are enabled, we need to reserve any VUs that may +// potentially have to finish running iterations when we're scaling their number +// down. This would prevent attempts from other schedulers to use them while the +// iterations are finishing up during their allotted gracefulRampDown periods. +// +// But we also need to be careful to not over-allocate more VUs than we actually +// need. We should never have more PlannedVUs than the max(startVUs, +// stage[n].target), even if we're quickly scaling VUs up and down multiple +// times, one after the other. In those cases, any previously reserved VUs +// finishing up interrupted iterations should be reused by the scheduler, +// instead of new ones being requested from the executor. +// +// Here's an example with graceful ramp-town (i.e. "uninterruptible" +// iterations), where stars represent actively scheduled VUs and dots are used +// for VUs that are potentially finishing up iterations: +// +// +// ^ +// | +// VUs 6| *.............................. +// 5| ***.......*.............................. +// 4|*****.....***.....**.............................. +// 3|******...*****...***.............................. +// 2|*******.*******.****.............................. +// 1|***********************.............................. +// 0--------------------------------------------------------> time(s) +// 012345678901234567890123456789012345678901234567890123 (t%10) +// 000000000011111111112222222222333333333344444444445555 (t/10) +// +// We start with 4 VUs, scale to 6, scale down to 1, scale up to 5, scale down +// to 1 again, scale up to 4, back to 1, and finally back down to 0. If our +// gracefulStop timeout was 30s (the default), then we'll stay with 6 PlannedVUs +// until t=32 in the test above, and the actual scheduler could run until t=52. +// See TestVariableLoopingVUsConfigExecutionPlanExample() for the above example +// as a unit test. +// +// The algorithm we use below to reserve VUs so that ramping-down VUs can finish +// their last iterations is pretty simple. It just traverses the raw execution +// steps and whenever there's a scaling down of VUs, it prevents the number of +// VUs from faliing down for the configured gracefulRampDown period. +// +// Finishing up the test, i.e. making sure we have a step with 0 VUs at time +// schedulerEndOffset, is not handled here. Instead GetExecutionRequirements() +// takes care of that. But to make its job easier, this method won't add any +// steps with an offset that's greater or equal to schedulerEndOffset. +func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( + rawSteps []lib.ExecutionStep, schedulerEndOffset time.Duration) []lib.ExecutionStep { + + rawStepsLen := len(rawSteps) + gracefulRampDownPeriod := vlvc.GetGracefulRampDown() + newSteps := []lib.ExecutionStep{} + + lastPlannedVUs := uint64(0) + for rawStepNum := 0; rawStepNum < rawStepsLen; rawStepNum++ { + rawStep := rawSteps[rawStepNum] + // Add the first step or any step where the number of planned VUs us + // greater than the ones in the previous step. We don't need to worry + // about reserving time for ramping-down VUs when the number of planned + // VUs is growing. That's because the gracefulRampDown period is a fixed + // value and any timeouts from early steps with fewer VUs will get + // overshadowed by timeouts from latter steps with more VUs. + if rawStepNum == 0 || rawStep.PlannedVUs > lastPlannedVUs { + newSteps = append(newSteps, rawStep) + lastPlannedVUs = rawStep.PlannedVUs + continue + } + + // We simply skip steps with the same number of planned VUs + if rawStep.PlannedVUs == lastPlannedVUs { + continue + } + + // If we're here, we have a downward "slope" - thelastPlannedVUs are + // more than the current rawStep's planned VUs. We're going to look + // forward in time (up to gracefulRampDown) and inspect the rawSteps. + // There are a 3 possibilities: + // - We find a new step within the gracefulRampDown period which has + // the same number of VUs or greater than lastPlannedVUs. Which + // means that we can just advance rawStepNum to that number and we + // don't need to worry about any of the raw steps in the middle! + // Both their planned VUs and their gracefulRampDown periods will + // be lower than what we're going to set from that new rawStep - + // we've basically found a new upward slope or equal value again. + // - We reach schedulerEndOffset, in which case we are done - we can't + // add any new steps, since those will be after the scheduler end + // offset. + // - We reach the end of the rawSteps, or we don't find any higher or + // equal steps to prevStep in the next gracefulRampDown period. So + // we'll simply try to add an entry into newSteps with the values + // {prevStep.TimeOffset + gracefulRampDown, rawStep.PlannedVUs} and + // we'll continue with traversing the following rawSteps. + + skippedToNewRawStep := false + timeOffsetWithTimeout := rawStep.TimeOffset + gracefulRampDownPeriod + + for advStepNum := rawStepNum + 1; advStepNum < rawStepsLen; advStepNum++ { + advStep := rawSteps[advStepNum] + if advStep.TimeOffset > timeOffsetWithTimeout { + break + } + if advStep.PlannedVUs >= lastPlannedVUs { + rawStepNum = advStepNum - 1 + skippedToNewRawStep = true + break + } + } + + // Nothing more to do here, found a new "slope" with equal or grater + // PlannedVUs in the gracefulRampDownPeriod window, so we go to it. + if skippedToNewRawStep { + continue + } + + // We've reached the absolute scheduler end offset, and we were already + // on a downward "slope" (i.e. the previous planned VUs are more than + // the current planned VUs), so nothing more we can do here. + if timeOffsetWithTimeout >= schedulerEndOffset { + break } + + newSteps = append(newSteps, lib.ExecutionStep{ + TimeOffset: timeOffsetWithTimeout, + PlannedVUs: rawStep.PlannedVUs, + }) + lastPlannedVUs = rawStep.PlannedVUs + } + + return newSteps +} + +// GetExecutionRequirements very dynamically reserves exactly the number of +// required VUs for this scheduler at every moment of the test. +// +// If gracefulRampDown is specified, it will also be taken into account, and the +// number of needed VUs to handle that will also be reserved. See the +// documentation of reserveGracefulVUScalingDown() for more details. +// +// On the other hand, gracefulStop is handled here. To facilitate it, we'll +// ensure that the last execution step will have 0 VUs and will be at time +// offset (sum(stages.Duration)+gracefulStop). Any steps that would've been +// added after it will be ignored. Thus: +// - gracefulStop can be less than gracefulRampDown and can cut the graceful +// ramp-down periods of the last VUs short. +// - gracefulRampDown can be more than gracefulRampDown: +// - If the user manually ramped down VUs at the end of the test (i.e. the +// last stage's target is 0), then this will have no effect. +// - If the last stage's target is more than 0, the VUs at the end of the +// scheduler's life will have more time to finish their last iterations. +func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { + steps := vlvc.getRawExecutionSteps(es, false) + + schedulerEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) + // Handle graceful ramp-downs, if we have them + if vlvc.GracefulRampDown.Duration > 0 { + steps = vlvc.reserveVUsForGracefulRampDowns(steps, schedulerEndOffset) } - return maxVUs + + // If the last PlannedVUs value wasn't 0, add a last step with 0 + if steps[len(steps)-1].PlannedVUs != 0 { + steps = append(steps, lib.ExecutionStep{TimeOffset: schedulerEndOffset, PlannedVUs: 0}) + } + + return steps +} + +// NewScheduler creates a new VariableLoopingVUs scheduler +func (vlvc VariableLoopingVUsConfig) NewScheduler(es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + return VariableLoopingVUs{ + BaseScheduler: NewBaseScheduler(vlvc, es, logger), + config: vlvc, + }, nil } -// GetMaxDuration returns the maximum duration time for this scheduler, including -// the specified iterationTimeout, if the iterations are uninterruptible -func (vlvc VariableLoopingVUsConfig) GetMaxDuration() time.Duration { - var maxDuration types.Duration - for _, s := range vlvc.Stages { - maxDuration += s.Duration.Duration +// VariableLoopingVUs handles the old "stages" execution configuration - it +// loops iterations with a variable number of VUs for the sum of all of the +// specified stages' duration. +type VariableLoopingVUs struct { + *BaseScheduler + config VariableLoopingVUsConfig +} + +// Make sure we implement the lib.Scheduler interface. +var _ lib.Scheduler = &VariableLoopingVUs{} + +// Run constantly loops through as many iterations as possible on a variable +// number of VUs for the specified stages. +// +// TODO: split up? since this does a ton of things, unfortunately I can't think +// of a less complex way to implement it (besides the old "increment by 100ms +// and see what happens)... :/ so maybe see how it can be spit? +func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + segment := vlv.executorState.Options.ExecutionSegment + rawExecutionSteps := vlv.config.getRawExecutionSteps(segment, true) + regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) + if !isFinal { + return fmt.Errorf("%s expected raw end offset at %s to be final", vlv.config.GetName(), regularDuration) + } + + gracefulExecutionSteps := vlv.config.GetExecutionRequirements(segment) + maxDuration, isFinal := lib.GetEndOffset(gracefulExecutionSteps) + if !isFinal { + return fmt.Errorf("%s expected graceful end offset at %s to be final", vlv.config.GetName(), maxDuration) + } + maxVUs := lib.GetMaxPlannedVUs(gracefulExecutionSteps) + gracefulStop := maxDuration - regularDuration + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, regularDuration, gracefulStop) + defer cancel() + + // Make sure the log and the progress bar have accurate information + vlv.logger.WithFields(logrus.Fields{ + "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(segment), "maxVUs": maxVUs, + "duration": regularDuration, "numStages": len(vlv.config.Stages)}, + ).Debug("Starting scheduler run...") + + activeVUs := new(int64) + vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) + progresFn := func() (float64, string) { + spent := time.Since(startTime) + if spent > regularDuration { + return 1, fmt.Sprintf("variable looping VUs for %s", regularDuration) + } + currentlyActiveVUs := atomic.LoadInt64(activeVUs) + return float64(spent) / float64(regularDuration), fmt.Sprintf( + "currently "+vusFmt+" active looping VUs, %s/%s", currentlyActiveVUs, + pb.GetFixedLengthDuration(spent, regularDuration), regularDuration, + ) + } + vlv.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, vlv, progresFn) + + // Actually schedule the VUs and iterations, likely the most complicated + // scheduler among all of them... + wg := &sync.WaitGroup{} + + runIteration := getIterationRunner(vlv.executorState, vlv.logger, out) + getVU := func() (lib.VU, error) { + vu, err := vlv.executorState.GetPlannedVU(maxDurationCtx, vlv.logger) + if err != nil { + cancel() + } else { + wg.Add(1) + atomic.AddInt64(activeVUs, 1) + } + return vu, err + } + returnVU := func(vu lib.VU) { + vlv.executorState.ReturnVU(vu) + atomic.AddInt64(activeVUs, -1) + wg.Done() } - if !vlvc.Interruptible.Bool { - maxDuration += vlvc.IterationTimeout.Duration + + vuHandles := make([]*vuHandle, maxVUs) + for i := uint64(0); i < maxVUs; i++ { + vuHandle := newStoppedVUHandle(maxDurationCtx, getVU, returnVU, vlv.logger.WithField("vuNum", i)) + go vuHandle.runLoopsIfPossible(runIteration) + vuHandles[i] = vuHandle + } + + rawStepEvents := lib.StreamExecutionSteps(ctx, startTime, rawExecutionSteps, true) + gracefulLimitEvents := lib.StreamExecutionSteps(ctx, startTime, gracefulExecutionSteps, false) + + // 0 <= currentScheduledVUs <= currentMaxAllowedVUs <= maxVUs + var currentScheduledVUs, currentMaxAllowedVUs uint64 + + handleNewScheduledVUs := func(newScheduledVUs uint64) { + if newScheduledVUs > currentScheduledVUs { + for vuNum := currentScheduledVUs; vuNum < newScheduledVUs; vuNum++ { + vuHandles[vuNum].start() + } + } else { + for vuNum := newScheduledVUs; vuNum < currentScheduledVUs; vuNum++ { + vuHandles[vuNum].gracefulStop() + } + } + currentScheduledVUs = newScheduledVUs } - return time.Duration(maxDuration) + + handleNewMaxAllowedVUs := func(newMaxAllowedVUs uint64) { + if newMaxAllowedVUs < currentMaxAllowedVUs { + for vuNum := newMaxAllowedVUs; vuNum < currentMaxAllowedVUs; vuNum++ { + vuHandles[vuNum].hardStop() + } + } + currentMaxAllowedVUs = newMaxAllowedVUs + } + + handleAllRawSteps := func() bool { + for { + select { + case step, ok := <-rawStepEvents: + if !ok { + return true + } + handleNewScheduledVUs(step.PlannedVUs) + case step := <-gracefulLimitEvents: + if step.PlannedVUs > currentScheduledVUs { + // Handle the case where a value is read from the + // gracefulLimitEvents channel before rawStepEvents + handleNewScheduledVUs(step.PlannedVUs) + } + handleNewMaxAllowedVUs(step.PlannedVUs) + case <-ctx.Done(): + return false + } + } + } + + if handleAllRawSteps() { + // Handle any remaining graceful stops + go func() { + for { + select { + case step := <-gracefulLimitEvents: + handleNewMaxAllowedVUs(step.PlannedVUs) + case <-maxDurationCtx.Done(): + return + } + } + }() + } + + wg.Wait() + + return nil } diff --git a/lib/scheduler/vu_handle.go b/lib/scheduler/vu_handle.go new file mode 100644 index 00000000000..97f0a77eefe --- /dev/null +++ b/lib/scheduler/vu_handle.go @@ -0,0 +1,167 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package scheduler + +import ( + "context" + "sync" + + "github.com/sirupsen/logrus" + + "github.com/loadimpact/k6/lib" +) + +// This is a helper type used in schedulers where we have to dynamically control +// the number of VUs that are simultaneously running. For the moment, it is used in the VariableLoopingVUs and +// +// TODO: something simpler? +type vuHandle struct { + mutex *sync.RWMutex + parentCtx context.Context + getVU func() (lib.VU, error) + returnVU func(lib.VU) + + canStartIter chan struct{} + + ctx context.Context + cancel func() + logger *logrus.Entry +} + +func newStoppedVUHandle( + parentCtx context.Context, getVU func() (lib.VU, error), returnVU func(lib.VU), logger *logrus.Entry, +) *vuHandle { + lock := &sync.RWMutex{} + ctx, cancel := context.WithCancel(parentCtx) + return &vuHandle{ + mutex: lock, + parentCtx: parentCtx, + getVU: getVU, + returnVU: returnVU, + + canStartIter: make(chan struct{}), + + ctx: ctx, + cancel: cancel, + logger: logger, + } +} + +func (vh *vuHandle) start() { + vh.mutex.Lock() + vh.logger.Debugf("Start") + close(vh.canStartIter) + vh.mutex.Unlock() +} + +func (vh *vuHandle) gracefulStop() { + vh.mutex.Lock() + select { + case <-vh.canStartIter: + vh.canStartIter = make(chan struct{}) + vh.logger.Debugf("Graceful stop") + default: + // do nothing, the signalling channel was already closed by either + // hardStop() or gracefulStop() + } + vh.mutex.Unlock() +} + +func (vh *vuHandle) hardStop() { + vh.mutex.Lock() + vh.logger.Debugf("Hard stop") + vh.cancel() // cancel the previous context + vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) // create a new context + select { // if needed, + case <-vh.canStartIter: + vh.canStartIter = make(chan struct{}) + default: + // do nothing, the signalling channel was already closed by either + // hardStop() or gracefulStop() + } + vh.mutex.Unlock() +} + +//TODO: simplify this somehow - I feel like there should be a better way to +//implement this logic... maybe with sync.Cond? +func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.VU)) { + schedulerDone := vh.parentCtx.Done() + + var vu lib.VU + defer func() { + if vu != nil { + vh.returnVU(vu) + vu = nil + } + }() + +mainLoop: + for { + vh.mutex.RLock() + canStartIter, ctx := vh.canStartIter, vh.ctx + vh.mutex.RUnlock() + + // Wait for either the scheduler to be done, or for us to be unpaused + select { + case <-canStartIter: + // Best case, we're currently running, so we do nothing here, we + // just continue straight ahead. + case <-schedulerDone: + return // The whole scheduler is done, nothing more to do. + default: + // We're not running, but the scheduler isn't done yet, so we wait + // for either one of those conditions. But before that, we'll return + // our VU to the pool, if we have it. + if vu != nil { + vh.returnVU(vu) + vu = nil + } + select { + case <-canStartIter: + // continue on, we were unblocked... + case <-ctx.Done(): + // hardStop was called, start a fresh iteration to get the new + // context and signal channel + continue mainLoop + case <-schedulerDone: + return // The whole scheduler is done, nothing more to do. + } + } + + // Probably not needed, but just in case - if both running and + // schedulerDone were actice, check that the scheduler isn't done. + select { + case <-schedulerDone: + return + default: + } + + if vu == nil { // Ensure we have a VU + freshVU, err := vh.getVU() + if err != nil { + return + } + vu = freshVU + } + + runIter(ctx, vu) + } +} From d9dbfa2062282535ffc30e59671485d7d28dddfe Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:29:19 +0300 Subject: [PATCH 006/350] Update the cloud collector --- cmd/collectors.go | 7 +++-- stats/cloud/collector.go | 49 ++++++++++++++--------------------- stats/cloud/collector_test.go | 4 +-- 3 files changed, 26 insertions(+), 34 deletions(-) diff --git a/cmd/collectors.go b/cmd/collectors.go index aa395b672e6..5d0f2d58678 100644 --- a/cmd/collectors.go +++ b/cmd/collectors.go @@ -60,7 +60,10 @@ func parseCollector(s string) (t, arg string) { } } -func newCollector(collectorName, arg string, src *lib.SourceData, conf Config) (lib.Collector, error) { +func newCollector( + collectorName, arg string, src *lib.SourceData, conf Config, executionPlan []lib.ExecutionStep, +) (lib.Collector, error) { + getCollector := func() (lib.Collector, error) { switch collectorName { case collectorJSON: @@ -84,7 +87,7 @@ func newCollector(collectorName, arg string, src *lib.SourceData, conf Config) ( if arg != "" { config.Name = null.StringFrom(arg) } - return cloud.New(config, src, conf.Options, Version) + return cloud.New(config, src, conf.Options, executionPlan, Version) case collectorKafka: config := kafka.NewConfig().Apply(conf.Collectors.Kafka) if err := envconfig.Process("k6", &config); err != nil { diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index d5108a11f40..307ea8473fb 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -47,9 +47,10 @@ type Collector struct { config Config referenceID string - duration int64 - thresholds map[string][]*stats.Threshold - client *Client + executionPlan []lib.ExecutionStep + duration int64 // in seconds + thresholds map[string][]*stats.Threshold + client *Client anonymous bool runStatus lib.RunStatus @@ -97,7 +98,9 @@ func MergeFromExternal(external map[string]json.RawMessage, conf *Config) error } // New creates a new cloud collector -func New(conf Config, src *lib.SourceData, opts lib.Options, version string) (*Collector, error) { +func New( + conf Config, src *lib.SourceData, opts lib.Options, executionPlan []lib.ExecutionStep, version string, +) (*Collector, error) { if err := MergeFromExternal(opts.External, &conf); err != nil { return nil, err } @@ -118,15 +121,8 @@ func New(conf Config, src *lib.SourceData, opts lib.Options, version string) (*C thresholds[name] = append(thresholds[name], t.Thresholds...) } - // Sum test duration from options. -1 for unknown duration. - var duration int64 = -1 - if len(opts.Stages) > 0 { - duration = sumStages(opts.Stages) - } else if opts.Duration.Valid { - duration = int64(time.Duration(opts.Duration.Duration).Seconds()) - } - - if duration == -1 { + duration, testEnds := lib.GetEndOffset(executionPlan) + if !testEnds { return nil, errors.New("Tests with unspecified duration are not allowed when using Load Impact Insights") } @@ -136,13 +132,14 @@ func New(conf Config, src *lib.SourceData, opts lib.Options, version string) (*C } return &Collector{ - config: conf, - thresholds: thresholds, - client: NewClient(conf.Token.String, conf.Host.String, version), - anonymous: !conf.Token.Valid, - duration: duration, - opts: opts, - aggrBuckets: map[int64]aggregationBucket{}, + config: conf, + thresholds: thresholds, + client: NewClient(conf.Token.String, conf.Host.String, version), + anonymous: !conf.Token.Valid, + executionPlan: executionPlan, + duration: int64(duration / time.Second), + opts: opts, + aggrBuckets: map[int64]aggregationBucket{}, }, nil } @@ -156,11 +153,12 @@ func (c *Collector) Init() error { thresholds[name] = append(thresholds[name], threshold.Source) } } + maxVUs := lib.GetMaxPossibleVUs(c.executionPlan) testRun := &TestRun{ Name: c.config.Name.String, ProjectID: c.config.ProjectID.Int64, - VUsMax: c.opts.VUsMax.Int64, + VUsMax: int64(maxVUs), Thresholds: thresholds, Duration: c.duration, } @@ -505,15 +503,6 @@ func (c *Collector) testFinished() { } } -func sumStages(stages []lib.Stage) int64 { - var total time.Duration - for _, stage := range stages { - total += time.Duration(stage.Duration.Duration) - } - - return int64(total.Seconds()) -} - // GetRequiredSystemTags returns which sample tags are needed by this collector func (c *Collector) GetRequiredSystemTags() lib.TagSet { return lib.GetTagSet("name", "method", "status", "error", "check", "group") diff --git a/stats/cloud/collector_test.go b/stats/cloud/collector_test.go index ccdec01b936..cdbd8eb2c49 100644 --- a/stats/cloud/collector_test.go +++ b/stats/cloud/collector_test.go @@ -147,7 +147,7 @@ func TestCloudCollector(t *testing.T) { Host: null.StringFrom(tb.ServerHTTP.URL), NoCompress: null.BoolFrom(true), }) - collector, err := New(config, script, options, "1.0") + collector, err := New(config, script, options, []lib.ExecutionStep{}, "1.0") require.NoError(t, err) assert.True(t, collector.config.Host.Valid) @@ -293,7 +293,7 @@ func TestCloudCollectorMaxPerPacket(t *testing.T) { Host: null.StringFrom(tb.ServerHTTP.URL), NoCompress: null.BoolFrom(true), }) - collector, err := New(config, script, options, "1.0") + collector, err := New(config, script, options, []lib.ExecutionStep{}, "1.0") require.NoError(t, err) now := time.Now() tags := stats.IntoSampleTags(&map[string]string{"test": "mest", "a": "b"}) From a9cc1309e4fc8317ab105dd474d56740b1f0a197 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:41:26 +0300 Subject: [PATCH 007/350] Add the new progressbars and fixed-length formatters This doesn't include all of the changes in run/cmd.go, since those are quite intertwined with a lot of others. --- cmd/cloud.go | 37 ++++++--- cmd/common.go | 19 ----- cmd/root.go | 18 +++- cmd/ui.go | 155 ++++++++++++++++++++++++++++++++++ lib/types/types.go | 37 ++++++++- lib/types/types_test.go | 55 +++++++++++++ ui/pb/helpers.go | 131 +++++++++++++++++++++++++++++ ui/pb/helpers_test.go | 169 ++++++++++++++++++++++++++++++++++++++ ui/pb/progressbar.go | 142 ++++++++++++++++++++++++++++++++ ui/pb/progressbar_test.go | 23 ++++++ ui/progress_bar.go | 68 --------------- 11 files changed, 750 insertions(+), 104 deletions(-) create mode 100644 cmd/ui.go create mode 100644 ui/pb/helpers.go create mode 100644 ui/pb/helpers_test.go create mode 100644 ui/pb/progressbar.go create mode 100644 ui/pb/progressbar_test.go delete mode 100644 ui/progress_bar.go diff --git a/cmd/cloud.go b/cmd/cloud.go index 89f71aaf0a6..80db546618e 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -23,6 +23,7 @@ package cmd import ( "bytes" "encoding/json" + "fmt" "os" "os/signal" "path/filepath" @@ -33,6 +34,7 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats/cloud" "github.com/loadimpact/k6/ui" + "github.com/loadimpact/k6/ui/pb" "github.com/pkg/errors" "github.com/spf13/afero" "github.com/spf13/cobra" @@ -57,11 +59,9 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud RunE: func(cmd *cobra.Command, args []string) error { //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", Banner) - initBar := ui.ProgressBar{ - Width: 60, - Left: func() string { return " uploading script" }, - } - fprintf(stdout, "%s \r", initBar.String()) + + progressBar := pb.New(pb.WithConstLeft(" Init")) + printBar(progressBar, "Parsing script") // Runner pwd, err := os.Getwd() @@ -81,11 +81,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } + printBar(progressBar, "Getting script options") r, err := newRunner(src, runType, fs, runtimeOptions) if err != nil { return err } + printBar(progressBar, "Consolidating options") cliOpts, err := getOptions(cmd.Flags()) if err != nil { return err @@ -113,6 +115,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return errors.New("Not logged in, please use `k6 login cloud`.") } + printBar(progressBar, "Building the archive") arc := r.MakeArchive() // TODO: Fix this // We reuse cloud.Config for parsing options.ext.loadimpact, but this probably shouldn't be @@ -161,22 +164,28 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud } // Start cloud test run + printBar(progressBar, "Validating script options") client := cloud.NewClient(cloudConfig.Token.String, cloudConfig.Host.String, Version) if err := client.ValidateOptions(arc.Options); err != nil { return err } + printBar(progressBar, "Uploading archive") refID, err := client.StartCloudTestRun(name, cloudConfig.ProjectID.Int64, arc) if err != nil { return err } + progressBar.Modify(pb.WithConstLeft(" Run")) + printBar(progressBar, "Initializing the cloud test") testURL := cloud.URLForResults(refID, cloudConfig) fprintf(stdout, "\n\n") - fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprint("cloud")) + fprintf(stdout, " executor: %s\n", ui.ValueColor.Sprint("cloud")) fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) + //TODO: print schedulers information fprintf(stdout, "\n") + printBar(progressBar, "Initializing the cloud test") // The quiet option hides the progress bar and disallow aborting the test if quiet { @@ -190,12 +199,15 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud var progressErr error testProgress := &cloud.TestProgressResponse{} - progress := ui.ProgressBar{ - Width: 60, - Left: func() string { - return " " + testProgress.RunStatusText - }, + percentageFmt := "[" + pb.GetFixedLengthFloatFormat(100, 2) + "%%] %s" + progressBar.Modify( + pb.WithProgress(func() (float64, string) { + if testProgress.RunStatus < lib.RunStatusRunning { + return 0, testProgress.RunStatusText } + return testProgress.Progress, fmt.Sprintf(percentageFmt, testProgress.Progress*100, testProgress.RunStatusText) + }), + ) ticker := time.NewTicker(time.Millisecond * 2000) shouldExitLoop := false @@ -209,8 +221,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } - progress.Progress = testProgress.Progress - fprintf(stdout, "%s\x1b[0K\r", progress.String()) + printBar(progressBar, "") } else { log.WithError(progressErr).Error("Test progress error") } diff --git a/cmd/common.go b/cmd/common.go index d3964bce45f..2c833ca93c9 100644 --- a/cmd/common.go +++ b/cmd/common.go @@ -21,11 +21,9 @@ package cmd import ( - "bytes" "fmt" "io" "os" - "sync" "github.com/loadimpact/k6/lib/types" "github.com/spf13/afero" @@ -51,23 +49,6 @@ type ExitCode struct { Code int } -// A writer that syncs writes with a mutex and, if the output is a TTY, clears before newlines. -type consoleWriter struct { - Writer io.Writer - IsTTY bool - Mutex *sync.Mutex -} - -func (w consoleWriter) Write(p []byte) (n int, err error) { - if w.IsTTY { - p = bytes.Replace(p, []byte{'\n'}, []byte{'\x1b', '[', '0', 'K', '\n'}, -1) - } - w.Mutex.Lock() - n, err = w.Writer.Write(p) - w.Mutex.Unlock() - return -} - //TODO: refactor the CLI config so these functions aren't needed - they // can mask errors by failing only at runtime, not at compile time func getNullBool(flags *pflag.FlagSet, key string) null.Bool { diff --git a/cmd/root.go b/cmd/root.go index 2ee71c81156..54091676863 100644 --- a/cmd/root.go +++ b/cmd/root.go @@ -54,12 +54,14 @@ var Banner = strings.Join([]string{ }, "\n") var BannerColor = color.New(color.FgCyan) +//TODO: remove these global variables +//nolint:gochecknoglobals var ( outMutex = &sync.Mutex{} stdoutTTY = isatty.IsTerminal(os.Stdout.Fd()) || isatty.IsCygwinTerminal(os.Stdout.Fd()) stderrTTY = isatty.IsTerminal(os.Stderr.Fd()) || isatty.IsCygwinTerminal(os.Stderr.Fd()) - stdout = consoleWriter{colorable.NewColorableStdout(), stdoutTTY, outMutex} - stderr = consoleWriter{colorable.NewColorableStderr(), stderrTTY, outMutex} + stdout = &consoleWriter{colorable.NewColorableStdout(), stdoutTTY, outMutex, nil} + stderr = &consoleWriter{colorable.NewColorableStderr(), stderrTTY, outMutex, nil} ) const defaultConfigFileName = "config.json" @@ -89,6 +91,18 @@ var RootCmd = &cobra.Command{ PersistentPreRun: func(cmd *cobra.Command, args []string) { setupLoggers(logFmt) if noColor { + // TODO: figure our something else... currently, with the wrappers + // below, we're stripping any colors from the output after we've + // added them. The problem is that, besides being very inefficient, + // this actually also strips other special characters from the + // intended output, like the progressbar formatting ones, which + // would otherwise be fine (in a TTY). + // + // It would be much better if we avoid messing with the output and + // instead have a parametrized instance of the color library. It + // will return colored output if colors are enabled and simply + // return the passed input as-is (i.e. be a noop) if colors are + // disabled... stdout.Writer = colorable.NewNonColorable(os.Stdout) stderr.Writer = colorable.NewNonColorable(os.Stderr) } diff --git a/cmd/ui.go b/cmd/ui.go new file mode 100644 index 00000000000..9080893c177 --- /dev/null +++ b/cmd/ui.go @@ -0,0 +1,155 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package cmd + +import ( + "bytes" + "context" + "fmt" + "io" + "strings" + "sync" + "time" + + "github.com/loadimpact/k6/core/local" + "github.com/loadimpact/k6/ui/pb" +) + +// A writer that syncs writes with a mutex and, if the output is a TTY, clears before newlines. +type consoleWriter struct { + Writer io.Writer + IsTTY bool + Mutex *sync.Mutex + + // Used for flicker-free persistent objects like the progressbars + PersistentText func() +} + +func (w *consoleWriter) Write(p []byte) (n int, err error) { + origLen := len(p) + if w.IsTTY { + //TODO: check how cross-platform this is... + p = bytes.ReplaceAll(p, []byte{'\n'}, []byte{'\x1b', '[', '0', 'K', '\n'}) + } + + w.Mutex.Lock() + n, err = w.Writer.Write(p) + if w.PersistentText != nil { + w.PersistentText() + } + w.Mutex.Unlock() + + if err != nil && n < origLen { + return n, err + } + return origLen, err +} + +func printBar(bar *pb.ProgressBar, rightText string) { + end := "\n" + if stdout.IsTTY { + //TODO: check for cross platform support + end = "\x1b[0K\r" + } + fprintf(stdout, "%s %s%s", bar.String(), rightText, end) +} + +func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { + lineEnd := "\n" + if isTTY { + //TODO: check for cross platform support + lineEnd = "\x1b[K\n" // erase till end of line + } + + pbsCount := len(pbs) + result := make([]string, pbsCount+2) + result[0] = lineEnd // start with an empty line + for i, pb := range pbs { + result[i+1] = pb.String() + lineEnd + } + if isTTY && goBack { + // Go back to the beginning + //TODO: check for cross platform support + result[pbsCount+1] = fmt.Sprintf("\r\x1b[%dA", pbsCount+1) + } else { + result[pbsCount+1] = "\n" + } + return strings.Join(result, "") +} + +//TODO: show other information here? +//TODO: add a no-progress option that will disable these +//TODO: don't use global variables... +func showProgress(ctx context.Context, wg *sync.WaitGroup, conf Config, executor *local.Executor) { + defer wg.Done() + if quiet || conf.HttpDebug.Valid && conf.HttpDebug.String != "" { + return + } + + pbs := []*pb.ProgressBar{executor.GetInitProgressBar()} + for _, s := range executor.GetSchedulers() { + pbs = append(pbs, s.GetProgress()) + } + + // For flicker-free progressbars! + progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, pbs)) + progressBarsPrint := func() { + _, _ = stdout.Writer.Write(progressBarsLastRender) + } + + //TODO: make configurable? + updateFreq := 1 * time.Second + //TODO: remove !noColor after we fix how we handle colors (see the related + //description in the TODO message in cmd/root.go) + if stdoutTTY && !noColor { + updateFreq = 100 * time.Millisecond + outMutex.Lock() + stdout.PersistentText = progressBarsPrint + stderr.PersistentText = progressBarsPrint + outMutex.Unlock() + defer func() { + outMutex.Lock() + stdout.PersistentText = nil + stderr.PersistentText = nil + if ctx.Err() != nil { + // Render a last plain-text progressbar in an error + progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, pbs)) + progressBarsPrint() + } + outMutex.Unlock() + }() + } + + ctxDone := ctx.Done() + ticker := time.NewTicker(updateFreq) + for { + select { + case <-ticker.C: + barText := renderMultipleBars(stdoutTTY, true, pbs) + outMutex.Lock() + progressBarsLastRender = []byte(barText) + progressBarsPrint() + outMutex.Unlock() + case <-ctxDone: + return + } + } +} diff --git a/lib/types/types.go b/lib/types/types.go index a6e6305837f..d59096ad16e 100644 --- a/lib/types/types.go +++ b/lib/types/types.go @@ -25,6 +25,8 @@ import ( "encoding/json" "fmt" "reflect" + "strconv" + "strings" "time" null "gopkg.in/guregu/null.v3" @@ -81,6 +83,8 @@ func NullDecoder(f reflect.Type, t reflect.Type, data interface{}) (interface{}, return data, nil } +//TODO: something better that won't reuire so much boilerplate and casts for NullDuration values... + // Duration is an alias for time.Duration that de/serialises to JSON as human-readable strings. type Duration time.Duration @@ -88,8 +92,37 @@ func (d Duration) String() string { return time.Duration(d).String() } +// ParseExtendedDuration is a helper function that allows for string duration +// values containing days. +func ParseExtendedDuration(data string) (result time.Duration, err error) { + dPos := strings.IndexByte(data, 'd') + if dPos < 0 { + return time.ParseDuration(data) + } + + var hours time.Duration + if dPos+1 < len(data) { // case "12d" + hours, err = time.ParseDuration(data[dPos+1:]) + if err != nil { + return + } + if hours < 0 { + return 0, fmt.Errorf("invalid time format '%s'", data[dPos+1:]) + } + } + + days, err := strconv.ParseInt(data[:dPos], 10, 64) + if err != nil { + return + } + if days < 0 { + hours = -hours + } + return time.Duration(days)*24*time.Hour + hours, nil +} + func (d *Duration) UnmarshalText(data []byte) error { - v, err := time.ParseDuration(string(data)) + v, err := ParseExtendedDuration(string(data)) if err != nil { return err } @@ -104,7 +137,7 @@ func (d *Duration) UnmarshalJSON(data []byte) error { return err } - v, err := time.ParseDuration(str) + v, err := ParseExtendedDuration(str) if err != nil { return err } diff --git a/lib/types/types_test.go b/lib/types/types_test.go index d98db6473f2..d7975c19646 100644 --- a/lib/types/types_test.go +++ b/lib/types/types_test.go @@ -23,6 +23,7 @@ package types import ( "encoding/json" "fmt" + "math" "testing" "time" @@ -99,6 +100,55 @@ func TestNullDecoder(t *testing.T) { } } +func TestParseExtendedDuration(t *testing.T) { + testCases := []struct { + durStr string + expErr bool + expDur time.Duration + }{ + {"", true, 0}, + {"d", true, 0}, + {"d2h", true, 0}, + {"d2h", true, 0}, + {"2.1d", true, 0}, + {"2d-2h", true, 0}, + {"-2d-2h", true, 0}, + {"2+d", true, 0}, + {"2da", true, 0}, + {"2-d", true, 0}, + {"1.12s", false, 1120 * time.Millisecond}, + {"0d1.12s", false, 1120 * time.Millisecond}, + {"10d1.12s", false, 240*time.Hour + 1120*time.Millisecond}, + {"1s", false, 1 * time.Second}, + {"1d", false, 24 * time.Hour}, + {"20d", false, 480 * time.Hour}, + {"1d23h", false, 47 * time.Hour}, + {"1d24h15m", false, 48*time.Hour + 15*time.Minute}, + {"1d25h80m", false, 50*time.Hour + 20*time.Minute}, + {"0d25h120m80s", false, 27*time.Hour + 80*time.Second}, + {"-1d2h", false, -26 * time.Hour}, + {"-1d24h", false, -48 * time.Hour}, + {"2d1ns", false, 48*time.Hour + 1}, + {"-2562047h47m16.854775807s", false, time.Duration(math.MinInt64 + 1)}, + {"-106751d23h47m16.854775807s", false, time.Duration(math.MinInt64 + 1)}, + {"2562047h47m16.854775807s", false, time.Duration(math.MaxInt64)}, + {"106751d23h47m16.854775807s", false, time.Duration(math.MaxInt64)}, + } + + for _, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("tc_%s_exp", tc.durStr), func(t *testing.T) { + result, err := ParseExtendedDuration(tc.durStr) + if tc.expErr { + assert.Error(t, err) + } else { + assert.NoError(t, err) + assert.Equal(t, tc.expDur, result) + } + }) + } +} + func TestDuration(t *testing.T) { t.Run("String", func(t *testing.T) { assert.Equal(t, "1m15s", Duration(75*time.Second).String()) @@ -120,6 +170,11 @@ func TestDuration(t *testing.T) { assert.NoError(t, json.Unmarshal([]byte(`"1m15s"`), &d)) assert.Equal(t, Duration(75*time.Second), d) }) + t.Run("Extended", func(t *testing.T) { + var d Duration + assert.NoError(t, json.Unmarshal([]byte(`"1d2h1m15s"`), &d)) + assert.Equal(t, Duration(26*time.Hour+75*time.Second), d) + }) }) t.Run("Marshal", func(t *testing.T) { d := Duration(75 * time.Second) diff --git a/ui/pb/helpers.go b/ui/pb/helpers.go new file mode 100644 index 00000000000..751d635d7ad --- /dev/null +++ b/ui/pb/helpers.go @@ -0,0 +1,131 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package pb + +import ( + "math" + "strconv" + "time" +) + +// GetFixedLengthIntFormat returns "%0__d" format argument for fmt functions +// that will produce a base-10 right-aligned zero-padded string representation +// of the supplied integer value. The number of characters (i.e. the actual +// number + how many zeros it will be padded on the left with) in the returned +// string corresponds to the number of digits in the supplied maxValue. +func GetFixedLengthIntFormat(maxValue int64) (formatStr string) { + resLen := 1 + if maxValue < 0 { + resLen++ + } + for maxValue /= 10; maxValue != 0; maxValue /= 10 { + resLen++ + } + return "%0" + strconv.Itoa(resLen) + "d" +} + +// GetFixedLengthFloatFormat returns "%0__.__f" format argument for fmt +// functions that will produce a base-10 right-aligned zero-padded string +// representation of the supplied float value, with the specified decimal +// precision. The number of characters (i.e. the actual number + maybe dot and +// precision + how many zeros it will be padded on the left with) in the +// returned string corresponds to the number of digits in the supplied maxValue +// and the desired precision. +func GetFixedLengthFloatFormat(maxValue float64, precision uint) (formatStr string) { + resLen := 1 + if maxValue < 0 { + maxValue = -maxValue + resLen++ + } + if maxValue >= 10 { + resLen += int(math.Log10(maxValue)) + } + if precision > 0 { + resLen += int(precision + 1) + } + return "%0" + strconv.Itoa(resLen) + "." + strconv.Itoa(int(precision)) + "f" +} + +// GetFixedLengthDuration takes a *positive* duration and its max value and +// returns a string with a fixed width so we can prevent UI elements jumping +// around. The format is "___d__h__m__s.s", but leading values can be omitted +// based on the maxDuration value, the results can be: "___h__m__s.s" +// +// This is code was inspired by the Go stdlib's time.Duration.String() code. +// TODO: more flexibility - negative values or variable precision? +func GetFixedLengthDuration(d, maxDuration time.Duration) (result string) { + const rounding = 100 * time.Millisecond + if d < 0 { + d = -d + } + if maxDuration < 0 { + maxDuration = -maxDuration + } + if maxDuration < d { + maxDuration = d + } + maxDuration = maxDuration.Round(rounding) + + // Largest time is "106751d23h47m16.9s", i.e. time.Duration(math.MaxInt64) + // Positions: 0 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 + buf := [18]byte{'0', '0', '0', '0', '0', '0', 'd', '0', '0', 'h', '0', '0', 'm', '0', '0', '.', '0', 's'} + + u := uint64(d.Round(rounding) / (rounding)) + u, buf[16] = u/10, byte(u%10)+'0' + u, buf[14] = u/10, byte(u%10)+'0' + if maxDuration < 10*time.Second { + return string(buf[14:]) + } + + u, buf[13] = u/6, byte(u%6)+'0' + if maxDuration < time.Minute { + return string(buf[13:]) + } + + u, buf[11] = u/10, byte(u%10)+'0' + if maxDuration < 10*time.Minute { + return string(buf[11:]) + } + + u, buf[10] = u/6, byte(u%6)+'0' + if maxDuration < time.Hour { + return string(buf[10:]) + } + + u, h := u/24, u%24 + buf[7], buf[8] = byte(h/10)+'0', byte(h%10)+'0' + if maxDuration < 10*time.Hour { + return string(buf[8:]) + } else if maxDuration < 24*time.Hour { + return string(buf[7:]) + } + + u, buf[5] = u/10, byte(u%10)+'0' + remDayPowers := maxDuration / (240 * time.Hour) + i := 5 + for remDayPowers > 0 { + i-- + u, buf[i] = u/10, byte(u%10)+'0' + remDayPowers /= 10 + } + + return string(buf[i:]) +} diff --git a/ui/pb/helpers_test.go b/ui/pb/helpers_test.go new file mode 100644 index 00000000000..7d07b1636db --- /dev/null +++ b/ui/pb/helpers_test.go @@ -0,0 +1,169 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package pb + +import ( + "fmt" + "math" + "strconv" + "testing" + "time" + + "github.com/loadimpact/k6/lib/types" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestGetFixedLengthInt(t *testing.T) { + testCases := []struct { + val, maxVal int64 + expRes string + }{ + {1, 0, "1"}, + {1, 1, "1"}, + {1, 5, "1"}, + {111, 5, "111"}, + {-1, 5, "-1"}, + {-1, -50, "-01"}, + {-1, 50, "-1"}, + + {1, 15, "01"}, + {1, 15, "01"}, + {1, 150, "001"}, + {1, 1500, "0001"}, + {999, 1500, "0999"}, + {-999, 1500, "-999"}, + {-9999, 1500, "-9999"}, + {1, 10000, "00001"}, + {1234567, 10000, "1234567"}, + {123456790, math.MaxInt64, "0000000000123456790"}, + {-123456790, math.MaxInt64, "-000000000123456790"}, + {math.MaxInt64, math.MaxInt64, "9223372036854775807"}, + {-123456790, math.MinInt64, "-0000000000123456790"}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.expRes, func(t *testing.T) { + fmtFormat := GetFixedLengthIntFormat(tc.maxVal) + res := fmt.Sprintf(fmtFormat, tc.val) + assert.Equal(t, tc.expRes, res) + back, err := strconv.ParseInt(res, 10, 64) + require.NoError(t, err) + assert.Equal(t, tc.val, back) + }) + } +} +func TestGetFixedLengthFloat(t *testing.T) { + testCases := []struct { + val, maxVal float64 + precision uint + expRes string + }{ + {0, 0, 0, "0"}, + {0, 0, 2, "0.00"}, + {0, 100, 2, "000.00"}, + {0, -100, 2, "0000.00"}, + {12, -100, 2, "0012.00"}, + {-12, -100, 2, "-012.00"}, + {12, 99, 2, "12.00"}, + {12, 100, 2, "012.00"}, + {1, 0, 0, "1"}, + {1, 0, 1, "1.0"}, + {1, 0, 2, "1.00"}, + {1.01, 0, 1, "1.0"}, + {1.01, 0, 1, "1.0"}, + {1.01, 0, 2, "1.01"}, + {1.007, 0, 2, "1.01"}, + {1.003, 0, 2, "1.00"}, + {1.003, 0, 3, "1.003"}, + {1.003, 0, 4, "1.0030"}, + {1.003, 1, 4, "1.0030"}, + {1.003, 9.999, 4, "1.0030"}, + {1.003, 10, 4, "01.0030"}, + {1.003, -10, 4, "001.0030"}, + {-1.003, -10, 4, "-01.0030"}, + {12.003, 1000, 4, "0012.0030"}, + } + + for i, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("tc%d_exp_%s", i, tc.expRes), func(t *testing.T) { + fmtFormat := GetFixedLengthFloatFormat(tc.maxVal, tc.precision) + res := fmt.Sprintf(fmtFormat, tc.val) + assert.Equal(t, tc.expRes, res) + back, err := strconv.ParseFloat(res, 64) + require.NoError(t, err) + + precPow := math.Pow(10, float64(tc.precision)) + expParseVal := math.Round(tc.val*precPow) / precPow + assert.Equal(t, expParseVal, back) + }) + } +} + +func TestGetFixedLengthDuration(t *testing.T) { + testCases := []struct { + val, maxVal time.Duration + expRes string + }{ + {0, 0, "0.0s"}, + {1 * time.Second, 0, "1.0s"}, + {9*time.Second + 940*time.Millisecond, 0, "9.9s"}, + {9*time.Second + 950*time.Millisecond, 0, "10.0s"}, + {1100 * time.Millisecond, 0, "1.1s"}, + {-1100 * time.Millisecond, 0, "1.1s"}, + {1100 * time.Millisecond, 10 * time.Second, "01.1s"}, + {1100 * time.Millisecond, 1 * time.Minute, "0m01.1s"}, + {1100 * time.Millisecond, -1 * time.Minute, "0m01.1s"}, + {-1100 * time.Millisecond, -1 * time.Minute, "0m01.1s"}, + {1100 * time.Millisecond, 10 * time.Minute, "00m01.1s"}, + {1100 * time.Millisecond, time.Hour, "0h00m01.1s"}, + {1100 * time.Millisecond, 10 * time.Hour, "00h00m01.1s"}, + {183 * time.Second, 10 * time.Minute, "03m03.0s"}, + {183 * time.Second, 120 * time.Minute, "0h03m03.0s"}, + {183 * time.Second, 10 * time.Hour, "00h03m03.0s"}, + {183 * time.Second, 25 * time.Hour, "0d00h03m03.0s"}, + {25 * time.Hour, 25 * time.Hour, "1d01h00m00.0s"}, + {482 * time.Hour, 25 * time.Hour, "20d02h00m00.0s"}, + {482 * time.Hour, 4800 * time.Hour, "020d02h00m00.0s"}, + {482*time.Hour + 671*time.Second + 65*time.Millisecond, time.Duration(math.MaxInt64), "000020d02h11m11.1s"}, + + // subtracting a second since rounding doesn't work as expected at the limits of int64 + {time.Duration(math.MaxInt64) - time.Second, time.Duration(math.MaxInt64), "106751d23h47m15.9s"}, + } + + for i, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("tc%d_exp_%s", i, tc.expRes), func(t *testing.T) { + res := GetFixedLengthDuration(tc.val, tc.maxVal) + assert.Equal(t, tc.expRes, res) + + expBackDur := tc.val.Round(100 * time.Millisecond) + if expBackDur < 0 { + expBackDur = -expBackDur + } + backDur, err := types.ParseExtendedDuration(res) + assert.NoError(t, err) + assert.Equal(t, expBackDur, backDur) + }) + } +} diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go new file mode 100644 index 00000000000..457edf0ea1f --- /dev/null +++ b/ui/pb/progressbar.go @@ -0,0 +1,142 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package pb + +import ( + "fmt" + "strings" + "sync" + + "github.com/fatih/color" +) + +const defaultWidth = 40 +const defaultBarColor = color.Faint + +// ProgressBar is just a simple thread-safe progressbar implementation with +// callbacks. +type ProgressBar struct { + mutex sync.RWMutex + width int + color *color.Color + + left func() string + progress func() (progress float64, right string) + hijack func() string +} + +// ProgressBarOption is used for helper functions that modify the progressbar +// parameters, either in the constructor or via the Modify() method. +type ProgressBarOption func(*ProgressBar) + +// WithLeft modifies the function that returns the left progressbar padding. +func WithLeft(left func() string) ProgressBarOption { + return func(pb *ProgressBar) { pb.left = left } +} + +// WithConstLeft sets the left progressbar padding to the supplied const. +func WithConstLeft(left string) ProgressBarOption { + return func(pb *ProgressBar) { + pb.left = func() string { return left } + } +} + +// WithProgress modifies the progress calculation function. +func WithProgress(progress func() (float64, string)) ProgressBarOption { + return func(pb *ProgressBar) { pb.progress = progress } +} + +// WithConstProgress sets the progress and right padding to the supplied consts. +func WithConstProgress(progress float64, right string) ProgressBarOption { + return func(pb *ProgressBar) { + pb.progress = func() (float64, string) { return progress, right } + } +} + +// WithHijack replaces the progressbar String function with the argument. +func WithHijack(hijack func() string) ProgressBarOption { + return func(pb *ProgressBar) { pb.hijack = hijack } +} + +// New creates and initializes a new ProgressBar struct, calling all of the +// supplied options +func New(options ...ProgressBarOption) *ProgressBar { + pb := &ProgressBar{ + mutex: sync.RWMutex{}, + width: defaultWidth, + color: color.New(defaultBarColor), + } + pb.Modify(options...) + return pb +} + +// Modify changes the progressbar options in a thread-safe way. +func (pb *ProgressBar) Modify(options ...ProgressBarOption) { + pb.mutex.Lock() + defer pb.mutex.Unlock() + for _, option := range options { + option(pb) + } +} + +// String locks the progressbar struct for reading and calls all of its methods +// to assemble the progress bar and return it as a string. +//TODO: something prettier? paddings, right-alighment of the left column, line trimming by terminal size +func (pb *ProgressBar) String() string { + pb.mutex.RLock() + defer pb.mutex.RUnlock() + + if pb.hijack != nil { + return pb.hijack() + } + + var left, right string + if pb.left != nil { + left = pb.left() + " " + } + + var progress float64 + if pb.progress != nil { + progress, right = pb.progress() + right = " " + right + } + + space := pb.width - 2 + filled := int(float64(space) * progress) + + filling := "" + caret := "" + if filled > 0 { + if filled < space { + filling = strings.Repeat("=", filled-1) + caret = ">" + } else { + filling = strings.Repeat("=", filled) + } + } + + padding := "" + if space > filled { + padding = pb.color.Sprint(strings.Repeat("-", space-filled)) + } + + return fmt.Sprintf("%s[%s%s%s]%s", left, filling, caret, padding, right) +} diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go new file mode 100644 index 00000000000..d36af9b0d3d --- /dev/null +++ b/ui/pb/progressbar_test.go @@ -0,0 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package pb + +//TODO diff --git a/ui/progress_bar.go b/ui/progress_bar.go deleted file mode 100644 index 5e098a42313..00000000000 --- a/ui/progress_bar.go +++ /dev/null @@ -1,68 +0,0 @@ -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2016 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package ui - -import ( - "fmt" - "strings" - - "github.com/fatih/color" -) - -var ( - faint = color.New(color.Faint) -) - -type ProgressBar struct { - Width int - Progress float64 - Left, Right func() string -} - -func (b ProgressBar) String() string { - space := b.Width - 2 - filled := int(float64(space) * b.Progress) - - filling := "" - caret := "" - if filled > 0 { - if filled < space { - filling = strings.Repeat("=", filled-1) - caret = ">" - } else { - filling = strings.Repeat("=", filled) - } - } - - padding := "" - if space > filled { - padding = faint.Sprint(strings.Repeat("-", space-filled)) - } - - var left, right string - if b.Left != nil { - left = b.Left() + " " - } - if b.Right != nil { - right = " " + b.Right() - } - return fmt.Sprintf("%s[%s%s%s]%s", left, filling, caret, padding, right) -} From d6fb5b3c34d2af5a7ce72453d3d3967fc23662f2 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:48:11 +0300 Subject: [PATCH 008/350] Add execution-segment, remove maxvus, fix no-setup/no-teardown options --- cmd/options.go | 23 ++++++++++++++++++++++- cmd/run.go | 42 ++---------------------------------------- js/bundle_test.go | 11 ----------- lib/options.go | 36 +++++++++++++++++++++++------------- lib/options_test.go | 22 +--------------------- 5 files changed, 48 insertions(+), 86 deletions(-) diff --git a/cmd/options.go b/cmd/options.go index 40164b8efb8..4d7c2cc35c0 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -45,11 +45,18 @@ func optionFlagSet() *pflag.FlagSet { flags := pflag.NewFlagSet("", 0) flags.SortFlags = false flags.Int64P("vus", "u", 1, "number of virtual users") + + //TODO: delete in a few versions flags.Int64P("max", "m", 0, "max available virtual users") + _ = flags.MarkDeprecated("max", "the global MaxVUs option is obsolete and doesn't affect the k6 script execution") + flags.DurationP("duration", "d", 0, "test duration limit") flags.Int64P("iterations", "i", 0, "script total iteration limit (among all VUs)") flags.StringSliceP("stage", "s", nil, "add a `stage`, as `[duration]:[target]`") + flags.String("execution-segment", "", "limit execution to the specified segment, e.g. 10%, 1/3, 0.2:2/3") flags.BoolP("paused", "p", false, "start the test in a paused state") + flags.Bool("no-setup", false, "don't run setup()") + flags.Bool("no-teardown", false, "don't run teardown()") flags.Int64("max-redirects", 10, "follow at most n redirects") flags.Int64("batch", 20, "max parallel batch reqs") flags.Int64("batch-per-host", 20, "max parallel batch reqs per host") @@ -75,10 +82,11 @@ func optionFlagSet() *pflag.FlagSet { func getOptions(flags *pflag.FlagSet) (lib.Options, error) { opts := lib.Options{ VUs: getNullInt64(flags, "vus"), - VUsMax: getNullInt64(flags, "max"), Duration: getNullDuration(flags, "duration"), Iterations: getNullInt64(flags, "iterations"), Paused: getNullBool(flags, "paused"), + NoSetup: getNullBool(flags, "no-setup"), + NoTeardown: getNullBool(flags, "no-teardown"), MaxRedirects: getNullInt64(flags, "max-redirects"), Batch: getNullInt64(flags, "batch"), RPS: getNullInt64(flags, "rps"), @@ -117,6 +125,19 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { } } + if flags.Lookup("execution-segment").Changed { + executionSegmentStr, err := flags.GetString("execution-segment") + if err != nil { + return opts, err + } + segment := new(lib.ExecutionSegment) + err = segment.UnmarshalText([]byte(executionSegmentStr)) + if err != nil { + return opts, err + } + opts.ExecutionSegment = segment + } + blacklistIPStrings, err := flags.GetStringSlice("blacklist-ip") if err != nil { return opts, err diff --git a/cmd/run.go b/cmd/run.go index 7376a0d16ab..7872f293b20 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -65,12 +65,9 @@ const ( invalidConfigErrorCode = 104 ) -var ( //TODO: fix this, global variables are not very testable... - runType = os.Getenv("K6_TYPE") - runNoSetup = os.Getenv("K6_NO_SETUP") != "" - runNoTeardown = os.Getenv("K6_NO_TEARDOWN") != "" -) +//nolint:gochecknoglobals +var runType = os.Getenv("K6_TYPE") // runCmd represents the run command. var runCmd = &cobra.Command{ @@ -142,36 +139,6 @@ a commandline interface for interacting with it.`, return err } - // If -m/--max isn't specified, figure out the max that should be needed. - if !conf.VUsMax.Valid { - conf.VUsMax = null.NewInt(conf.VUs.Int64, conf.VUs.Valid) - for _, stage := range conf.Stages { - if stage.Target.Valid && stage.Target.Int64 > conf.VUsMax.Int64 { - conf.VUsMax = stage.Target - } - } - } - - // If -d/--duration, -i/--iterations and -s/--stage are all unset, run to one iteration. - if !conf.Duration.Valid && !conf.Iterations.Valid && len(conf.Stages) == 0 { - conf.Iterations = null.IntFrom(1) - } - - if conf.Iterations.Valid && conf.Iterations.Int64 < conf.VUsMax.Int64 { - log.Warnf( - "All iterations (%d in this test run) are shared between all VUs, so some of the %d VUs will not execute even a single iteration!", - conf.Iterations.Int64, conf.VUsMax.Int64, - ) - } - - //TODO: move a bunch of the logic above to a config "constructor" and to the Validate() method - - // If duration is explicitly set to 0, it means run forever. - //TODO: just... handle this differently, e.g. as a part of the manual executor - if conf.Duration.Valid && conf.Duration.Duration == 0 { - conf.Duration = types.NullDuration{} - } - if cerr := validateConfig(conf); cerr != nil { return ExitCode{cerr, invalidConfigErrorCode} } @@ -486,11 +453,6 @@ func runCmdFlagSet() *pflag.FlagSet { // - and finally, global variables are not very testable... :/ flags.StringVarP(&runType, "type", "t", runType, "override file `type`, \"js\" or \"archive\"") flags.Lookup("type").DefValue = "" - flags.BoolVar(&runNoSetup, "no-setup", runNoSetup, "don't run setup()") - falseStr := "false" // avoiding goconst warnings... - flags.Lookup("no-setup").DefValue = falseStr - flags.BoolVar(&runNoTeardown, "no-teardown", runNoTeardown, "don't run teardown()") - flags.Lookup("no-teardown").DefValue = falseStr return flags } diff --git a/js/bundle_test.go b/js/bundle_test.go index 603afda382b..5354a736a84 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -139,17 +139,6 @@ func TestNewBundle(t *testing.T) { assert.Equal(t, null.IntFrom(100), b.Options.VUs) } }) - t.Run("VUsMax", func(t *testing.T) { - b, err := getSimpleBundle("/script.js", ` - export let options = { - vusMax: 100, - }; - export default function() {}; - `) - if assert.NoError(t, err) { - assert.Equal(t, null.IntFrom(100), b.Options.VUsMax) - } - }) t.Run("Duration", func(t *testing.T) { b, err := getSimpleBundle("/script.js", ` export let options = { diff --git a/lib/options.go b/lib/options.go index f4a4657e401..e8351a7f871 100644 --- a/lib/options.go +++ b/lib/options.go @@ -29,7 +29,6 @@ import ( "reflect" "strings" - "github.com/loadimpact/k6/lib/scheduler" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/pkg/errors" @@ -213,16 +212,23 @@ type Options struct { // See the Runner or Executor interfaces for more information. VUs null.Int `json:"vus" envconfig:"vus"` - //TODO: deprecate this? or reuse it in the manual control "scheduler"? - VUsMax null.Int `json:"vusMax" envconfig:"vus_max"` Duration types.NullDuration `json:"duration" envconfig:"duration"` Iterations null.Int `json:"iterations" envconfig:"iterations"` Stages []Stage `json:"stages" envconfig:"stages"` - Execution scheduler.ConfigMap `json:"execution,omitempty" envconfig:"-"` + // TODO: remove the `ignored:"true"` from the field tags, it's there so that + // the envconfig library will ignore those fields. + // + // We should support specifying execution segments via environment + // variables, but we currently can't, because envconfig has this nasty bug + // (among others): https://github.com/kelseyhightower/envconfig/issues/113 + Execution SchedulerConfigMap `json:"execution,omitempty" ignored:"true"` + ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` // Timeouts for the setup() and teardown() functions + NoSetup null.Bool `json:"noSetup" envconfig:"NO_SETUP"` SetupTimeout types.NullDuration `json:"setupTimeout" envconfig:"setup_timeout"` + NoTeardown null.Bool `json:"noTeardown" envconfig:"NO_TEARDOWN"` TeardownTimeout types.NullDuration `json:"teardownTimeout" envconfig:"teardown_timeout"` // Limit HTTP requests per second. @@ -317,9 +323,6 @@ func (o Options) Apply(opts Options) Options { if opts.VUs.Valid { o.VUs = opts.VUs } - if opts.VUsMax.Valid { - o.VUsMax = opts.VUsMax - } // Specifying duration, iterations, stages, or execution in a "higher" config tier // will overwrite all of the the previous execution settings (if any) from any @@ -328,12 +331,10 @@ func (o Options) Apply(opts Options) Options { // config tier, they will be preserved, so the validation after we've consolidated // all of the options can return an error. if opts.Duration.Valid || opts.Iterations.Valid || opts.Stages != nil || opts.Execution != nil { - //TODO: uncomment this after we start using the new schedulers - /* - o.Duration = types.NewNullDuration(0, false) - o.Iterations = null.NewInt(0, false) - o.Stages = nil - */ + //TODO: emit a warning or a notice log message if overwrite lower tier config options? + o.Duration = types.NewNullDuration(0, false) + o.Iterations = null.NewInt(0, false) + o.Stages = nil o.Execution = nil } @@ -358,9 +359,18 @@ func (o Options) Apply(opts Options) Options { if opts.Execution != nil { o.Execution = opts.Execution } + if opts.ExecutionSegment != nil { + o.ExecutionSegment = opts.ExecutionSegment + } + if opts.NoSetup.Valid { + o.NoSetup = opts.NoSetup + } if opts.SetupTimeout.Valid { o.SetupTimeout = opts.SetupTimeout } + if opts.NoTeardown.Valid { + o.NoTeardown = opts.NoTeardown + } if opts.TeardownTimeout.Valid { o.TeardownTimeout = opts.TeardownTimeout } diff --git a/lib/options_test.go b/lib/options_test.go index 63cf4ae891e..68f07a3cee4 100644 --- a/lib/options_test.go +++ b/lib/options_test.go @@ -30,7 +30,6 @@ import ( "time" "github.com/kelseyhightower/envconfig" - "github.com/loadimpact/k6/lib/scheduler" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" @@ -49,11 +48,6 @@ func TestOptions(t *testing.T) { assert.True(t, opts.VUs.Valid) assert.Equal(t, int64(12345), opts.VUs.Int64) }) - t.Run("VUsMax", func(t *testing.T) { - opts := Options{}.Apply(Options{VUsMax: null.IntFrom(12345)}) - assert.True(t, opts.VUsMax.Valid) - assert.Equal(t, int64(12345), opts.VUsMax.Int64) - }) t.Run("Duration", func(t *testing.T) { opts := Options{}.Apply(Options{Duration: types.NullDurationFrom(2 * time.Minute)}) assert.True(t, opts.Duration.Valid) @@ -88,17 +82,7 @@ func TestOptions(t *testing.T) { assert.Equal(t, oneStage, opts.Apply(Options{Stages: oneStage}).Stages) assert.Equal(t, oneStage, Options{}.Apply(opts).Apply(Options{Stages: oneStage}).Apply(Options{Stages: oneStage}).Stages) }) - t.Run("Execution", func(t *testing.T) { - sched := scheduler.NewConstantLoopingVUsConfig("test") - sched.VUs = null.IntFrom(123) - sched.Duration = types.NullDurationFrom(3 * time.Minute) - opts := Options{}.Apply(Options{Execution: scheduler.ConfigMap{"test": sched}}) - cs, ok := opts.Execution["test"].(scheduler.ConstantLoopingVUsConfig) - assert.True(t, ok) - assert.Equal(t, int64(123), cs.VUs.Int64) - assert.Equal(t, "3m0s", cs.Duration.String()) - }) - //TODO: test that any execution option overwrites any other lower-level options + // Execution overwriting is tested by the config consolidation test in cmd t.Run("RPS", func(t *testing.T) { opts := Options{}.Apply(Options{RPS: null.IntFrom(12345)}) assert.True(t, opts.RPS.Valid) @@ -415,10 +399,6 @@ func TestOptionsEnv(t *testing.T) { "": null.Int{}, "123": null.IntFrom(123), }, - {"VUsMax", "K6_VUS_MAX"}: { - "": null.Int{}, - "123": null.IntFrom(123), - }, {"Duration", "K6_DURATION"}: { "": types.NullDuration{}, "10s": types.NullDurationFrom(10 * time.Second), From 755f69b20fa0791c7fdc7604dc1bfde16c35ba90 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:50:18 +0300 Subject: [PATCH 009/350] Add the updated engine and the new local executor --- core/engine.go | 82 ++--- core/engine_test.go | 362 ++++++------------- core/local/local.go | 730 +++++++++++++++------------------------ core/local/local_test.go | 444 ++++++++++++++---------- core/local/util.go | 64 ---- core/local/util_test.go | 13 +- 6 files changed, 668 insertions(+), 1027 deletions(-) delete mode 100644 core/local/util.go diff --git a/core/engine.go b/core/engine.go index 3e948836c88..db9ba7c555f 100644 --- a/core/engine.go +++ b/core/engine.go @@ -22,15 +22,15 @@ package core import ( "context" + "errors" "strings" "sync" "time" - "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/stats" - log "github.com/sirupsen/logrus" + "github.com/sirupsen/logrus" "gopkg.in/guregu/null.v3" ) @@ -47,15 +47,18 @@ const ( // The Engine is the beating heart of K6. type Engine struct { - runLock sync.Mutex + runLock sync.Mutex // y tho? TODO: remove? + + //TODO: make most of the stuff here private! + Executor lib.Executor + executorState *lib.ExecutorState - Executor lib.Executor Options lib.Options Collectors []lib.Collector NoThresholds bool NoSummary bool - logger *log.Logger + logger *logrus.Logger Metrics map[string]*stats.Metric MetricsLock sync.Mutex @@ -70,29 +73,21 @@ type Engine struct { thresholdsTainted bool } -func NewEngine(ex lib.Executor, o lib.Options) (*Engine, error) { +// NewEngine instantiates a new Engine, without doing any heavy initialization. +func NewEngine(ex lib.Executor, o lib.Options, logger *logrus.Logger) (*Engine, error) { if ex == nil { - ex = local.New(nil) + return nil, errors.New("missing executor instance") } e := &Engine{ - Executor: ex, - Options: o, - Metrics: make(map[string]*stats.Metric), - Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), - } - e.SetLogger(log.StandardLogger()) + Executor: ex, + executorState: ex.GetState(), - if err := ex.SetVUsMax(o.VUsMax.Int64); err != nil { - return nil, err + Options: o, + Metrics: make(map[string]*stats.Metric), + Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), + logger: logger, } - if err := ex.SetVUs(o.VUs.Int64); err != nil { - return nil, err - } - ex.SetPaused(o.Paused.Bool) - ex.SetStages(o.Stages) - ex.SetEndTime(o.Duration) - ex.SetEndIterations(o.Iterations) e.thresholds = o.Thresholds e.submetrics = make(map[string][]*stats.Submetric) @@ -108,6 +103,11 @@ func NewEngine(ex lib.Executor, o lib.Options) (*Engine, error) { return e, nil } +// Init is used to initialize the executor. That's a costly operation, since it initializes all of +func (e *Engine) Init(ctx context.Context) error { + return e.Executor.Init(ctx, e.Samples) +} + func (e *Engine) setRunStatus(status lib.RunStatus) { if len(e.Collectors) == 0 { return @@ -123,25 +123,6 @@ func (e *Engine) Run(ctx context.Context) error { defer e.runLock.Unlock() e.logger.Debug("Engine: Starting with parameters...") - for i, st := range e.Executor.GetStages() { - fields := make(log.Fields) - if st.Target.Valid { - fields["tgt"] = st.Target.Int64 - } - if st.Duration.Valid { - fields["d"] = st.Duration.Duration - } - e.logger.WithFields(fields).Debugf(" - stage #%d", i) - } - - fields := make(log.Fields) - if endTime := e.Executor.GetEndTime(); endTime.Valid { - fields["time"] = endTime.Duration - } - if endIter := e.Executor.GetEndIterations(); endIter.Valid { - fields["iter"] = endIter.Int64 - } - e.logger.WithFields(fields).Debug(" - end conditions (if any)") collectorwg := sync.WaitGroup{} collectorctx, collectorcancel := context.WithCancel(context.Background()) @@ -208,9 +189,6 @@ func (e *Engine) Run(ctx context.Context) error { e.processSamples(sampleContainers) } - // Emit final metrics. - e.emitMetrics() - // Process final thresholds. if !e.NoThresholds { e.processThresholds(nil) @@ -252,15 +230,6 @@ func (e *Engine) IsTainted() bool { return e.thresholdsTainted } -func (e *Engine) SetLogger(l *log.Logger) { - e.logger = l - e.Executor.SetLogger(l) -} - -func (e *Engine) GetLogger() *log.Logger { - return e.logger -} - func (e *Engine) runMetricsEmission(ctx context.Context) { ticker := time.NewTicker(MetricsRate) for { @@ -276,17 +245,18 @@ func (e *Engine) runMetricsEmission(ctx context.Context) { func (e *Engine) emitMetrics() { t := time.Now() + executorState := e.Executor.GetState() e.processSamples([]stats.SampleContainer{stats.ConnectedSamples{ Samples: []stats.Sample{ { Time: t, Metric: metrics.VUs, - Value: float64(e.Executor.GetVUs()), + Value: float64(executorState.GetCurrentlyActiveVUsCount()), Tags: e.Options.RunTags, }, { Time: t, Metric: metrics.VUsMax, - Value: float64(e.Executor.GetVUsMax()), + Value: float64(executorState.GetInitializedVUsCount()), Tags: e.Options.RunTags, }, }, @@ -311,7 +281,7 @@ func (e *Engine) processThresholds(abort func()) { e.MetricsLock.Lock() defer e.MetricsLock.Unlock() - t := e.Executor.GetTime() + t := e.executorState.GetCurrentTestRunDuration() abortOnFail := false e.thresholdsTainted = false diff --git a/core/engine_test.go b/core/engine_test.go index fe81be047a1..3d41c19b9b8 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -26,19 +26,20 @@ import ( "testing" "time" + "github.com/spf13/afero" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" + "github.com/loadimpact/k6/lib/scheduler" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/dummy" - log "github.com/sirupsen/logrus" - logtest "github.com/sirupsen/logrus/hooks/test" - "github.com/spf13/afero" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" + "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) @@ -47,184 +48,46 @@ type testErrorWithString string func (e testErrorWithString) Error() string { return string(e) } func (e testErrorWithString) String() string { return string(e) } -// Apply a null logger to the engine and return the hook. -func applyNullLogger(e *Engine) *logtest.Hook { - logger, hook := logtest.NewNullLogger() - e.SetLogger(logger) - return hook -} - -// Wrapper around newEngine that applies a null logger. -func newTestEngine(ex lib.Executor, opts lib.Options) (*Engine, error) { - if !opts.MetricSamplesBufferSize.Valid { - opts.MetricSamplesBufferSize = null.IntFrom(200) +// Wrapper around NewEngine that applies a logger and manages the options. +func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts lib.Options) *Engine { //nolint: golint + if runner == nil { + runner = &lib.MiniRunner{} } - e, err := NewEngine(ex, opts) - if err != nil { - return e, err + if ctx == nil { + ctx = context.Background() } - applyNullLogger(e) - return e, nil -} -func LF(fn func(ctx context.Context, out chan<- stats.SampleContainer) error) lib.Executor { - return local.New(&lib.MiniRunner{Fn: fn}) -} + newOpts, err := scheduler.BuildExecutionConfig(lib.Options{ + MetricSamplesBufferSize: null.NewInt(200, false), + }.Apply(runner.GetOptions()).Apply(opts)) + require.NoError(t, err) + require.Empty(t, newOpts.Validate()) -func TestNewEngine(t *testing.T) { - _, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) -} + require.NoError(t, runner.SetOptions(newOpts)) -func TestNewEngineOptions(t *testing.T) { - t.Run("Duration", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Duration: types.NullDurationFrom(10 * time.Second), - }) - assert.NoError(t, err) - assert.Nil(t, e.Executor.GetStages()) - assert.Equal(t, types.NullDurationFrom(10*time.Second), e.Executor.GetEndTime()) - - t.Run("Infinite", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{Duration: types.NullDuration{}}) - assert.NoError(t, err) - assert.Nil(t, e.Executor.GetStages()) - assert.Equal(t, types.NullDuration{}, e.Executor.GetEndTime()) - }) - }) - t.Run("Stages", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Stages: []lib.Stage{ - {Duration: types.NullDurationFrom(10 * time.Second), Target: null.IntFrom(10)}, - }, - }) - assert.NoError(t, err) - if assert.Len(t, e.Executor.GetStages(), 1) { - assert.Equal(t, e.Executor.GetStages()[0], lib.Stage{Duration: types.NullDurationFrom(10 * time.Second), Target: null.IntFrom(10)}) - } - }) - t.Run("Stages/Duration", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Duration: types.NullDurationFrom(60 * time.Second), - Stages: []lib.Stage{ - {Duration: types.NullDurationFrom(10 * time.Second), Target: null.IntFrom(10)}, - }, - }) - assert.NoError(t, err) - if assert.Len(t, e.Executor.GetStages(), 1) { - assert.Equal(t, e.Executor.GetStages()[0], lib.Stage{Duration: types.NullDurationFrom(10 * time.Second), Target: null.IntFrom(10)}) - } - assert.Equal(t, types.NullDurationFrom(60*time.Second), e.Executor.GetEndTime()) - }) - t.Run("Iterations", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{Iterations: null.IntFrom(100)}) - assert.NoError(t, err) - assert.Equal(t, null.IntFrom(100), e.Executor.GetEndIterations()) - }) - t.Run("VUsMax", func(t *testing.T) { - t.Run("not set", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) - assert.Equal(t, int64(0), e.Executor.GetVUsMax()) - assert.Equal(t, int64(0), e.Executor.GetVUs()) - }) - t.Run("set", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - VUsMax: null.IntFrom(10), - }) - assert.NoError(t, err) - assert.Equal(t, int64(10), e.Executor.GetVUsMax()) - assert.Equal(t, int64(0), e.Executor.GetVUs()) - }) - }) - t.Run("VUs", func(t *testing.T) { - t.Run("no max", func(t *testing.T) { - _, err := newTestEngine(nil, lib.Options{ - VUs: null.IntFrom(10), - }) - assert.EqualError(t, err, "can't raise vu count (to 10) above vu cap (0)") - }) - t.Run("negative max", func(t *testing.T) { - _, err := newTestEngine(nil, lib.Options{ - VUsMax: null.IntFrom(-1), - }) - assert.EqualError(t, err, "vu cap can't be negative") - }) - t.Run("max too low", func(t *testing.T) { - _, err := newTestEngine(nil, lib.Options{ - VUsMax: null.IntFrom(1), - VUs: null.IntFrom(10), - }) - assert.EqualError(t, err, "can't raise vu count (to 10) above vu cap (1)") - }) - t.Run("max higher", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - VUsMax: null.IntFrom(10), - VUs: null.IntFrom(1), - }) - assert.NoError(t, err) - assert.Equal(t, int64(10), e.Executor.GetVUsMax()) - assert.Equal(t, int64(1), e.Executor.GetVUs()) - }) - t.Run("max just right", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - VUsMax: null.IntFrom(10), - VUs: null.IntFrom(10), - }) - assert.NoError(t, err) - assert.Equal(t, int64(10), e.Executor.GetVUsMax()) - assert.Equal(t, int64(10), e.Executor.GetVUs()) - }) - }) - t.Run("Paused", func(t *testing.T) { - t.Run("not set", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) - assert.False(t, e.Executor.IsPaused()) - }) - t.Run("false", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Paused: null.BoolFrom(false), - }) - assert.NoError(t, err) - assert.False(t, e.Executor.IsPaused()) - }) - t.Run("true", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Paused: null.BoolFrom(true), - }) - assert.NoError(t, err) - assert.True(t, e.Executor.IsPaused()) - }) - }) - t.Run("thresholds", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Thresholds: map[string]stats.Thresholds{ - "my_metric": {}, - }, - }) - assert.NoError(t, err) - assert.Contains(t, e.thresholds, "my_metric") + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) - t.Run("submetrics", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ - Thresholds: map[string]stats.Thresholds{ - "my_metric{tag:value}": {}, - }, - }) - assert.NoError(t, err) - assert.Contains(t, e.thresholds, "my_metric{tag:value}") - assert.Contains(t, e.submetrics, "my_metric") - }) - }) + executor, err := local.New(runner, logger) + require.NoError(t, err) + + engine, err := NewEngine(executor, opts, logger) + require.NoError(t, err) + + require.NoError(t, engine.Init(ctx)) + + return engine +} + +func TestNewEngine(t *testing.T) { + newTestEngine(t, nil, nil, lib.Options{}) } func TestEngineRun(t *testing.T) { - log.SetLevel(log.DebugLevel) + logrus.SetLevel(logrus.DebugLevel) t.Run("exits with context", func(t *testing.T) { duration := 100 * time.Millisecond - e, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{}) ctx, cancel := context.WithTimeout(context.Background(), duration) defer cancel() @@ -233,50 +96,36 @@ func TestEngineRun(t *testing.T) { assert.WithinDuration(t, startTime.Add(duration), time.Now(), 100*time.Millisecond) }) t.Run("exits with executor", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{ + e := newTestEngine(t, nil, nil, lib.Options{ VUs: null.IntFrom(10), - VUsMax: null.IntFrom(10), Iterations: null.IntFrom(100), }) - assert.NoError(t, err) assert.NoError(t, e.Run(context.Background())) - assert.Equal(t, int64(100), e.Executor.GetIterations()) + assert.Equal(t, uint64(100), e.Executor.GetState().GetFullIterationCount()) }) - // Make sure samples are discarded after context close (using "cutoff" timestamp in local.go) t.Run("collects samples", func(t *testing.T) { testMetric := stats.New("test_metric", stats.Trend) signalChan := make(chan interface{}) - var e *Engine - e, err := newTestEngine(LF(func(ctx context.Context, samples chan<- stats.SampleContainer) error { - samples <- stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1} + + runner := &lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + stats.PushIfNotCancelled(ctx, out, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1}) close(signalChan) <-ctx.Done() - - // HACK(robin): Add a sleep here to temporarily workaround two problems with this test: - // 1. The sample times are compared against the `cutoff` in core/local/local.go and sometimes the - // second sample (below) gets a `Time` smaller than `cutoff` because the lines below get executed - // before the `<-ctx.Done()` select in local.go:Run() on multi-core systems where - // goroutines can run in parallel. - // 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 <- stats.Sample{Metric: testMetric, Time: time.Now(), Value: 2} + stats.PushIfNotCancelled(ctx, out, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1}) return nil - }), lib.Options{ + }} + + ctx, cancel := context.WithCancel(context.Background()) + e := newTestEngine(t, ctx, runner, lib.Options{ VUs: null.IntFrom(1), - VUsMax: null.IntFrom(1), Iterations: null.IntFrom(1), }) - if !assert.NoError(t, err) { - return - } c := &dummy.Collector{} e.Collectors = []lib.Collector{c} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- e.Run(ctx) }() <-signalChan @@ -296,8 +145,10 @@ func TestEngineRun(t *testing.T) { } func TestEngineAtTime(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{ + VUs: null.IntFrom(2), + Duration: types.NullDurationFrom(20 * time.Second), + }) ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) defer cancel() @@ -307,11 +158,12 @@ 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, out chan<- stats.SampleContainer) error { + runner := &lib.MiniRunner{Fn: 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) + }} + + e := newTestEngine(t, nil, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) c := &dummy.Collector{} e.Collectors = []lib.Collector{c} @@ -339,8 +191,7 @@ func TestEngine_processSamples(t *testing.T) { metric := stats.New("my_metric", stats.Gauge) t.Run("metric", func(t *testing.T) { - e, err := newTestEngine(nil, lib.Options{}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{}) e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, @@ -352,12 +203,11 @@ func TestEngine_processSamples(t *testing.T) { ths, err := stats.NewThresholds([]string{`1+1==2`}) assert.NoError(t, err) - e, err := newTestEngine(nil, lib.Options{ + e := newTestEngine(t, nil, nil, lib.Options{ Thresholds: map[string]stats.Thresholds{ "my_metric{a:1}": ths, }, }) - assert.NoError(t, err) sms := e.submetrics["my_metric"] assert.Len(t, sms, 1) @@ -383,8 +233,7 @@ func TestEngine_runThresholds(t *testing.T) { t.Run("aborted", func(t *testing.T) { ths.Thresholds[0].AbortOnFail = true thresholds[metric.Name] = ths - e, err := newTestEngine(nil, lib.Options{Thresholds: thresholds}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, @@ -406,8 +255,7 @@ func TestEngine_runThresholds(t *testing.T) { t.Run("canceled", func(t *testing.T) { ths.Abort = false thresholds[metric.Name] = ths - e, err := newTestEngine(nil, lib.Options{Thresholds: thresholds}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, @@ -459,8 +307,7 @@ func TestEngine_processThresholds(t *testing.T) { thresholds[m] = ths } - e, err := newTestEngine(nil, lib.Options{Thresholds: thresholds}) - assert.NoError(t, err) + e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, @@ -551,7 +398,7 @@ func TestSentReceivedMetrics(t *testing.T) { type testCase struct{ Iterations, VUs int64 } testCases := []testCase{ - {1, 1}, {1, 2}, {2, 1}, {5, 2}, {25, 2}, {50, 5}, + {1, 1}, {2, 2}, {2, 1}, {5, 2}, {25, 2}, {50, 5}, } runTest := func(t *testing.T, ts testScript, tc testCase, noConnReuse bool) (float64, float64) { @@ -562,23 +409,18 @@ func TestSentReceivedMetrics(t *testing.T) { ) require.NoError(t, err) - options := lib.Options{ + ctx, cancel := context.WithCancel(context.Background()) + engine := newTestEngine(t, ctx, r, lib.Options{ Iterations: null.IntFrom(tc.Iterations), VUs: null.IntFrom(tc.VUs), - VUsMax: null.IntFrom(tc.VUs), Hosts: tb.Dialer.Hosts, InsecureSkipTLSVerify: null.BoolFrom(true), NoVUConnectionReuse: null.BoolFrom(noConnReuse), - } - - r.SetOptions(options) - engine, err := NewEngine(local.New(r), options) - require.NoError(t, err) + }) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- engine.Run(ctx) }() @@ -703,24 +545,19 @@ func TestRunTags(t *testing.T) { ) require.NoError(t, err) - options := lib.Options{ + ctx, cancel := context.WithCancel(context.Background()) + engine := newTestEngine(t, ctx, r, lib.Options{ Iterations: null.IntFrom(3), VUs: null.IntFrom(2), - VUsMax: null.IntFrom(2), Hosts: tb.Dialer.Hosts, RunTags: runTags, SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), InsecureSkipTLSVerify: null.BoolFrom(true), - } - - r.SetOptions(options) - engine, err := NewEngine(local.New(r), options) - require.NoError(t, err) + }) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- engine.Run(ctx) }() @@ -802,18 +639,15 @@ func TestSetupTeardownThresholds(t *testing.T) { lib.RuntimeOptions{}, ) require.NoError(t, err) - runner.SetOptions(runner.GetOptions().Apply(lib.Options{ + + ctx, cancel := context.WithCancel(context.Background()) + engine := newTestEngine(t, ctx, runner, 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) }() @@ -839,15 +673,20 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { export let options = { systemTags: ["iter", "vu", "url"], - - // Start with 2 VUs for 4 seconds and then quickly scale down to 1 for the next 4s and then quit - vus: 2, - vusMax: 2, - stages: [ - { duration: "4s", target: 2 }, - { duration: "1s", target: 1 }, - { duration: "3s", target: 1 }, - ], + execution: { + we_need_hard_stop_and_ramp_down: { + type: "variable-looping-vus", + // Start with 2 VUs for 4 seconds and then quickly scale down to 1 for the next 4s and then quit + startVUs: 2, + stages: [ + { duration: "4s", target: 2 }, + { duration: "0s", target: 1 }, + { duration: "4s", target: 1 }, + ], + gracefulStop: "0s", + gracefulRampDown: "0s", + }, + }, }; export default function () { @@ -866,13 +705,12 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { ) require.NoError(t, err) - engine, err := NewEngine(local.New(runner), runner.GetOptions()) - require.NoError(t, err) + ctx, cancel := context.WithCancel(context.Background()) + engine := newTestEngine(t, ctx, runner, lib.Options{}) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- engine.Run(ctx) }() @@ -886,7 +724,7 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { require.False(t, engine.IsTainted()) } - // The 1.7 sleep in the default function would cause the first VU to comlete 2 full iterations + // The 3.1 sleep in the default function would cause the first VU to comlete 2 full iterations // and stat executing its third one, while the second VU will only fully complete 1 iteration // and will be canceled in the middle of its second one. assert.Equal(t, 3.0, getMetricSum(collector, metrics.Iterations.Name)) @@ -895,11 +733,13 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { // and one each from the two iterations that would be canceled in the middle of their execution assert.Equal(t, 8.0, getMetricSum(collector, metrics.HTTPReqs.Name)) - // But we expect to only see the data_received for only 7 of those requests. The data for the 8th - // request (the 3rd one in the first VU before the test ends) gets cut off by the engine because - // it's emitted after the test officially ends - dataReceivedExpectedMin := 15000.0 * 7 - dataReceivedExpectedMax := (15000.0 + expectedHeaderMaxLength) * 7 + // And we expect to see the data_received for all 8 of those requests. Previously, the data for + // the 8th request (the 3rd one in the first VU before the test ends) was cut off by the engine + // because it was emitted after the test officially ended. But that was mostly an unintended + // consequence of the fact that those metrics were emitted only after an iteration ended when + // it was interrupted. + dataReceivedExpectedMin := 15000.0 * 8 + dataReceivedExpectedMax := (15000.0 + expectedHeaderMaxLength) * 8 dataReceivedActual := getMetricSum(collector, metrics.DataReceived.Name) if dataReceivedActual < dataReceivedExpectedMin || dataReceivedActual > dataReceivedExpectedMax { t.Errorf( @@ -926,10 +766,15 @@ func TestMinIterationDuration(t *testing.T) { let testCounter = new Counter("testcounter"); export let options = { + execution: { + we_need_hard_stop: { + type: "constant-looping-vus", + vus: 2, + duration: "1.9s", + gracefulStop: "0s", + }, + }, minIterationDuration: "1s", - vus: 2, - vusMax: 2, - duration: "1.9s", }; export default function () { @@ -940,13 +785,12 @@ func TestMinIterationDuration(t *testing.T) { ) require.NoError(t, err) - engine, err := NewEngine(local.New(runner), runner.GetOptions()) - require.NoError(t, err) + ctx, cancel := context.WithCancel(context.Background()) + engine := newTestEngine(t, ctx, runner, lib.Options{}) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- engine.Run(ctx) }() diff --git a/core/local/local.go b/core/local/local.go index d6bef798c4d..8307586336a 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -23,527 +23,363 @@ package local import ( "context" "fmt" - "sync" + "runtime" "sync/atomic" "time" + "github.com/loadimpact/k6/ui/pb" + "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" - log "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "github.com/sirupsen/logrus" ) -// 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 incomprehensible 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 { - sync.RWMutex - vu lib.VU - ctx context.Context - cancel context.CancelFunc +// Executor is the local implementation of lib.Executor +type Executor struct { + runner lib.Runner + options lib.Options + logger *logrus.Logger + + initProgress *pb.ProgressBar + schedulers []lib.Scheduler // sorted by (startTime, ID) + executionPlan []lib.ExecutionStep + maxDuration time.Duration // cached value derived from the execution plan + maxPossibleVUs uint64 // cached value derived from the execution plan + state *lib.ExecutorState } -func (h *vuHandle) run(logger *log.Logger, flow <-chan int64, iterDone chan<- struct{}) { - h.RLock() - ctx := h.ctx - h.RUnlock() +// Check to see if we implement the lib.Executor interface +var _ lib.Executor = &Executor{} - for { - select { - case _, ok := <-flow: - if !ok { - return - } - case <-ctx.Done(): - return +// New creates and returns a new local lib.Executor instance, without +// initializing it beyond the bare minimum. Specifically, it creates the needed +// schedulers instances and a lot of state placeholders, but it doesn't +// initialize the schedulers and it doesn't initialize or run any VUs. +func New(runner lib.Runner, logger *logrus.Logger) (*Executor, error) { + options := runner.GetOptions() + + executionPlan := options.Execution.GetFullExecutionRequirements(options.ExecutionSegment) + maxPlannedVUs := lib.GetMaxPlannedVUs(executionPlan) + maxPossibleVUs := lib.GetMaxPossibleVUs(executionPlan) + + executorState := lib.NewExecutorState(options, maxPlannedVUs, maxPossibleVUs) + maxDuration, _ := lib.GetEndOffset(executionPlan) // we don't care if the end offset is final + + schedulerConfigs := options.Execution.GetSortedSchedulerConfigs() + schedulers := make([]lib.Scheduler, len(schedulerConfigs)) + for i, sc := range schedulerConfigs { + s, err := sc.NewScheduler(executorState, logger.WithField("scheduler", sc.GetName())) + if err != nil { + return nil, err } + schedulers[i] = s + } - if h.vu != nil { - err := h.vu.RunOnce(ctx) - select { - case <-ctx.Done(): - // Don't log errors or emit iterations metrics from cancelled iterations - default: - if err != nil { - if s, ok := err.(fmt.Stringer); ok { - logger.Error(s.String()) - } else { - logger.Error(err.Error()) - } - } - iterDone <- struct{}{} - } - } else { - iterDone <- struct{}{} + if options.Paused.Bool { + if err := executorState.Pause(); err != nil { + return nil, err } } -} - -type Executor struct { - Runner lib.Runner - Logger *log.Logger - runLock sync.Mutex - wg sync.WaitGroup - - runSetup bool - runTeardown bool - - vus []*vuHandle - vusLock sync.RWMutex - numVUs int64 - numVUsMax int64 - nextVUID int64 - - iters int64 // Completed iterations - partIters int64 // Partial, incomplete iterations - endIters int64 // End test at this many iterations + return &Executor{ + runner: runner, + logger: logger, + options: options, + + initProgress: pb.New(pb.WithConstLeft("Init")), + schedulers: schedulers, + executionPlan: executionPlan, + maxDuration: maxDuration, + maxPossibleVUs: maxPossibleVUs, + state: executorState, + }, nil +} - time int64 // Current time - endTime int64 // End test at this timestamp +// GetRunner returns the wrapped lib.Runner instance. +func (e *Executor) GetRunner() lib.Runner { + return e.runner +} - pauseLock sync.RWMutex - pause chan interface{} +// GetState returns a pointer to the executor state struct for the local +// executor. It's guaranteed to be initialized and present, though see +// the documentation in lib/executor.go for caveats about its usage. +// The most important one is that none of the methods beyond the pause-related +// ones should be used for synchronization. +func (e *Executor) GetState() *lib.ExecutorState { + return e.state +} - stages []lib.Stage +// GetSchedulers returns the slice of configured scheduler instances, sorted by +// their (startTime, name) in an ascending order. +func (e *Executor) GetSchedulers() []lib.Scheduler { + return e.schedulers +} - // Lock for: ctx, flow, out - lock sync.RWMutex +// GetInitProgressBar returns a the progress bar assotiated with the Init +// function. After the Init is done, it is "hijacked" to display real-time +// execution statistics as a text bar. +func (e *Executor) GetInitProgressBar() *pb.ProgressBar { + return e.initProgress +} - // Current context, nil if a test isn't running right now. - ctx context.Context +// GetExecutionPlan is a helper method so users of the local executor don't have +// to calculate the execution plan again. +func (e *Executor) GetExecutionPlan() []lib.ExecutionStep { + return e.executionPlan +} - // Output channel to which VUs send samples. - vuOut chan stats.SampleContainer +// initVU is just a helper method that's used to both initialize the planned VUs +// in the Init() method, and also passed to schedulers so they can initialize +// any unplanned VUs themselves. +//TODO: actually use the context... +func (e *Executor) initVU( + _ context.Context, logger *logrus.Entry, engineOut chan<- stats.SampleContainer, +) (lib.VU, error) { + + vu, err := e.runner.NewVU(engineOut) + if err != nil { + return nil, fmt.Errorf("error while initializing a VU: '%s'", err) + } - // Channel on which VUs sigal that iterations are completed - iterDone chan struct{} + // Get the VU ID here, so that the VUs are (mostly) ordered by their + // number in the channel buffer + vuID := e.state.GetUniqueVUIdentifier() + if err := vu.Reconfigure(int64(vuID)); err != nil { + return nil, fmt.Errorf("error while reconfiguring VU #%d: '%s'", vuID, err) - // Flow control for VUs; iterations are run only after reading from this channel. - flow chan int64 + } + logger.Debugf("Initialized VU #%d", vuID) + return vu, nil } -func New(r lib.Runner) *Executor { - var bufferSize int64 - if r != nil { - bufferSize = r.GetOptions().MetricSamplesBufferSize.Int64 +// getRunStats is a helper function that can be used as the executor's +// progressbar substitute (i.e. hijack). +func (e *Executor) getRunStats() string { + status := "running" + if e.state.IsPaused() { + status = "paused" } - - return &Executor{ - Runner: r, - Logger: log.StandardLogger(), - runSetup: true, - runTeardown: true, - endIters: -1, - endTime: -1, - vuOut: make(chan stats.SampleContainer, bufferSize), - iterDone: make(chan struct{}), + if e.state.HasStarted() { + dur := e.state.GetCurrentTestRunDuration() + status = fmt.Sprintf("%s (%s)", status, pb.GetFixedLengthDuration(dur, e.maxDuration)) } -} -func (e *Executor) Run(parent context.Context, engineOut chan<- stats.SampleContainer) (reterr error) { - e.runLock.Lock() - defer e.runLock.Unlock() + vusFmt := pb.GetFixedLengthIntFormat(int64(e.maxPossibleVUs)) + return fmt.Sprintf( + "%s, "+vusFmt+"/"+vusFmt+" VUs, %d complete and %d incomplete iterations", + status, e.state.GetCurrentlyActiveVUsCount(), e.state.GetInitializedVUsCount(), + e.state.GetFullIterationCount(), e.state.GetPartialIterationCount(), + ) +} - if e.Runner != nil && e.runSetup { - if err := e.Runner.Setup(parent, engineOut); err != nil { - return err +// Init concurrently initializes all of the planned VUs and then sequentially +// initializes all of the configured schedulers. +func (e *Executor) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { + logger := e.logger.WithField("phase", "local-executor-init") + + vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) + logger.WithFields(logrus.Fields{ + "neededVUs": vusToInitialize, + "schedulersCount": len(e.schedulers), + }).Debugf("Start of initialization") + + doneInits := make(chan error, vusToInitialize) // poor man's early-return waitgroup + //TODO: make this an option? + initConcurrency := runtime.NumCPU() + limiter := make(chan struct{}, initConcurrency) + subctx, cancel := context.WithCancel(ctx) + defer cancel() + + initPlannedVU := func() { + newVU, err := e.initVU(ctx, logger, engineOut) + if err == nil { + e.state.AddInitializedVU(newVU) + <-limiter } + doneInits <- err } - ctx, cancel := context.WithCancel(parent) - vuFlow := make(chan int64) - e.lock.Lock() - vuOut := e.vuOut - iterDone := e.iterDone - e.ctx = ctx - e.flow = vuFlow - e.lock.Unlock() - - var cutoff time.Time - defer func() { - if e.Runner != nil && e.runTeardown { - err := e.Runner.Teardown(parent, engineOut) - if reterr == nil { - reterr = err - } else if err != nil { - reterr = fmt.Errorf("teardown error %#v\nPrevious error: %#v", err, reterr) - } - } - - close(vuFlow) - cancel() - - e.lock.Lock() - e.ctx = nil - e.vuOut = nil - e.flow = nil - e.lock.Unlock() - - wait := make(chan interface{}) - go func() { - e.wg.Wait() - close(wait) - }() - - for { + go func() { + for vuNum := uint64(0); vuNum < vusToInitialize; vuNum++ { 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 - } else if csc, ok := newSampleContainer.(stats.ConnectedSampleContainer); ok && csc.GetTime().Before(cutoff) { - engineOut <- newSampleContainer - } else { - for _, s := range newSampleContainer.GetSamples() { - if s.Time.Before(cutoff) { - engineOut <- s - } - } - } - case <-wait: - } - select { - case <-wait: - close(vuOut) + case limiter <- struct{}{}: + go initPlannedVU() + case <-subctx.Done(): return - default: } } }() - startVUs := atomic.LoadInt64(&e.numVUs) - if err := e.scale(ctx, lib.Max(0, startVUs)); err != nil { - return err - } - - ticker := time.NewTicker(1 * time.Millisecond) - defer ticker.Stop() - - lastTick := time.Now() - for { - // If the test is paused, sleep until either the pause or the test ends. - // Also shift the last tick to omit time spent paused, but not partial ticks. - e.pauseLock.RLock() - pause := e.pause - e.pauseLock.RUnlock() - if pause != nil { - e.Logger.Debug("Local: Pausing!") - leftovers := time.Since(lastTick) - select { - case <-pause: - e.Logger.Debug("Local: No longer paused") - lastTick = time.Now().Add(-leftovers) - case <-ctx.Done(): - e.Logger.Debug("Local: Terminated while in paused state") - return nil - } - } - - // Dumb hack: we don't wanna start any more iterations than the max, but we can't - // conditionally select on a channel either...so, we cheat: swap out the flow channel for a - // nil channel (writing to nil always blocks) if we don't wanna write an iteration. - flow := vuFlow - end := atomic.LoadInt64(&e.endIters) - partials := atomic.LoadInt64(&e.partIters) - if end >= 0 && partials >= end { - flow = nil - } - + initializedVUs := new(uint64) + vusFmt := pb.GetFixedLengthIntFormat(int64(vusToInitialize)) + e.initProgress.Modify( + pb.WithProgress(func() (float64, string) { + doneVUs := atomic.LoadUint64(initializedVUs) + return float64(doneVUs) / float64(vusToInitialize), + fmt.Sprintf(vusFmt+"/%d VUs initialized", doneVUs, vusToInitialize) + }), + ) + + for vuNum := uint64(0); vuNum < vusToInitialize; vuNum++ { select { - case flow <- partials: - // Start an iteration if there's a VU waiting. See also: the big comment block above. - atomic.AddInt64(&e.partIters, 1) - case t := <-ticker.C: - // Every tick, increment the clock, see if we passed the end point, and process stages. - // If the test ends this way, set a cutoff point; any samples collected past the cutoff - // point are excluded. - d := t.Sub(lastTick) - lastTick = t - - end := time.Duration(atomic.LoadInt64(&e.endTime)) - at := time.Duration(atomic.AddInt64(&e.time, int64(d))) - if end >= 0 && at >= end { - e.Logger.WithFields(log.Fields{"at": at, "end": end}).Debug("Local: Hit time limit") - cutoff = time.Now() - return nil - } - - stages := e.stages - if len(stages) > 0 { - vus, keepRunning := ProcessStages(startVUs, stages, at) - if !keepRunning { - e.Logger.WithField("at", at).Debug("Local: Ran out of stages") - cutoff = time.Now() - return nil - } - if vus.Valid { - if err := e.SetVUs(vus.Int64); err != nil { - return err - } - } - } - 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! - 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) - at := atomic.AddInt64(&e.iters, 1) - if end >= 0 && at >= end { - e.Logger.WithFields(log.Fields{"at": at, "end": end}).Debug("Local: Hit iteration limit") - return nil + case err := <-doneInits: + if err != nil { + return err } + atomic.AddUint64(initializedVUs, 1) case <-ctx.Done(): - // If the test is cancelled, just set the cutoff point to now and proceed down the same - // logic as if the time limit was hit. - e.Logger.Debug("Local: Exiting with context") - cutoff = time.Now() - return nil + return ctx.Err() } } -} -func (e *Executor) scale(ctx context.Context, num int64) error { - e.Logger.WithField("num", num).Debug("Local: Scaling...") - - e.vusLock.Lock() - defer e.vusLock.Unlock() - - e.lock.RLock() - flow := e.flow - iterDone := e.iterDone - e.lock.RUnlock() - - for i, handle := range e.vus { - handle := handle - handle.RLock() - cancel := handle.cancel - handle.RUnlock() - - if i < int(num) { - if cancel == nil { - vuctx, cancel := context.WithCancel(ctx) - handle.Lock() - handle.ctx = vuctx - handle.cancel = cancel - handle.Unlock() - - if handle.vu != nil { - if err := handle.vu.Reconfigure(atomic.AddInt64(&e.nextVUID, 1)); err != nil { - return err - } - } - - e.wg.Add(1) - go func() { - handle.run(e.Logger, flow, iterDone) - e.wg.Done() - }() - } - } else if cancel != nil { - handle.Lock() - handle.cancel() - handle.cancel = nil - handle.Unlock() + e.state.SetInitVUFunc(func(ctx context.Context, logger *logrus.Entry) (lib.VU, error) { + return e.initVU(ctx, logger, engineOut) + }) + + logger.Debugf("Finished initializing needed VUs, start initializing schedulers...") + for _, sched := range e.schedulers { + schedConfig := sched.GetConfig() + + if err := sched.Init(ctx); err != nil { + return fmt.Errorf("error while initializing scheduler %s: %s", schedConfig.GetName(), err) } + logger.Debugf("Initialized scheduler %s", schedConfig.GetName()) } - atomic.StoreInt64(&e.numVUs, num) + logger.Debugf("Initization completed") return nil } -func (e *Executor) IsRunning() bool { - e.lock.RLock() - defer e.lock.RUnlock() - return e.ctx != nil -} - -func (e *Executor) GetRunner() lib.Runner { - return e.Runner -} - -func (e *Executor) SetLogger(l *log.Logger) { - e.Logger = l -} - -func (e *Executor) GetLogger() *log.Logger { - return e.Logger -} - -func (e *Executor) GetStages() []lib.Stage { - return e.stages -} +// Run the Executor, funneling all generated metric samples through the supplied +// out channel. +func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error { + schedulersCount := len(e.schedulers) + logger := e.logger.WithField("phase", "local-executor-run") + e.initProgress.Modify(pb.WithConstLeft("Run")) -func (e *Executor) SetStages(s []lib.Stage) { - e.stages = s -} - -func (e *Executor) GetIterations() int64 { - return atomic.LoadInt64(&e.iters) -} - -func (e *Executor) GetEndIterations() null.Int { - v := atomic.LoadInt64(&e.endIters) - if v < 0 { - return null.Int{} - } - return null.IntFrom(v) -} - -func (e *Executor) SetEndIterations(i null.Int) { - if !i.Valid { - i.Int64 = -1 + if e.state.IsPaused() { + logger.Debug("Execution is paused, waiting for resume or interrupt...") + e.initProgress.Modify(pb.WithConstProgress(1, "paused")) + select { + case <-e.state.ResumeNotify(): + // continue + case <-ctx.Done(): + return nil + } } - e.Logger.WithField("i", i.Int64).Debug("Local: Setting end iterations") - atomic.StoreInt64(&e.endIters, i.Int64) -} -func (e *Executor) GetTime() time.Duration { - return time.Duration(atomic.LoadInt64(&e.time)) -} - -func (e *Executor) GetEndTime() types.NullDuration { - v := atomic.LoadInt64(&e.endTime) - if v < 0 { - return types.NullDuration{} - } - return types.NullDurationFrom(time.Duration(v)) -} + e.state.MarkStarted() + defer e.state.MarkEnded() + e.initProgress.Modify(pb.WithConstProgress(1, "running")) -func (e *Executor) SetEndTime(t types.NullDuration) { - if !t.Valid { - t.Duration = -1 - } - e.Logger.WithField("d", t.Duration).Debug("Local: Setting end time") - atomic.StoreInt64(&e.endTime, int64(t.Duration)) -} + logger.WithFields(logrus.Fields{"schedulersCount": schedulersCount}).Debugf("Start of test run") -func (e *Executor) IsPaused() bool { - e.pauseLock.RLock() - defer e.pauseLock.RUnlock() - return e.pause != nil -} + runResults := make(chan error, schedulersCount) // nil values are successful runs -func (e *Executor) SetPaused(paused bool) { - e.Logger.WithField("paused", paused).Debug("Local: Setting paused") - e.pauseLock.Lock() - defer e.pauseLock.Unlock() + runCtx, cancel := context.WithCancel(ctx) + defer cancel() // just in case, and to shut up go vet... - if paused && e.pause == nil { - e.pause = make(chan interface{}) - } else if !paused && e.pause != nil { - close(e.pause) - e.pause = nil + // Run setup() before any schedulers, if it's not disabled + if !e.options.NoSetup.Bool { + logger.Debug("Running setup()") + e.initProgress.Modify(pb.WithConstProgress(1, "setup()")) + if err := e.runner.Setup(runCtx, engineOut); err != nil { + logger.WithField("error", err).Debug("setup() aborted by error") + return err + } } -} - -func (e *Executor) GetVUs() int64 { - return atomic.LoadInt64(&e.numVUs) -} + e.initProgress.Modify(pb.WithHijack(e.getRunStats)) + + runCtxDone := runCtx.Done() + runScheduler := func(sched lib.Scheduler) { + schedConfig := sched.GetConfig() + schedStartTime := schedConfig.GetStartTime() + schedLogger := logger.WithFields(logrus.Fields{ + "scheduler": schedConfig.GetName(), + "type": schedConfig.GetType(), + "startTime": schedStartTime, + }) + schedProgress := sched.GetProgress() + + // Check if we have to wait before starting the actual scheduler execution + if schedStartTime > 0 { + startTime := time.Now() + schedProgress.Modify(pb.WithProgress(func() (float64, string) { + remWait := (schedStartTime - time.Since(startTime)) + return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, schedStartTime)) + })) + + schedLogger.Debugf("Waiting for scheduler start time...") + select { + case <-runCtxDone: + runResults <- nil // no error since scheduler hasn't started yet + return + case <-time.After(schedStartTime): + // continue + } + } -func (e *Executor) SetVUs(num int64) error { - if num < 0 { - return errors.New("vu count can't be negative") + schedProgress.Modify(pb.WithConstProgress(0, "started")) + schedLogger.Debugf("Starting scheduler") + err := sched.Run(runCtx, engineOut) // scheduler should handle context cancel itself + if err == nil { + schedLogger.Debugf("Scheduler finished successfully") + } else { + schedLogger.WithField("error", err).Errorf("Scheduler error") + } + runResults <- err } - if atomic.LoadInt64(&e.numVUs) == num { - return nil + // Start all schedulers at their particular startTime in a separate goroutine... + logger.Debug("Start all schedulers...") + for _, sched := range e.schedulers { + go runScheduler(sched) } - e.Logger.WithField("vus", num).Debug("Local: Setting VUs") - - if numVUsMax := atomic.LoadInt64(&e.numVUsMax); num > numVUsMax { - return errors.Errorf("can't raise vu count (to %d) above vu cap (%d)", num, numVUsMax) + // Wait for all schedulers to finish + var firstErr error + for range e.schedulers { + err := <-runResults + if err != nil && firstErr == nil { + firstErr = err + cancel() + } } - if ctx := e.ctx; ctx != nil { - if err := e.scale(ctx, num); err != nil { + // Run teardown() after all schedulers are done, if it's not disabled + if !e.options.NoTeardown.Bool { + logger.Debug("Running teardown()") + if err := e.runner.Teardown(ctx, engineOut); err != nil { + logger.WithField("error", err).Debug("teardown() aborted by error") return err } - } else { - atomic.StoreInt64(&e.numVUs, num) } - return nil -} - -func (e *Executor) GetVUsMax() int64 { - return atomic.LoadInt64(&e.numVUsMax) + return firstErr } -func (e *Executor) SetVUsMax(max int64) error { - e.Logger.WithField("max", max).Debug("Local: Setting max VUs") - if max < 0 { - return errors.New("vu cap can't be negative") - } - - numVUsMax := atomic.LoadInt64(&e.numVUsMax) - - if numVUsMax == max { - return nil - } - - if numVUs := atomic.LoadInt64(&e.numVUs); max < numVUs { - return errors.Errorf("can't lower vu cap (to %d) below vu count (%d)", max, numVUs) - } - - if max < numVUsMax { - e.vus = e.vus[:max] - atomic.StoreInt64(&e.numVUsMax, max) - return nil +// SetPaused pauses a test, if called with true. And if called with +// false, tries to start/resume it. See the lib.Executor interface documentation +// of the methods for the various caveats about its usage. +func (e *Executor) SetPaused(pause bool) error { + if !e.state.HasStarted() && e.state.IsPaused() { + if pause { + return fmt.Errorf("execution is already paused") + } + e.logger.Debug("Starting execution") + return e.state.Resume() } - e.lock.RLock() - vuOut := e.vuOut - e.lock.RUnlock() - - e.vusLock.Lock() - defer e.vusLock.Unlock() - - vus := e.vus - for i := numVUsMax; i < max; i++ { - var handle vuHandle - if e.Runner != nil { - vu, err := e.Runner.NewVU(vuOut) - if err != nil { - return err - } - handle.vu = vu + for _, sched := range e.schedulers { + if !sched.IsPausable() { + return fmt.Errorf( + "%s scheduler '%s' doesn't support pause and resume operations after its start", + sched.GetConfig().GetType(), sched.GetConfig().GetName(), + ) + } + if err := sched.LiveUpdate(pause, nil); err != nil { + return err } - vus = append(vus, &handle) } - e.vus = vus - - atomic.StoreInt64(&e.numVUsMax, max) - return nil } - -func (e *Executor) SetRunSetup(r bool) { - e.runSetup = r -} - -func (e *Executor) SetRunTeardown(r bool) { - e.runTeardown = r -} diff --git a/core/local/local_test.go b/core/local/local_test.go index 23c27d88179..8bc507836bd 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -22,20 +22,22 @@ package local import ( "context" + "errors" "net" "runtime" "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/netext" + "github.com/loadimpact/k6/lib/scheduler" + "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" - "github.com/pkg/errors" + "github.com/sirupsen/logrus" logtest "github.com/sirupsen/logrus/hooks/test" "github.com/spf13/afero" "github.com/stretchr/testify/assert" @@ -43,30 +45,61 @@ import ( null "gopkg.in/guregu/null.v3" ) -func TestExecutorRun(t *testing.T) { - e := New(nil) - assert.NoError(t, e.SetVUsMax(10)) - assert.NoError(t, e.SetVUs(10)) +func newTestExecutor( + t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, //nolint: golint +) (ctx context.Context, cancel func(), executor *Executor, samples chan stats.SampleContainer) { + if runner == nil { + runner = &lib.MiniRunner{} + } + ctx, cancel = context.WithCancel(context.Background()) + newOpts, err := scheduler.BuildExecutionConfig(lib.Options{ + MetricSamplesBufferSize: null.NewInt(200, false), + }.Apply(runner.GetOptions()).Apply(opts)) + require.NoError(t, err) + require.Empty(t, newOpts.Validate()) - ctx, cancel := context.WithCancel(context.Background()) - err := make(chan error, 1) - samples := make(chan stats.SampleContainer, 100) - defer close(samples) + require.NoError(t, runner.SetOptions(newOpts)) + + if logger == nil { + logger = logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + } + + executor, err = New(runner, logger) + require.NoError(t, err) + + samples = make(chan stats.SampleContainer, newOpts.MetricSamplesBufferSize.Int64) go func() { - for range samples { + for { + select { + case <-samples: + case <-ctx.Done(): + return + } } }() - go func() { err <- e.Run(ctx, samples) }() - cancel() + require.NoError(t, executor.Init(ctx, samples)) + + return ctx, cancel, executor, samples +} + +func TestExecutorRun(t *testing.T) { + t.Parallel() + ctx, cancel, executor, samples := newTestExecutor(t, nil, nil, lib.Options{}) + defer cancel() + + err := make(chan error, 1) + go func() { err <- executor.Run(ctx, samples) }() assert.NoError(t, <-err) } func TestExecutorSetupTeardownRun(t *testing.T) { + t.Parallel() t.Run("Normal", func(t *testing.T) { setupC := make(chan struct{}) teardownC := make(chan struct{}) - e := New(&lib.MiniRunner{ + runner := &lib.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { close(setupC) return nil, nil @@ -75,214 +108,259 @@ func TestExecutorSetupTeardownRun(t *testing.T) { close(teardownC) return nil }, - }) + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{}) - ctx, cancel := context.WithCancel(context.Background()) err := make(chan error, 1) - go func() { err <- e.Run(ctx, make(chan stats.SampleContainer, 100)) }() - cancel() + go func() { err <- executor.Run(ctx, samples) }() + defer cancel() <-setupC <-teardownC assert.NoError(t, <-err) }) t.Run("Setup Error", func(t *testing.T) { - e := New(&lib.MiniRunner{ + runner := &lib.MiniRunner{ + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { + return nil, errors.New("setup error") + }, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{}) + defer cancel() + assert.EqualError(t, executor.Run(ctx, samples), "setup error") + }) + t.Run("Don't Run Setup", func(t *testing.T) { + runner := &lib.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, errors.New("setup error") }, TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + NoSetup: null.BoolFrom(true), + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), }) - 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, out chan<- stats.SampleContainer) ([]byte, error) { - return nil, errors.New("setup error") - }, - TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { - return errors.New("teardown error") - }, - }) - e.SetRunSetup(false) - e.SetEndIterations(null.IntFrom(1)) - assert.NoError(t, e.SetVUsMax(1)) - assert.NoError(t, e.SetVUs(1)) - assert.EqualError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100)), "teardown error") - }) + defer cancel() + assert.EqualError(t, executor.Run(ctx, samples), "teardown error") }) + t.Run("Teardown Error", func(t *testing.T) { - e := New(&lib.MiniRunner{ + runner := &lib.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, nil }, TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return errors.New("teardown error") }, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), }) - e.SetEndIterations(null.IntFrom(1)) - assert.NoError(t, e.SetVUsMax(1)) - assert.NoError(t, e.SetVUs(1)) - 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, out chan<- stats.SampleContainer) ([]byte, error) { - return nil, nil - }, - TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { - return errors.New("teardown error") - }, - }) - e.SetRunTeardown(false) - e.SetEndIterations(null.IntFrom(1)) - assert.NoError(t, e.SetVUsMax(1)) - assert.NoError(t, e.SetVUs(1)) - assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 100))) + defer cancel() + + assert.EqualError(t, executor.Run(ctx, samples), "teardown error") + }) + t.Run("Don't Run Teardown", func(t *testing.T) { + runner := &lib.MiniRunner{ + SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { + return nil, nil + }, + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + return errors.New("teardown error") + }, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + NoTeardown: null.BoolFrom(true), + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), }) + defer cancel() + assert.NoError(t, executor.Run(ctx, samples)) }) } -func TestExecutorSetLogger(t *testing.T) { - logger, _ := logtest.NewNullLogger() - e := New(nil) - e.SetLogger(logger) - assert.Equal(t, logger, e.GetLogger()) -} - func TestExecutorStages(t *testing.T) { + t.Parallel() testdata := map[string]struct { Duration time.Duration Stages []lib.Stage }{ "one": { 1 * time.Second, - []lib.Stage{{Duration: types.NullDurationFrom(1 * time.Second)}}, + []lib.Stage{{Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1)}}, }, "two": { 2 * time.Second, []lib.Stage{ - {Duration: types.NullDurationFrom(1 * time.Second)}, - {Duration: types.NullDurationFrom(1 * time.Second)}, + {Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1)}, + {Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(2)}, }, }, - "two/targeted": { - 2 * time.Second, + "four": { + 4 * time.Second, []lib.Stage{ {Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(5)}, - {Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(10)}, + {Duration: types.NullDurationFrom(3 * time.Second), Target: null.IntFrom(10)}, }, }, } for name, data := range testdata { + data := data t.Run(name, func(t *testing.T) { - e := New(&lib.MiniRunner{ + t.Parallel() + runner := &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), - }, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + VUs: null.IntFrom(1), + Stages: data.Stages, }) - assert.NoError(t, e.SetVUsMax(10)) - e.SetStages(data.Stages) - assert.NoError(t, e.Run(context.Background(), make(chan stats.SampleContainer, 500))) - assert.True(t, e.GetTime() >= data.Duration) + defer cancel() + assert.NoError(t, executor.Run(ctx, samples)) + assert.True(t, executor.GetState().GetCurrentTestRunDuration() >= data.Duration) }) } } func TestExecutorEndTime(t *testing.T) { - e := New(&lib.MiniRunner{ + t.Parallel() + runner := &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)}, + } + ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + VUs: null.IntFrom(10), + Duration: types.NullDurationFrom(1 * time.Second), }) - 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()) + defer cancel() + + endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + assert.Equal(t, 31*time.Second, endTime) // because of the default 30s gracefulStop + assert.True(t, isFinal) startTime := time.Now() - 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, 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)) - assert.Equal(t, types.NullDurationFrom(100*time.Millisecond), e.GetEndTime()) + assert.NoError(t, executor.Run(ctx, samples)) + runTime := time.Since(startTime) + assert.True(t, runTime > 1*time.Second, "test did not take 1s") + assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") +} + +func TestExecutorRuntimeErrors(t *testing.T) { + t.Parallel() + runner := &lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(10 * time.Millisecond) + return errors.New("hi") + }, + Options: lib.Options{ + VUs: null.IntFrom(10), + Duration: types.NullDurationFrom(1 * time.Second), + }, + } + logger, hook := logtest.NewNullLogger() + ctx, cancel, executor, samples := newTestExecutor(t, runner, logger, lib.Options{}) + defer cancel() - l, hook := logtest.NewNullLogger() - e.SetLogger(l) + endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + assert.Equal(t, 31*time.Second, endTime) // because of the default 30s gracefulStop + assert.True(t, isFinal) - startTime := time.Now() - 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") + startTime := time.Now() + assert.NoError(t, executor.Run(ctx, samples)) + runTime := time.Since(startTime) + assert.True(t, runTime > 1*time.Second, "test did not take 1s") + assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") + + assert.NotEmpty(t, hook.Entries) + for _, e := range hook.Entries { + assert.Equal(t, "hi", e.Message) + } +} - assert.NotEmpty(t, hook.Entries) - for _, e := range hook.Entries { - assert.Equal(t, "hi", e.Message) - } - }) +func TestExecutorEndErrors(t *testing.T) { + t.Parallel() - t.Run("End Errors", func(t *testing.T) { - 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)) - assert.Equal(t, types.NullDurationFrom(100*time.Millisecond), e.GetEndTime()) + scheduler := scheduler.NewConstantLoopingVUsConfig("we_need_hard_stop") + scheduler.VUs = null.IntFrom(10) + scheduler.Duration = types.NullDurationFrom(1 * time.Second) + scheduler.GracefulStop = types.NullDurationFrom(0 * time.Second) - l, hook := logtest.NewNullLogger() - e.SetLogger(l) + runner := &lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + <-ctx.Done() + return errors.New("hi") + }, + Options: lib.Options{ + Execution: lib.SchedulerConfigMap{scheduler.GetName(): scheduler}, + }, + } + logger, hook := logtest.NewNullLogger() + ctx, cancel, executor, samples := newTestExecutor(t, runner, logger, lib.Options{}) + defer cancel() - startTime := time.Now() - 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") + endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + assert.Equal(t, 1*time.Second, endTime) // because of the 0s gracefulStop + assert.True(t, isFinal) - assert.Empty(t, hook.Entries) - }) + startTime := time.Now() + assert.NoError(t, executor.Run(ctx, samples)) + runTime := time.Since(startTime) + assert.True(t, runTime > 1*time.Second, "test did not take 1s") + assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") + + assert.Empty(t, hook.Entries) } func TestExecutorEndIterations(t *testing.T) { + t.Parallel() metric := &stats.Metric{Name: "test_metric"} + options, err := scheduler.BuildExecutionConfig(lib.Options{ + VUs: null.IntFrom(1), + Iterations: null.IntFrom(100), + }) + require.NoError(t, err) + require.Empty(t, options.Validate()) + var i int64 - e := New(&lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { - select { - case <-ctx.Done(): - default: - atomic.AddInt64(&i, 1) - } - 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, 201) - assert.NoError(t, e.Run(context.Background(), samples)) - assert.Equal(t, int64(100), e.GetIterations()) + runner := &lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + select { + case <-ctx.Done(): + default: + atomic.AddInt64(&i, 1) + } + out <- stats.Sample{Metric: metric, Value: 1.0} + return nil + }, + Options: options, + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + + executor, err := New(runner, logger) + require.NoError(t, err) + + samples := make(chan stats.SampleContainer, 300) + require.NoError(t, executor.Init(ctx, samples)) + require.NoError(t, executor.Run(ctx, samples)) + + assert.Equal(t, uint64(100), executor.GetState().GetFullIterationCount()) + assert.Equal(t, uint64(0), executor.GetState().GetPartialIterationCount()) assert.Equal(t, int64(100), i) + require.Equal(t, 200, len(samples)) for i := 0; i < 100; i++ { mySample, ok := <-samples require.True(t, ok) @@ -297,53 +375,30 @@ func TestExecutorEndIterations(t *testing.T) { } func TestExecutorIsRunning(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - e := New(nil) + t.Parallel() + runner := &lib.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + <-ctx.Done() + return nil + }, + } + ctx, cancel, executor, _ := newTestExecutor(t, runner, nil, lib.Options{}) + state := executor.GetState() err := make(chan error) - go func() { err <- e.Run(ctx, nil) }() - for !e.IsRunning() { + go func() { err <- executor.Run(ctx, nil) }() + for !state.HasStarted() { + time.Sleep(10 * time.Microsecond) } cancel() - for e.IsRunning() { + for !state.HasEnded() { + time.Sleep(10 * time.Microsecond) } assert.NoError(t, <-err) } -func TestExecutorSetVUsMax(t *testing.T) { - t.Run("Negative", func(t *testing.T) { - assert.EqualError(t, New(nil).SetVUsMax(-1), "vu cap can't be negative") - }) - - t.Run("Raise", func(t *testing.T) { - e := New(nil) - - assert.NoError(t, e.SetVUsMax(50)) - assert.Equal(t, int64(50), e.GetVUsMax()) - - assert.NoError(t, e.SetVUsMax(100)) - assert.Equal(t, int64(100), e.GetVUsMax()) - - t.Run("Lower", func(t *testing.T) { - assert.NoError(t, e.SetVUsMax(50)) - assert.Equal(t, int64(50), e.GetVUsMax()) - }) - }) - - t.Run("TooLow", func(t *testing.T) { - e := New(nil) - e.ctx = context.Background() - - assert.NoError(t, e.SetVUsMax(100)) - assert.Equal(t, int64(100), e.GetVUsMax()) - - assert.NoError(t, e.SetVUs(100)) - assert.Equal(t, int64(100), e.GetVUs()) - - assert.EqualError(t, e.SetVUsMax(50), "can't lower vu cap (to 50) below vu count (100)") - }) -} - +/* +//TODO: convert for the manual-execution scheduler func TestExecutorSetVUs(t *testing.T) { t.Run("Negative", func(t *testing.T) { assert.EqualError(t, New(nil).SetVUs(-1), "vu count can't be negative") @@ -437,6 +492,7 @@ func TestExecutorSetVUs(t *testing.T) { }) }) } +*/ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { if runtime.GOOS == "windows" { @@ -487,23 +543,29 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { ) require.NoError(t, err) - options := lib.Options{ + options, err := scheduler.BuildExecutionConfig(lib.Options{ + Iterations: null.IntFrom(2), + VUs: null.IntFrom(1), SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), SetupTimeout: types.NullDurationFrom(4 * time.Second), TeardownTimeout: types.NullDurationFrom(4 * time.Second), - } - runner.SetOptions(options) + }.Apply(runner.GetOptions())) + require.NoError(t, err) + require.NoError(t, runner.SetOptions(options)) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) - executor := New(runner) - executor.SetEndIterations(null.IntFrom(2)) - require.NoError(t, executor.SetVUsMax(1)) - require.NoError(t, executor.SetVUs(1)) + executor, err := New(runner, logger) + require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() + done := make(chan struct{}) sampleContainers := make(chan stats.SampleContainer) go func() { + require.NoError(t, executor.Init(ctx, sampleContainers)) assert.NoError(t, executor.Run(ctx, sampleContainers)) close(done) }() diff --git a/core/local/util.go b/core/local/util.go deleted file mode 100644 index 612021af378..00000000000 --- a/core/local/util.go +++ /dev/null @@ -1,64 +0,0 @@ -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2016 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package local - -import ( - "time" - - "github.com/loadimpact/k6/lib" - "gopkg.in/guregu/null.v3" -) - -// Returns the VU count and whether to keep going at the specified time. -func ProcessStages(startVUs int64, stages []lib.Stage, t time.Duration) (null.Int, bool) { - vus := null.NewInt(startVUs, false) - - var start time.Duration - for _, stage := range stages { - // Infinite stages keep running forever, with the last valid end point, or its own target. - if !stage.Duration.Valid { - if stage.Target.Valid { - vus = stage.Target - } - return vus, true - } - - // If the stage has already ended, still record the end VU count for interpolation. - end := start + time.Duration(stage.Duration.Duration) - if end < t { - if stage.Target.Valid { - vus = stage.Target - } - start = end - continue - } - - // If there's a VU target, use linear interpolation to reach it. - if stage.Target.Valid { - prog := lib.Clampf(float64(t-start)/float64(stage.Duration.Duration), 0.0, 1.0) - vus = null.IntFrom(lib.Lerp(vus.Int64, stage.Target.Int64, prog)) - } - - // We found a stage, so keep running. - return vus, true - } - return vus, false -} diff --git a/core/local/util_test.go b/core/local/util_test.go index 1a0733cc022..6801c5b3620 100644 --- a/core/local/util_test.go +++ b/core/local/util_test.go @@ -20,16 +20,8 @@ package local -import ( - "testing" - "time" - - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" - "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" -) - +//TODO: translate this test to the new paradigm +/* func TestProcessStages(t *testing.T) { type checkpoint struct { D time.Duration @@ -291,3 +283,4 @@ func TestProcessStages(t *testing.T) { }) } } +*/ From dc87fed736c092383daed1a30106ba92866cdfd9 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:51:21 +0300 Subject: [PATCH 010/350] Update cmd/run.go to use the latest changes --- cmd/run.go | 257 +++++++++++++++++++++++------------------------------ 1 file changed, 110 insertions(+), 147 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 7872f293b20..b66e8037f00 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -24,16 +24,12 @@ import ( "archive/tar" "bytes" "context" - "encoding/json" - "fmt" "io" "io/ioutil" - "net/http" "os" "os/signal" "path/filepath" - "runtime" - "strings" + "sync" "syscall" "time" @@ -42,15 +38,14 @@ import ( "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/ui" + "github.com/loadimpact/k6/ui/pb" "github.com/pkg/errors" - log "github.com/sirupsen/logrus" + "github.com/sirupsen/logrus" "github.com/spf13/afero" "github.com/spf13/cobra" "github.com/spf13/pflag" - null "gopkg.in/guregu/null.v3" ) const ( @@ -65,7 +60,7 @@ const ( invalidConfigErrorCode = 104 ) - //TODO: fix this, global variables are not very testable... +//TODO: fix this, global variables are not very testable... //nolint:gochecknoglobals var runType = os.Getenv("K6_TYPE") @@ -100,13 +95,10 @@ a commandline interface for interacting with it.`, //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", Banner) - initBar := ui.ProgressBar{ - Width: 60, - Left: func() string { return " init" }, - } + initBar := pb.New(pb.WithConstLeft(" init")) // Create the Runner. - fprintf(stdout, "%s runner\r", initBar.String()) + fprintf(stdout, "%s runner\r", initBar.String()) //TODO pwd, err := os.Getwd() if err != nil { return err @@ -153,23 +145,33 @@ a commandline interface for interacting with it.`, return err } + //TODO: don't use a global... or maybe change the logger? + logger := logrus.StandardLogger() + + ctx, cancel := context.WithCancel(context.Background()) //TODO: move even earlier? + defer cancel() + // Create a local executor wrapping the runner. fprintf(stdout, "%s executor\r", initBar.String()) - ex := local.New(r) - if runNoSetup { - ex.SetRunSetup(false) - } - if runNoTeardown { - ex.SetRunTeardown(false) + executor, err := local.New(r, logger) + if err != nil { + return err } + executorState := executor.GetState() + initBar = executor.GetInitProgressBar() + progressBarWG := &sync.WaitGroup{} + progressBarWG.Add(1) + go showProgress(ctx, progressBarWG, conf, executor) + // Create an engine. - fprintf(stdout, "%s engine\r", initBar.String()) - engine, err := core.NewEngine(ex, conf.Options) + initBar.Modify(pb.WithConstProgress(0, "Init engine")) + engine, err := core.NewEngine(executor, conf.Options, logger) if err != nil { return err } + //TODO: the engine should just probably have a copy of the config... // Configure the engine. if conf.NoThresholds.Valid { engine.NoThresholds = conf.NoThresholds.Bool @@ -179,10 +181,10 @@ a commandline interface for interacting with it.`, } // Create a collector and assign it to the engine if requested. - fprintf(stdout, "%s collector\r", initBar.String()) + initBar.Modify(pb.WithConstProgress(0, "Init metric outputs")) for _, out := range conf.Out { t, arg := parseCollector(out) - collector, err := newCollector(t, arg, src, conf) + collector, err := newCollector(t, arg, src, conf, executor.GetExecutionPlan()) if err != nil { return err } @@ -193,12 +195,14 @@ a commandline interface for interacting with it.`, } // Create an API server. - fprintf(stdout, "%s server\r", initBar.String()) - go func() { - if err := api.ListenAndServe(address, engine); err != nil { - log.WithError(err).Warn("Error from API server") - } - }() + if address != "" { + initBar.Modify(pb.WithConstProgress(0, "Init API server")) + go func() { + if err := api.ListenAndServe(address, engine); err != nil { + logger.WithError(err).Warn("Error from API server") + } + }() + } // Write the big banner. { @@ -218,39 +222,38 @@ a commandline interface for interacting with it.`, } } - fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprint("local")) + fprintf(stdout, " executor: %s\n", ui.ValueColor.Sprint("local")) fprintf(stdout, " output: %s%s\n", ui.ValueColor.Sprint(out), ui.ExtraColor.Sprint(link)) fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) fprintf(stdout, "\n") - duration := ui.GrayColor.Sprint("-") - iterations := ui.GrayColor.Sprint("-") - if conf.Duration.Valid { - duration = ui.ValueColor.Sprint(conf.Duration.Duration) - } - if conf.Iterations.Valid { - iterations = ui.ValueColor.Sprint(conf.Iterations.Int64) - } - vus := ui.ValueColor.Sprint(conf.VUs.Int64) - max := ui.ValueColor.Sprint(conf.VUsMax.Int64) - - leftWidth := ui.StrWidth(duration) - if l := ui.StrWidth(vus); l > leftWidth { - leftWidth = l + plan := executor.GetExecutionPlan() + schedulers := executor.GetSchedulers() + maxDuration, _ := lib.GetEndOffset(plan) + + fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprintf( + "(%.2f%%) %d schedulers, %d max VUs, %s max duration (incl. graceful stop):", + conf.ExecutionSegment.FloatLength()*100, len(schedulers), + lib.GetMaxPossibleVUs(plan), maxDuration), + ) + for _, sched := range schedulers { + fprintf(stdout, " * %s: %s\n", + sched.GetConfig().GetName(), sched.GetConfig().GetDescription(conf.ExecutionSegment)) } - durationPad := strings.Repeat(" ", leftWidth-ui.StrWidth(duration)) - vusPad := strings.Repeat(" ", leftWidth-ui.StrWidth(vus)) - - fprintf(stdout, " duration: %s,%s iterations: %s\n", duration, durationPad, iterations) - fprintf(stdout, " vus: %s,%s max: %s\n", vus, vusPad, max) fprintf(stdout, "\n") } // Run the engine with a cancellable context. - fprintf(stdout, "%s starting\r", initBar.String()) - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { + initBar.Modify(pb.WithConstProgress(0, "Init VUs")) + if err := engine.Init(ctx); err != nil { + errC <- err + } else { + initBar.Modify(pb.WithConstProgress(0, "Start test")) + errC <- engine.Run(ctx) + } + }() // Trap Interrupts, SIGINTs and SIGTERMs. sigC := make(chan os.Signal, 1) @@ -259,64 +262,38 @@ a commandline interface for interacting with it.`, // If the user hasn't opted out: report usage. if !conf.NoUsageReport.Bool { - go func() { - u := "http://k6reports.loadimpact.com/" - mime := "application/json" - var endTSeconds float64 - if endT := engine.Executor.GetEndTime(); endT.Valid { - endTSeconds = time.Duration(endT.Duration).Seconds() - } - var stagesEndTSeconds float64 - if stagesEndT := lib.SumStages(engine.Executor.GetStages()); stagesEndT.Valid { - stagesEndTSeconds = time.Duration(stagesEndT.Duration).Seconds() - } - body, err := json.Marshal(map[string]interface{}{ - "k6_version": Version, - "vus_max": engine.Executor.GetVUsMax(), - "iterations": engine.Executor.GetEndIterations(), - "duration": endTSeconds, - "st_duration": stagesEndTSeconds, - "goos": runtime.GOOS, - "goarch": runtime.GOARCH, - }) - if err != nil { - panic(err) // This should never happen!! - } - _, _ = http.Post(u, mime, bytes.NewBuffer(body)) - }() - } - - // Prepare a progress bar. - progress := ui.ProgressBar{ - Width: 60, - Left: func() string { - if engine.Executor.IsPaused() { - return " paused" - } else if engine.Executor.IsRunning() { - return " running" - } else { - return " done" - } - }, - Right: func() string { - if endIt := engine.Executor.GetEndIterations(); endIt.Valid { - return fmt.Sprintf("%d / %d", engine.Executor.GetIterations(), endIt.Int64) - } - precision := 100 * time.Millisecond - atT := engine.Executor.GetTime() - stagesEndT := lib.SumStages(engine.Executor.GetStages()) - endT := engine.Executor.GetEndTime() - if !endT.Valid || (stagesEndT.Valid && endT.Duration > stagesEndT.Duration) { - endT = stagesEndT - } - if endT.Valid { - return fmt.Sprintf("%s / %s", - (atT/precision)*precision, - (time.Duration(endT.Duration)/precision)*precision, - ) - } - return ((atT / precision) * precision).String() - }, + //TODO: fix + //TODO: move to a separate function + /* + go func() { + u := "http://k6reports.loadimpact.com/" + mime := "application/json" + var endTSeconds float64 + if endT := engine.Executor.GetEndTime(); endT.Valid { + endTSeconds = time.Duration(endT.Duration).Seconds() + } + var stagesEndTSeconds float64 + if stagesEndT := lib.SumStages(engine.Executor.GetStages()); stagesEndT.Valid { + stagesEndTSeconds = time.Duration(stagesEndT.Duration).Seconds() + } + body, err := json.Marshal(map[string]interface{}{ + "k6_version": Version, + "vus_max": engine.Executor.GetVUsMax(), + "iterations": engine.Executor.GetEndIterations(), + "duration": endTSeconds, + "st_duration": stagesEndTSeconds, + "goos": runtime.GOOS, + "goarch": runtime.GOARCH, + }) + if err != nil { + panic(err) // This should never happen!! + } + if err != nil { + panic(err) // This should never happen!! + } + _, _ = http.Post(u, mime, bytes.NewBuffer(body)) + }() + */ } // Ticker for progress bar updates. Less frequent updates for non-TTYs, none if quiet. @@ -333,41 +310,25 @@ a commandline interface for interacting with it.`, select { case <-ticker.C: if quiet || !stdoutTTY { - l := log.WithFields(log.Fields{ - "t": engine.Executor.GetTime(), - "i": engine.Executor.GetIterations(), + l := logrus.WithFields(logrus.Fields{ + "t": executorState.GetCurrentTestRunDuration(), + "i": executorState.GetFullIterationCount(), }) fn := l.Info if quiet { fn = l.Debug } - if engine.Executor.IsPaused() { + if executorState.IsPaused() { fn("Paused") } else { fn("Running") } break } - - var prog float64 - if endIt := engine.Executor.GetEndIterations(); endIt.Valid { - prog = float64(engine.Executor.GetIterations()) / float64(endIt.Int64) - } else { - stagesEndT := lib.SumStages(engine.Executor.GetStages()) - endT := engine.Executor.GetEndTime() - if !endT.Valid || (stagesEndT.Valid && endT.Duration > stagesEndT.Duration) { - endT = stagesEndT - } - if endT.Valid { - prog = float64(engine.Executor.GetTime()) / float64(endT.Duration) - } - } - progress.Progress = prog - fprintf(stdout, "%s\x1b[0K\r", progress.String()) case err := <-errC: cancel() if err == nil { - log.Debug("Engine terminated cleanly") + logger.Debug("Engine terminated cleanly") break mainLoop } @@ -375,42 +336,44 @@ a commandline interface for interacting with it.`, case lib.TimeoutError: switch string(e) { case "setup": - log.WithError(err).Error("Setup timeout") + logger.WithError(err).Error("Setup timeout") return ExitCode{errors.New("Setup timeout"), setupTimeoutErrorCode} case "teardown": - log.WithError(err).Error("Teardown timeout") + logger.WithError(err).Error("Teardown timeout") return ExitCode{errors.New("Teardown timeout"), teardownTimeoutErrorCode} default: - log.WithError(err).Error("Engine timeout") + logger.WithError(err).Error("Engine timeout") return ExitCode{errors.New("Engine timeout"), genericTimeoutErrorCode} } default: - log.WithError(err).Error("Engine error") + logger.WithError(err).Error("Engine error") return ExitCode{errors.New("Engine Error"), genericEngineErrorCode} } case sig := <-sigC: - log.WithField("sig", sig).Debug("Exiting in response to signal") + logger.WithField("sig", sig).Debug("Exiting in response to signal") cancel() + //TODO: Actually exit on a second Ctrl+C, even if some of the iterations are stuck. + // This is currently problematic because of https://github.com/loadimpact/k6/issues/971, + // but with uninterruptible iterations it will be even more problematic. } } if quiet || !stdoutTTY { - e := log.WithFields(log.Fields{ - "t": engine.Executor.GetTime(), - "i": engine.Executor.GetIterations(), + e := logger.WithFields(logrus.Fields{ + "t": executorState.GetCurrentTestRunDuration(), + "i": executorState.GetFullIterationCount(), }) fn := e.Info if quiet { fn = e.Debug } fn("Test finished") - } else { - progress.Progress = 1 - fprintf(stdout, "%s\x1b[0K\n", progress.String()) } + progressBarWG.Wait() + // Warn if no iterations could be completed. - if engine.Executor.GetIterations() == 0 { - log.Warn("No data generated, because no script iterations finished, consider making the test duration longer") + if executorState.GetFullIterationCount() == 0 { + logger.Warn("No data generated, because no script iterations finished, consider making the test duration longer") } // Print the end-of-test summary. @@ -420,13 +383,13 @@ a commandline interface for interacting with it.`, Opts: conf.Options, Root: engine.Executor.GetRunner().GetDefaultGroup(), Metrics: engine.Metrics, - Time: engine.Executor.GetTime(), + Time: executorState.GetCurrentTestRunDuration(), }) fprintf(stdout, "\n") } if conf.Linger.Bool { - log.Info("Linger set; waiting for Ctrl+C...") + logger.Info("Linger set; waiting for Ctrl+C...") <-sigC } From c5e63020e6b5f0aa563fb1c33ba83944f113bd9e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:52:10 +0300 Subject: [PATCH 011/350] Fix most of the REST API --- api/server_test.go | 17 +++++++++------- api/v1/group_routes_test.go | 8 ++++++-- api/v1/metric_routes.go | 4 ++-- api/v1/metric_routes_test.go | 17 ++++++++++++---- api/v1/setup_teardown_routes_test.go | 10 ++++++---- api/v1/status.go | 9 +++++---- api/v1/status_routes.go | 30 +++++++++++++++++----------- api/v1/status_routes_test.go | 14 +++++++++---- 8 files changed, 70 insertions(+), 39 deletions(-) diff --git a/api/server_test.go b/api/server_test.go index f83010497b5..97d43fa10f9 100644 --- a/api/server_test.go +++ b/api/server_test.go @@ -26,12 +26,15 @@ import ( "net/http/httptest" "testing" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/api/common" "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" - log "github.com/sirupsen/logrus" logtest "github.com/sirupsen/logrus/hooks/test" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/urfave/negroni" ) @@ -51,7 +54,7 @@ func TestLogger(t *testing.T) { r := httptest.NewRequest(method, "http://example.com"+path, nil) l, hook := logtest.NewNullLogger() - l.Level = log.DebugLevel + l.Level = logrus.DebugLevel NewLogger(l)(negroni.NewResponseWriter(rw), r, testHTTPHandler) res := rw.Result() @@ -63,7 +66,7 @@ func TestLogger(t *testing.T) { } e := hook.LastEntry() - assert.Equal(t, log.DebugLevel, e.Level) + assert.Equal(t, logrus.DebugLevel, e.Level) assert.Equal(t, fmt.Sprintf("%s %s", method, path), e.Message) assert.Equal(t, http.StatusOK, e.Data["status"]) }) @@ -73,10 +76,10 @@ func TestLogger(t *testing.T) { } func TestWithEngine(t *testing.T) { - engine, err := core.NewEngine(nil, lib.Options{}) - if !assert.NoError(t, err) { - return - } + executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + require.NoError(t, err) rw := httptest.NewRecorder() r := httptest.NewRequest("GET", "http://example.com/", nil) diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index 17af242676f..1f4af119e0c 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -30,7 +30,9 @@ import ( "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/manyminds/api2go/jsonapi" + "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestGetGroups(t *testing.T) { @@ -41,8 +43,10 @@ func TestGetGroups(t *testing.T) { g2, err := g1.Group("group 2") assert.NoError(t, err) - engine, err := core.NewEngine(local.New(&lib.MiniRunner{Group: g0}), lib.Options{}) - assert.NoError(t, err) + executor, err := local.New(&lib.MiniRunner{Group: g0}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + require.NoError(t, err) t.Run("list", func(t *testing.T) { rw := httptest.NewRecorder() diff --git a/api/v1/metric_routes.go b/api/v1/metric_routes.go index 08016c9dda7..a38936b5e41 100644 --- a/api/v1/metric_routes.go +++ b/api/v1/metric_routes.go @@ -34,7 +34,7 @@ func HandleGetMetrics(rw http.ResponseWriter, r *http.Request, p httprouter.Para var t time.Duration if engine.Executor != nil { - t = engine.Executor.GetTime() + t = engine.Executor.GetState().GetCurrentTestRunDuration() } metrics := make([]Metric, 0) @@ -56,7 +56,7 @@ func HandleGetMetric(rw http.ResponseWriter, r *http.Request, p httprouter.Param var t time.Duration if engine.Executor != nil { - t = engine.Executor.GetTime() + t = engine.Executor.GetState().GetCurrentTestRunDuration() } var metric Metric diff --git a/api/v1/metric_routes_test.go b/api/v1/metric_routes_test.go index 265871fd9c6..390942cab35 100644 --- a/api/v1/metric_routes_test.go +++ b/api/v1/metric_routes_test.go @@ -26,7 +26,12 @@ import ( "net/http/httptest" "testing" + "github.com/stretchr/testify/require" + + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" "github.com/manyminds/api2go/jsonapi" @@ -35,8 +40,10 @@ import ( ) func TestGetMetrics(t *testing.T) { - engine, err := core.NewEngine(nil, lib.Options{}) - assert.NoError(t, err) + executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + require.NoError(t, err) engine.Metrics = map[string]*stats.Metric{ "my_metric": stats.New("my_metric", stats.Trend, stats.Time), @@ -74,8 +81,10 @@ func TestGetMetrics(t *testing.T) { } func TestGetMetric(t *testing.T) { - engine, err := core.NewEngine(nil, lib.Options{}) - assert.NoError(t, err) + executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + require.NoError(t, err) engine.Metrics = map[string]*stats.Metric{ "my_metric": stats.New("my_metric", stats.Trend, stats.Time), diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index ef352d6a260..0e5dd89d21d 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -29,6 +29,8 @@ import ( "testing" "time" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js" @@ -140,14 +142,14 @@ func TestSetupData(t *testing.T) { runner.SetOptions(lib.Options{ Paused: null.BoolFrom(true), VUs: null.IntFrom(2), - VUsMax: null.IntFrom(2), Iterations: null.IntFrom(3), + NoSetup: null.BoolFrom(true), SetupTimeout: types.NullDurationFrom(1 * time.Second), TeardownTimeout: types.NullDurationFrom(1 * time.Second), }) - executor := local.New(runner) - executor.SetRunSetup(false) - engine, err := core.NewEngine(executor, runner.GetOptions()) + executor, err := local.New(runner, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, runner.GetOptions(), logrus.StandardLogger()) require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) diff --git a/api/v1/status.go b/api/v1/status.go index 3472e25cf1d..e7a3cee49b1 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -36,11 +36,12 @@ type Status struct { } func NewStatus(engine *core.Engine) Status { + executorState := engine.Executor.GetState() return Status{ - Paused: null.BoolFrom(engine.Executor.IsPaused()), - VUs: null.IntFrom(engine.Executor.GetVUs()), - VUsMax: null.IntFrom(engine.Executor.GetVUsMax()), - Running: engine.Executor.IsRunning(), + Running: executorState.HasStarted(), + Paused: null.BoolFrom(executorState.IsPaused()), + VUs: null.IntFrom(int64(executorState.GetCurrentlyActiveVUsCount())), + VUsMax: null.IntFrom(int64(executorState.GetInitializedVUsCount())), Tainted: engine.IsTainted(), } } diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index 5205d8ad7c1..97cc76e4f71 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -56,22 +56,28 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par return } - if status.VUsMax.Valid { - if err := engine.Executor.SetVUsMax(status.VUsMax.Int64); err != nil { - apiError(rw, "Couldn't change cap", err.Error(), http.StatusBadRequest) - return - } - } - if status.VUs.Valid { - if err := engine.Executor.SetVUs(status.VUs.Int64); err != nil { - apiError(rw, "Couldn't scale", err.Error(), http.StatusBadRequest) + if status.Paused.Valid { + if err = engine.Executor.SetPaused(status.Paused.Bool); err != nil { + apiError(rw, "Pause error", err.Error(), http.StatusInternalServerError) return } } - if status.Paused.Valid { - engine.Executor.SetPaused(status.Paused.Bool) - } + /* + //TODO: handle manual executor update + if status.VUsMax.Valid { + if err := engine.Executor.SetVUsMax(status.VUsMax.Int64); err != nil { + apiError(rw, "Couldn't change cap", err.Error(), http.StatusBadRequest) + return + } + } + if status.VUs.Valid { + if err := engine.Executor.SetVUs(status.VUs.Int64); err != nil { + apiError(rw, "Couldn't scale", err.Error(), http.StatusBadRequest) + return + } + } + */ data, err := jsonapi.Marshal(NewStatus(engine)) if err != nil { apiError(rw, "Encoding error", err.Error(), http.StatusInternalServerError) diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index 8e33d0a85df..e6bb6fc4a50 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -21,22 +21,25 @@ package v1 import ( - "bytes" "encoding/json" "net/http" "net/http/httptest" "testing" "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/manyminds/api2go/jsonapi" + "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" - "gopkg.in/guregu/null.v3" + "github.com/stretchr/testify/require" ) func TestGetStatus(t *testing.T) { - engine, err := core.NewEngine(nil, lib.Options{}) - assert.NoError(t, err) + executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + require.NoError(t, err) rw := httptest.NewRecorder() NewHandler().ServeHTTP(rw, newRequestWithEngine(engine, "GET", "/v1/status", nil)) @@ -62,6 +65,8 @@ func TestGetStatus(t *testing.T) { }) } +//TODO: fix after the manual executor +/* func TestPatchStatus(t *testing.T) { testdata := map[string]struct { StatusCode int @@ -108,3 +113,4 @@ func TestPatchStatus(t *testing.T) { }) } } +*/ From 18328e554867fbe95a9b8a6915cdc89d67df7b15 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 17:52:49 +0300 Subject: [PATCH 012/350] Clean up and add various TODOs --- cmd/cloud.go | 7 ++++++- js/bundle.go | 1 + js/modules/k6/http/limiter.go | 1 + js/runner.go | 3 +++ js/runner_test.go | 28 +++++++++++++++++++++------- lib/runner.go | 3 +++ lib/util.go | 13 ------------- lib/util_test.go | 5 +++-- 8 files changed, 38 insertions(+), 23 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 80db546618e..37fa1260013 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -101,6 +101,11 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return ExitCode{cerr, invalidConfigErrorCode} } + //TODO: warn about lack of support for --no-setup and --no-teardown in the cloud? + //TODO: validate for usage of execution segment + //TODO: validate for manual exacution (i.e. schedulers that aren't distributable) + //TODO: move those validations to a separate function and reuse validateConfig()? + err = r.SetOptions(conf.Options) if err != nil { return err @@ -204,7 +209,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud pb.WithProgress(func() (float64, string) { if testProgress.RunStatus < lib.RunStatusRunning { return 0, testProgress.RunStatusText - } + } return testProgress.Progress, fmt.Sprintf(percentageFmt, testProgress.Progress*100, testProgress.RunStatusText) }), ) diff --git a/js/bundle.go b/js/bundle.go index 3b5de91f38e..6da51dda544 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -198,6 +198,7 @@ func (b *Bundle) makeArchive() *lib.Archive { // Instantiate creates a new runtime from this bundle. func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { + //TODO: actually use a real context here, so that the instantiation can be killed // Placeholder for a real context. ctxPtr := new(context.Context) diff --git a/js/modules/k6/http/limiter.go b/js/modules/k6/http/limiter.go index bd5c0bee305..6a3bb8a65ca 100644 --- a/js/modules/k6/http/limiter.go +++ b/js/modules/k6/http/limiter.go @@ -65,6 +65,7 @@ type MultiSlotLimiter struct { } // NewMultiSlotLimiter initializes and returns a new MultiSlotLimiter with the given slot count +//TODO: move to lib and use something better than a mutex? sync.Map perhaps? func NewMultiSlotLimiter(slots int) *MultiSlotLimiter { return &MultiSlotLimiter{make(map[string]SlotLimiter), slots, sync.Mutex{}} } diff --git a/js/runner.go b/js/runner.go index 603161369b4..eaee1c44f5f 100644 --- a/js/runner.go +++ b/js/runner.go @@ -271,6 +271,8 @@ func (r *Runner) SetOptions(opts lib.Options) error { r.RPSLimit = rate.NewLimiter(rate.Limit(rps.Int64), 1) } + //TODO: validate that all exec values are either nil or valid exported methods (or HTTP requests in the future) + if opts.ConsoleOutput.Valid { c, err := newFileConsole(opts.ConsoleOutput.String) if err != nil { @@ -317,6 +319,7 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, if deadline, ok := ctx.Deadline(); ok && time.Now().After(deadline) { // we could have an error that is not errInterrupt in which case we should return it instead if err, ok := err.(*goja.InterruptedError); ok && v != nil && err.Value() != errInterrupt { + //TODO: silence this error? return v, err } // otherwise we have timeouted diff --git a/js/runner_test.go b/js/runner_test.go index bd15201293d..e08729bcd63 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -37,6 +37,8 @@ import ( "testing" "time" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js/common" @@ -46,6 +48,7 @@ import ( "github.com/loadimpact/k6/js/modules/k6/ws" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" + _ "github.com/loadimpact/k6/lib/scheduler" //TODO: figure out something better "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" @@ -259,11 +262,15 @@ func TestSetupDataIsolation(t *testing.T) { import { Counter } from "k6/metrics"; export let options = { - vus: 2, - vusMax: 10, - iterations: 500, - teardownTimeout: "1s", - setupTimeout: "1s", + execution: { + shared_iters: { + type: "shared-iterations", + vus: 5, + iterations: 500, + }, + }, + teardownTimeout: "5s", + setupTimeout: "5s", }; let myCounter = new Counter("mycounter"); @@ -294,13 +301,20 @@ func TestSetupDataIsolation(t *testing.T) { ) require.NoError(t, err) - engine, err := core.NewEngine(local.New(runner), runner.GetOptions()) + options := runner.GetOptions() + require.Empty(t, options.Validate()) + + executor, err := local.New(runner, logrus.StandardLogger()) require.NoError(t, err) + engine, err := core.NewEngine(executor, options, logrus.StandardLogger()) + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + require.NoError(t, engine.Init(ctx)) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} - ctx, cancel := context.WithCancel(context.Background()) errC := make(chan error) go func() { errC <- engine.Run(ctx) }() diff --git a/lib/runner.go b/lib/runner.go index e3fde944104..853836c98b8 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -44,6 +44,7 @@ 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. + //TODO: pass context.Context, so initialization can be killed properly... NewVU(out chan<- stats.SampleContainer) (VU, error) // Runs pre-test setup, if applicable. @@ -76,10 +77,12 @@ type VU interface { // 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. + //TODO: support reconfiguring of env vars, tags and exec Reconfigure(id int64) error } // MiniRunner wraps a function in a runner whose VUs will simply call that function. +//TODO: move to testutils, or somewhere else that's not lib... type MiniRunner struct { Fn func(ctx context.Context, out chan<- stats.SampleContainer) error SetupFn func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) diff --git a/lib/util.go b/lib/util.go index 60c6966e22e..2be0b770c5b 100644 --- a/lib/util.go +++ b/lib/util.go @@ -22,21 +22,8 @@ package lib import ( "strings" - - "github.com/loadimpact/k6/lib/types" ) -// Returns the total sum of time taken by the given set of stages. -func SumStages(stages []Stage) (d types.NullDuration) { - for _, stage := range stages { - d.Valid = stage.Duration.Valid - if stage.Duration.Valid { - d.Duration += stage.Duration.Duration - } - } - return d -} - // Splits a string in the form "key=value". func SplitKV(s string) (key, value string) { parts := strings.SplitN(s, "=", 2) diff --git a/lib/util_test.go b/lib/util_test.go index ea484bab297..119f43ba132 100644 --- a/lib/util_test.go +++ b/lib/util_test.go @@ -24,12 +24,12 @@ import ( "fmt" "strconv" "testing" - "time" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" ) +//TODO: update test +/* func TestSumStages(t *testing.T) { testdata := map[string]struct { Time types.NullDuration @@ -59,6 +59,7 @@ func TestSumStages(t *testing.T) { }) } } +*/ func TestSplitKV(t *testing.T) { testdata := map[string]struct { From 736c50585b1b73f0bcc43f5a8c4164fec180a110 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 18:18:33 +0300 Subject: [PATCH 013/350] Remove bytes.ReplaceAll since it was added only in Go 1.12 --- cmd/ui.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ui.go b/cmd/ui.go index 9080893c177..3fca03f7323 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -47,7 +47,7 @@ func (w *consoleWriter) Write(p []byte) (n int, err error) { origLen := len(p) if w.IsTTY { //TODO: check how cross-platform this is... - p = bytes.ReplaceAll(p, []byte{'\n'}, []byte{'\x1b', '[', '0', 'K', '\n'}) + p = bytes.Replace(p, []byte{'\n'}, []byte{'\x1b', '[', '0', 'K', '\n'}, -1) } w.Mutex.Lock() From 246baf2d3b2f854c95d60da0437d492197261da4 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 23 Apr 2019 18:32:34 +0300 Subject: [PATCH 014/350] Fix some linter errors --- cmd/run.go | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index b66e8037f00..1cbe1d878c1 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -261,10 +261,10 @@ a commandline interface for interacting with it.`, defer signal.Stop(sigC) // If the user hasn't opted out: report usage. - if !conf.NoUsageReport.Bool { - //TODO: fix - //TODO: move to a separate function - /* + //TODO: fix + //TODO: move to a separate function + /* + if !conf.NoUsageReport.Bool { go func() { u := "http://k6reports.loadimpact.com/" mime := "application/json" @@ -293,8 +293,8 @@ a commandline interface for interacting with it.`, } _, _ = http.Post(u, mime, bytes.NewBuffer(body)) }() - */ - } + } + */ // Ticker for progress bar updates. Less frequent updates for non-TTYs, none if quiet. updateFreq := 50 * time.Millisecond @@ -323,7 +323,6 @@ a commandline interface for interacting with it.`, } else { fn("Running") } - break } case err := <-errC: cancel() From 0a3c63f121b9cd2eeee769c6f9cbbab48fb17f61 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 24 Apr 2019 10:39:27 +0300 Subject: [PATCH 015/350] Make sure we wait for and return VUs in all schedulers Additionally, now getting a VU from the common buffer won't return an error if the context was cancelled. --- lib/executor.go | 6 ++--- lib/scheduler/constant_arrival_rate.go | 36 +++++++++++++------------- lib/scheduler/constant_looping_vus.go | 12 +++++---- lib/scheduler/per_vu_iterations.go | 12 +++++---- lib/scheduler/shared_iterations.go | 12 +++++---- lib/scheduler/variable_arrival_rate.go | 36 +++++++++++++------------- lib/scheduler/variable_looping_vus.go | 19 +++++++------- 7 files changed, 68 insertions(+), 65 deletions(-) diff --git a/lib/executor.go b/lib/executor.go index 18b07aa65d2..143fe8f538d 100644 --- a/lib/executor.go +++ b/lib/executor.go @@ -458,15 +458,13 @@ func (es *ExecutorState) ResumeNotify() <-chan struct{} { // we reach that timeout more than MaxRetriesGetPlannedVU number of times, this // function will return an error, since we either have a bug with some // scheduler, or the machine is very, very overloaded. -func (es *ExecutorState) GetPlannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { +func (es *ExecutorState) GetPlannedVU(logger *logrus.Entry) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { case vu := <-es.vus: atomic.AddUint64(es.activeVUs, 1) //TODO: set environment and exec return vu, nil - case <-ctx.Done(): - return nil, ctx.Err() case <-time.After(MaxTimeToWaitForPlannedVU): logger.Warnf("Could not get a VU from the buffer for %s", time.Duration(i)*MaxTimeToWaitForPlannedVU) } @@ -499,7 +497,7 @@ func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entr if remVUs < 0 { logger.Debug("Reusing a previously initialized unplanned VU") atomic.AddInt64(es.uninitializedUnplannedVUs, 1) - return es.GetPlannedVU(ctx, logger) + return es.GetPlannedVU(logger) } if es.initVUFunc == nil { return nil, fmt.Errorf("initVUFunc wasn't set in the executor state") diff --git a/lib/scheduler/constant_arrival_rate.go b/lib/scheduler/constant_arrival_rate.go index 504684dc9b5..c390d1eface 100644 --- a/lib/scheduler/constant_arrival_rate.go +++ b/lib/scheduler/constant_arrival_rate.go @@ -196,26 +196,36 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC "tickerPeriod": tickerPeriod, "type": car.config.GetType(), }).Debug("Starting scheduler run...") - // Pre-allocate VUs, but reserve space in the buffer for up to MaxVUs + // Pre-allocate the VUs local shared buffer vus := make(chan lib.VU, maxVUs) + + initialisedVUs := uint64(0) + // Make sure we put back planned and unplanned VUs back in the global + // buffer, and as an extra incentive, this replaces a waitgroup. + defer func() { + // no need for atomics, since initialisedVUs is mutated only in the select{} + for i := uint64(0); i < initialisedVUs; i++ { + car.executorState.ReturnVU(<-vus) + } + }() + + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := car.executorState.GetPlannedVU(ctx, car.logger) + vu, err := car.executorState.GetPlannedVU(car.logger) if err != nil { return err } + initialisedVUs++ vus <- vu } - initialisedVUs := new(uint64) - *initialisedVUs = uint64(preAllocatedVUs) - vusFmt := pb.GetFixedLengthIntFormat(maxVUs) fmtStr := pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 2) + " iters/s, " + vusFmt + " out of " + vusFmt + " VUs active" progresFn := func() (float64, string) { spent := time.Since(startTime) - currentInitialisedVUs := atomic.LoadUint64(initialisedVUs) + currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) vusInBuffer := uint64(len(vus)) return math.Min(1, float64(spent)/float64(duration)), fmt.Sprintf(fmtStr, arrivalRatePerSec, currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, @@ -232,15 +242,6 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs - // Make sure we put back planned and unplanned VUs back in the global - // buffer, and as an extra incentive, this replaces a waitgroup. - defer func() { - unplannedVUs := maxVUs - remainingUnplannedVUs - for i := int64(0); i < unplannedVUs; i++ { - car.executorState.ReturnVU(<-vus) - } - }() - for { select { case <-ticker.C: @@ -254,13 +255,12 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } - remainingUnplannedVUs-- vu, err := car.executorState.GetUnplannedVU(maxDurationCtx, car.logger) if err != nil { - remainingUnplannedVUs++ return err } - atomic.AddUint64(initialisedVUs, 1) + remainingUnplannedVUs-- + atomic.AddUint64(&initialisedVUs, 1) go runIteration(vu) } case <-regDurationDone: diff --git a/lib/scheduler/constant_looping_vus.go b/lib/scheduler/constant_looping_vus.go index 19b2610fe15..967b8860fa2 100644 --- a/lib/scheduler/constant_looping_vus.go +++ b/lib/scheduler/constant_looping_vus.go @@ -161,13 +161,15 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo go trackProgress(ctx, maxDurationCtx, regDurationCtx, clv, progresFn) // Actually schedule the VUs and iterations... - wg := sync.WaitGroup{} + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() + regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executorState, clv.logger, out) handleVU := func(vu lib.VU) { defer clv.executorState.ReturnVU(vu) - defer wg.Done() + defer activeVUs.Done() for { select { @@ -181,14 +183,14 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } for i := int64(0); i < numVUs; i++ { - wg.Add(1) - vu, err := clv.executorState.GetPlannedVU(ctx, clv.logger) + vu, err := clv.executorState.GetPlannedVU(clv.logger) if err != nil { + cancel() return err } + activeVUs.Add(1) go handleVU(vu) } - wg.Wait() return nil } diff --git a/lib/scheduler/per_vu_iterations.go b/lib/scheduler/per_vu_iterations.go index 8fe991643bc..ea657f8f4f8 100644 --- a/lib/scheduler/per_vu_iterations.go +++ b/lib/scheduler/per_vu_iterations.go @@ -170,13 +170,15 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai go trackProgress(ctx, maxDurationCtx, regDurationCtx, pvi, progresFn) // Actually schedule the VUs and iterations... - wg := sync.WaitGroup{} + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() + regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executorState, pvi.logger, out) handleVU := func(vu lib.VU) { defer pvi.executorState.ReturnVU(vu) - defer wg.Done() + defer activeVUs.Done() for i := int64(0); i < iterations; i++ { select { @@ -191,14 +193,14 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - wg.Add(1) - vu, err := pvi.executorState.GetPlannedVU(ctx, pvi.logger) + vu, err := pvi.executorState.GetPlannedVU(pvi.logger) if err != nil { + cancel() return err } + activeVUs.Add(1) go handleVU(vu) } - wg.Wait() return nil } diff --git a/lib/scheduler/shared_iterations.go b/lib/scheduler/shared_iterations.go index 110ba25899b..f7d07dfb3e5 100644 --- a/lib/scheduler/shared_iterations.go +++ b/lib/scheduler/shared_iterations.go @@ -175,14 +175,16 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai go trackProgress(ctx, maxDurationCtx, regDurationCtx, si, progresFn) // Actually schedule the VUs and iterations... - wg := sync.WaitGroup{} + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() + regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(si.executorState, si.logger, out) attemptedIters := new(uint64) handleVU := func(vu lib.VU) { defer si.executorState.ReturnVU(vu) - defer wg.Done() + defer activeVUs.Done() for { attemptedIterNumber := atomic.AddUint64(attemptedIters, 1) @@ -202,14 +204,14 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - wg.Add(1) - vu, err := si.executorState.GetPlannedVU(ctx, si.logger) + vu, err := si.executorState.GetPlannedVU(si.logger) if err != nil { + cancel() return err } + activeVUs.Add(1) go handleVU(vu) } - wg.Wait() return nil } diff --git a/lib/scheduler/variable_arrival_rate.go b/lib/scheduler/variable_arrival_rate.go index df72ceb95f3..f68c20efab5 100644 --- a/lib/scheduler/variable_arrival_rate.go +++ b/lib/scheduler/variable_arrival_rate.go @@ -312,26 +312,36 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample "startTickerPeriod": startTickerPeriod.Duration, "type": varr.config.GetType(), }).Debug("Starting scheduler run...") - // Pre-allocate VUs, but reserve space in the buffer for up to MaxVUs + // Pre-allocate the VUs local shared buffer vus := make(chan lib.VU, maxVUs) + + initialisedVUs := uint64(0) + // Make sure we put back planned and unplanned VUs back in the global + // buffer, and as an extra incentive, this replaces a waitgroup. + defer func() { + // no need for atomics, since initialisedVUs is mutated only in the select{} + for i := uint64(0); i < initialisedVUs; i++ { + varr.executorState.ReturnVU(<-vus) + } + }() + + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := varr.executorState.GetPlannedVU(ctx, varr.logger) + vu, err := varr.executorState.GetPlannedVU(varr.logger) if err != nil { return err } + initialisedVUs++ vus <- vu } - initialisedVUs := new(uint64) - *initialisedVUs = uint64(preAllocatedVUs) - tickerPeriod := new(int64) *tickerPeriod = int64(startTickerPeriod.Duration) fmtStr := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 2) + " iters/s, " + pb.GetFixedLengthIntFormat(maxVUs) + " out of " + pb.GetFixedLengthIntFormat(maxVUs) + " VUs active" progresFn := func() (float64, string) { - currentInitialisedVUs := atomic.LoadUint64(initialisedVUs) + currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) currentTickerPeriod := atomic.LoadInt64(tickerPeriod) vusInBuffer := uint64(len(vus)) @@ -354,15 +364,6 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample } remainingUnplannedVUs := maxVUs - preAllocatedVUs - // Make sure we put back planned and unplanned VUs back in the global - // buffer, and as an extra incentive, this replaces a waitgroup. - defer func() { - unplannedVUs := maxVUs - remainingUnplannedVUs - for i := int64(0); i < unplannedVUs; i++ { - varr.executorState.ReturnVU(<-vus) - } - }() - rateChangesStream := varr.streamRateChanges(maxDurationCtx, startTime) for { @@ -385,13 +386,12 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } - remainingUnplannedVUs-- vu, err := varr.executorState.GetUnplannedVU(maxDurationCtx, varr.logger) if err != nil { - remainingUnplannedVUs++ return err } - atomic.AddUint64(initialisedVUs, 1) + remainingUnplannedVUs-- + atomic.AddUint64(&initialisedVUs, 1) go runIteration(vu) } case <-regDurationDone: diff --git a/lib/scheduler/variable_looping_vus.go b/lib/scheduler/variable_looping_vus.go index 4207d8018ef..278f05dce5f 100644 --- a/lib/scheduler/variable_looping_vus.go +++ b/lib/scheduler/variable_looping_vus.go @@ -500,14 +500,14 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo "duration": regularDuration, "numStages": len(vlv.config.Stages)}, ).Debug("Starting scheduler run...") - activeVUs := new(int64) + activeVUsCount := new(int64) vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) progresFn := func() (float64, string) { spent := time.Since(startTime) if spent > regularDuration { return 1, fmt.Sprintf("variable looping VUs for %s", regularDuration) } - currentlyActiveVUs := atomic.LoadInt64(activeVUs) + currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) return float64(spent) / float64(regularDuration), fmt.Sprintf( "currently "+vusFmt+" active looping VUs, %s/%s", currentlyActiveVUs, pb.GetFixedLengthDuration(spent, regularDuration), regularDuration, @@ -518,23 +518,24 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo // Actually schedule the VUs and iterations, likely the most complicated // scheduler among all of them... - wg := &sync.WaitGroup{} + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() runIteration := getIterationRunner(vlv.executorState, vlv.logger, out) getVU := func() (lib.VU, error) { - vu, err := vlv.executorState.GetPlannedVU(maxDurationCtx, vlv.logger) + vu, err := vlv.executorState.GetPlannedVU(vlv.logger) if err != nil { cancel() } else { - wg.Add(1) - atomic.AddInt64(activeVUs, 1) + activeVUs.Add(1) + atomic.AddInt64(activeVUsCount, 1) } return vu, err } returnVU := func(vu lib.VU) { vlv.executorState.ReturnVU(vu) - atomic.AddInt64(activeVUs, -1) - wg.Done() + atomic.AddInt64(activeVUsCount, -1) + activeVUs.Done() } vuHandles := make([]*vuHandle, maxVUs) @@ -607,7 +608,5 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo }() } - wg.Wait() - return nil } From 8165f28e2751f2ab7e51ddd7d9b8dc3e0defe2a9 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 14 May 2019 20:55:01 +0300 Subject: [PATCH 016/350] Implement the manual-execution scheduler --- api/v1/status.go | 4 +- api/v1/status_routes.go | 41 +- core/local/local.go | 10 +- lib/executor.go | 72 +++- lib/helpers.go | 18 +- lib/scheduler/base_scheduler.go | 16 - lib/scheduler/constant_arrival_rate.go | 6 +- lib/scheduler/constant_looping_vus.go | 4 +- lib/scheduler/manual.go | 515 +++++++++++++++++++++---- lib/scheduler/per_vu_iterations.go | 4 +- lib/scheduler/shared_iterations.go | 4 +- lib/scheduler/variable_arrival_rate.go | 6 +- lib/scheduler/variable_looping_vus.go | 4 +- lib/schedulers.go | 20 +- 14 files changed, 570 insertions(+), 154 deletions(-) diff --git a/api/v1/status.go b/api/v1/status.go index e7a3cee49b1..d9b030c822c 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -40,8 +40,8 @@ func NewStatus(engine *core.Engine) Status { return Status{ Running: executorState.HasStarted(), Paused: null.BoolFrom(executorState.IsPaused()), - VUs: null.IntFrom(int64(executorState.GetCurrentlyActiveVUsCount())), - VUsMax: null.IntFrom(int64(executorState.GetInitializedVUsCount())), + VUs: null.IntFrom(executorState.GetCurrentlyActiveVUsCount()), + VUsMax: null.IntFrom(executorState.GetInitializedVUsCount()), Tainted: engine.IsTainted(), } } diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index 97cc76e4f71..c414a2f6dca 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -21,11 +21,14 @@ package v1 import ( + "fmt" "io/ioutil" "net/http" "github.com/julienschmidt/httprouter" "github.com/loadimpact/k6/api/common" + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/scheduler" "github.com/manyminds/api2go/jsonapi" ) @@ -41,6 +44,16 @@ func HandleGetStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Param _, _ = rw.Write(data) } +func getFirstManualExecutionScheduler(executor lib.Executor) (*scheduler.ManualExecution, error) { + schedulers := executor.GetSchedulers() + for _, s := range schedulers { + if mex, ok := s.(*scheduler.ManualExecution); ok { + return mex, nil + } + } + return nil, fmt.Errorf("a manual-execution scheduler needs to be configured for live configuration updates") +} + func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { engine := common.GetEngine(r.Context()) @@ -63,21 +76,27 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par } } - /* - //TODO: handle manual executor update + if status.VUsMax.Valid || status.VUs.Valid { + //TODO: add ability to specify the actual scheduler id? though thus should + //likely be in the v2 REST API, where we could implement it in a way that + //may allow us to eventually support other scheduler types + scheduler, uptateErr := getFirstManualExecutionScheduler(engine.Executor) + if uptateErr != nil { + apiError(rw, "Execution config error", uptateErr.Error(), http.StatusInternalServerError) + return + } + newConfig := scheduler.GetCurrentConfig().ManualExecutionControlConfig if status.VUsMax.Valid { - if err := engine.Executor.SetVUsMax(status.VUsMax.Int64); err != nil { - apiError(rw, "Couldn't change cap", err.Error(), http.StatusBadRequest) - return - } + newConfig.MaxVUs = status.VUsMax } if status.VUs.Valid { - if err := engine.Executor.SetVUs(status.VUs.Int64); err != nil { - apiError(rw, "Couldn't scale", err.Error(), http.StatusBadRequest) - return - } + newConfig.VUs = status.VUs + } + if uptateErr := scheduler.UpdateConfig(r.Context(), newConfig); err != nil { + apiError(rw, "Config update error", uptateErr.Error(), http.StatusInternalServerError) + return } - */ + } data, err := jsonapi.Marshal(NewStatus(engine)) if err != nil { apiError(rw, "Encoding error", err.Error(), http.StatusInternalServerError) diff --git a/core/local/local.go b/core/local/local.go index 8307586336a..49498d54171 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -371,15 +371,19 @@ func (e *Executor) SetPaused(pause bool) error { } for _, sched := range e.schedulers { - if !sched.IsPausable() { + pausableSched, ok := sched.(lib.PausableScheduler) + if !ok { return fmt.Errorf( "%s scheduler '%s' doesn't support pause and resume operations after its start", sched.GetConfig().GetType(), sched.GetConfig().GetName(), ) } - if err := sched.LiveUpdate(pause, nil); err != nil { + if err := pausableSched.SetPaused(pause); err != nil { return err } } - return nil + if pause { + return e.state.Pause() + } + return e.state.Resume() } diff --git a/lib/executor.go b/lib/executor.go index 143fe8f538d..8e9880d2c0d 100644 --- a/lib/executor.go +++ b/lib/executor.go @@ -167,8 +167,10 @@ type ExecutorState struct { // Total number of currently initialized VUs. Generally equal to // currentVUIdentifier minus 1, since initializedVUs starts from 0 and is // incremented only after a VU is initialized, while CurrentVUIdentifier is - // incremented before a VU is initialized. - initializedVUs *uint64 + // incremented before a VU is initialized. It should always be greater than + // or equal to 0, but int64 is used for simplification of the used atomic + // arithmetic operations. + initializedVUs *int64 // Total number of unplanned VUs we haven't initialized yet. It starts // being equal to GetMaxPossibleVUs(executionPlan)-GetMaxPlannedVUs(), and @@ -182,8 +184,10 @@ type ExecutorState struct { // The number of VUs that are currently executing the test script. This also // includes any VUs that are in the process of gracefully winding down, - // either at the end of the test, or when VUs are ramping down. - activeVUs *uint64 + // either at the end of the test, or when VUs are ramping down. It should + // always be greater than or equal to 0, but int64 is used for + // simplification of the used atomic arithmetic operations. + activeVUs *int64 // The total number of full (i.e uninterrupted) iterations that have been // completed so far. @@ -250,9 +254,9 @@ func NewExecutorState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *Ex vus: make(chan VU, maxPossibleVUs), currentVUIdentifier: new(uint64), - initializedVUs: new(uint64), + initializedVUs: new(int64), uninitializedUnplannedVUs: &maxUnplannedUninitializedVUs, - activeVUs: new(uint64), + activeVUs: new(int64), fullIterationsCount: new(uint64), partialIterationsCount: new(uint64), startTime: new(int64), @@ -277,8 +281,15 @@ func (es *ExecutorState) GetUniqueVUIdentifier() uint64 { // exported script options and for the execution of setup() and teardown() // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetInitializedVUsCount() uint64 { - return atomic.LoadUint64(es.initializedVUs) +func (es *ExecutorState) GetInitializedVUsCount() int64 { + return atomic.LoadInt64(es.initializedVUs) +} + +// ModInitializedVUsCount changes the total number of currently initialized VUs. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) ModInitializedVUsCount(mod int64) int64 { + return atomic.AddInt64(es.initializedVUs, mod) } // GetCurrentlyActiveVUsCount returns the number of VUs that are currently @@ -286,8 +297,15 @@ func (es *ExecutorState) GetInitializedVUsCount() uint64 { // of gracefullt winding down. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetCurrentlyActiveVUsCount() uint64 { - return atomic.LoadUint64(es.activeVUs) +func (es *ExecutorState) GetCurrentlyActiveVUsCount() int64 { + return atomic.LoadInt64(es.activeVUs) +} + +// ModCurrentlyActiveVUsCount changes the total number of currently active VUs. +// +// IMPORTANT: for UI/information purposes only, don't use for synchronization. +func (es *ExecutorState) ModCurrentlyActiveVUsCount(mod int64) int64 { + return atomic.AddInt64(es.activeVUs, mod) } // GetFullIterationCount returns the total of full (i.e uninterrupted) iterations @@ -458,11 +476,13 @@ func (es *ExecutorState) ResumeNotify() <-chan struct{} { // we reach that timeout more than MaxRetriesGetPlannedVU number of times, this // function will return an error, since we either have a bug with some // scheduler, or the machine is very, very overloaded. -func (es *ExecutorState) GetPlannedVU(logger *logrus.Entry) (VU, error) { +func (es *ExecutorState) GetPlannedVU(logger *logrus.Entry, modifyAtiveVUCount bool) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { case vu := <-es.vus: - atomic.AddUint64(es.activeVUs, 1) + if modifyAtiveVUCount { + es.ModCurrentlyActiveVUsCount(+1) + } //TODO: set environment and exec return vu, nil case <-time.After(MaxTimeToWaitForPlannedVU): @@ -497,18 +517,28 @@ func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entr if remVUs < 0 { logger.Debug("Reusing a previously initialized unplanned VU") atomic.AddInt64(es.uninitializedUnplannedVUs, 1) - return es.GetPlannedVU(logger) + return es.GetPlannedVU(logger, true) } + + logger.Debug("Initializing an unplanned VU, this may affect test results") + vu, err := es.InitializeNewVU(ctx, logger) + if err != nil { + es.ModCurrentlyActiveVUsCount(+1) + } + return vu, err +} + +// InitializeNewVU creates and returns a brand new VU, updating the relevant +// tracking counters. +func (es *ExecutorState) InitializeNewVU(ctx context.Context, logger *logrus.Entry) (VU, error) { if es.initVUFunc == nil { return nil, fmt.Errorf("initVUFunc wasn't set in the executor state") } - logger.Debug("Initializing an unplanned VU, this may affect test results") newVU, err := es.initVUFunc(ctx, logger) if err != nil { return nil, err } - atomic.AddUint64(es.activeVUs, 1) - atomic.AddUint64(es.initializedVUs, 1) + es.ModInitializedVUsCount(+1) return newVU, err } @@ -516,14 +546,14 @@ func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entr // increases the initialized VUs counter. func (es *ExecutorState) AddInitializedVU(vu VU) { es.vus <- vu - atomic.AddUint64(es.initializedVUs, 1) + es.ModInitializedVUsCount(+1) } // ReturnVU is a helper function that puts VUs back into the buffer and // decreases the active VUs counter. -func (es *ExecutorState) ReturnVU(vu VU) { +func (es *ExecutorState) ReturnVU(vu VU, wasActive bool) { es.vus <- vu - // From the official atomic.AddUint64() docs: "to subtract a signed positive - // constant value c from x, do AddUint64(&x, ^uint64(c-1))" - atomic.AddUint64(es.activeVUs, ^uint64(0)) + if wasActive { + es.ModCurrentlyActiveVUsCount(-1) + } } diff --git a/lib/helpers.go b/lib/helpers.go index 20659a5cc4a..cc34484d22c 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -72,10 +72,10 @@ func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { // // As an example, imagine that you have a scheduler with MaxPlannedVUs=20 and // MaxUnaplannedVUs=0, followed immediately after by another scheduler with -// MaxPlannedVUs=10 and MaxUnaplannedVUs=10. If you MaxPlannedVUs for the whole -// test is 20, and MaxUnaplannedVUs, but since those schedulers won't run -// concurrently, MaxVUs for the whole test is not 30, rather it's 20, since 20 -// VUs will be sufficient to run the test. +// MaxPlannedVUs=10 and MaxUnaplannedVUs=10. The MaxPlannedVUs number for the +// whole test is 20, and MaxUnaplannedVUs is 10, but since those schedulers +// won't run concurrently, MaxVUs for the whole test is not 30, rather it's 20, +// since 20 VUs will be sufficient to run the whole test. // // IMPORTANT 2: this has one very important exception. The manual execution // scheduler doesn't use the MaxUnplannedVUs (i.e. this function will return 0), @@ -93,7 +93,7 @@ func GetMaxPossibleVUs(steps []ExecutionStep) (result uint64) { // GetEndOffset returns the time offset of the last step of the execution plan, // and whether that step is a final one, i.e. whether the number of planned or -// unplanned +// unplanned is 0 func GetEndOffset(steps []ExecutionStep) (lastStepOffset time.Duration, isFinal bool) { if len(steps) == 0 { return 0, true @@ -102,8 +102,12 @@ func GetEndOffset(steps []ExecutionStep) (lastStepOffset time.Duration, isFinal return lastStep.TimeOffset, (lastStep.PlannedVUs == 0 && lastStep.MaxUnplannedVUs == 0) } -// A helper function for joining error messages into a single string -func concatErrors(errors []error, separator string) string { +// ConcatErrors is a a helper function for joining error messages into a single +// string. +// +// TODO: use Go 2.0/xerrors style errors so we don't loose error type information and +// metadata. +func ConcatErrors(errors []error, separator string) string { errStrings := make([]string, len(errors)) for i, e := range errors { errStrings[i] = e.Error() diff --git a/lib/scheduler/base_scheduler.go b/lib/scheduler/base_scheduler.go index 4e5469210e1..c2ff61bff52 100644 --- a/lib/scheduler/base_scheduler.go +++ b/lib/scheduler/base_scheduler.go @@ -22,7 +22,6 @@ package scheduler import ( "context" - "fmt" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/ui/pb" @@ -72,18 +71,3 @@ func (bs BaseScheduler) GetLogger() *logrus.Entry { func (bs BaseScheduler) GetProgress() *pb.ProgressBar { return bs.progress } - -// IsPausable just returns false, since most schedulers are not pausable after -// they have been started... -func (BaseScheduler) IsPausable() bool { - return false -} - -// LiveUpdate just returns false, since most schedulers' configs cannot be -// updated in-flight. -func (bs BaseScheduler) LiveUpdate(_ bool, _ lib.SchedulerConfig) error { - return fmt.Errorf( - "%s scheduler '%s' doesn't support pausing or live configuration updates", - bs.config.GetType(), bs.config.GetName(), - ) -} diff --git a/lib/scheduler/constant_arrival_rate.go b/lib/scheduler/constant_arrival_rate.go index c390d1eface..eab1f5214f9 100644 --- a/lib/scheduler/constant_arrival_rate.go +++ b/lib/scheduler/constant_arrival_rate.go @@ -172,7 +172,7 @@ type ConstantArrivalRate struct { // Make sure we implement the lib.Scheduler interface. var _ lib.Scheduler = &ConstantArrivalRate{} -// Run executes a specific number of iterations with each confugured VU. +// Run executes a constant number of iterations per second. // // TODO: Reuse the variable arrival rate method? func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { @@ -205,13 +205,13 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} for i := uint64(0); i < initialisedVUs; i++ { - car.executorState.ReturnVU(<-vus) + car.executorState.ReturnVU(<-vus, true) } }() // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := car.executorState.GetPlannedVU(car.logger) + vu, err := car.executorState.GetPlannedVU(car.logger, true) if err != nil { return err } diff --git a/lib/scheduler/constant_looping_vus.go b/lib/scheduler/constant_looping_vus.go index 967b8860fa2..f0c74eacde9 100644 --- a/lib/scheduler/constant_looping_vus.go +++ b/lib/scheduler/constant_looping_vus.go @@ -168,7 +168,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo runIteration := getIterationRunner(clv.executorState, clv.logger, out) handleVU := func(vu lib.VU) { - defer clv.executorState.ReturnVU(vu) + defer clv.executorState.ReturnVU(vu, true) defer activeVUs.Done() for { @@ -183,7 +183,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } for i := int64(0); i < numVUs; i++ { - vu, err := clv.executorState.GetPlannedVU(clv.logger) + vu, err := clv.executorState.GetPlannedVU(clv.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/manual.go b/lib/scheduler/manual.go index ef13b85a07e..94dd1bc2121 100644 --- a/lib/scheduler/manual.go +++ b/lib/scheduler/manual.go @@ -21,31 +21,79 @@ package scheduler import ( + "context" "errors" "fmt" + "math" + "sync" + "sync/atomic" "time" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" ) -const manualExecution = "manual-execution" +const manualExecutionType = "manual-execution" -// ManualExecutionConfig stores VUs and duration -type ManualExecutionConfig struct { - StartVUs null.Int - MaxVUs null.Int - Duration types.NullDuration +func init() { + lib.RegisterSchedulerConfigType( + manualExecutionType, + func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + config := ManualExecutionConfig{BaseConfig: NewBaseConfig(name, manualExecutionType)} + err := lib.StrictJSONUnmarshal(rawJSON, &config) + if err != nil { + return config, err + } + if !config.MaxVUs.Valid { + config.MaxVUs = config.VUs + } + return config, nil + }, + ) +} + +// ManualExecutionControlConfig contains all of the options that actually +// determine the scheduling of VUs in the manual execution scheduler. +type ManualExecutionControlConfig struct { + VUs null.Int `json:"vus"` + Duration types.NullDuration `json:"duration"` + MaxVUs null.Int `json:"maxVUs"` } -// NewManualExecutionConfig returns a ManualExecutionConfig with default values -func NewManualExecutionConfig(startVUs, maxVUs null.Int, duration types.NullDuration) ManualExecutionConfig { - if !maxVUs.Valid { - maxVUs = startVUs +// Validate just checks the control options in isolation. +func (mecc ManualExecutionControlConfig) Validate() (errors []error) { + if mecc.VUs.Int64 < 0 { + errors = append(errors, fmt.Errorf("the number of VUs shouldn't be negative")) } - return ManualExecutionConfig{startVUs, maxVUs, duration} + + if mecc.MaxVUs.Int64 < mecc.VUs.Int64 { + errors = append(errors, fmt.Errorf( + "the specified maxVUs (%d) should more than or equal to the the number of active VUs (%d)", + mecc.MaxVUs.Int64, mecc.VUs.Int64, + )) + } + + if !mecc.Duration.Valid { + errors = append(errors, fmt.Errorf("the duration should be specified, for infinite duration use 0")) + } else if time.Duration(mecc.Duration.Duration) < 0 { + errors = append(errors, fmt.Errorf( + "the duration shouldn't be negative, for infinite duration use 0", + )) + } + + return errors +} + +// ManualExecutionConfig stores the number of currently active VUs, the max +// number of VUs and the scheduler duration. The duration can be 0, which means +// "infinite duration", i.e. the user has to manually abort the script. +type ManualExecutionConfig struct { + BaseConfig + ManualExecutionControlConfig } // Make sure we implement the lib.SchedulerConfig interface @@ -53,58 +101,48 @@ var _ lib.SchedulerConfig = &ManualExecutionConfig{} // GetDescription returns a human-readable description of the scheduler options func (mec ManualExecutionConfig) GetDescription(_ *lib.ExecutionSegment) string { - duration := "" + duration := "infinite" if mec.Duration.Duration != 0 { - duration = fmt.Sprintf(" and duration %s", mec.Duration) + duration = mec.Duration.String() } return fmt.Sprintf( - "Manual execution with %d starting and %d initialized VUs%s", - mec.StartVUs.Int64, mec.MaxVUs.Int64, duration, + "Manually controlled execution with %d VUs, %d max VUs, %s duration", + mec.VUs.Int64, mec.MaxVUs.Int64, duration, ) } // Validate makes sure all options are configured and valid func (mec ManualExecutionConfig) Validate() []error { - var errors []error - if mec.StartVUs.Int64 <= 0 { - errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) - } - - if mec.MaxVUs.Int64 < mec.StartVUs.Int64 { - errors = append(errors, fmt.Errorf("the number of MaxVUs should more than or equal to the starting number of VUs")) - } - - if !mec.Duration.Valid { - errors = append(errors, fmt.Errorf("the duration should be specified, for infinite duration use 0")) - } else if time.Duration(mec.Duration.Duration) < 0 { + errors := append(mec.BaseConfig.Validate(), mec.ManualExecutionControlConfig.Validate()...) + if mec.GracefulStop.Valid { errors = append(errors, fmt.Errorf( - "the duration shouldn't be negative, for infinite duration use 0", + "gracefulStop is not supported by the manual execution scheduler", )) } - return errors } -// GetExecutionRequirements just reserves the number of starting VUs for the whole -// duration of the scheduler, so these VUs can be initialized in the beginning of the -// test. +// GetExecutionRequirements just reserves the specified number of max VUs for +// the whole duration of the scheduler, so these VUs can be initialized in the +// beginning of the test. // -// Importantly, if 0 (i.e. infinite) duration is configured, this scheduler doesn't -// emit the last step to relinquish these VUs. +// Importantly, if 0 (i.e. infinite) duration is configured, this scheduler +// doesn't emit the last step to relinquish these VUs. // -// Also, the manual execution scheduler doesn't set MaxUnplannedVUs in the returned steps, -// since their initialization and usage is directly controlled by the user and is effectively -// bounded only by the resources of the machine k6 is running on. +// Also, the manual execution scheduler doesn't set MaxUnplannedVUs in the +// returned steps, since their initialization and usage is directly controlled +// by the user and is effectively bounded only by the resources of the machine +// k6 is running on. // -// This is not a problem, because the MaxUnplannedVUs are mostly meant to be used for -// calculating the maximum possble number of initialized VUs at any point during a test -// run. That's used for sizing purposes and for user qouta checking in the cloud execution, -// where the manual scheduler isn't supported. +// This is not a problem, because the MaxUnplannedVUs are mostly meant to be +// used for calculating the maximum possble number of initialized VUs at any +// point during a test run. That's used for sizing purposes and for user qouta +// checking in the cloud execution, where the manual scheduler isn't supported. func (mec ManualExecutionConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { startVUs := lib.ExecutionStep{ TimeOffset: 0, - PlannedVUs: uint64(es.Scale(mec.StartVUs.Int64)), - MaxUnplannedVUs: 0, // intentional, see function comment + PlannedVUs: uint64(es.Scale(mec.MaxVUs.Int64)), // use + MaxUnplannedVUs: 0, // intentional, see function comment } maxDuration := time.Duration(mec.Duration.Duration) @@ -119,51 +157,378 @@ func (mec ManualExecutionConfig) GetExecutionRequirements(es *lib.ExecutionSegme }} } -// GetName always returns manual-execution, since this config can't be -// specified in the exported script options. -func (ManualExecutionConfig) GetName() string { - return manualExecution +// IsDistributable simply returns false because there's no way to reliably +// distribute the manual execution scheduler. +func (ManualExecutionConfig) IsDistributable() bool { + return false } -// GetType always returns manual-execution, since that's this special -// config's type... -func (ManualExecutionConfig) GetType() string { - return manualExecution +// NewScheduler creates a new ManualExecution "scheduler" +func (mec ManualExecutionConfig) NewScheduler( + es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { + + return &ManualExecution{ + startConfig: mec, + currentControlConfig: mec.ManualExecutionControlConfig, + configLock: &sync.RWMutex{}, + newControlConfigs: make(chan updateConfigEvent), + pauseEvents: make(chan pauseEvent), + hasStarted: make(chan struct{}), + + executorState: es, + logger: logger, + progress: pb.New(pb.WithLeft(mec.GetName)), + }, nil } -// GetStartTime always returns 0, since the manual execution scheduler -// always starts in the beginning and is always the only scheduler. -func (ManualExecutionConfig) GetStartTime() time.Duration { - return 0 +type pauseEvent struct { + isPaused bool + err chan error } -// GetGracefulStop always returns 0, since we still don't support graceful -// stops or ramp downs in the manual execution mode. -//TODO: implement? -func (ManualExecutionConfig) GetGracefulStop() time.Duration { - return 0 +type updateConfigEvent struct { + newConfig ManualExecutionControlConfig + err chan error } -// GetEnv returns an empty map, since the manual executor doesn't support custom -// environment variables. -func (ManualExecutionConfig) GetEnv() map[string]string { +// ManualExecution is an implementation of the old k6 scheduler that could be +// controlled externally, via the k6 REST API. It implements both the +// lib.PausableScheduler and the lib.LiveUpdatableScheduler interfaces. +type ManualExecution struct { + startConfig ManualExecutionConfig + currentControlConfig ManualExecutionControlConfig + configLock *sync.RWMutex + newControlConfigs chan updateConfigEvent + pauseEvents chan pauseEvent + hasStarted chan struct{} + + executorState *lib.ExecutorState + logger *logrus.Entry + progress *pb.ProgressBar +} + +// Make sure we implement all the interfaces +var _ lib.Scheduler = &ManualExecution{} +var _ lib.PausableScheduler = &ManualExecution{} +var _ lib.LiveUpdatableScheduler = &ManualExecution{} + +// GetCurrentConfig just returns the scheduler's current configuration. +func (mex *ManualExecution) GetCurrentConfig() ManualExecutionConfig { + mex.configLock.RLock() + defer mex.configLock.RUnlock() + return ManualExecutionConfig{ + BaseConfig: mex.startConfig.BaseConfig, + ManualExecutionControlConfig: mex.currentControlConfig, + } +} + +// GetConfig just returns the scheduler's current configuration, it's basically +// an alias of GetCurrentConfig that implements the more generic interface. +func (mex *ManualExecution) GetConfig() lib.SchedulerConfig { + return mex.GetCurrentConfig() +} + +// GetProgress just returns the scheduler's progress bar instance. +func (mex ManualExecution) GetProgress() *pb.ProgressBar { + return mex.progress +} + +// GetLogger just returns the scheduler's logger instance. +func (mex ManualExecution) GetLogger() *logrus.Entry { + return mex.logger +} + +// Init doesn't do anything... +func (mex ManualExecution) Init(ctx context.Context) error { return nil } -// GetExec always returns nil, for now there's no way to execute custom funcions in -// the manual execution mode. -func (ManualExecutionConfig) GetExec() null.String { - return null.NewString("", false) +// SetPaused pauses or resumes the scheduler. +func (mex *ManualExecution) SetPaused(paused bool) error { + select { + case <-mex.hasStarted: + event := pauseEvent{isPaused: paused, err: make(chan error)} + mex.pauseEvents <- event + return <-event.err + default: + return fmt.Errorf("cannot pause the manual scheduler before it has started") + } +} + +// UpdateConfig validates the supplied config and updates it in real time. It is +// possible to update the configuration even when k6 is paused, either in the +// beginning (i.e. when running k6 with --paused) or in the middle of the script +// execution. +func (mex *ManualExecution) UpdateConfig(ctx context.Context, newConf interface{}) error { + newManualConfig, ok := newConf.(ManualExecutionControlConfig) + if !ok { + return errors.New("invalid config type") + } + if errs := newManualConfig.Validate(); len(errs) != 0 { + return fmt.Errorf("invalid confiuguration supplied: %s", lib.ConcatErrors(errs, ", ")) + } + + if newManualConfig.Duration != mex.startConfig.Duration { + return fmt.Errorf("the manual scheduler duration cannot be changed") + } + if newManualConfig.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { + // This limitation is because the manual execution scheduler is still a + // scheduler that participates in the overall k6 scheduling. Thus, any + // VUs that were explicitly specified by the user in the config may be + // reused from or by other schedulers. + return fmt.Errorf( + "the new number of max VUs cannot be lower than the starting number of max VUs (%d)", + mex.startConfig.MaxVUs.Int64, + ) + } + + mex.configLock.Lock() + select { + case <-mex.hasStarted: + mex.configLock.Unlock() + event := updateConfigEvent{newConfig: newManualConfig, err: make(chan error)} + mex.newControlConfigs <- event + return <-event.err + case <-ctx.Done(): + mex.configLock.Unlock() + return ctx.Err() + default: + mex.currentControlConfig = newManualConfig + mex.configLock.Unlock() + return nil + } } -// IsDistributable simply returns false because there's no way to reliably -// distribute the manual execution scheduler. -func (ManualExecutionConfig) IsDistributable() bool { - return false +// This is a helper function that is used in run for non-infinite durations. +func (mex *ManualExecution) stopWhenDurationIsReached(ctx context.Context, duration time.Duration, cancel func()) { + ctxDone := ctx.Done() + checkInterval := time.NewTicker(100 * time.Millisecond) + for { + select { + case <-ctxDone: + checkInterval.Stop() + return + + //TODO: something more optimized that sleeps for pauses? + case <-checkInterval.C: + if mex.executorState.GetCurrentTestRunDuration() >= duration { + cancel() + return + } + } + } } -// NewScheduler creates a new ManualExecution "scheduler" -func (mec ManualExecutionConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { - return nil, errors.New("not implemented 4") //TODO +// manualVUHandle is a wrapper around the vuHandle helper, used in the +// variable-looping-vus scheduler. Here, instead of using its getVU and returnVU +// methods to retrieve and return a VU from the global buffer, we use them to +// accurately update the local and global active VU counters and to ensure that +// the pausing and reducing VUs operations wait for VUs to fully finish +// executing their current iterations before returning. +type manualVUHandle struct { + *vuHandle + vu lib.VU + wg *sync.WaitGroup + + // This is the cancel of the local context, used to kill its goroutine when + // we reduce the number of MaxVUs, so that the Go GC can clean up the VU. + cancelVU func() +} + +func newManualVUHandle( + parentCtx context.Context, state *lib.ExecutorState, localActiveVUsCount *int64, vu lib.VU, logger *logrus.Entry, +) *manualVUHandle { + + wg := sync.WaitGroup{} + getVU := func() (lib.VU, error) { + wg.Add(1) + state.ModCurrentlyActiveVUsCount(+1) + atomic.AddInt64(localActiveVUsCount, +1) + return vu, nil + } + returnVU := func(_ lib.VU) { + state.ModCurrentlyActiveVUsCount(-1) + atomic.AddInt64(localActiveVUsCount, -1) + wg.Done() + } + ctx, cancel := context.WithCancel(parentCtx) + return &manualVUHandle{ + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, logger), + vu: vu, + wg: &wg, + cancelVU: cancel, + } +} + +// Run constantly loops through as many iterations as possible on a variable +// dynamically controlled number of VUs either for the specified duration, or +// until the test is manually stopped. +// +//TODO: split this up? somehow... :/ +func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { + mex.configLock.RLock() + // Safely get the current config - it's important that the close of the + // hasStarted channel is inside of the lock, so that there are no data races + // between it and the UpdateConfig() method. + currentControlConfig := mex.currentControlConfig + close(mex.hasStarted) + mex.configLock.RUnlock() + + segment := mex.executorState.Options.ExecutionSegment + duration := time.Duration(currentControlConfig.Duration.Duration) + + ctx, cancel := context.WithCancel(parentCtx) + defer cancel() + if duration > 0 { // Only keep track of duration if it's not infinite + go mex.stopWhenDurationIsReached(ctx, duration, cancel) + } + + mex.logger.WithFields( + logrus.Fields{"type": manualExecutionType, "duration": duration}, + ).Debug("Starting scheduler run...") + + // Retrieve and initialize the (scaled) number of MaxVUs from the global VU + // buffer that the user originally specified in the JS config. + startMaxVUs := segment.Scale(mex.startConfig.MaxVUs.Int64) + vuHandles := make([]*manualVUHandle, startMaxVUs) + activeVUsCount := new(int64) + runIteration := getIterationRunner(mex.executorState, mex.logger, out) + for i := int64(0); i < startMaxVUs; i++ { // get the initial planned VUs from the common buffer + vu, vuGetErr := mex.executorState.GetPlannedVU(mex.logger, false) + if vuGetErr != nil { + return vuGetErr + } + vuHandle := newManualVUHandle( + parentCtx, mex.executorState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), + ) + go vuHandle.runLoopsIfPossible(runIteration) + vuHandles[i] = vuHandle + } + + // Keep track of the progress + maxVUs := new(int64) + *maxVUs = startMaxVUs + progresFn := func() (float64, string) { + spent := mex.executorState.GetCurrentTestRunDuration() + progress := 0.0 + if duration > 0 { + progress = math.Min(1, float64(spent)/float64(duration)) + } + //TODO: simulate spinner for the other case or cycle 0-100? + currentActiveVUs := atomic.LoadInt64(activeVUsCount) + currentMaxVUs := atomic.LoadInt64(maxVUs) + vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) + return progress, fmt.Sprintf( + "currently "+vusFmt+" out of "+vusFmt+" active looping VUs, %s/%s", currentActiveVUs, currentMaxVUs, + pb.GetFixedLengthDuration(spent, duration), duration, + ) + } + mex.progress.Modify(pb.WithProgress(progresFn)) + go trackProgress(parentCtx, ctx, ctx, mex, progresFn) + + currentlyPaused := false + waitVUs := func(from, to int64) { + for i := from; i < to; i++ { + vuHandles[i].wg.Wait() + } + } + handleConfigChange := func(oldControlConfig, newControlConfig ManualExecutionControlConfig) error { + oldActiveVUs := segment.Scale(oldControlConfig.VUs.Int64) + oldMaxVUs := segment.Scale(oldControlConfig.MaxVUs.Int64) + newActiveVUs := segment.Scale(newControlConfig.VUs.Int64) + newMaxVUs := segment.Scale(newControlConfig.MaxVUs.Int64) + + mex.logger.WithFields(logrus.Fields{ + "oldActiveVUs": oldActiveVUs, "oldMaxVUs": oldMaxVUs, + "newActiveVUs": newActiveVUs, "newMaxVUs": newMaxVUs, + }).Debug("Updating execution configuration...") + + for i := oldMaxVUs; i < newMaxVUs; i++ { + vu, vuInitErr := mex.executorState.InitializeNewVU(ctx, mex.logger) + if vuInitErr != nil { + return vuInitErr + } + vuHandle := newManualVUHandle( + ctx, mex.executorState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), + ) + go vuHandle.runLoopsIfPossible(runIteration) + vuHandles = append(vuHandles, vuHandle) + } + + if oldActiveVUs < newActiveVUs { + for i := oldActiveVUs; i < newActiveVUs; i++ { + + if !currentlyPaused { + vuHandles[i].start() + } + } + } else { + for i := newActiveVUs; i < oldActiveVUs; i++ { + vuHandles[i].hardStop() + } + waitVUs(newActiveVUs, oldActiveVUs) + } + + if oldMaxVUs > newMaxVUs { + for i := newMaxVUs; i < oldMaxVUs; i++ { + vuHandles[i].cancelVU() + if i < startMaxVUs { + // return the initial planned VUs to the common buffer + mex.executorState.ReturnVU(vuHandles[i].vu, false) + } else { + mex.executorState.ModInitializedVUsCount(-1) + } + vuHandles[i] = nil + } + vuHandles = vuHandles[:newMaxVUs] + } + + atomic.StoreInt64(maxVUs, newMaxVUs) + return nil + } + + err = handleConfigChange(ManualExecutionControlConfig{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig) + if err != nil { + return err + } + defer func() { + err = handleConfigChange(currentControlConfig, ManualExecutionControlConfig{}) + }() + + for { + select { + case <-ctx.Done(): + return nil + case updateConfigEvent := <-mex.newControlConfigs: + err := handleConfigChange(currentControlConfig, updateConfigEvent.newConfig) + if err != nil { + updateConfigEvent.err <- err + return err + } + currentControlConfig = updateConfigEvent.newConfig + mex.configLock.Lock() + mex.currentControlConfig = updateConfigEvent.newConfig + mex.configLock.Unlock() + updateConfigEvent.err <- nil + + case pauseEvent := <-mex.pauseEvents: + if pauseEvent.isPaused == currentlyPaused { + pauseEvent.err <- nil + continue + } + activeVUs := currentControlConfig.VUs.Int64 + if pauseEvent.isPaused { + for i := int64(0); i < activeVUs; i++ { + vuHandles[i].gracefulStop() + } + waitVUs(0, activeVUs) + } else { + for i := int64(0); i < activeVUs; i++ { + vuHandles[i].start() + } + } + currentlyPaused = pauseEvent.isPaused + pauseEvent.err <- nil + } + } } diff --git a/lib/scheduler/per_vu_iterations.go b/lib/scheduler/per_vu_iterations.go index ea657f8f4f8..81ddea9977f 100644 --- a/lib/scheduler/per_vu_iterations.go +++ b/lib/scheduler/per_vu_iterations.go @@ -177,7 +177,7 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai runIteration := getIterationRunner(pvi.executorState, pvi.logger, out) handleVU := func(vu lib.VU) { - defer pvi.executorState.ReturnVU(vu) + defer pvi.executorState.ReturnVU(vu, true) defer activeVUs.Done() for i := int64(0); i < iterations; i++ { @@ -193,7 +193,7 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - vu, err := pvi.executorState.GetPlannedVU(pvi.logger) + vu, err := pvi.executorState.GetPlannedVU(pvi.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/shared_iterations.go b/lib/scheduler/shared_iterations.go index f7d07dfb3e5..14f36de4292 100644 --- a/lib/scheduler/shared_iterations.go +++ b/lib/scheduler/shared_iterations.go @@ -183,7 +183,7 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai attemptedIters := new(uint64) handleVU := func(vu lib.VU) { - defer si.executorState.ReturnVU(vu) + defer si.executorState.ReturnVU(vu, true) defer activeVUs.Done() for { @@ -204,7 +204,7 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - vu, err := si.executorState.GetPlannedVU(si.logger) + vu, err := si.executorState.GetPlannedVU(si.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/variable_arrival_rate.go b/lib/scheduler/variable_arrival_rate.go index f68c20efab5..40bfdc73a13 100644 --- a/lib/scheduler/variable_arrival_rate.go +++ b/lib/scheduler/variable_arrival_rate.go @@ -284,7 +284,7 @@ func (varr VariableArrivalRate) streamRateChanges(ctx context.Context, startTime return ch } -// Run executes a specific number of iterations with each confugured VU. +// Run executes a variable number of iterations per second. func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executorState.Options.ExecutionSegment gracefulStop := varr.config.GetGracefulStop() @@ -321,13 +321,13 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} for i := uint64(0); i < initialisedVUs; i++ { - varr.executorState.ReturnVU(<-vus) + varr.executorState.ReturnVU(<-vus, true) } }() // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := varr.executorState.GetPlannedVU(varr.logger) + vu, err := varr.executorState.GetPlannedVU(varr.logger, true) if err != nil { return err } diff --git a/lib/scheduler/variable_looping_vus.go b/lib/scheduler/variable_looping_vus.go index 278f05dce5f..1dc9b9b6868 100644 --- a/lib/scheduler/variable_looping_vus.go +++ b/lib/scheduler/variable_looping_vus.go @@ -523,7 +523,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo runIteration := getIterationRunner(vlv.executorState, vlv.logger, out) getVU := func() (lib.VU, error) { - vu, err := vlv.executorState.GetPlannedVU(vlv.logger) + vu, err := vlv.executorState.GetPlannedVU(vlv.logger, true) if err != nil { cancel() } else { @@ -533,7 +533,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo return vu, err } returnVU := func(vu lib.VU) { - vlv.executorState.ReturnVU(vu) + vlv.executorState.ReturnVU(vu, true) atomic.AddInt64(activeVUsCount, -1) activeVUs.Done() } diff --git a/lib/schedulers.go b/lib/schedulers.go index d1e8514bfd4..450a0082d72 100644 --- a/lib/schedulers.go +++ b/lib/schedulers.go @@ -109,12 +109,22 @@ type Scheduler interface { GetProgress() *pb.ProgressBar GetLogger() *logrus.Entry - Init(ctx context.Context) error + Init(ctx context.Context) error //TODO: remove, since it's currently unused? Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error +} + +// PausableScheduler should be implemented by the schedulers that can be paused +// and resumend in the middle of the test execution. Currently, only the manual +// execution scheduler implements it. +type PausableScheduler interface { + SetPaused(bool) error +} - // Currently used only in the manual executor - IsPausable() bool // TODO: make a separate interfaces - LiveUpdate(pause bool, newConfig SchedulerConfig) error +// LiveUpdatableScheduler should be implemented for the schedulers whoose +// configuration can be modified in the middle of the test execution. Currently, +// only the manual execution scheduler implements it. +type LiveUpdatableScheduler interface { + UpdateConfig(ctx context.Context, newConfig interface{}) error } // SchedulerConfigConstructor is a simple function that returns a concrete @@ -182,7 +192,7 @@ func (scs SchedulerConfigMap) Validate() (errors []error) { for name, scheduler := range scs { if schedErr := scheduler.Validate(); len(schedErr) != 0 { errors = append(errors, - fmt.Errorf("scheduler %s has errors: %s", name, concatErrors(schedErr, ", "))) + fmt.Errorf("scheduler %s has errors: %s", name, ConcatErrors(schedErr, ", "))) } } return errors From 70448ea39789c636d61cb4d0bdff2adc440514cb Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 14 May 2019 21:49:34 +0300 Subject: [PATCH 017/350] Fix issues pointed out in the code review --- cmd/root.go | 2 +- cmd/run.go | 8 ++++---- cmd/ui.go | 3 +-- lib/helpers.go | 10 +++++++--- lib/scheduler/base_config.go | 5 ++--- 5 files changed, 15 insertions(+), 13 deletions(-) diff --git a/cmd/root.go b/cmd/root.go index 54091676863..693d915398a 100644 --- a/cmd/root.go +++ b/cmd/root.go @@ -91,7 +91,7 @@ var RootCmd = &cobra.Command{ PersistentPreRun: func(cmd *cobra.Command, args []string) { setupLoggers(logFmt) if noColor { - // TODO: figure our something else... currently, with the wrappers + // TODO: figure out something else... currently, with the wrappers // below, we're stripping any colors from the output after we've // added them. The problem is that, besides being very inefficient, // this actually also strips other special characters from the diff --git a/cmd/run.go b/cmd/run.go index 1cbe1d878c1..4c0e837b5e9 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -162,7 +162,10 @@ a commandline interface for interacting with it.`, initBar = executor.GetInitProgressBar() progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) - go showProgress(ctx, progressBarWG, conf, executor) + go func() { + showProgress(ctx, conf, executor) + progressBarWG.Done() + }() // Create an engine. initBar.Modify(pb.WithConstProgress(0, "Init engine")) @@ -288,9 +291,6 @@ a commandline interface for interacting with it.`, if err != nil { panic(err) // This should never happen!! } - if err != nil { - panic(err) // This should never happen!! - } _, _ = http.Post(u, mime, bytes.NewBuffer(body)) }() } diff --git a/cmd/ui.go b/cmd/ui.go index 3fca03f7323..b5336de313f 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -98,8 +98,7 @@ func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { //TODO: show other information here? //TODO: add a no-progress option that will disable these //TODO: don't use global variables... -func showProgress(ctx context.Context, wg *sync.WaitGroup, conf Config, executor *local.Executor) { - defer wg.Done() +func showProgress(ctx context.Context, conf Config, executor *local.Executor) { if quiet || conf.HttpDebug.Valid && conf.HttpDebug.String != "" { return } diff --git a/lib/helpers.go b/lib/helpers.go index cc34484d22c..04ffb3bf02d 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -93,7 +93,7 @@ func GetMaxPossibleVUs(steps []ExecutionStep) (result uint64) { // GetEndOffset returns the time offset of the last step of the execution plan, // and whether that step is a final one, i.e. whether the number of planned or -// unplanned is 0 +// unplanned is 0. func GetEndOffset(steps []ExecutionStep) (lastStepOffset time.Duration, isFinal bool) { if len(steps) == 0 { return 0, true @@ -142,8 +142,12 @@ func StreamExecutionSteps( } select { case <-ctx.Done(): - return // exit if context is cancelled - case ch <- step: // send the step + // exit if context is cancelled + return + case ch <- step: + // ... otherwise, just send the step - the out channel is + // unbuffered, so we don't need to worry whether the other side + // will keep reading after the context is done. } } diff --git a/lib/scheduler/base_config.go b/lib/scheduler/base_config.go index 4a18fb3a643..377a9907532 100644 --- a/lib/scheduler/base_config.go +++ b/lib/scheduler/base_config.go @@ -31,6 +31,7 @@ import ( ) var schedulerNameWhitelist = regexp.MustCompile(`^[0-9a-zA-Z_-]+$`) //nolint:gochecknoglobals +const schedulerNameErr = "the scheduler name should contain only numbers, latin letters, underscores, and dashes" // BaseConfig contains the common config fields for all schedulers type BaseConfig struct { @@ -61,9 +62,7 @@ func (bc BaseConfig) Validate() (errors []error) { errors = append(errors, fmt.Errorf("scheduler name shouldn't be empty")) } if !schedulerNameWhitelist.MatchString(bc.Name) { - errors = append(errors, fmt.Errorf( - "the scheduler name should contain only numbers, latin letters, underscores, and dashes", - )) + errors = append(errors, fmt.Errorf(schedulerNameErr)) } if bc.Exec.Valid && bc.Exec.String == "" { errors = append(errors, fmt.Errorf("exec value cannot be empty")) From c9a534fd258b59068c4f996361243c436c07131c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 15 May 2019 09:23:47 +0300 Subject: [PATCH 018/350] Emit a warning when users only specify VUs without exec shortcuts --- cmd/config_consolidation_test.go | 5 +++-- lib/scheduler/execution_config_shortcuts.go | 12 ++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 87b04f56d3d..8efcb55bd38 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -254,8 +254,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { {opts{cli: []string{"--vus", "10", "-i", "6"}}, exp{validationErrors: true}, verifySharedIters(I(10), I(6))}, {opts{cli: []string{"-s", "10s:5", "-s", "10s:"}}, exp{validationErrors: true}, nil}, {opts{fs: defaultConfig(`{"stages": [{"duration": "20s"}], "vus": 10}`)}, exp{validationErrors: true}, nil}, - // These should emit a warning - //TODO: these should probably emit a validation error? + // These should emit a consolidation error {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{consolidationError: true}, nil}, {opts{cli: []string{"-u", "2", "-d", "10s", "-s", "10s:20"}}, exp{consolidationError: true}, nil}, {opts{cli: []string{"-u", "3", "-i", "5", "-s", "10s:20"}}, exp{consolidationError: true}, nil}, @@ -285,6 +284,8 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { {opts{fs: getFS(nil), cli: []string{"--config", "/my/config.file"}}, exp{consolidationError: true}, nil}, // Test combinations between options and levels + {opts{cli: []string{"--vus", "1"}}, exp{}, verifyOneIterPerOneVU}, + {opts{cli: []string{"--vus", "10"}}, exp{logWarning: true}, verifyOneIterPerOneVU}, { opts{ fs: getFS([]file{{"/my/config.file", `{"vus": 8, "duration": "2m"}`}}), diff --git a/lib/scheduler/execution_config_shortcuts.go b/lib/scheduler/execution_config_shortcuts.go index 2efb1eeb2f5..62e468e2852 100644 --- a/lib/scheduler/execution_config_shortcuts.go +++ b/lib/scheduler/execution_config_shortcuts.go @@ -23,7 +23,6 @@ package scheduler import ( "github.com/loadimpact/k6/lib" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) // ExecutionConflictError is a custom error type used for all of the errors in @@ -111,8 +110,15 @@ func BuildExecutionConfig(opts lib.Options) (lib.Options, error) { case len(opts.Execution) > 0: // Do nothing, execution was explicitly specified + default: // Check if we should emit some warnings + if opts.VUs.Valid && opts.VUs.Int64 != 1 { + logrus.Warnf( + "the `vus=%d` option will be ignored, it only works in conjunction with `iterations`, `duration`, or `stages`", + opts.VUs.Int64, + ) + } if opts.Stages != nil && len(opts.Stages) == 0 { // No someone explicitly set stages to empty logrus.Warnf("`stages` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") @@ -122,12 +128,10 @@ func BuildExecutionConfig(opts lib.Options) (lib.Options, error) { logrus.Warnf("`execution` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") } // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config - // with 1 VU and 1 iteration. We're choosing the per-VU config, since that one could also - // be executed both locally, and in the cloud. + // with 1 VU and 1 iteration. result.Execution = lib.SchedulerConfigMap{ lib.DefaultSchedulerName: NewPerVUIterationsConfig(lib.DefaultSchedulerName), } - result.Iterations = null.NewInt(1, false) } //TODO: validate the config; questions: From 53833064220999b7648d1ea4581601ba1012efe5 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 17 May 2019 17:48:20 +0300 Subject: [PATCH 019/350] Add a helper Split() method to execution segments --- lib/execution_segment.go | 35 ++++++++++++++++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 2e8ad8be09e..e349c86a660 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -150,7 +150,40 @@ func (es *ExecutionSegment) FloatLength() float64 { return res } -//TODO: add a NewFromString() and Split() methods +// Split evenly dividies the execution segment into the specified number of +// equal consecutive execution sub-segments. +func (es *ExecutionSegment) Split(numParts int64) ([]*ExecutionSegment, error) { + if numParts < 1 { + return nil, fmt.Errorf("the number of parts should be at least 1, %d received", numParts) + } + + from, to := zeroRat, oneRat + if es != nil { + from, to = es.from, es.to + } + + increment := new(big.Rat).Sub(to, from) + increment.Denom().Mul(increment.Denom(), big.NewInt(numParts)) + + results := make([]*ExecutionSegment, numParts) + for i := int64(0); i < numParts; i++ { + segmentTo := new(big.Rat).Add(from, increment) + segment, err := NewExecutionSegment(from, segmentTo) + if err != nil { + return nil, err + } + results[i] = segment + from = segmentTo + } + + if from.Cmp(to) != 0 { + return nil, fmt.Errorf("Expected %s and %s to be equal", from, to) + } + + return results, nil +} + +//TODO: add a NewFromString() method // helper function for rounding (up) of rational numbers to big.Int values func roundUp(rat *big.Rat) *big.Int { From 05c9dcc7d04aade49f1d6942d3eef64ba4c54999 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 17 May 2019 17:53:25 +0300 Subject: [PATCH 020/350] Fix linter issues and code comments --- core/engine.go | 3 ++- lib/execution_segment.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/engine.go b/core/engine.go index db9ba7c555f..48aca3bd565 100644 --- a/core/engine.go +++ b/core/engine.go @@ -103,7 +103,8 @@ func NewEngine(ex lib.Executor, o lib.Options, logger *logrus.Logger) (*Engine, return e, nil } -// Init is used to initialize the executor. That's a costly operation, since it initializes all of +// Init is used to initialize the executor. That's a costly operation, since it +// initializes all of the planned VUs and could potentially take a long time. func (e *Engine) Init(ctx context.Context) error { return e.Executor.Init(ctx, e.Samples) } diff --git a/lib/execution_segment.go b/lib/execution_segment.go index e349c86a660..a82bd10c26a 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -177,7 +177,7 @@ func (es *ExecutionSegment) Split(numParts int64) ([]*ExecutionSegment, error) { } if from.Cmp(to) != 0 { - return nil, fmt.Errorf("Expected %s and %s to be equal", from, to) + return nil, fmt.Errorf("expected %s and %s to be equal", from, to) } return results, nil From 104ce6ef1a7b2ff3072ab0a2aba5b3a1b806bff2 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 16 Aug 2019 23:13:11 +0300 Subject: [PATCH 021/350] Rename schedulers to executors without renaming files or packages This includes all of the renames announced in https://github.com/loadimpact/k6/pull/1007#issuecomment-497319205: - schedulers are renamed as executors - lib.ExecutorState is now lib.ExecutionState - the lib.Executor interface and the local.Executor are now ExecutionSchedulers - the `manual-execution` executor is renamed to `externally-controlled` - incomplete iterations are now referred to as interrupted in the CLI UI All of these changes are in the code and comments only, no files or packages have been moved or renamed. This will happen in a follow-up commit, so that the git move tracking doesn't break, due to all of the changes in the file contents. --- api/server_test.go | 4 +- api/v1/group_routes.go | 4 +- api/v1/group_routes_test.go | 4 +- api/v1/metric_routes.go | 8 +- api/v1/metric_routes_test.go | 8 +- api/v1/setup_teardown_routes.go | 8 +- api/v1/setup_teardown_routes_test.go | 6 +- api/v1/status.go | 10 +- api/v1/status_routes.go | 25 +-- api/v1/status_routes_test.go | 6 +- cmd/cloud.go | 4 +- cmd/config_consolidation_test.go | 12 +- cmd/run.go | 42 ++--- cmd/ui.go | 6 +- core/engine.go | 34 ++-- core/engine_test.go | 6 +- core/local/local.go | 192 ++++++++++---------- core/local/local_test.go | 106 +++++------ js/runner_test.go | 4 +- lib/doc.go | 11 +- lib/executor.go | 187 +++++++++---------- lib/executor_test.go | 4 +- lib/helpers.go | 27 ++- lib/old_archive_test.go | 2 +- lib/options.go | 16 +- lib/runner.go | 12 +- lib/scheduler/base_config.go | 24 +-- lib/scheduler/base_scheduler.go | 42 ++--- lib/scheduler/constant_arrival_rate.go | 44 ++--- lib/scheduler/constant_looping_vus.go | 38 ++-- lib/scheduler/execution_config_shortcuts.go | 26 +-- lib/scheduler/helpers.go | 22 +-- lib/scheduler/manual.go | 189 ++++++++++--------- lib/scheduler/per_vu_iterations.go | 42 ++--- lib/scheduler/schedulers_test.go | 30 +-- lib/scheduler/shared_iterations.go | 42 ++--- lib/scheduler/variable_arrival_rate.go | 42 ++--- lib/scheduler/variable_looping_vus.go | 76 ++++---- lib/scheduler/vu_handle.go | 20 +- lib/schedulers.go | 140 +++++++------- 40 files changed, 765 insertions(+), 760 deletions(-) diff --git a/api/server_test.go b/api/server_test.go index 92c0f607dbf..adade23f848 100644 --- a/api/server_test.go +++ b/api/server_test.go @@ -76,9 +76,9 @@ func TestLogger(t *testing.T) { } func TestWithEngine(t *testing.T) { - executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) rw := httptest.NewRecorder() diff --git a/api/v1/group_routes.go b/api/v1/group_routes.go index ead29a3bb36..24d65fda565 100644 --- a/api/v1/group_routes.go +++ b/api/v1/group_routes.go @@ -31,7 +31,7 @@ import ( func HandleGetGroups(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { engine := common.GetEngine(r.Context()) - root := NewGroup(engine.Executor.GetRunner().GetDefaultGroup(), nil) + root := NewGroup(engine.ExecutionScheduler.GetRunner().GetDefaultGroup(), nil) groups := FlattenGroup(root) data, err := jsonapi.Marshal(groups) @@ -47,7 +47,7 @@ func HandleGetGroup(rw http.ResponseWriter, r *http.Request, p httprouter.Params engine := common.GetEngine(r.Context()) - root := NewGroup(engine.Executor.GetRunner().GetDefaultGroup(), nil) + root := NewGroup(engine.ExecutionScheduler.GetRunner().GetDefaultGroup(), nil) groups := FlattenGroup(root) var group *Group diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index 1f4af119e0c..4d4e6c40bc5 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -43,9 +43,9 @@ func TestGetGroups(t *testing.T) { g2, err := g1.Group("group 2") assert.NoError(t, err) - executor, err := local.New(&lib.MiniRunner{Group: g0}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{Group: g0}, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) t.Run("list", func(t *testing.T) { diff --git a/api/v1/metric_routes.go b/api/v1/metric_routes.go index a38936b5e41..6178f69eb6f 100644 --- a/api/v1/metric_routes.go +++ b/api/v1/metric_routes.go @@ -33,8 +33,8 @@ func HandleGetMetrics(rw http.ResponseWriter, r *http.Request, p httprouter.Para engine := common.GetEngine(r.Context()) var t time.Duration - if engine.Executor != nil { - t = engine.Executor.GetState().GetCurrentTestRunDuration() + if engine.ExecutionScheduler != nil { + t = engine.ExecutionScheduler.GetState().GetCurrentTestRunDuration() } metrics := make([]Metric, 0) @@ -55,8 +55,8 @@ func HandleGetMetric(rw http.ResponseWriter, r *http.Request, p httprouter.Param engine := common.GetEngine(r.Context()) var t time.Duration - if engine.Executor != nil { - t = engine.Executor.GetState().GetCurrentTestRunDuration() + if engine.ExecutionScheduler != nil { + t = engine.ExecutionScheduler.GetState().GetCurrentTestRunDuration() } var metric Metric diff --git a/api/v1/metric_routes_test.go b/api/v1/metric_routes_test.go index 390942cab35..7d18a2117c6 100644 --- a/api/v1/metric_routes_test.go +++ b/api/v1/metric_routes_test.go @@ -40,9 +40,9 @@ import ( ) func TestGetMetrics(t *testing.T) { - executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) engine.Metrics = map[string]*stats.Metric{ @@ -81,9 +81,9 @@ func TestGetMetrics(t *testing.T) { } func TestGetMetric(t *testing.T) { - executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) engine.Metrics = map[string]*stats.Metric{ diff --git a/api/v1/setup_teardown_routes.go b/api/v1/setup_teardown_routes.go index 1ed62a27697..513115f38ef 100644 --- a/api/v1/setup_teardown_routes.go +++ b/api/v1/setup_teardown_routes.go @@ -71,7 +71,7 @@ func handleSetupDataOutput(rw http.ResponseWriter, setupData json.RawMessage) { // HandleGetSetupData just returns the current JSON-encoded setup data func HandleGetSetupData(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { - runner := common.GetEngine(r.Context()).Executor.GetRunner() + runner := common.GetEngine(r.Context()).ExecutionScheduler.GetRunner() handleSetupDataOutput(rw, runner.GetSetupData()) } @@ -91,7 +91,7 @@ func HandleSetSetupData(rw http.ResponseWriter, r *http.Request, p httprouter.Pa } } - runner := common.GetEngine(r.Context()).Executor.GetRunner() + runner := common.GetEngine(r.Context()).ExecutionScheduler.GetRunner() if len(body) == 0 { runner.SetSetupData(nil) @@ -105,7 +105,7 @@ 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) { engine := common.GetEngine(r.Context()) - runner := engine.Executor.GetRunner() + runner := engine.ExecutionScheduler.GetRunner() if err := runner.Setup(r.Context(), engine.Samples); err != nil { apiError(rw, "Error executing setup", err.Error(), http.StatusInternalServerError) @@ -118,7 +118,7 @@ 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() + runner := common.GetEngine(r.Context()).ExecutionScheduler.GetRunner() if err := runner.Teardown(r.Context(), engine.Samples); err != nil { apiError(rw, "Error executing teardown", err.Error(), http.StatusInternalServerError) diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index 9e57e19649b..f722a55619a 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -149,9 +149,9 @@ func TestSetupData(t *testing.T) { SetupTimeout: types.NullDurationFrom(1 * time.Second), TeardownTimeout: types.NullDurationFrom(1 * time.Second), }) - executor, err := local.New(runner, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(runner, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, runner.GetOptions(), logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, runner.GetOptions(), logrus.StandardLogger()) require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) @@ -181,7 +181,7 @@ func TestSetupData(t *testing.T) { checkSetup(setupRun[0], setupRun[1], setupRun[2]) } - engine.Executor.SetPaused(false) + require.NoError(t, engine.ExecutionScheduler.SetPaused(false)) select { case <-time.After(10 * time.Second): diff --git a/api/v1/status.go b/api/v1/status.go index d9b030c822c..93ecf945602 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -36,12 +36,12 @@ type Status struct { } func NewStatus(engine *core.Engine) Status { - executorState := engine.Executor.GetState() + executionState := engine.ExecutionScheduler.GetState() return Status{ - Running: executorState.HasStarted(), - Paused: null.BoolFrom(executorState.IsPaused()), - VUs: null.IntFrom(executorState.GetCurrentlyActiveVUsCount()), - VUsMax: null.IntFrom(executorState.GetInitializedVUsCount()), + Running: executionState.HasStarted(), + Paused: null.BoolFrom(executionState.IsPaused()), + VUs: null.IntFrom(executionState.GetCurrentlyActiveVUsCount()), + VUsMax: null.IntFrom(executionState.GetInitializedVUsCount()), Tainted: engine.IsTainted(), } } diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index c414a2f6dca..4b2b1993bfa 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -44,14 +44,17 @@ func HandleGetStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Param _, _ = rw.Write(data) } -func getFirstManualExecutionScheduler(executor lib.Executor) (*scheduler.ManualExecution, error) { - schedulers := executor.GetSchedulers() - for _, s := range schedulers { - if mex, ok := s.(*scheduler.ManualExecution); ok { +func getFirstExternallyControlledExecutor( + execScheduler lib.ExecutionScheduler, +) (*scheduler.ExternallyControlled, error) { + + executors := execScheduler.GetExecutors() + for _, s := range executors { + if mex, ok := s.(*scheduler.ExternallyControlled); ok { return mex, nil } } - return nil, fmt.Errorf("a manual-execution scheduler needs to be configured for live configuration updates") + return nil, fmt.Errorf("a externally-controlled executor needs to be configured for live configuration updates") } func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { @@ -70,29 +73,29 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par } if status.Paused.Valid { - if err = engine.Executor.SetPaused(status.Paused.Bool); err != nil { + if err = engine.ExecutionScheduler.SetPaused(status.Paused.Bool); err != nil { apiError(rw, "Pause error", err.Error(), http.StatusInternalServerError) return } } if status.VUsMax.Valid || status.VUs.Valid { - //TODO: add ability to specify the actual scheduler id? though thus should + //TODO: add ability to specify the actual executor id? though thus should //likely be in the v2 REST API, where we could implement it in a way that - //may allow us to eventually support other scheduler types - scheduler, uptateErr := getFirstManualExecutionScheduler(engine.Executor) + //may allow us to eventually support other executor types + executor, uptateErr := getFirstExternallyControlledExecutor(engine.ExecutionScheduler) if uptateErr != nil { apiError(rw, "Execution config error", uptateErr.Error(), http.StatusInternalServerError) return } - newConfig := scheduler.GetCurrentConfig().ManualExecutionControlConfig + newConfig := executor.GetCurrentConfig().ExternallyControlledConfigParams if status.VUsMax.Valid { newConfig.MaxVUs = status.VUsMax } if status.VUs.Valid { newConfig.VUs = status.VUs } - if uptateErr := scheduler.UpdateConfig(r.Context(), newConfig); err != nil { + if uptateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { apiError(rw, "Config update error", uptateErr.Error(), http.StatusInternalServerError) return } diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index e6bb6fc4a50..25163ae25b6 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -36,9 +36,9 @@ import ( ) func TestGetStatus(t *testing.T) { - executor, err := local.New(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, lib.Options{}, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) rw := httptest.NewRecorder() @@ -65,7 +65,7 @@ func TestGetStatus(t *testing.T) { }) } -//TODO: fix after the manual executor +//TODO: fix after the externally-controlled executor /* func TestPatchStatus(t *testing.T) { testdata := map[string]struct { diff --git a/cmd/cloud.go b/cmd/cloud.go index 457a288b0b3..59f4e2ad577 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -106,7 +106,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud //TODO: warn about lack of support for --no-setup and --no-teardown in the cloud? //TODO: validate for usage of execution segment - //TODO: validate for manual exacution (i.e. schedulers that aren't distributable) + //TODO: validate for externally controlled executor (i.e. executors that aren't distributable) //TODO: move those validations to a separate function and reuse validateConfig()? err = r.SetOptions(conf.Options) @@ -191,7 +191,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud fprintf(stdout, " executor: %s\n", ui.ValueColor.Sprint("cloud")) fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) - //TODO: print schedulers information + //TODO: print executors information fprintf(stdout, "\n") printBar(progressBar, "Initializing the cloud test") diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index b88188a3303..27621738fc1 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -74,7 +74,7 @@ func setEnv(t *testing.T, newEnv []string) (restoreEnv func()) { func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config - sched := c.Execution[lib.DefaultSchedulerName] + sched := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, sched) require.IsType(t, scheduler.PerVUIteationsConfig{}, sched) perVuIters, ok := sched.(scheduler.PerVUIteationsConfig) @@ -85,7 +85,7 @@ func verifyOneIterPerOneVU(t *testing.T, c Config) { func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultSchedulerName] + sched := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, sched) require.IsType(t, scheduler.SharedIteationsConfig{}, sched) sharedIterConfig, ok := sched.(scheduler.SharedIteationsConfig) @@ -99,7 +99,7 @@ func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultSchedulerName] + sched := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, sched) require.IsType(t, scheduler.ConstantLoopingVUsConfig{}, sched) clvc, ok := sched.(scheduler.ConstantLoopingVUsConfig) @@ -113,7 +113,7 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing func verifyVarLoopingVUs(startVus null.Int, stages []scheduler.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultSchedulerName] + sched := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, sched) require.IsType(t, scheduler.VariableLoopingVUsConfig{}, sched) clvc, ok := sched.(scheduler.VariableLoopingVUsConfig) @@ -254,7 +254,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) - sharedIterConfig := c.Execution[lib.DefaultSchedulerName].(scheduler.SharedIteationsConfig) + sharedIterConfig := c.Execution[lib.DefaultExecutorName].(scheduler.SharedIteationsConfig) assert.Equal(t, time.Duration(sharedIterConfig.MaxDuration.Duration), 10*time.Second) }}, // This should get a validation error since VUs are more than the shared iterations @@ -357,7 +357,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { exp{}, verifySharedIters(I(12), I(25)), }, - //TODO: test manual execution + //TODO: test the externally controlled executor //TODO: test execution-segment // Just in case, verify that no options will result in the same 1 vu 1 iter config diff --git a/cmd/run.go b/cmd/run.go index a477cd4bb73..ac91a3c2506 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -151,25 +151,25 @@ a commandline interface for interacting with it.`, ctx, cancel := context.WithCancel(context.Background()) //TODO: move even earlier? defer cancel() - // Create a local executor wrapping the runner. - fprintf(stdout, "%s executor\r", initBar.String()) - executor, err := local.New(r, logger) + // Create a local execution scheduler wrapping the runner. + fprintf(stdout, "%s execution scheduler\r", initBar.String()) + execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err } - executorState := executor.GetState() - initBar = executor.GetInitProgressBar() + executionState := execScheduler.GetState() + initBar = execScheduler.GetInitProgressBar() progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) go func() { - showProgress(ctx, conf, executor) + showProgress(ctx, conf, execScheduler) progressBarWG.Done() }() // Create an engine. initBar.Modify(pb.WithConstProgress(0, "Init engine")) - engine, err := core.NewEngine(executor, conf.Options, logger) + engine, err := core.NewEngine(execScheduler, conf.Options, logger) if err != nil { return err } @@ -187,7 +187,7 @@ a commandline interface for interacting with it.`, initBar.Modify(pb.WithConstProgress(0, "Init metric outputs")) for _, out := range conf.Out { t, arg := parseCollector(out) - collector, err := newCollector(t, arg, src, conf, executor.GetExecutionPlan()) + collector, err := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) if err != nil { return err } @@ -230,16 +230,16 @@ a commandline interface for interacting with it.`, fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) fprintf(stdout, "\n") - plan := executor.GetExecutionPlan() - schedulers := executor.GetSchedulers() + plan := execScheduler.GetExecutionPlan() + executors := execScheduler.GetExecutors() maxDuration, _ := lib.GetEndOffset(plan) fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprintf( - "(%.2f%%) %d schedulers, %d max VUs, %s max duration (incl. graceful stop):", - conf.ExecutionSegment.FloatLength()*100, len(schedulers), + "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", + conf.ExecutionSegment.FloatLength()*100, len(executors), lib.GetMaxPossibleVUs(plan), maxDuration), ) - for _, sched := range schedulers { + for _, sched := range executors { fprintf(stdout, " * %s: %s\n", sched.GetConfig().GetName(), sched.GetConfig().GetDescription(conf.ExecutionSegment)) } @@ -311,14 +311,14 @@ a commandline interface for interacting with it.`, case <-ticker.C: if quiet || !stdoutTTY { l := logrus.WithFields(logrus.Fields{ - "t": executorState.GetCurrentTestRunDuration(), - "i": executorState.GetFullIterationCount(), + "t": executionState.GetCurrentTestRunDuration(), + "i": executionState.GetFullIterationCount(), }) fn := l.Info if quiet { fn = l.Debug } - if executorState.IsPaused() { + if executionState.IsPaused() { fn("Paused") } else { fn("Running") @@ -358,8 +358,8 @@ a commandline interface for interacting with it.`, } if quiet || !stdoutTTY { e := logger.WithFields(logrus.Fields{ - "t": executorState.GetCurrentTestRunDuration(), - "i": executorState.GetFullIterationCount(), + "t": executionState.GetCurrentTestRunDuration(), + "i": executionState.GetFullIterationCount(), }) fn := e.Info if quiet { @@ -371,7 +371,7 @@ a commandline interface for interacting with it.`, progressBarWG.Wait() // Warn if no iterations could be completed. - if executorState.GetFullIterationCount() == 0 { + if executionState.GetFullIterationCount() == 0 { logger.Warn("No data generated, because no script iterations finished, consider making the test duration longer") } @@ -380,9 +380,9 @@ a commandline interface for interacting with it.`, fprintf(stdout, "\n") ui.Summarize(stdout, "", ui.SummaryData{ Opts: conf.Options, - Root: engine.Executor.GetRunner().GetDefaultGroup(), + Root: engine.ExecutionScheduler.GetRunner().GetDefaultGroup(), Metrics: engine.Metrics, - Time: executorState.GetCurrentTestRunDuration(), + Time: executionState.GetCurrentTestRunDuration(), }) fprintf(stdout, "\n") } diff --git a/cmd/ui.go b/cmd/ui.go index b5336de313f..0ed128135e2 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -98,13 +98,13 @@ func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { //TODO: show other information here? //TODO: add a no-progress option that will disable these //TODO: don't use global variables... -func showProgress(ctx context.Context, conf Config, executor *local.Executor) { +func showProgress(ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler) { if quiet || conf.HttpDebug.Valid && conf.HttpDebug.String != "" { return } - pbs := []*pb.ProgressBar{executor.GetInitProgressBar()} - for _, s := range executor.GetSchedulers() { + pbs := []*pb.ProgressBar{execScheduler.GetInitProgressBar()} + for _, s := range execScheduler.GetExecutors() { pbs = append(pbs, s.GetProgress()) } diff --git a/core/engine.go b/core/engine.go index 48aca3bd565..7a53f4e65e0 100644 --- a/core/engine.go +++ b/core/engine.go @@ -50,8 +50,8 @@ type Engine struct { runLock sync.Mutex // y tho? TODO: remove? //TODO: make most of the stuff here private! - Executor lib.Executor - executorState *lib.ExecutorState + ExecutionScheduler lib.ExecutionScheduler + executionState *lib.ExecutionState Options lib.Options Collectors []lib.Collector @@ -74,14 +74,14 @@ type Engine struct { } // NewEngine instantiates a new Engine, without doing any heavy initialization. -func NewEngine(ex lib.Executor, o lib.Options, logger *logrus.Logger) (*Engine, error) { +func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) (*Engine, error) { if ex == nil { - return nil, errors.New("missing executor instance") + return nil, errors.New("missing ExecutionScheduler instance") } e := &Engine{ - Executor: ex, - executorState: ex.GetState(), + ExecutionScheduler: ex, + executionState: ex.GetState(), Options: o, Metrics: make(map[string]*stats.Metric), @@ -103,10 +103,10 @@ func NewEngine(ex lib.Executor, o lib.Options, logger *logrus.Logger) (*Engine, return e, nil } -// Init is used to initialize the executor. That's a costly operation, since it +// Init is used to initialize the execuction scheduler. That's a costly operation, since it // initializes all of the planned VUs and could potentially take a long time. func (e *Engine) Init(ctx context.Context) error { - return e.Executor.Init(ctx, e.Samples) + return e.ExecutionScheduler.Init(ctx, e.Samples) } func (e *Engine) setRunStatus(status lib.RunStatus) { @@ -158,12 +158,12 @@ func (e *Engine) Run(ctx context.Context) error { }() } - // Run the executor. + // Run the execution scheduler. errC := make(chan error) subwg.Add(1) go func() { - errC <- e.Executor.Run(subctx, e.Samples) - e.logger.Debug("Engine: Executor terminated") + errC <- e.ExecutionScheduler.Run(subctx, e.Samples) + e.logger.Debug("Engine: Execution scheduler terminated") subwg.Done() }() @@ -213,11 +213,11 @@ func (e *Engine) Run(ctx context.Context) error { case err := <-errC: errC = nil if err != nil { - e.logger.WithError(err).Debug("run: executor returned an error") + e.logger.WithError(err).Debug("run: execution scheduler returned an error") e.setRunStatus(lib.RunStatusAbortedSystem) return err } - e.logger.Debug("run: executor terminated") + e.logger.Debug("run: execution scheduler terminated") return nil case <-ctx.Done(): e.logger.Debug("run: context expired; exiting...") @@ -246,18 +246,18 @@ func (e *Engine) runMetricsEmission(ctx context.Context) { func (e *Engine) emitMetrics() { t := time.Now() - executorState := e.Executor.GetState() + executionState := e.ExecutionScheduler.GetState() e.processSamples([]stats.SampleContainer{stats.ConnectedSamples{ Samples: []stats.Sample{ { Time: t, Metric: metrics.VUs, - Value: float64(executorState.GetCurrentlyActiveVUsCount()), + Value: float64(executionState.GetCurrentlyActiveVUsCount()), Tags: e.Options.RunTags, }, { Time: t, Metric: metrics.VUsMax, - Value: float64(executorState.GetInitializedVUsCount()), + Value: float64(executionState.GetInitializedVUsCount()), Tags: e.Options.RunTags, }, }, @@ -282,7 +282,7 @@ func (e *Engine) processThresholds(abort func()) { e.MetricsLock.Lock() defer e.MetricsLock.Unlock() - t := e.executorState.GetCurrentTestRunDuration() + t := e.executionState.GetCurrentTestRunDuration() abortOnFail := false e.thresholdsTainted = false diff --git a/core/engine_test.go b/core/engine_test.go index effb0361304..d1141fdb481 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -69,10 +69,10 @@ func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts li logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - executor, err := local.New(runner, logger) + execScheduler, err := local.NewExecutionScheduler(runner, logger) require.NoError(t, err) - engine, err := NewEngine(executor, opts, logger) + engine, err := NewEngine(execScheduler, opts, logger) require.NoError(t, err) require.NoError(t, engine.Init(ctx)) @@ -102,7 +102,7 @@ func TestEngineRun(t *testing.T) { Iterations: null.IntFrom(100), }) assert.NoError(t, e.Run(context.Background())) - assert.Equal(t, uint64(100), e.Executor.GetState().GetFullIterationCount()) + assert.Equal(t, uint64(100), e.ExecutionScheduler.GetState().GetFullIterationCount()) }) // Make sure samples are discarded after context close (using "cutoff" timestamp in local.go) t.Run("collects samples", func(t *testing.T) { diff --git a/core/local/local.go b/core/local/local.go index 49498d54171..844ed3da25f 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -34,105 +34,105 @@ import ( "github.com/sirupsen/logrus" ) -// Executor is the local implementation of lib.Executor -type Executor struct { +// ExecutionScheduler is the local implementation of lib.ExecutionScheduler +type ExecutionScheduler struct { runner lib.Runner options lib.Options logger *logrus.Logger initProgress *pb.ProgressBar - schedulers []lib.Scheduler // sorted by (startTime, ID) + executors []lib.Executor // sorted by (startTime, ID) executionPlan []lib.ExecutionStep maxDuration time.Duration // cached value derived from the execution plan maxPossibleVUs uint64 // cached value derived from the execution plan - state *lib.ExecutorState + state *lib.ExecutionState } -// Check to see if we implement the lib.Executor interface -var _ lib.Executor = &Executor{} +// Check to see if we implement the lib.ExecutionScheduler interface +var _ lib.ExecutionScheduler = &ExecutionScheduler{} -// New creates and returns a new local lib.Executor instance, without -// initializing it beyond the bare minimum. Specifically, it creates the needed -// schedulers instances and a lot of state placeholders, but it doesn't -// initialize the schedulers and it doesn't initialize or run any VUs. -func New(runner lib.Runner, logger *logrus.Logger) (*Executor, error) { +// NewExecutionScheduler creates and returns a new local lib.ExecutionScheduler +// instance, without initializing it beyond the bare minimum. Specifically, it +// creates the needed executor instances and a lot of state placeholders, but it +// doesn't initialize the executors and it doesn't initialize or run VUs. +func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*ExecutionScheduler, error) { options := runner.GetOptions() executionPlan := options.Execution.GetFullExecutionRequirements(options.ExecutionSegment) maxPlannedVUs := lib.GetMaxPlannedVUs(executionPlan) maxPossibleVUs := lib.GetMaxPossibleVUs(executionPlan) - executorState := lib.NewExecutorState(options, maxPlannedVUs, maxPossibleVUs) + executionState := lib.NewExecutionState(options, maxPlannedVUs, maxPossibleVUs) maxDuration, _ := lib.GetEndOffset(executionPlan) // we don't care if the end offset is final - schedulerConfigs := options.Execution.GetSortedSchedulerConfigs() - schedulers := make([]lib.Scheduler, len(schedulerConfigs)) - for i, sc := range schedulerConfigs { - s, err := sc.NewScheduler(executorState, logger.WithField("scheduler", sc.GetName())) + executorConfigs := options.Execution.GetSortedConfigs() + executors := make([]lib.Executor, len(executorConfigs)) + for i, sc := range executorConfigs { + s, err := sc.NewExecutor(executionState, logger.WithField("executor", sc.GetName())) if err != nil { return nil, err } - schedulers[i] = s + executors[i] = s } if options.Paused.Bool { - if err := executorState.Pause(); err != nil { + if err := executionState.Pause(); err != nil { return nil, err } } - return &Executor{ + return &ExecutionScheduler{ runner: runner, logger: logger, options: options, initProgress: pb.New(pb.WithConstLeft("Init")), - schedulers: schedulers, + executors: executors, executionPlan: executionPlan, maxDuration: maxDuration, maxPossibleVUs: maxPossibleVUs, - state: executorState, + state: executionState, }, nil } // GetRunner returns the wrapped lib.Runner instance. -func (e *Executor) GetRunner() lib.Runner { +func (e *ExecutionScheduler) GetRunner() lib.Runner { return e.runner } -// GetState returns a pointer to the executor state struct for the local -// executor. It's guaranteed to be initialized and present, though see -// the documentation in lib/executor.go for caveats about its usage. -// The most important one is that none of the methods beyond the pause-related -// ones should be used for synchronization. -func (e *Executor) GetState() *lib.ExecutorState { +// GetState returns a pointer to the execution state struct for the local +// execution scheduler. It's guaranteed to be initialized and present, though +// see the documentation in lib/execution.go for caveats about its usage. The +// most important one is that none of the methods beyond the pause-related ones +// should be used for synchronization. +func (e *ExecutionScheduler) GetState() *lib.ExecutionState { return e.state } -// GetSchedulers returns the slice of configured scheduler instances, sorted by +// GetExecutors returns the slice of configured executor instances, sorted by // their (startTime, name) in an ascending order. -func (e *Executor) GetSchedulers() []lib.Scheduler { - return e.schedulers +func (e *ExecutionScheduler) GetExecutors() []lib.Executor { + return e.executors } // GetInitProgressBar returns a the progress bar assotiated with the Init // function. After the Init is done, it is "hijacked" to display real-time // execution statistics as a text bar. -func (e *Executor) GetInitProgressBar() *pb.ProgressBar { +func (e *ExecutionScheduler) GetInitProgressBar() *pb.ProgressBar { return e.initProgress } -// GetExecutionPlan is a helper method so users of the local executor don't have -// to calculate the execution plan again. -func (e *Executor) GetExecutionPlan() []lib.ExecutionStep { +// GetExecutionPlan is a helper method so users of the local execution scheduler +// don't have to calculate the execution plan again. +func (e *ExecutionScheduler) GetExecutionPlan() []lib.ExecutionStep { return e.executionPlan } // initVU is just a helper method that's used to both initialize the planned VUs -// in the Init() method, and also passed to schedulers so they can initialize +// in the Init() method, and also passed to executors so they can initialize // any unplanned VUs themselves. //TODO: actually use the context... -func (e *Executor) initVU( +func (e *ExecutionScheduler) initVU( _ context.Context, logger *logrus.Entry, engineOut chan<- stats.SampleContainer, ) (lib.VU, error) { @@ -152,9 +152,9 @@ func (e *Executor) initVU( return vu, nil } -// getRunStats is a helper function that can be used as the executor's -// progressbar substitute (i.e. hijack). -func (e *Executor) getRunStats() string { +// getRunStats is a helper function that can be used as the execution +// scheduler's progressbar substitute (i.e. hijack). +func (e *ExecutionScheduler) getRunStats() string { status := "running" if e.state.IsPaused() { status = "paused" @@ -166,21 +166,21 @@ func (e *Executor) getRunStats() string { vusFmt := pb.GetFixedLengthIntFormat(int64(e.maxPossibleVUs)) return fmt.Sprintf( - "%s, "+vusFmt+"/"+vusFmt+" VUs, %d complete and %d incomplete iterations", + "%s, "+vusFmt+"/"+vusFmt+" VUs, %d complete and %d interrupted iterations", status, e.state.GetCurrentlyActiveVUsCount(), e.state.GetInitializedVUsCount(), e.state.GetFullIterationCount(), e.state.GetPartialIterationCount(), ) } // Init concurrently initializes all of the planned VUs and then sequentially -// initializes all of the configured schedulers. -func (e *Executor) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { - logger := e.logger.WithField("phase", "local-executor-init") +// initializes all of the configured executors. +func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { + logger := e.logger.WithField("phase", "local-execution-scheduler-init") vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) logger.WithFields(logrus.Fields{ - "neededVUs": vusToInitialize, - "schedulersCount": len(e.schedulers), + "neededVUs": vusToInitialize, + "executorsCount": len(e.executors), }).Debugf("Start of initialization") doneInits := make(chan error, vusToInitialize) // poor man's early-return waitgroup @@ -236,25 +236,25 @@ func (e *Executor) Init(ctx context.Context, engineOut chan<- stats.SampleContai return e.initVU(ctx, logger, engineOut) }) - logger.Debugf("Finished initializing needed VUs, start initializing schedulers...") - for _, sched := range e.schedulers { - schedConfig := sched.GetConfig() + logger.Debugf("Finished initializing needed VUs, start initializing executors...") + for _, executor := range e.executors { + executorConfig := executor.GetConfig() - if err := sched.Init(ctx); err != nil { - return fmt.Errorf("error while initializing scheduler %s: %s", schedConfig.GetName(), err) + if err := executor.Init(ctx); err != nil { + return fmt.Errorf("error while initializing executor %s: %s", executorConfig.GetName(), err) } - logger.Debugf("Initialized scheduler %s", schedConfig.GetName()) + logger.Debugf("Initialized executor %s", executorConfig.GetName()) } logger.Debugf("Initization completed") return nil } -// Run the Executor, funneling all generated metric samples through the supplied +// Run the ExecutionScheduler, funneling all generated metric samples through the supplied // out channel. -func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error { - schedulersCount := len(e.schedulers) - logger := e.logger.WithField("phase", "local-executor-run") +func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error { + executorsCount := len(e.executors) + logger := e.logger.WithField("phase", "local-execution-scheduler-run") e.initProgress.Modify(pb.WithConstLeft("Run")) if e.state.IsPaused() { @@ -272,14 +272,14 @@ func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContain defer e.state.MarkEnded() e.initProgress.Modify(pb.WithConstProgress(1, "running")) - logger.WithFields(logrus.Fields{"schedulersCount": schedulersCount}).Debugf("Start of test run") + logger.WithFields(logrus.Fields{"executorsCount": executorsCount}).Debugf("Start of test run") - runResults := make(chan error, schedulersCount) // nil values are successful runs + runResults := make(chan error, executorsCount) // nil values are successful runs runCtx, cancel := context.WithCancel(ctx) defer cancel() // just in case, and to shut up go vet... - // Run setup() before any schedulers, if it's not disabled + // Run setup() before any executors, if it's not disabled if !e.options.NoSetup.Bool { logger.Debug("Running setup()") e.initProgress.Modify(pb.WithConstProgress(1, "setup()")) @@ -291,54 +291,54 @@ func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContain e.initProgress.Modify(pb.WithHijack(e.getRunStats)) runCtxDone := runCtx.Done() - runScheduler := func(sched lib.Scheduler) { - schedConfig := sched.GetConfig() - schedStartTime := schedConfig.GetStartTime() - schedLogger := logger.WithFields(logrus.Fields{ - "scheduler": schedConfig.GetName(), - "type": schedConfig.GetType(), - "startTime": schedStartTime, + runExecutor := func(executor lib.Executor) { + executorConfig := executor.GetConfig() + executorStartTime := executorConfig.GetStartTime() + executorLogger := logger.WithFields(logrus.Fields{ + "executor": executorConfig.GetName(), + "type": executorConfig.GetType(), + "startTime": executorStartTime, }) - schedProgress := sched.GetProgress() + executorProgress := executor.GetProgress() - // Check if we have to wait before starting the actual scheduler execution - if schedStartTime > 0 { + // Check if we have to wait before starting the actual executor execution + if executorStartTime > 0 { startTime := time.Now() - schedProgress.Modify(pb.WithProgress(func() (float64, string) { - remWait := (schedStartTime - time.Since(startTime)) - return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, schedStartTime)) + executorProgress.Modify(pb.WithProgress(func() (float64, string) { + remWait := (executorStartTime - time.Since(startTime)) + return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) })) - schedLogger.Debugf("Waiting for scheduler start time...") + executorLogger.Debugf("Waiting for executor start time...") select { case <-runCtxDone: - runResults <- nil // no error since scheduler hasn't started yet + runResults <- nil // no error since executor hasn't started yet return - case <-time.After(schedStartTime): + case <-time.After(executorStartTime): // continue } } - schedProgress.Modify(pb.WithConstProgress(0, "started")) - schedLogger.Debugf("Starting scheduler") - err := sched.Run(runCtx, engineOut) // scheduler should handle context cancel itself + executorProgress.Modify(pb.WithConstProgress(0, "started")) + executorLogger.Debugf("Starting executor") + err := executor.Run(runCtx, engineOut) // executor should handle context cancel itself if err == nil { - schedLogger.Debugf("Scheduler finished successfully") + executorLogger.Debugf("Executor finished successfully") } else { - schedLogger.WithField("error", err).Errorf("Scheduler error") + executorLogger.WithField("error", err).Errorf("Executor error") } runResults <- err } - // Start all schedulers at their particular startTime in a separate goroutine... - logger.Debug("Start all schedulers...") - for _, sched := range e.schedulers { - go runScheduler(sched) + // Start all executors at their particular startTime in a separate goroutine... + logger.Debug("Start all executors...") + for _, executor := range e.executors { + go runExecutor(executor) } - // Wait for all schedulers to finish + // Wait for all executors to finish var firstErr error - for range e.schedulers { + for range e.executors { err := <-runResults if err != nil && firstErr == nil { firstErr = err @@ -346,7 +346,7 @@ func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContain } } - // Run teardown() after all schedulers are done, if it's not disabled + // Run teardown() after all executors are done, if it's not disabled if !e.options.NoTeardown.Bool { logger.Debug("Running teardown()") if err := e.runner.Teardown(ctx, engineOut); err != nil { @@ -358,10 +358,10 @@ func (e *Executor) Run(ctx context.Context, engineOut chan<- stats.SampleContain return firstErr } -// SetPaused pauses a test, if called with true. And if called with -// false, tries to start/resume it. See the lib.Executor interface documentation -// of the methods for the various caveats about its usage. -func (e *Executor) SetPaused(pause bool) error { +// SetPaused pauses a test, if called with true. And if called with false, tries +// to start/resume it. See the lib.ExecutionScheduler interface documentation of +// the methods for the various caveats about its usage. +func (e *ExecutionScheduler) SetPaused(pause bool) error { if !e.state.HasStarted() && e.state.IsPaused() { if pause { return fmt.Errorf("execution is already paused") @@ -370,15 +370,15 @@ func (e *Executor) SetPaused(pause bool) error { return e.state.Resume() } - for _, sched := range e.schedulers { - pausableSched, ok := sched.(lib.PausableScheduler) + for _, executor := range e.executors { + pausableExecutor, ok := executor.(lib.PausableExecutor) if !ok { return fmt.Errorf( - "%s scheduler '%s' doesn't support pause and resume operations after its start", - sched.GetConfig().GetType(), sched.GetConfig().GetName(), + "%s executor '%s' doesn't support pause and resume operations after its start", + executor.GetConfig().GetType(), executor.GetConfig().GetName(), ) } - if err := pausableSched.SetPaused(pause); err != nil { + if err := pausableExecutor.SetPaused(pause); err != nil { return err } } diff --git a/core/local/local_test.go b/core/local/local_test.go index 8cf2c0a0f42..eea635e8513 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -46,9 +46,9 @@ import ( null "gopkg.in/guregu/null.v3" ) -func newTestExecutor( +func newTestExecutionScheduler( t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, //nolint: golint -) (ctx context.Context, cancel func(), executor *Executor, samples chan stats.SampleContainer) { +) (ctx context.Context, cancel func(), execScheduler *ExecutionScheduler, samples chan stats.SampleContainer) { if runner == nil { runner = &lib.MiniRunner{} } @@ -66,7 +66,7 @@ func newTestExecutor( logger.SetOutput(testutils.NewTestOutput(t)) } - executor, err = New(runner, logger) + execScheduler, err = NewExecutionScheduler(runner, logger) require.NoError(t, err) samples = make(chan stats.SampleContainer, newOpts.MetricSamplesBufferSize.Int64) @@ -80,22 +80,22 @@ func newTestExecutor( } }() - require.NoError(t, executor.Init(ctx, samples)) + require.NoError(t, execScheduler.Init(ctx, samples)) - return ctx, cancel, executor, samples + return ctx, cancel, execScheduler, samples } -func TestExecutorRun(t *testing.T) { +func TestExecutionSchedulerRun(t *testing.T) { t.Parallel() - ctx, cancel, executor, samples := newTestExecutor(t, nil, nil, lib.Options{}) + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, nil, nil, lib.Options{}) defer cancel() err := make(chan error, 1) - go func() { err <- executor.Run(ctx, samples) }() + go func() { err <- execScheduler.Run(ctx, samples) }() assert.NoError(t, <-err) } -func TestExecutorSetupTeardownRun(t *testing.T) { +func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Parallel() t.Run("Normal", func(t *testing.T) { setupC := make(chan struct{}) @@ -110,10 +110,10 @@ func TestExecutorSetupTeardownRun(t *testing.T) { return nil }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{}) + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{}) err := make(chan error, 1) - go func() { err <- executor.Run(ctx, samples) }() + go func() { err <- execScheduler.Run(ctx, samples) }() defer cancel() <-setupC <-teardownC @@ -125,9 +125,9 @@ func TestExecutorSetupTeardownRun(t *testing.T) { return nil, errors.New("setup error") }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{}) + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{}) defer cancel() - assert.EqualError(t, executor.Run(ctx, samples), "setup error") + assert.EqualError(t, execScheduler.Run(ctx, samples), "setup error") }) t.Run("Don't Run Setup", func(t *testing.T) { runner := &lib.MiniRunner{ @@ -138,13 +138,13 @@ func TestExecutorSetupTeardownRun(t *testing.T) { return errors.New("teardown error") }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{ NoSetup: null.BoolFrom(true), VUs: null.IntFrom(1), Iterations: null.IntFrom(1), }) defer cancel() - assert.EqualError(t, executor.Run(ctx, samples), "teardown error") + assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") }) t.Run("Teardown Error", func(t *testing.T) { @@ -156,13 +156,13 @@ func TestExecutorSetupTeardownRun(t *testing.T) { return errors.New("teardown error") }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{ VUs: null.IntFrom(1), Iterations: null.IntFrom(1), }) defer cancel() - assert.EqualError(t, executor.Run(ctx, samples), "teardown error") + assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") }) t.Run("Don't Run Teardown", func(t *testing.T) { runner := &lib.MiniRunner{ @@ -173,17 +173,17 @@ func TestExecutorSetupTeardownRun(t *testing.T) { return errors.New("teardown error") }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{ NoTeardown: null.BoolFrom(true), VUs: null.IntFrom(1), Iterations: null.IntFrom(1), }) defer cancel() - assert.NoError(t, executor.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, samples)) }) } -func TestExecutorStages(t *testing.T) { +func TestExecutionSchedulerStages(t *testing.T) { t.Parallel() testdata := map[string]struct { Duration time.Duration @@ -218,18 +218,18 @@ func TestExecutorStages(t *testing.T) { return nil }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{ VUs: null.IntFrom(1), Stages: data.Stages, }) defer cancel() - assert.NoError(t, executor.Run(ctx, samples)) - assert.True(t, executor.GetState().GetCurrentTestRunDuration() >= data.Duration) + assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.True(t, execScheduler.GetState().GetCurrentTestRunDuration() >= data.Duration) }) } } -func TestExecutorEndTime(t *testing.T) { +func TestExecutionSchedulerEndTime(t *testing.T) { t.Parallel() runner := &lib.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { @@ -237,24 +237,24 @@ func TestExecutorEndTime(t *testing.T) { return nil }, } - ctx, cancel, executor, samples := newTestExecutor(t, runner, nil, lib.Options{ + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{ VUs: null.IntFrom(10), Duration: types.NullDurationFrom(1 * time.Second), }) defer cancel() - endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + endTime, isFinal := lib.GetEndOffset(execScheduler.GetExecutionPlan()) assert.Equal(t, 31*time.Second, endTime) // because of the default 30s gracefulStop assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, executor.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") } -func TestExecutorRuntimeErrors(t *testing.T) { +func TestExecutionSchedulerRuntimeErrors(t *testing.T) { t.Parallel() runner := &lib.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { @@ -267,15 +267,15 @@ func TestExecutorRuntimeErrors(t *testing.T) { }, } logger, hook := logtest.NewNullLogger() - ctx, cancel, executor, samples := newTestExecutor(t, runner, logger, lib.Options{}) + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, logger, lib.Options{}) defer cancel() - endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + endTime, isFinal := lib.GetEndOffset(execScheduler.GetExecutionPlan()) assert.Equal(t, 31*time.Second, endTime) // because of the default 30s gracefulStop assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, executor.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") @@ -286,7 +286,7 @@ func TestExecutorRuntimeErrors(t *testing.T) { } } -func TestExecutorEndErrors(t *testing.T) { +func TestExecutionSchedulerEndErrors(t *testing.T) { t.Parallel() scheduler := scheduler.NewConstantLoopingVUsConfig("we_need_hard_stop") @@ -300,19 +300,19 @@ func TestExecutorEndErrors(t *testing.T) { return errors.New("hi") }, Options: lib.Options{ - Execution: lib.SchedulerConfigMap{scheduler.GetName(): scheduler}, + Execution: lib.ExecutorConfigMap{scheduler.GetName(): scheduler}, }, } logger, hook := logtest.NewNullLogger() - ctx, cancel, executor, samples := newTestExecutor(t, runner, logger, lib.Options{}) + ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, logger, lib.Options{}) defer cancel() - endTime, isFinal := lib.GetEndOffset(executor.GetExecutionPlan()) + endTime, isFinal := lib.GetEndOffset(execScheduler.GetExecutionPlan()) assert.Equal(t, 1*time.Second, endTime) // because of the 0s gracefulStop assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, executor.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") @@ -320,7 +320,7 @@ func TestExecutorEndErrors(t *testing.T) { assert.Empty(t, hook.Entries) } -func TestExecutorEndIterations(t *testing.T) { +func TestExecutionSchedulerEndIterations(t *testing.T) { t.Parallel() metric := &stats.Metric{Name: "test_metric"} @@ -351,15 +351,15 @@ func TestExecutorEndIterations(t *testing.T) { logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - executor, err := New(runner, logger) + execScheduler, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) samples := make(chan stats.SampleContainer, 300) - require.NoError(t, executor.Init(ctx, samples)) - require.NoError(t, executor.Run(ctx, samples)) + require.NoError(t, execScheduler.Init(ctx, samples)) + require.NoError(t, execScheduler.Run(ctx, samples)) - assert.Equal(t, uint64(100), executor.GetState().GetFullIterationCount()) - assert.Equal(t, uint64(0), executor.GetState().GetPartialIterationCount()) + assert.Equal(t, uint64(100), execScheduler.GetState().GetFullIterationCount()) + assert.Equal(t, uint64(0), execScheduler.GetState().GetPartialIterationCount()) assert.Equal(t, int64(100), i) require.Equal(t, 200, len(samples)) for i := 0; i < 100; i++ { @@ -375,7 +375,7 @@ func TestExecutorEndIterations(t *testing.T) { } } -func TestExecutorIsRunning(t *testing.T) { +func TestExecutionSchedulerIsRunning(t *testing.T) { t.Parallel() runner := &lib.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { @@ -383,11 +383,11 @@ func TestExecutorIsRunning(t *testing.T) { return nil }, } - ctx, cancel, executor, _ := newTestExecutor(t, runner, nil, lib.Options{}) - state := executor.GetState() + ctx, cancel, execScheduler, _ := newTestExecutionScheduler(t, runner, nil, lib.Options{}) + state := execScheduler.GetState() err := make(chan error) - go func() { err <- executor.Run(ctx, nil) }() + go func() { err <- execScheduler.Run(ctx, nil) }() for !state.HasStarted() { time.Sleep(10 * time.Microsecond) } @@ -399,8 +399,8 @@ func TestExecutorIsRunning(t *testing.T) { } /* -//TODO: convert for the manual-execution scheduler -func TestExecutorSetVUs(t *testing.T) { +//TODO: convert for the externally-controlled scheduler +func TestExecutionSchedulerSetVUs(t *testing.T) { t.Run("Negative", func(t *testing.T) { assert.EqualError(t, New(nil).SetVUs(-1), "vu count can't be negative") }) @@ -553,7 +553,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - executor, err := New(runner, logger) + execScheduler, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) @@ -562,8 +562,8 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { done := make(chan struct{}) sampleContainers := make(chan stats.SampleContainer) go func() { - require.NoError(t, executor.Init(ctx, sampleContainers)) - assert.NoError(t, executor.Run(ctx, sampleContainers)) + require.NoError(t, execScheduler.Init(ctx, sampleContainers)) + assert.NoError(t, execScheduler.Run(ctx, sampleContainers)) close(done) }() @@ -622,7 +622,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { return netext.NewDialer(net.Dialer{}).GetTrail(time.Now(), time.Now(), true, getTags("group", group)) } - // Initially give a long time (5s) for the executor to start + // Initially give a long time (5s) for the execScheduler 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")) @@ -646,7 +646,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { 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") + t.Fatalf("Local execScheduler took way to long to finish") case <-done: return // Exit normally } diff --git a/js/runner_test.go b/js/runner_test.go index 7c845dd4700..2b90d2b1a6b 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -277,9 +277,9 @@ func TestSetupDataIsolation(t *testing.T) { options := runner.GetOptions() require.Empty(t, options.Validate()) - executor, err := local.New(runner, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(runner, logrus.StandardLogger()) require.NoError(t, err) - engine, err := core.NewEngine(executor, options, logrus.StandardLogger()) + engine, err := core.NewEngine(execScheduler, options, logrus.StandardLogger()) require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) diff --git a/lib/doc.go b/lib/doc.go index 4853c84ab0b..d653a9e24f5 100644 --- a/lib/doc.go +++ b/lib/doc.go @@ -18,12 +18,13 @@ * */ -// Package lib is a kitchen sink of... basically anything that doesn't belong in a specific part of -// the codebase, ranging from utility functions to universal types to core interfaces. +// Package lib is a kitchen sink of... basically anything that doesn't belong in +// a specific part of the codebase, ranging from utility functions to universal +// types to core interfaces. // -// Some of the things in lib do not actually belong in lib, most notably the Executor and Runner, -// which arguably belong in core. Other things are in files that are far too long, or that do not -// actually make sense. +// Some of the things in lib do not actually belong in lib, and need to be moved +// into either submodules, or separate modules like core. Other things are in +// files that are far too long, or that do not actually make sense. // // Feel free to move these things. package lib diff --git a/lib/executor.go b/lib/executor.go index 8e9880d2c0d..79bd138a57e 100644 --- a/lib/executor.go +++ b/lib/executor.go @@ -33,80 +33,81 @@ import ( "github.com/pkg/errors" ) -// An Executor is in charge of initializing schedulers and using them to -// initialize and schedule VUs created by a wrapped Runner. It decouples how a -// swarm of VUs is controlled from the details of how or even where they're +// An ExecutionScheduler is in charge of initializing executors and using them +// to initialize and schedule VUs created by a wrapped Runner. It decouples how +// a swarm of VUs is controlled from the details of how or even where they're // scheduled. // -// The core/local executor schedules VUs on the local machine, but the same -// interface may be implemented to control a test running on a cluster or in the -// cloud. -//TODO: flesh out the interface after actually having more than one implementation... -type Executor interface { +// The core/local execution scheduler schedules VUs on the local machine, but +// the same interface may be implemented to control a test running on a cluster +// or in the cloud. +// +// TODO: flesh out the interface after actually having more than one +// implementation... +type ExecutionScheduler interface { // Returns the wrapped runner. May return nil if not applicable, eg. // if we're remote controlling a test running on another machine. GetRunner() Runner - // Return the ExecutorState instance from which different statistics for the + // Return the ExecutionState instance from which different statistics for the // current state of the runner could be retrieved. - GetState() *ExecutorState + GetState() *ExecutionState - // Return the instances of the configured schedulers - GetSchedulers() []Scheduler + // Return the instances of the configured executors + GetExecutors() []Executor - // Init initializes all schedulers, including all of their needed VUs. + // Init initializes all executors, including all of their needed VUs. Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error - // Run the Executor, funneling generated metric samples through the supplied - // out channel. + // Run the ExecutionScheduler, funneling the generated metric samples + // through the supplied out channel. Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error // Pause a test, or start/resume it. To check if a test is paused, use // GetState().IsPaused(). // - // Currently, any test can be started in a paused state. This will cause k6 - // to initialize all needed VUs, but it won't actually start the test. - // Later, the test can be started for real be resuming/unpausing it from the - // REST API. + // Currently, any executor, so any test, can be started in a paused state. + // This will cause k6 to initialize all needed VUs, but it won't actually + // start the test. Later, the test can be started for real be + // resuming/unpausing it from the REST API. // // After a test is actually started, it may become impossible to pause it // again. That is denoted by having SetPaused(true) return an error. The - // likely cause is that some of the schedulers for the test don't support + // likely cause is that some of the executors for the test don't support // pausing after the test has been started. // - // IMPORTANT: Currently only the manual scheduler can be paused and resumed - // multiple times in the middle of the test execution! Even then, "pausing" - // is a bit misleading, since k6 won't pause in the middle of the currently - // executing iterations. It will allow the currently in progress iterations - // to finish, and it just won't start any new ones nor will it increment - // the value returned by GetCurrentTestRunDuration(). + // IMPORTANT: Currently only the externally controlled executor can be + // paused and resumed multiple times in the middle of the test execution! + // Even then, "pausing" is a bit misleading, since k6 won't pause in the + // middle of the currently executing iterations. It will allow the currently + // in progress iterations to finish, and it just won't start any new ones + // nor will it increment the value returned by GetCurrentTestRunDuration(). SetPaused(paused bool) error } -// MaxTimeToWaitForPlannedVU specifies the maximum allowable time for a -// scheduler to wait for a planned VU to be retrieved from the -// ExecutorState.PlannedVUs buffer. If it's exceeded, k6 will emit a warning log -// message, since it either means that there's a bug in the k6 scheduling code, -// or that the machine is overloaded and the scheduling code suffers from -// delays. +// MaxTimeToWaitForPlannedVU specifies the maximum allowable time for a executor +// to wait for a planned VU to be retrieved from the ExecutionState.PlannedVUs +// buffer. If it's exceeded, k6 will emit a warning log message, since it either +// means that there's a bug in the k6 scheduling code, or that the machine is +// overloaded and the scheduling code suffers from delays. // -// Critically, exceeding this time *doesn't* result in an aborted test -// or any test errors, and the scheduler will continue to try and borrow the VU +// Critically, exceeding this time *doesn't* result in an aborted test or any +// test errors, and the executor will continue to try and borrow the VU // (potentially resulting in further warnings). We likely should emit a k6 -// metric about it in the future. -// TODO: emit a metric every time this is exceeded? +// metric about it in the future. TODO: emit a metric every time this is +// exceeded? const MaxTimeToWaitForPlannedVU = 400 * time.Millisecond // MaxRetriesGetPlannedVU how many times we should wait for // MaxTimeToWaitForPlannedVU before we actually return an error. const MaxRetriesGetPlannedVU = 5 -// ExecutorState contains a few different things: -// - Some convenience items, that are needed by all schedulers, like the +// ExecutionState contains a few different things: +// - Some convenience items, that are needed by all executors, like the // execution segment and the unique VU ID generator. By keeping those here, -// we can just pass the ExecutorState to the different schedulers, instead of +// we can just pass the ExecutionState to the different executors, instead of // individually passing them each item. -// - Mutable counters that different schedulers modify and other parts of +// - Mutable counters that different executors modify and other parts of // k6 can read, e.g. for the vus and vus_max metrics k6 emits every second. // - Pausing controls and statistics. // @@ -119,8 +120,8 @@ const MaxRetriesGetPlannedVU = 5 // The only functionality indended for synchronization is the one revolving // around pausing, and uninitializedUnplannedVUs for restrictring the number of // unplanned VUs being initialized. -type ExecutorState struct { - // A copy of the options, so the different schedulers have access to them. +type ExecutionState struct { + // A copy of the options, so the different executors have access to them. // They will need to access things like the current execution segment, the // per-run metrics tags, etc. // @@ -130,24 +131,24 @@ type ExecutorState struct { Options Options // vus is the shared channel buffer that contains all of the VUs that have - // been initialized and aren't currently being used by a scheduler. + // been initialized and aren't currently being used by a executor. // // It contains both pre-initialized (i.e. planned) VUs, as well as any // unplanned VUs. Planned VUs are initialized before a test begins, while // unplanned VUS can be initialized in the middle of the test run by a - // scheduler and have been relinquished after it has finished working with + // executor and have been relinquished after it has finished working with // them. Usually, unplanned VUs are initialized by one of the arrival-rate - // schedulers, after they have exhausted their PreAllocatedVUs. After the - // scheduler is done with the VUs, it will put in this channel, so it could - // potentially be reused by other schedulers further along in the test. + // executors, after they have exhausted their PreAllocatedVUs. After the + // executor is done with the VUs, it will put in this channel, so it could + // potentially be reused by other executors further along in the test. // - // Different schedulers cooperatively borrow VUs from here when they are + // Different executors cooperatively borrow VUs from here when they are // needed and return them when they are done with them. There's no central - // enforcement of correctness, i.e. that a scheduler takes more VUs from + // enforcement of correctness, i.e. that a executor takes more VUs from // here than its execution plan has stipulated. The correctness guarantee - // lies with the actual schedulers - bugs in one can affect others. + // lies with the actual executors - bugs in one can affect others. // - // That's why the field is private and we force schedulers to use the + // That's why the field is private and we force executors to use the // GetPlannedVU(), GetUnplannedVU(), and ReturnVU() methods instead of work // directly with the channel. These methods will emit a warning or can even // return an error if retrieving a VU takes more than @@ -178,7 +179,7 @@ type ExecutorState struct { // no more unplanned VUs can be initialized. uninitializedUnplannedVUs *int64 - // Injected when the executor's init function is called, used for + // Injected when the execution scheduler's Init function is called, used for // initializing unplanned VUs. initVUFunc InitVUFunc @@ -196,7 +197,8 @@ type ExecutorState struct { // The total number of iterations that have been interrupted during their // execution. The potential interruption causes vary - end of a specified // script `duration`, scaling down of VUs via `stages`, a user hitting - // Ctrl+C, change of `vus` via the manual executor's REST API, etc. + // Ctrl+C, change of `vus` via the externally controlled executor's REST + // API, etc. partialIterationsCount *uint64 // A nanosecond UNIX timestamp that is set when the test is actually @@ -209,8 +211,8 @@ type ExecutorState struct { // used to denote that the test hasn't ended yet... endTime *int64 - // Stuff related to pausing follows. Read the docs in Executor for more - // information regarding how pausing works in k6. + // Stuff related to pausing follows. Read the docs in ExecutionScheduler for + // more information regarding how pausing works in k6. // // When we pause the execution in the middle of the test, we save the // current timestamp in currentPauseTime. When we resume the execution, we @@ -240,16 +242,16 @@ type ExecutorState struct { resumeNotify chan struct{} } -// NewExecutorState initializes all of the pointers in the ExecutorState +// NewExecutionState initializes all of the pointers in the ExecutionState // with zeros. It also makes sure that the initial state is unpaused, by // setting resumeNotify to an already closed channel. -func NewExecutorState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *ExecutorState { +func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *ExecutionState { resumeNotify := make(chan struct{}) - close(resumeNotify) // By default the ExecutorState starts unpaused + close(resumeNotify) // By default the ExecutionState starts unpaused maxUnplannedUninitializedVUs := int64(maxPossibleVUs - maxPlannedVUs) - return &ExecutorState{ + return &ExecutionState{ Options: options, vus: make(chan VU, maxPossibleVUs), @@ -270,7 +272,7 @@ func NewExecutorState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *Ex // GetUniqueVUIdentifier returns an auto-incrementing unique VU ID, used for __VU. // It starts from 1 (for backwards compatibility...) -func (es *ExecutorState) GetUniqueVUIdentifier() uint64 { +func (es *ExecutionState) GetUniqueVUIdentifier() uint64 { return atomic.AddUint64(es.currentVUIdentifier, 1) } @@ -281,14 +283,14 @@ func (es *ExecutorState) GetUniqueVUIdentifier() uint64 { // exported script options and for the execution of setup() and teardown() // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetInitializedVUsCount() int64 { +func (es *ExecutionState) GetInitializedVUsCount() int64 { return atomic.LoadInt64(es.initializedVUs) } // ModInitializedVUsCount changes the total number of currently initialized VUs. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) ModInitializedVUsCount(mod int64) int64 { +func (es *ExecutionState) ModInitializedVUsCount(mod int64) int64 { return atomic.AddInt64(es.initializedVUs, mod) } @@ -297,14 +299,14 @@ func (es *ExecutorState) ModInitializedVUsCount(mod int64) int64 { // of gracefullt winding down. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetCurrentlyActiveVUsCount() int64 { +func (es *ExecutionState) GetCurrentlyActiveVUsCount() int64 { return atomic.LoadInt64(es.activeVUs) } // ModCurrentlyActiveVUsCount changes the total number of currently active VUs. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) ModCurrentlyActiveVUsCount(mod int64) int64 { +func (es *ExecutionState) ModCurrentlyActiveVUsCount(mod int64) int64 { return atomic.AddInt64(es.activeVUs, mod) } @@ -312,7 +314,7 @@ func (es *ExecutorState) ModCurrentlyActiveVUsCount(mod int64) int64 { // that have been completed so far. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetFullIterationCount() uint64 { +func (es *ExecutionState) GetFullIterationCount() uint64 { return atomic.LoadUint64(es.fullIterationsCount) } @@ -320,7 +322,7 @@ func (es *ExecutorState) GetFullIterationCount() uint64 { // by the provided amount. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) AddFullIterations(count uint64) uint64 { +func (es *ExecutionState) AddFullIterations(count uint64) uint64 { return atomic.AddUint64(es.fullIterationsCount, count) } @@ -328,7 +330,7 @@ func (es *ExecutorState) AddFullIterations(count uint64) uint64 { // iterations that have been completed so far. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetPartialIterationCount() uint64 { +func (es *ExecutionState) GetPartialIterationCount() uint64 { return atomic.LoadUint64(es.partialIterationsCount) } @@ -336,7 +338,7 @@ func (es *ExecutorState) GetPartialIterationCount() uint64 { // iterations by the provided amount. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) AddPartialIterations(count uint64) uint64 { +func (es *ExecutionState) AddPartialIterations(count uint64) uint64 { return atomic.AddUint64(es.partialIterationsCount, count) } @@ -344,9 +346,9 @@ func (es *ExecutorState) AddPartialIterations(count uint64) uint64 { // // CAUTION: Calling MarkStarted() a second time for the same execution state will // result in a panic! -func (es *ExecutorState) MarkStarted() { +func (es *ExecutionState) MarkStarted() { if !atomic.CompareAndSwapInt64(es.startTime, 0, time.Now().UnixNano()) { - panic("the executor was started a second time") + panic("the execution scheduler was started a second time") } } @@ -354,9 +356,9 @@ func (es *ExecutorState) MarkStarted() { // // CAUTION: Calling MarkEnded() a second time for the same execution state will // result in a panic! -func (es *ExecutorState) MarkEnded() { +func (es *ExecutionState) MarkEnded() { if !atomic.CompareAndSwapInt64(es.endTime, 0, time.Now().UnixNano()) { - panic("the executor was started a second time") + panic("the execution scheduler was stopped a second time") } } @@ -364,20 +366,20 @@ func (es *ExecutorState) MarkEnded() { // It will return false while a test is in the init phase, or if it has // been initially paused. But if will return true if a test is paused // midway through its execution (see above for details regarind the -// feasibility of that pausing for normal schedulers). -func (es *ExecutorState) HasStarted() bool { +// feasibility of that pausing for normal executors). +func (es *ExecutionState) HasStarted() bool { return atomic.LoadInt64(es.startTime) != 0 } // HasEnded returns true if the test has finished executing. It will return // false until MarkEnded() is called. -func (es *ExecutorState) HasEnded() bool { +func (es *ExecutionState) HasEnded() bool { return atomic.LoadInt64(es.endTime) != 0 } // IsPaused quickly returns whether the test is currently paused, by reading // the atomic currentPauseTime timestamp -func (es *ExecutorState) IsPaused() bool { +func (es *ExecutionState) IsPaused() bool { return atomic.LoadInt64(es.currentPauseTime) != 0 } @@ -387,7 +389,7 @@ func (es *ExecutorState) IsPaused() bool { // And if it's currently running, it will return the time since the start time. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutorState) GetCurrentTestRunDuration() time.Duration { +func (es *ExecutionState) GetCurrentTestRunDuration() time.Duration { startTime := atomic.LoadInt64(es.startTime) if startTime == 0 { // The test hasn't started yet @@ -416,7 +418,7 @@ func (es *ExecutorState) GetCurrentTestRunDuration() time.Duration { // the current timestamp in currentPauseTime, and makes a new // channel for resumeNotify. // Pause can return an error if the test was already paused. -func (es *ExecutorState) Pause() error { +func (es *ExecutionState) Pause() error { es.pauseStateLock.Lock() defer es.pauseStateLock.Unlock() @@ -431,7 +433,7 @@ func (es *ExecutorState) Pause() error { // yet started, it calculates the duration between now and // the old currentPauseTime and adds it to // Resume will emit an error if the test wasn't paused. -func (es *ExecutorState) Resume() error { +func (es *ExecutionState) Resume() error { es.pauseStateLock.Lock() defer es.pauseStateLock.Unlock() @@ -464,7 +466,7 @@ func (es *ExecutorState) Resume() error { // if executionState.IsPaused() { // <-executionState.ResumeNotify() // } -func (es *ExecutorState) ResumeNotify() <-chan struct{} { +func (es *ExecutionState) ResumeNotify() <-chan struct{} { es.pauseStateLock.RLock() defer es.pauseStateLock.RUnlock() return es.resumeNotify @@ -475,8 +477,8 @@ func (es *ExecutorState) ResumeNotify() <-chan struct{} { // doesn't happen for MaxTimeToWaitForPlannedVU, a warning will be printed. If // we reach that timeout more than MaxRetriesGetPlannedVU number of times, this // function will return an error, since we either have a bug with some -// scheduler, or the machine is very, very overloaded. -func (es *ExecutorState) GetPlannedVU(logger *logrus.Entry, modifyAtiveVUCount bool) (VU, error) { +// executor, or the machine is very, very overloaded. +func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyAtiveVUCount bool) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { case vu := <-es.vus: @@ -495,11 +497,12 @@ func (es *ExecutorState) GetPlannedVU(logger *logrus.Entry, modifyAtiveVUCount b ) } -// SetInitVUFunc is called by the executor's init function, and it's used for -// setting the "constructor" function used for the initializing unplanned VUs. +// SetInitVUFunc is called by the execution scheduler's init function, and it's +// used for setting the "constructor" function used for the initializing +// unplanned VUs. // // TODO: figure out a better dependency injection method? -func (es *ExecutorState) SetInitVUFunc(initVUFunc InitVUFunc) { +func (es *ExecutionState) SetInitVUFunc(initVUFunc InitVUFunc) { es.initVUFunc = initVUFunc } @@ -508,11 +511,11 @@ func (es *ExecutorState) SetInitVUFunc(initVUFunc InitVUFunc) { // been initialized, it returns one from the global vus buffer. // // IMPORTANT: GetUnplannedVU() doesn't do any checking if the requesting -// scheduler is actually allowed to have the VU at this particular time. -// Schedulers are trusted to correctly declare their needs (via their +// executor is actually allowed to have the VU at this particular time. +// Executors are trusted to correctly declare their needs (via their // GetExecutionRequirements() methods) and then to never ask for more VUs than // they have specified in those requirements. -func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { +func (es *ExecutionState) GetUnplannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { remVUs := atomic.AddInt64(es.uninitializedUnplannedVUs, -1) if remVUs < 0 { logger.Debug("Reusing a previously initialized unplanned VU") @@ -530,9 +533,9 @@ func (es *ExecutorState) GetUnplannedVU(ctx context.Context, logger *logrus.Entr // InitializeNewVU creates and returns a brand new VU, updating the relevant // tracking counters. -func (es *ExecutorState) InitializeNewVU(ctx context.Context, logger *logrus.Entry) (VU, error) { +func (es *ExecutionState) InitializeNewVU(ctx context.Context, logger *logrus.Entry) (VU, error) { if es.initVUFunc == nil { - return nil, fmt.Errorf("initVUFunc wasn't set in the executor state") + return nil, fmt.Errorf("initVUFunc wasn't set in the execution state") } newVU, err := es.initVUFunc(ctx, logger) if err != nil { @@ -544,14 +547,14 @@ func (es *ExecutorState) InitializeNewVU(ctx context.Context, logger *logrus.Ent // AddInitializedVU is a helper function that adds VUs into the buffer and // increases the initialized VUs counter. -func (es *ExecutorState) AddInitializedVU(vu VU) { +func (es *ExecutionState) AddInitializedVU(vu VU) { es.vus <- vu es.ModInitializedVUsCount(+1) } // ReturnVU is a helper function that puts VUs back into the buffer and // decreases the active VUs counter. -func (es *ExecutorState) ReturnVU(vu VU, wasActive bool) { +func (es *ExecutionState) ReturnVU(vu VU, wasActive bool) { es.vus <- vu if wasActive { es.ModCurrentlyActiveVUsCount(-1) diff --git a/lib/executor_test.go b/lib/executor_test.go index dd3d6c10176..fbc9840ac0a 100644 --- a/lib/executor_test.go +++ b/lib/executor_test.go @@ -29,8 +29,8 @@ import ( "github.com/stretchr/testify/assert" ) -func TestExecutorStateVUIDs(t *testing.T) { - es := NewExecutorState(Options{}, 0, 0) //TODO: fix +func TestExecutionStateVUIDs(t *testing.T) { + es := NewExecutionState(Options{}, 0, 0) //TODO: fix assert.Equal(t, uint64(1), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(2), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(3), es.GetUniqueVUIdentifier()) diff --git a/lib/helpers.go b/lib/helpers.go index 04ffb3bf02d..7953f288ad5 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -47,7 +47,7 @@ func StrictJSONUnmarshal(data []byte, v interface{}) error { } // GetMaxPlannedVUs returns the maximum number of planned VUs at any stage of -// the scheduler execution plan. +// the execution plan. func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { for _, s := range steps { stepMaxPlannedVUs := s.PlannedVUs @@ -59,26 +59,25 @@ func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { } // GetMaxPossibleVUs returns the maximum number of planned + unplanned (i.e. -// initialized mid-test) VUs at any stage of the scheduler execution plan. -// Unplanned VUs are possible in some schedulers, like the arrival-rate ones, as -// a way to have a low number of pre-allocated VUs, but be able to initialize -// new ones in the middle of the test, if needed. For example, if the remote -// system starts responding very slowly and all of the pre-allocated VUs are -// waiting for it. +// initialized mid-test) VUs at any stage of the execution plan. Unplanned VUs +// are possible in some executors, like the arrival-rate ones, as a way to have +// a low number of pre-allocated VUs, but be able to initialize new ones in the +// middle of the test, if needed. For example, if the remote system starts +// responding very slowly and all of the pre-allocated VUs are waiting for it. // // IMPORTANT 1: Getting planned and unplanned VUs separately for the whole // duration of a test can often lead to mistakes. That's why this function is // called GetMaxPossibleVUs() and why there is no GetMaxUnplannedVUs() function. // -// As an example, imagine that you have a scheduler with MaxPlannedVUs=20 and -// MaxUnaplannedVUs=0, followed immediately after by another scheduler with +// As an example, imagine that you have a executor with MaxPlannedVUs=20 and +// MaxUnaplannedVUs=0, followed immediately after by another executor with // MaxPlannedVUs=10 and MaxUnaplannedVUs=10. The MaxPlannedVUs number for the -// whole test is 20, and MaxUnaplannedVUs is 10, but since those schedulers -// won't run concurrently, MaxVUs for the whole test is not 30, rather it's 20, -// since 20 VUs will be sufficient to run the whole test. +// whole test is 20, and MaxUnaplannedVUs is 10, but since those executors won't +// run concurrently, MaxVUs for the whole test is not 30, rather it's 20, since +// 20 VUs will be sufficient to run the whole test. // -// IMPORTANT 2: this has one very important exception. The manual execution -// scheduler doesn't use the MaxUnplannedVUs (i.e. this function will return 0), +// IMPORTANT 2: this has one very important exception. The externally controlled +// executor doesn't use the MaxUnplannedVUs (i.e. this function will return 0), // since their initialization and usage is directly controlled by the user and // is effectively bounded only by the resources of the machine k6 is running on. func GetMaxPossibleVUs(steps []ExecutionStep) (result uint64) { diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index d294dfd2ca7..d87ef2dde1c 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -209,7 +209,7 @@ func TestFilenamePwdResolve(t *testing.T) { } func TestDerivedExecutionDiscarding(t *testing.T) { - var emptyConfigMap SchedulerConfigMap + var emptyConfigMap ExecutorConfigMap var tests = []struct { metadata string expExecution interface{} diff --git a/lib/options.go b/lib/options.go index dee96a8279b..0d88ca8e98e 100644 --- a/lib/options.go +++ b/lib/options.go @@ -35,15 +35,14 @@ import ( "gopkg.in/guregu/null.v3" ) -// DefaultSchedulerName is used as the default key/ID of the scheduler config entries +// DefaultExecutorName is used as the default key/ID of the executor config entries // that were created due to the use of the shortcut execution control options (i.e. duration+vus, // iterations+vus, or stages) -const DefaultSchedulerName = "default" +const DefaultExecutorName = "default" // DefaultSystemTagList includes all of the system tags emitted with metrics by default. // Other tags that are not enabled by default include: iter, vu, ocsp_status, ip var DefaultSystemTagList = []string{ - "proto", "subproto", "status", "method", "url", "name", "group", "check", "error", "error_code", "tls_version", } @@ -243,10 +242,9 @@ type Options struct { // Should the test start in a paused state? Paused null.Bool `json:"paused" envconfig:"paused"` - // Initial values for VUs, max VUs, duration cap, iteration cap, and stages. - // See the Runner or Executor interfaces for more information. - VUs null.Int `json:"vus" envconfig:"vus"` - + // Execution shortcut options - see the execution option below for all + // possibilities and the executors and ExecutionScheduler for more info. + VUs null.Int `json:"vus" envconfig:"vus"` Duration types.NullDuration `json:"duration" envconfig:"duration"` Iterations null.Int `json:"iterations" envconfig:"iterations"` Stages []Stage `json:"stages" envconfig:"stages"` @@ -257,8 +255,8 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Execution SchedulerConfigMap `json:"execution,omitempty" ignored:"true"` - ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` + Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` + ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` // Timeouts for the setup() and teardown() functions NoSetup null.Bool `json:"noSetup" envconfig:"NO_SETUP"` diff --git a/lib/runner.go b/lib/runner.go index 853836c98b8..c7f4a62bd09 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -30,12 +30,14 @@ import ( 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. -// The Runner doesn't actually *do* anything in itself, the Executor is responsible for wrapping -// and scheduling these VUs for execution. +// 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. The Runner doesn't actually *do* anything in +// itself, the ExecutionScheduler is responsible for wrapping and scheduling +// these VUs for execution. // -// TODO: Rename this to something more obvious? This name made sense a very long time ago. +// TODO: Rename this to something more obvious? This name made sense a very long +// time ago. type Runner interface { // Creates an Archive of the runner. There should be a corresponding NewFromArchive() function // that will restore the runner from the archive. diff --git a/lib/scheduler/base_config.go b/lib/scheduler/base_config.go index 377a9907532..6fe6aaa0114 100644 --- a/lib/scheduler/base_config.go +++ b/lib/scheduler/base_config.go @@ -30,10 +30,10 @@ import ( null "gopkg.in/guregu/null.v3" ) -var schedulerNameWhitelist = regexp.MustCompile(`^[0-9a-zA-Z_-]+$`) //nolint:gochecknoglobals -const schedulerNameErr = "the scheduler name should contain only numbers, latin letters, underscores, and dashes" +var executorNameWhitelist = regexp.MustCompile(`^[0-9a-zA-Z_-]+$`) //nolint:gochecknoglobals +const executorNameErr = "the executor name should contain only numbers, latin letters, underscores, and dashes" -// BaseConfig contains the common config fields for all schedulers +// BaseConfig contains the common config fields for all executors type BaseConfig struct { Name string `json:"-"` // set via the JS object key Type string `json:"type"` @@ -59,10 +59,10 @@ func (bc BaseConfig) Validate() (errors []error) { // Some just-in-case checks, since those things are likely checked in other places or // even assigned by us: if bc.Name == "" { - errors = append(errors, fmt.Errorf("scheduler name shouldn't be empty")) + errors = append(errors, fmt.Errorf("executor name shouldn't be empty")) } - if !schedulerNameWhitelist.MatchString(bc.Name) { - errors = append(errors, fmt.Errorf(schedulerNameErr)) + if !executorNameWhitelist.MatchString(bc.Name) { + errors = append(errors, fmt.Errorf(executorNameErr)) } if bc.Exec.Valid && bc.Exec.String == "" { errors = append(errors, fmt.Errorf("exec value cannot be empty")) @@ -80,24 +80,24 @@ func (bc BaseConfig) Validate() (errors []error) { return errors } -// GetName returns the name of the scheduler. +// GetName returns the name of the executor. func (bc BaseConfig) GetName() string { return bc.Name } -// GetType returns the scheduler's type as a string ID. +// GetType returns the executor's type as a string ID. func (bc BaseConfig) GetType() string { return bc.Type } // GetStartTime returns the starting time, relative to the beginning of the -// actual test, that this scheduler is supposed to execute. +// actual test, that this executor is supposed to execute. func (bc BaseConfig) GetStartTime() time.Duration { return time.Duration(bc.StartTime.Duration) } // GetGracefulStop returns how long k6 is supposed to wait for any still -// running iterations to finish executing at the end of the normal scheduler +// running iterations to finish executing at the end of the normal executor // duration, before it actually kills them. // // Of course, that doesn't count when the user manually interrupts the test, @@ -107,7 +107,7 @@ func (bc BaseConfig) GetGracefulStop() time.Duration { } // GetEnv returns any specific environment key=value pairs that -// are configured for the scheduler. +// are configured for the executor. func (bc BaseConfig) GetEnv() map[string]string { return bc.Env } @@ -117,7 +117,7 @@ func (bc BaseConfig) GetExec() null.String { return bc.Exec } -// IsDistributable returns true since by default all schedulers could be run in +// IsDistributable returns true since by default all executors could be run in // a distributed manner. func (bc BaseConfig) IsDistributable() bool { return true diff --git a/lib/scheduler/base_scheduler.go b/lib/scheduler/base_scheduler.go index c2ff61bff52..d511ab1d3a5 100644 --- a/lib/scheduler/base_scheduler.go +++ b/lib/scheduler/base_scheduler.go @@ -28,46 +28,46 @@ import ( "github.com/sirupsen/logrus" ) -// BaseScheduler is a helper struct that contains common properties and methods -// between most schedulers. It is intended to be used as an anonymous struct -// inside of most of the schedulers, for the purpose of reducing boilerplate +// BaseExecutor is a helper struct that contains common properties and methods +// between most executors. It is intended to be used as an anonymous struct +// inside of most of the executors, for the purpose of reducing boilerplate // code. -type BaseScheduler struct { - config lib.SchedulerConfig - executorState *lib.ExecutorState - logger *logrus.Entry - progress *pb.ProgressBar +type BaseExecutor struct { + config lib.ExecutorConfig + executionState *lib.ExecutionState + logger *logrus.Entry + progress *pb.ProgressBar } -// NewBaseScheduler just returns an initialized BaseScheduler -func NewBaseScheduler(config lib.SchedulerConfig, es *lib.ExecutorState, logger *logrus.Entry) *BaseScheduler { - return &BaseScheduler{ - config: config, - executorState: es, - logger: logger, +// NewBaseExecutor just returns an initialized BaseExecutor +func NewBaseExecutor(config lib.ExecutorConfig, es *lib.ExecutionState, logger *logrus.Entry) *BaseExecutor { + return &BaseExecutor{ + config: config, + executionState: es, + logger: logger, progress: pb.New( pb.WithLeft(config.GetName), ), } } -// Init doesn't do anything for most schedulers, since initialization of all +// Init doesn't do anything for most executors, since initialization of all // planned VUs is handled by the executor. -func (bs *BaseScheduler) Init(_ context.Context) error { +func (bs *BaseExecutor) Init(_ context.Context) error { return nil } -// GetConfig returns the configuration with which this scheduler was launched. -func (bs BaseScheduler) GetConfig() lib.SchedulerConfig { +// GetConfig returns the configuration with which this executor was launched. +func (bs BaseExecutor) GetConfig() lib.ExecutorConfig { return bs.config } -// GetLogger returns the scheduler logger entry. -func (bs BaseScheduler) GetLogger() *logrus.Entry { +// GetLogger returns the executor logger entry. +func (bs BaseExecutor) GetLogger() *logrus.Entry { return bs.logger } // GetProgress just returns the progressbar pointer. -func (bs BaseScheduler) GetProgress() *pb.ProgressBar { +func (bs BaseExecutor) GetProgress() *pb.ProgressBar { return bs.progress } diff --git a/lib/scheduler/constant_arrival_rate.go b/lib/scheduler/constant_arrival_rate.go index eab1f5214f9..8374e93bac3 100644 --- a/lib/scheduler/constant_arrival_rate.go +++ b/lib/scheduler/constant_arrival_rate.go @@ -38,9 +38,9 @@ import ( const constantArrivalRateType = "constant-arrival-rate" func init() { - lib.RegisterSchedulerConfigType( + lib.RegisterExecutorConfigType( constantArrivalRateType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewConstantArrivalRateConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -48,7 +48,7 @@ func init() { ) } -// ConstantArrivalRateConfig stores config for the constant arrival-rate scheduler +// ConstantArrivalRateConfig stores config for the constant arrival-rate executor type ConstantArrivalRateConfig struct { BaseConfig Rate null.Int `json:"rate"` @@ -57,7 +57,7 @@ type ConstantArrivalRateConfig struct { // Initialize `PreAllocatedVUs` number of VUs, and if more than that are needed, // they will be dynamically allocated, until `MaxVUs` is reached, which is an - // absolutely hard limit on the number of VUs the scheduler will use + // absolutely hard limit on the number of VUs the executor will use PreAllocatedVUs null.Int `json:"preAllocatedVUs"` MaxVUs null.Int `json:"maxVUs"` } @@ -70,8 +70,8 @@ func NewConstantArrivalRateConfig(name string) ConstantArrivalRateConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &ConstantArrivalRateConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &ConstantArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. func (carc ConstantArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { @@ -83,7 +83,7 @@ func (carc ConstantArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 return es.Scale(carc.MaxVUs.Int64) } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (carc ConstantArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { preAllocatedVUs, maxVUs := carc.GetPreAllocatedVUs(es), carc.GetMaxVUs(es) maxVUsRange := fmt.Sprintf("maxVUs: %d", preAllocatedVUs) @@ -136,7 +136,7 @@ func (carc ConstantArrivalRateConfig) Validate() []error { } // GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the scheduler, including the maximum waiting time for +// whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ @@ -152,31 +152,31 @@ func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.Execution } } -// NewScheduler creates a new ConstantArrivalRate scheduler -func (carc ConstantArrivalRateConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new ConstantArrivalRate executor +func (carc ConstantArrivalRateConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return ConstantArrivalRate{ - BaseScheduler: NewBaseScheduler(carc, es, logger), - config: carc, + BaseExecutor: NewBaseExecutor(carc, es, logger), + config: carc, }, nil } // ConstantArrivalRate tries to execute a specific number of iterations for a // specific period. type ConstantArrivalRate struct { - *BaseScheduler + *BaseExecutor config ConstantArrivalRateConfig } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &ConstantArrivalRate{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &ConstantArrivalRate{} // Run executes a constant number of iterations per second. // // TODO: Reuse the variable arrival rate method? func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := car.executorState.Options.ExecutionSegment + segment := car.executionState.Options.ExecutionSegment gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) preAllocatedVUs := car.config.GetPreAllocatedVUs(segment) @@ -194,7 +194,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC car.logger.WithFields(logrus.Fields{ "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, "tickerPeriod": tickerPeriod, "type": car.config.GetType(), - }).Debug("Starting scheduler run...") + }).Debug("Starting executor run...") // Pre-allocate the VUs local shared buffer vus := make(chan lib.VU, maxVUs) @@ -205,13 +205,13 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} for i := uint64(0); i < initialisedVUs; i++ { - car.executorState.ReturnVU(<-vus, true) + car.executionState.ReturnVU(<-vus, true) } }() // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := car.executorState.GetPlannedVU(car.logger, true) + vu, err := car.executionState.GetPlannedVU(car.logger, true) if err != nil { return err } @@ -235,7 +235,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC go trackProgress(ctx, maxDurationCtx, regDurationCtx, car, progresFn) regDurationDone := regDurationCtx.Done() - runIterationBasic := getIterationRunner(car.executorState, car.logger, out) + runIterationBasic := getIterationRunner(car.executionState, car.logger, out) runIteration := func(vu lib.VU) { runIterationBasic(maxDurationCtx, vu) vus <- vu @@ -255,7 +255,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } - vu, err := car.executorState.GetUnplannedVU(maxDurationCtx, car.logger) + vu, err := car.executionState.GetUnplannedVU(maxDurationCtx, car.logger) if err != nil { return err } diff --git a/lib/scheduler/constant_looping_vus.go b/lib/scheduler/constant_looping_vus.go index f0c74eacde9..8cfea79f6af 100644 --- a/lib/scheduler/constant_looping_vus.go +++ b/lib/scheduler/constant_looping_vus.go @@ -37,9 +37,9 @@ import ( const constantLoopingVUsType = "constant-looping-vus" func init() { - lib.RegisterSchedulerConfigType( + lib.RegisterExecutorConfigType( constantLoopingVUsType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewConstantLoopingVUsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -66,15 +66,15 @@ func NewConstantLoopingVUsConfig(name string) ConstantLoopingVUsConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &ConstantLoopingVUsConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &ConstantLoopingVUsConfig{} -// GetVUs returns the scaled VUs for the scheduler. +// GetVUs returns the scaled VUs for the executor. func (clvc ConstantLoopingVUsConfig) GetVUs(es *lib.ExecutionSegment) int64 { return es.Scale(clvc.VUs.Int64) } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (clvc ConstantLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { return fmt.Sprintf("%d looping VUs for %s%s", clvc.GetVUs(es), clvc.Duration.Duration, clvc.getBaseInfo()) @@ -99,7 +99,7 @@ func (clvc ConstantLoopingVUsConfig) Validate() []error { } // GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the scheduler, including the maximum waiting time for +// whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ @@ -114,28 +114,28 @@ func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionS } } -// NewScheduler creates a new ConstantLoopingVUs scheduler -func (clvc ConstantLoopingVUsConfig) NewScheduler(es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new ConstantLoopingVUs executor +func (clvc ConstantLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return ConstantLoopingVUs{ - BaseScheduler: NewBaseScheduler(clvc, es, logger), - config: clvc, + BaseExecutor: NewBaseExecutor(clvc, es, logger), + config: clvc, }, nil } // ConstantLoopingVUs maintains a constant number of VUs running for the // specified duration. type ConstantLoopingVUs struct { - *BaseScheduler + *BaseExecutor config ConstantLoopingVUsConfig } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &ConstantLoopingVUs{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &ConstantLoopingVUs{} // Run constantly loops through as many iterations as possible on a fixed number // of VUs for the specified duration. func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := clv.executorState.Options.ExecutionSegment + segment := clv.executionState.Options.ExecutionSegment numVUs := clv.config.GetVUs(segment) duration := time.Duration(clv.config.Duration.Duration) gracefulStop := clv.config.GetGracefulStop() @@ -146,7 +146,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo // Make sure the log and the progress bar have accurate information clv.logger.WithFields( logrus.Fields{"vus": numVUs, "duration": duration, "type": clv.config.GetType()}, - ).Debug("Starting scheduler run...") + ).Debug("Starting executor run...") progresFn := func() (float64, string) { spent := time.Since(startTime) @@ -165,10 +165,10 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(clv.executorState, clv.logger, out) + runIteration := getIterationRunner(clv.executionState, clv.logger, out) handleVU := func(vu lib.VU) { - defer clv.executorState.ReturnVU(vu, true) + defer clv.executionState.ReturnVU(vu, true) defer activeVUs.Done() for { @@ -183,7 +183,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } for i := int64(0); i < numVUs; i++ { - vu, err := clv.executorState.GetPlannedVU(clv.logger, true) + vu, err := clv.executionState.GetPlannedVU(clv.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/execution_config_shortcuts.go b/lib/scheduler/execution_config_shortcuts.go index 28651caada6..2b4aa2f97cb 100644 --- a/lib/scheduler/execution_config_shortcuts.go +++ b/lib/scheduler/execution_config_shortcuts.go @@ -37,37 +37,37 @@ func (e ExecutionConflictError) Error() string { var _ error = ExecutionConflictError("") -func getConstantLoopingVUsExecution(duration types.NullDuration, vus null.Int) lib.SchedulerConfigMap { - ds := NewConstantLoopingVUsConfig(lib.DefaultSchedulerName) +func getConstantLoopingVUsExecution(duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { + ds := NewConstantLoopingVUsConfig(lib.DefaultExecutorName) ds.VUs = vus ds.Duration = duration - return lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } -func getVariableLoopingVUsExecution(stages []lib.Stage, startVUs null.Int) lib.SchedulerConfigMap { - ds := NewVariableLoopingVUsConfig(lib.DefaultSchedulerName) +func getVariableLoopingVUsExecution(stages []lib.Stage, startVUs null.Int) lib.ExecutorConfigMap { + ds := NewVariableLoopingVUsConfig(lib.DefaultExecutorName) ds.StartVUs = startVUs for _, s := range stages { if s.Duration.Valid { ds.Stages = append(ds.Stages, Stage{Duration: s.Duration, Target: s.Target}) } } - return lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } -func getSharedIterationsExecution(iters null.Int, duration types.NullDuration, vus null.Int) lib.SchedulerConfigMap { - ds := NewSharedIterationsConfig(lib.DefaultSchedulerName) +func getSharedIterationsExecution(iters null.Int, duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { + ds := NewSharedIterationsConfig(lib.DefaultExecutorName) ds.VUs = vus ds.Iterations = iters if duration.Valid { ds.MaxDuration = duration } - return lib.SchedulerConfigMap{lib.DefaultSchedulerName: ds} + return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } // DeriveExecutionFromShortcuts checks for conflicting options and turns any // shortcut options (i.e. duration, iterations, stages) into the proper -// long-form scheduler configuration in the execution property. +// long-form executor configuration in the execution property. func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { result := opts @@ -99,7 +99,7 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { if opts.Duration.Duration <= 0 { //TODO: move this validation to Validate()? return result, ExecutionConflictError( - "`duration` should be more than 0, for infinite duration use the manual-execution scheduler", + "`duration` should be more than 0, for infinite duration use the externally-controlled executor", ) } result.Execution = getConstantLoopingVUsExecution(opts.Duration, opts.VUs) @@ -133,8 +133,8 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { } // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config // with 1 VU and 1 iteration. - result.Execution = lib.SchedulerConfigMap{ - lib.DefaultSchedulerName: NewPerVUIterationsConfig(lib.DefaultSchedulerName), + result.Execution = lib.ExecutorConfigMap{ + lib.DefaultExecutorName: NewPerVUIterationsConfig(lib.DefaultExecutorName), } } diff --git a/lib/scheduler/helpers.go b/lib/scheduler/helpers.go index de0a0aff61f..cbf968a23fa 100644 --- a/lib/scheduler/helpers.go +++ b/lib/scheduler/helpers.go @@ -76,9 +76,9 @@ func validateStages(stages []Stage) []error { } // getIterationRunner is a helper function that returns an iteration executor -// closure. It takes care of updating metrics, executor stat statistics and +// closure. It takes care of updating metrics, execution state statistics, and // warning messages. -func getIterationRunner(executorState *lib.ExecutorState, logger *logrus.Entry, out chan<- stats.SampleContainer, +func getIterationRunner(executionState *lib.ExecutionState, logger *logrus.Entry, out chan<- stats.SampleContainer, ) func(context.Context, lib.VU) { return func(ctx context.Context, vu lib.VU) { @@ -91,7 +91,7 @@ func getIterationRunner(executorState *lib.ExecutorState, logger *logrus.Entry, select { case <-ctx.Done(): // Don't log errors or emit iterations metrics from cancelled iterations - executorState.AddPartialIterations(1) + executionState.AddPartialIterations(1) default: if err != nil { if s, ok := err.(fmt.Stringer); ok { @@ -106,19 +106,19 @@ func getIterationRunner(executorState *lib.ExecutorState, logger *logrus.Entry, Time: time.Now(), Metric: metrics.Iterations, Value: 1, - Tags: executorState.Options.RunTags, + Tags: executionState.Options.RunTags, } - executorState.AddFullIterations(1) + executionState.AddFullIterations(1) } } } // getDurationContexts is used to create sub-contexts that can restrict a -// scheduler to only run for its allotted time. +// executor to only run for its allotted time. // -// If the scheduler doesn't have a graceful stop period for iterations, then +// If the executor doesn't have a graceful stop period for iterations, then // both returned sub-contexts will be the same one, with a timeout equal to -// supplied regular scheduler duration. +// supplied regular executor duration. // // But if a graceful stop is enabled, then the first returned context (and the // cancel func) will be for the "outer" sub-context. Its timeout will include @@ -130,7 +130,7 @@ func getIterationRunner(executorState *lib.ExecutorState, logger *logrus.Entry, // - As long as the regDurationCtx isn't done, new iterations can be started. // - After regDurationCtx is done, no new iterations should be started; every // VU that finishes an iteration from now on can be returned to the buffer -// pool in the executor state struct. +// pool in the ExecutionState struct. // - After maxDurationCtx is done, any VUs with iterations will be // interrupted by the context's closing and will be returned to the buffer. // - If you want to interrupt the execution of all VUs prematurely (e.g. there @@ -153,10 +153,10 @@ func getDurationContexts(parentCtx context.Context, regularDuration, gracefulSto } // trackProgress is a helper function that monitors certain end-events in a -// scheduler and updates it's progressbar accordingly. +// executor and updates it's progressbar accordingly. func trackProgress( parentCtx, maxDurationCtx, regDurationCtx context.Context, - sched lib.Scheduler, snapshot func() (float64, string), + sched lib.Executor, snapshot func() (float64, string), ) { progressBar := sched.GetProgress() logger := sched.GetLogger() diff --git a/lib/scheduler/manual.go b/lib/scheduler/manual.go index 94dd1bc2121..7068cab08cd 100644 --- a/lib/scheduler/manual.go +++ b/lib/scheduler/manual.go @@ -37,13 +37,13 @@ import ( null "gopkg.in/guregu/null.v3" ) -const manualExecutionType = "manual-execution" +const externallyControlledType = "externally-controlled" func init() { - lib.RegisterSchedulerConfigType( - manualExecutionType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { - config := ManualExecutionConfig{BaseConfig: NewBaseConfig(name, manualExecutionType)} + lib.RegisterExecutorConfigType( + externallyControlledType, + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { + config := ExternallyControlledConfig{BaseConfig: NewBaseConfig(name, externallyControlledType)} err := lib.StrictJSONUnmarshal(rawJSON, &config) if err != nil { return config, err @@ -56,16 +56,16 @@ func init() { ) } -// ManualExecutionControlConfig contains all of the options that actually -// determine the scheduling of VUs in the manual execution scheduler. -type ManualExecutionControlConfig struct { +// ExternallyControlledConfigParams contains all of the options that actually +// determine the scheduling of VUs in the externally controlled executor. +type ExternallyControlledConfigParams struct { VUs null.Int `json:"vus"` Duration types.NullDuration `json:"duration"` MaxVUs null.Int `json:"maxVUs"` } // Validate just checks the control options in isolation. -func (mecc ManualExecutionControlConfig) Validate() (errors []error) { +func (mecc ExternallyControlledConfigParams) Validate() (errors []error) { if mecc.VUs.Int64 < 0 { errors = append(errors, fmt.Errorf("the number of VUs shouldn't be negative")) } @@ -88,48 +88,48 @@ func (mecc ManualExecutionControlConfig) Validate() (errors []error) { return errors } -// ManualExecutionConfig stores the number of currently active VUs, the max -// number of VUs and the scheduler duration. The duration can be 0, which means +// ExternallyControlledConfig stores the number of currently active VUs, the max +// number of VUs and the executor duration. The duration can be 0, which means // "infinite duration", i.e. the user has to manually abort the script. -type ManualExecutionConfig struct { +type ExternallyControlledConfig struct { BaseConfig - ManualExecutionControlConfig + ExternallyControlledConfigParams } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &ManualExecutionConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &ExternallyControlledConfig{} -// GetDescription returns a human-readable description of the scheduler options -func (mec ManualExecutionConfig) GetDescription(_ *lib.ExecutionSegment) string { +// GetDescription returns a human-readable description of the executor options +func (mec ExternallyControlledConfig) GetDescription(_ *lib.ExecutionSegment) string { duration := "infinite" if mec.Duration.Duration != 0 { duration = mec.Duration.String() } return fmt.Sprintf( - "Manually controlled execution with %d VUs, %d max VUs, %s duration", + "Externally controlled execution with %d VUs, %d max VUs, %s duration", mec.VUs.Int64, mec.MaxVUs.Int64, duration, ) } // Validate makes sure all options are configured and valid -func (mec ManualExecutionConfig) Validate() []error { - errors := append(mec.BaseConfig.Validate(), mec.ManualExecutionControlConfig.Validate()...) +func (mec ExternallyControlledConfig) Validate() []error { + errors := append(mec.BaseConfig.Validate(), mec.ExternallyControlledConfigParams.Validate()...) if mec.GracefulStop.Valid { errors = append(errors, fmt.Errorf( - "gracefulStop is not supported by the manual execution scheduler", + "gracefulStop is not supported by the externally controlled executor", )) } return errors } // GetExecutionRequirements just reserves the specified number of max VUs for -// the whole duration of the scheduler, so these VUs can be initialized in the +// the whole duration of the executor, so these VUs can be initialized in the // beginning of the test. // -// Importantly, if 0 (i.e. infinite) duration is configured, this scheduler +// Importantly, if 0 (i.e. infinite) duration is configured, this executor // doesn't emit the last step to relinquish these VUs. // -// Also, the manual execution scheduler doesn't set MaxUnplannedVUs in the +// Also, the externally controlled executor doesn't set MaxUnplannedVUs in the // returned steps, since their initialization and usage is directly controlled // by the user and is effectively bounded only by the resources of the machine // k6 is running on. @@ -137,8 +137,9 @@ func (mec ManualExecutionConfig) Validate() []error { // This is not a problem, because the MaxUnplannedVUs are mostly meant to be // used for calculating the maximum possble number of initialized VUs at any // point during a test run. That's used for sizing purposes and for user qouta -// checking in the cloud execution, where the manual scheduler isn't supported. -func (mec ManualExecutionConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +// checking in the cloud execution, where the externally controlled executor +// isn't supported. +func (mec ExternallyControlledConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { startVUs := lib.ExecutionStep{ TimeOffset: 0, PlannedVUs: uint64(es.Scale(mec.MaxVUs.Int64)), // use @@ -158,26 +159,24 @@ func (mec ManualExecutionConfig) GetExecutionRequirements(es *lib.ExecutionSegme } // IsDistributable simply returns false because there's no way to reliably -// distribute the manual execution scheduler. -func (ManualExecutionConfig) IsDistributable() bool { +// distribute the externally controlled executor. +func (ExternallyControlledConfig) IsDistributable() bool { return false } -// NewScheduler creates a new ManualExecution "scheduler" -func (mec ManualExecutionConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { - - return &ManualExecution{ +// NewExecutor creates a new ExternallyControlled executor +func (mec ExternallyControlledConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { + return &ExternallyControlled{ startConfig: mec, - currentControlConfig: mec.ManualExecutionControlConfig, + currentControlConfig: mec.ExternallyControlledConfigParams, configLock: &sync.RWMutex{}, newControlConfigs: make(chan updateConfigEvent), pauseEvents: make(chan pauseEvent), hasStarted: make(chan struct{}), - executorState: es, - logger: logger, - progress: pb.New(pb.WithLeft(mec.GetName)), + executionState: es, + logger: logger, + progress: pb.New(pb.WithLeft(mec.GetName)), }, nil } @@ -187,71 +186,71 @@ type pauseEvent struct { } type updateConfigEvent struct { - newConfig ManualExecutionControlConfig + newConfig ExternallyControlledConfigParams err chan error } -// ManualExecution is an implementation of the old k6 scheduler that could be +// ExternallyControlled is an implementation of the old k6 executor that could be // controlled externally, via the k6 REST API. It implements both the -// lib.PausableScheduler and the lib.LiveUpdatableScheduler interfaces. -type ManualExecution struct { - startConfig ManualExecutionConfig - currentControlConfig ManualExecutionControlConfig +// lib.PausableExecutor and the lib.LiveUpdatableExecutor interfaces. +type ExternallyControlled struct { + startConfig ExternallyControlledConfig + currentControlConfig ExternallyControlledConfigParams configLock *sync.RWMutex newControlConfigs chan updateConfigEvent pauseEvents chan pauseEvent hasStarted chan struct{} - executorState *lib.ExecutorState - logger *logrus.Entry - progress *pb.ProgressBar + executionState *lib.ExecutionState + logger *logrus.Entry + progress *pb.ProgressBar } // Make sure we implement all the interfaces -var _ lib.Scheduler = &ManualExecution{} -var _ lib.PausableScheduler = &ManualExecution{} -var _ lib.LiveUpdatableScheduler = &ManualExecution{} +var _ lib.Executor = &ExternallyControlled{} +var _ lib.PausableExecutor = &ExternallyControlled{} +var _ lib.LiveUpdatableExecutor = &ExternallyControlled{} -// GetCurrentConfig just returns the scheduler's current configuration. -func (mex *ManualExecution) GetCurrentConfig() ManualExecutionConfig { +// GetCurrentConfig just returns the executor's current configuration. +func (mex *ExternallyControlled) GetCurrentConfig() ExternallyControlledConfig { mex.configLock.RLock() defer mex.configLock.RUnlock() - return ManualExecutionConfig{ - BaseConfig: mex.startConfig.BaseConfig, - ManualExecutionControlConfig: mex.currentControlConfig, + return ExternallyControlledConfig{ + BaseConfig: mex.startConfig.BaseConfig, + ExternallyControlledConfigParams: mex.currentControlConfig, } } -// GetConfig just returns the scheduler's current configuration, it's basically +// GetConfig just returns the executor's current configuration, it's basically // an alias of GetCurrentConfig that implements the more generic interface. -func (mex *ManualExecution) GetConfig() lib.SchedulerConfig { +func (mex *ExternallyControlled) GetConfig() lib.ExecutorConfig { return mex.GetCurrentConfig() } -// GetProgress just returns the scheduler's progress bar instance. -func (mex ManualExecution) GetProgress() *pb.ProgressBar { +// GetProgress just returns the executor's progress bar instance. +func (mex ExternallyControlled) GetProgress() *pb.ProgressBar { return mex.progress } -// GetLogger just returns the scheduler's logger instance. -func (mex ManualExecution) GetLogger() *logrus.Entry { +// GetLogger just returns the executor's logger instance. +func (mex ExternallyControlled) GetLogger() *logrus.Entry { return mex.logger } // Init doesn't do anything... -func (mex ManualExecution) Init(ctx context.Context) error { +func (mex ExternallyControlled) Init(ctx context.Context) error { return nil } -// SetPaused pauses or resumes the scheduler. -func (mex *ManualExecution) SetPaused(paused bool) error { +// SetPaused pauses or resumes the executor. +func (mex *ExternallyControlled) SetPaused(paused bool) error { select { case <-mex.hasStarted: event := pauseEvent{isPaused: paused, err: make(chan error)} mex.pauseEvents <- event return <-event.err default: - return fmt.Errorf("cannot pause the manual scheduler before it has started") + return fmt.Errorf("cannot pause the externally controlled executor before it has started") } } @@ -259,23 +258,23 @@ func (mex *ManualExecution) SetPaused(paused bool) error { // possible to update the configuration even when k6 is paused, either in the // beginning (i.e. when running k6 with --paused) or in the middle of the script // execution. -func (mex *ManualExecution) UpdateConfig(ctx context.Context, newConf interface{}) error { - newManualConfig, ok := newConf.(ManualExecutionControlConfig) +func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf interface{}) error { + newConfigParams, ok := newConf.(ExternallyControlledConfigParams) if !ok { return errors.New("invalid config type") } - if errs := newManualConfig.Validate(); len(errs) != 0 { + if errs := newConfigParams.Validate(); len(errs) != 0 { return fmt.Errorf("invalid confiuguration supplied: %s", lib.ConcatErrors(errs, ", ")) } - if newManualConfig.Duration != mex.startConfig.Duration { - return fmt.Errorf("the manual scheduler duration cannot be changed") + if newConfigParams.Duration != mex.startConfig.Duration { + return fmt.Errorf("the externally controlled executor duration cannot be changed") } - if newManualConfig.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { - // This limitation is because the manual execution scheduler is still a - // scheduler that participates in the overall k6 scheduling. Thus, any - // VUs that were explicitly specified by the user in the config may be - // reused from or by other schedulers. + if newConfigParams.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { + // This limitation is because the externally controlled executor is + // still a executor that participates in the overall k6 scheduling. + // Thus, any VUs that were explicitly specified by the user in the + // config may be reused from or by other executors. return fmt.Errorf( "the new number of max VUs cannot be lower than the starting number of max VUs (%d)", mex.startConfig.MaxVUs.Int64, @@ -286,21 +285,21 @@ func (mex *ManualExecution) UpdateConfig(ctx context.Context, newConf interface{ select { case <-mex.hasStarted: mex.configLock.Unlock() - event := updateConfigEvent{newConfig: newManualConfig, err: make(chan error)} + event := updateConfigEvent{newConfig: newConfigParams, err: make(chan error)} mex.newControlConfigs <- event return <-event.err case <-ctx.Done(): mex.configLock.Unlock() return ctx.Err() default: - mex.currentControlConfig = newManualConfig + mex.currentControlConfig = newConfigParams mex.configLock.Unlock() return nil } } // This is a helper function that is used in run for non-infinite durations. -func (mex *ManualExecution) stopWhenDurationIsReached(ctx context.Context, duration time.Duration, cancel func()) { +func (mex *ExternallyControlled) stopWhenDurationIsReached(ctx context.Context, duration time.Duration, cancel func()) { ctxDone := ctx.Done() checkInterval := time.NewTicker(100 * time.Millisecond) for { @@ -311,7 +310,7 @@ func (mex *ManualExecution) stopWhenDurationIsReached(ctx context.Context, durat //TODO: something more optimized that sleeps for pauses? case <-checkInterval.C: - if mex.executorState.GetCurrentTestRunDuration() >= duration { + if mex.executionState.GetCurrentTestRunDuration() >= duration { cancel() return } @@ -320,7 +319,7 @@ func (mex *ManualExecution) stopWhenDurationIsReached(ctx context.Context, durat } // manualVUHandle is a wrapper around the vuHandle helper, used in the -// variable-looping-vus scheduler. Here, instead of using its getVU and returnVU +// variable-looping-vus executor. Here, instead of using its getVU and returnVU // methods to retrieve and return a VU from the global buffer, we use them to // accurately update the local and global active VU counters and to ensure that // the pausing and reducing VUs operations wait for VUs to fully finish @@ -336,7 +335,7 @@ type manualVUHandle struct { } func newManualVUHandle( - parentCtx context.Context, state *lib.ExecutorState, localActiveVUsCount *int64, vu lib.VU, logger *logrus.Entry, + parentCtx context.Context, state *lib.ExecutionState, localActiveVUsCount *int64, vu lib.VU, logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} @@ -365,7 +364,7 @@ func newManualVUHandle( // until the test is manually stopped. // //TODO: split this up? somehow... :/ -func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { +func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { mex.configLock.RLock() // Safely get the current config - it's important that the close of the // hasStarted channel is inside of the lock, so that there are no data races @@ -374,7 +373,7 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp close(mex.hasStarted) mex.configLock.RUnlock() - segment := mex.executorState.Options.ExecutionSegment + segment := mex.executionState.Options.ExecutionSegment duration := time.Duration(currentControlConfig.Duration.Duration) ctx, cancel := context.WithCancel(parentCtx) @@ -384,22 +383,22 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp } mex.logger.WithFields( - logrus.Fields{"type": manualExecutionType, "duration": duration}, - ).Debug("Starting scheduler run...") + logrus.Fields{"type": externallyControlledType, "duration": duration}, + ).Debug("Starting executor run...") // Retrieve and initialize the (scaled) number of MaxVUs from the global VU // buffer that the user originally specified in the JS config. startMaxVUs := segment.Scale(mex.startConfig.MaxVUs.Int64) vuHandles := make([]*manualVUHandle, startMaxVUs) activeVUsCount := new(int64) - runIteration := getIterationRunner(mex.executorState, mex.logger, out) + runIteration := getIterationRunner(mex.executionState, mex.logger, out) for i := int64(0); i < startMaxVUs; i++ { // get the initial planned VUs from the common buffer - vu, vuGetErr := mex.executorState.GetPlannedVU(mex.logger, false) + vu, vuGetErr := mex.executionState.GetPlannedVU(mex.logger, false) if vuGetErr != nil { return vuGetErr } vuHandle := newManualVUHandle( - parentCtx, mex.executorState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), + parentCtx, mex.executionState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(runIteration) vuHandles[i] = vuHandle @@ -409,7 +408,7 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp maxVUs := new(int64) *maxVUs = startMaxVUs progresFn := func() (float64, string) { - spent := mex.executorState.GetCurrentTestRunDuration() + spent := mex.executionState.GetCurrentTestRunDuration() progress := 0.0 if duration > 0 { progress = math.Min(1, float64(spent)/float64(duration)) @@ -432,7 +431,7 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp vuHandles[i].wg.Wait() } } - handleConfigChange := func(oldControlConfig, newControlConfig ManualExecutionControlConfig) error { + handleConfigChange := func(oldControlConfig, newControlConfig ExternallyControlledConfigParams) error { oldActiveVUs := segment.Scale(oldControlConfig.VUs.Int64) oldMaxVUs := segment.Scale(oldControlConfig.MaxVUs.Int64) newActiveVUs := segment.Scale(newControlConfig.VUs.Int64) @@ -444,12 +443,12 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp }).Debug("Updating execution configuration...") for i := oldMaxVUs; i < newMaxVUs; i++ { - vu, vuInitErr := mex.executorState.InitializeNewVU(ctx, mex.logger) + vu, vuInitErr := mex.executionState.InitializeNewVU(ctx, mex.logger) if vuInitErr != nil { return vuInitErr } vuHandle := newManualVUHandle( - ctx, mex.executorState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), + ctx, mex.executionState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(runIteration) vuHandles = append(vuHandles, vuHandle) @@ -474,9 +473,9 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp vuHandles[i].cancelVU() if i < startMaxVUs { // return the initial planned VUs to the common buffer - mex.executorState.ReturnVU(vuHandles[i].vu, false) + mex.executionState.ReturnVU(vuHandles[i].vu, false) } else { - mex.executorState.ModInitializedVUsCount(-1) + mex.executionState.ModInitializedVUsCount(-1) } vuHandles[i] = nil } @@ -487,12 +486,12 @@ func (mex *ManualExecution) Run(parentCtx context.Context, out chan<- stats.Samp return nil } - err = handleConfigChange(ManualExecutionControlConfig{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig) + err = handleConfigChange(ExternallyControlledConfigParams{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig) if err != nil { return err } defer func() { - err = handleConfigChange(currentControlConfig, ManualExecutionControlConfig{}) + err = handleConfigChange(currentControlConfig, ExternallyControlledConfigParams{}) }() for { diff --git a/lib/scheduler/per_vu_iterations.go b/lib/scheduler/per_vu_iterations.go index 81ddea9977f..a6f325f31ba 100644 --- a/lib/scheduler/per_vu_iterations.go +++ b/lib/scheduler/per_vu_iterations.go @@ -38,7 +38,7 @@ import ( const perVUIterationsType = "per-vu-iterations" func init() { - lib.RegisterSchedulerConfigType(perVUIterationsType, func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + lib.RegisterExecutorConfigType(perVUIterationsType, func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewPerVUIterationsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -63,23 +63,23 @@ func NewPerVUIterationsConfig(name string) PerVUIteationsConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &PerVUIteationsConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &PerVUIteationsConfig{} -// GetVUs returns the scaled VUs for the scheduler. +// GetVUs returns the scaled VUs for the executor. func (pvic PerVUIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { return es.Scale(pvic.VUs.Int64) } -// GetIterations returns the UNSCALED iteration count for the scheduler. It's -// important to note that scaling per-VU iteration scheduler affects only the +// GetIterations returns the UNSCALED iteration count for the executor. It's +// important to note that scaling per-VU iteration executor affects only the // number of VUs. If we also scaled the iterations, scaling would have quadratic // effects instead of just linear. func (pvic PerVUIteationsConfig) GetIterations() int64 { return pvic.Iterations.Int64 } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (pvic PerVUIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { return fmt.Sprintf("%d iterations for each of %d VUs%s", pvic.GetIterations(), pvic.GetVUs(es), @@ -107,7 +107,7 @@ func (pvic PerVUIteationsConfig) Validate() []error { } // GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the scheduler, including the maximum waiting time for +// whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. func (pvic PerVUIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ @@ -122,28 +122,28 @@ func (pvic PerVUIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegme } } -// NewScheduler creates a new PerVUIteations scheduler -func (pvic PerVUIteationsConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new PerVUIteations executor +func (pvic PerVUIteationsConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return PerVUIteations{ - BaseScheduler: NewBaseScheduler(pvic, es, logger), - config: pvic, + BaseExecutor: NewBaseExecutor(pvic, es, logger), + config: pvic, }, nil } // PerVUIteations executes a specific number of iterations with each VU. type PerVUIteations struct { - *BaseScheduler + *BaseExecutor config PerVUIteationsConfig } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &PerVUIteations{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &PerVUIteations{} // Run executes a specific number of iterations with each confugured VU. func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := pvi.executorState.Options.ExecutionSegment + segment := pvi.executionState.Options.ExecutionSegment numVUs := pvi.config.GetVUs(segment) iterations := pvi.config.GetIterations() duration := time.Duration(pvi.config.MaxDuration.Duration) @@ -155,7 +155,7 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai // Make sure the log and the progress bar have accurate information pvi.logger.WithFields(logrus.Fields{ "vus": numVUs, "iterations": iterations, "maxDuration": duration, "type": pvi.config.GetType(), - }).Debug("Starting scheduler run...") + }).Debug("Starting executor run...") totalIters := uint64(numVUs * iterations) doneIters := new(uint64) @@ -174,10 +174,10 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(pvi.executorState, pvi.logger, out) + runIteration := getIterationRunner(pvi.executionState, pvi.logger, out) handleVU := func(vu lib.VU) { - defer pvi.executorState.ReturnVU(vu, true) + defer pvi.executionState.ReturnVU(vu, true) defer activeVUs.Done() for i := int64(0); i < iterations; i++ { @@ -193,7 +193,7 @@ func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - vu, err := pvi.executorState.GetPlannedVU(pvi.logger, true) + vu, err := pvi.executionState.GetPlannedVU(pvi.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/schedulers_test.go b/lib/scheduler/schedulers_test.go index 7f503682f37..f62964645f8 100644 --- a/lib/scheduler/schedulers_test.go +++ b/lib/scheduler/schedulers_test.go @@ -36,7 +36,7 @@ import ( type exp struct { parseError bool validationError bool - custom func(t *testing.T, cm lib.SchedulerConfigMap) + custom func(t *testing.T, cm lib.ExecutorConfigMap) } type configMapTestCase struct { @@ -51,11 +51,11 @@ var configMapTestCases = []configMapTestCase{ {"asdf", exp{parseError: true}}, {"'adsf'", exp{parseError: true}}, {"[]", exp{parseError: true}}, - {"{}", exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { - assert.Equal(t, cm, lib.SchedulerConfigMap{}) + {"{}", exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + assert.Equal(t, cm, lib.ExecutorConfigMap{}) }}}, {"{}asdf", exp{parseError: true}}, - {"null", exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + {"null", exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { assert.Nil(t, cm) }}}, {`{"someKey": {}}`, exp{parseError: true}}, @@ -66,7 +66,7 @@ var configMapTestCases = []configMapTestCase{ // Validation errors for constant-looping-vus and the base config {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewConstantLoopingVUsConfig("someKey") sched.VUs = null.IntFrom(10) sched.Duration = types.NullDurationFrom(1 * time.Minute) @@ -74,7 +74,7 @@ var configMapTestCases = []configMapTestCase{ sched.StartTime = types.NullDurationFrom(70 * time.Second) sched.Exec = null.StringFrom("someFunc") sched.Env = map[string]string{"test": "mest"} - require.Equal(t, cm, lib.SchedulerConfigMap{"someKey": sched}) + require.Equal(t, cm, lib.ExecutorConfigMap{"someKey": sched}) require.Equal(t, sched.BaseConfig.Name, cm["someKey"].GetName()) require.Equal(t, sched.BaseConfig.Type, cm["someKey"].GetType()) require.Equal(t, sched.BaseConfig.GetGracefulStop(), cm["someKey"].GetGracefulStop()) @@ -119,7 +119,7 @@ var configMapTestCases = []configMapTestCase{ // variable-looping-vus {`{"varloops": {"type": "variable-looping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", "startTime": "23s", "stages": [{"duration": "60s", "target": 30}, {"duration": "130s", "target": 10}]}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewVariableLoopingVUsConfig("varloops") sched.GracefulStop = types.NullDurationFrom(15 * time.Second) sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) @@ -129,7 +129,7 @@ var configMapTestCases = []configMapTestCase{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(60 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(130 * time.Second)}, } - require.Equal(t, cm, lib.SchedulerConfigMap{"varloops": sched}) + require.Equal(t, cm, lib.ExecutorConfigMap{"varloops": sched}) assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) @@ -161,7 +161,7 @@ var configMapTestCases = []configMapTestCase{ {`{"varloops": {"type": "variable-looping-vus"}}`, exp{validationError: true}}, // shared-iterations {`{"ishared": {"type": "shared-iterations", "iterations": 22, "vus": 12, "maxDuration": "100s"}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewSharedIterationsConfig("ishared") sched.Iterations = null.IntFrom(22) sched.MaxDuration = types.NullDurationFrom(100 * time.Second) @@ -194,7 +194,7 @@ var configMapTestCases = []configMapTestCase{ {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 30}}`, exp{validationError: true}}, // per-vu-iterations {`{"ipervu": {"type": "per-vu-iterations", "iterations": 23, "vus": 13, "gracefulStop": 0}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewPerVUIterationsConfig("ipervu") sched.Iterations = null.IntFrom(23) sched.GracefulStop = types.NullDurationFrom(0) @@ -227,7 +227,7 @@ var configMapTestCases = []configMapTestCase{ // constant-arrival-rate {`{"carrival": {"type": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewConstantArrivalRateConfig("carrival") sched.Rate = null.IntFrom(30) sched.Duration = types.NullDurationFrom(10 * time.Minute) @@ -265,7 +265,7 @@ var configMapTestCases = []configMapTestCase{ // variable-arrival-rate {`{"varrival": {"type": "variable-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, - exp{custom: func(t *testing.T, cm lib.SchedulerConfigMap) { + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewVariableArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) sched.Stages = []Stage{ @@ -275,7 +275,7 @@ var configMapTestCases = []configMapTestCase{ sched.TimeUnit = types.NullDurationFrom(30 * time.Second) sched.PreAllocatedVUs = null.IntFrom(20) sched.MaxVUs = null.IntFrom(50) - require.Equal(t, cm, lib.SchedulerConfigMap{"varrival": sched}) + require.Equal(t, cm, lib.ExecutorConfigMap{"varrival": sched}) assert.Empty(t, cm["varrival"].Validate()) assert.Empty(t, cm.Validate()) @@ -302,7 +302,7 @@ var configMapTestCases = []configMapTestCase{ {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, exp{validationError: true}}, {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}], "timeUnit": "-1s"}}`, exp{validationError: true}}, {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 30, "maxVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - //TODO: more tests of mixed schedulers and execution plans + //TODO: more tests of mixed executors and execution plans } func TestConfigMapParsingAndValidation(t *testing.T) { @@ -311,7 +311,7 @@ func TestConfigMapParsingAndValidation(t *testing.T) { tc := tc t.Run(fmt.Sprintf("TestCase#%d", i), func(t *testing.T) { t.Logf(tc.rawJSON) - var result lib.SchedulerConfigMap + var result lib.ExecutorConfigMap err := json.Unmarshal([]byte(tc.rawJSON), &result) if tc.expected.parseError { require.Error(t, err) diff --git a/lib/scheduler/shared_iterations.go b/lib/scheduler/shared_iterations.go index 14f36de4292..3f7bdc7ebc6 100644 --- a/lib/scheduler/shared_iterations.go +++ b/lib/scheduler/shared_iterations.go @@ -38,9 +38,9 @@ import ( const sharedIterationsType = "shared-iterations" func init() { - lib.RegisterSchedulerConfigType( + lib.RegisterExecutorConfigType( sharedIterationsType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewSharedIterationsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -66,20 +66,20 @@ func NewSharedIterationsConfig(name string) SharedIteationsConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &SharedIteationsConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &SharedIteationsConfig{} -// GetVUs returns the scaled VUs for the scheduler. +// GetVUs returns the scaled VUs for the executor. func (sic SharedIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { return es.Scale(sic.VUs.Int64) } -// GetIterations returns the scaled iteration count for the scheduler. +// GetIterations returns the scaled iteration count for the executor. func (sic SharedIteationsConfig) GetIterations(es *lib.ExecutionSegment) int64 { return es.Scale(sic.Iterations.Int64) } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (sic SharedIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { return fmt.Sprintf("%d iterations shared among %d VUs%s", sic.GetIterations(es), sic.GetVUs(es), @@ -110,7 +110,7 @@ func (sic SharedIteationsConfig) Validate() []error { } // GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the scheduler, including the maximum waiting time for +// whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. func (sic SharedIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ @@ -125,30 +125,30 @@ func (sic SharedIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegme } } -// NewScheduler creates a new SharedIteations scheduler -func (sic SharedIteationsConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new SharedIteations executor +func (sic SharedIteationsConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return SharedIteations{ - BaseScheduler: NewBaseScheduler(sic, es, logger), - config: sic, + BaseExecutor: NewBaseExecutor(sic, es, logger), + config: sic, }, nil } // SharedIteations executes a specific total number of iterations, which are // all shared by the configured VUs. type SharedIteations struct { - *BaseScheduler + *BaseExecutor config SharedIteationsConfig } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &PerVUIteations{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &PerVUIteations{} // Run executes a specific total number of iterations, which are all shared by // the configured VUs. func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := si.executorState.Options.ExecutionSegment + segment := si.executionState.Options.ExecutionSegment numVUs := si.config.GetVUs(segment) iterations := si.config.GetIterations(segment) duration := time.Duration(si.config.MaxDuration.Duration) @@ -160,7 +160,7 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai // Make sure the log and the progress bar have accurate information si.logger.WithFields(logrus.Fields{ "vus": numVUs, "iterations": iterations, "maxDuration": duration, "type": si.config.GetType(), - }).Debug("Starting scheduler run...") + }).Debug("Starting executor run...") totalIters := uint64(iterations) doneIters := new(uint64) @@ -179,11 +179,11 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(si.executorState, si.logger, out) + runIteration := getIterationRunner(si.executionState, si.logger, out) attemptedIters := new(uint64) handleVU := func(vu lib.VU) { - defer si.executorState.ReturnVU(vu, true) + defer si.executionState.ReturnVU(vu, true) defer activeVUs.Done() for { @@ -204,7 +204,7 @@ func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContai } for i := int64(0); i < numVUs; i++ { - vu, err := si.executorState.GetPlannedVU(si.logger, true) + vu, err := si.executionState.GetPlannedVU(si.logger, true) if err != nil { cancel() return err diff --git a/lib/scheduler/variable_arrival_rate.go b/lib/scheduler/variable_arrival_rate.go index 40bfdc73a13..b7bdd204b90 100644 --- a/lib/scheduler/variable_arrival_rate.go +++ b/lib/scheduler/variable_arrival_rate.go @@ -43,9 +43,9 @@ const variableArrivalRateType = "variable-arrival-rate" const minIntervalBetweenRateAdjustments = 250 * time.Millisecond func init() { - lib.RegisterSchedulerConfigType( + lib.RegisterExecutorConfigType( variableArrivalRateType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewVariableArrivalRateConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -53,7 +53,7 @@ func init() { ) } -// VariableArrivalRateConfig stores config for the variable arrival-rate scheduler +// VariableArrivalRateConfig stores config for the variable arrival-rate executor type VariableArrivalRateConfig struct { BaseConfig StartRate null.Int `json:"startRate"` @@ -62,7 +62,7 @@ type VariableArrivalRateConfig struct { // Initialize `PreAllocatedVUs` number of VUs, and if more than that are needed, // they will be dynamically allocated, until `MaxVUs` is reached, which is an - // absolutely hard limit on the number of VUs the scheduler will use + // absolutely hard limit on the number of VUs the executor will use PreAllocatedVUs null.Int `json:"preAllocatedVUs"` MaxVUs null.Int `json:"maxVUs"` } @@ -75,8 +75,8 @@ func NewVariableArrivalRateConfig(name string) VariableArrivalRateConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &VariableArrivalRateConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &VariableArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { @@ -88,7 +88,7 @@ func (varc VariableArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 return es.Scale(varc.MaxVUs.Int64) } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (varc VariableArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { //TODO: something better? always show iterations per second? maxVUsRange := fmt.Sprintf("maxVUs: %d", es.Scale(varc.PreAllocatedVUs.Int64)) @@ -135,7 +135,7 @@ func (varc VariableArrivalRateConfig) Validate() []error { } // GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the scheduler, including the maximum waiting time for +// whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. func (varc VariableArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ @@ -236,12 +236,12 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi return rateChanges } -// NewScheduler creates a new VariableArrivalRate scheduler -func (varc VariableArrivalRateConfig) NewScheduler( - es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new VariableArrivalRate executor +func (varc VariableArrivalRateConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return VariableArrivalRate{ - BaseScheduler: NewBaseScheduler(varc, es, logger), + BaseExecutor: NewBaseExecutor(varc, es, logger), config: varc, plannedRateChanges: varc.getPlannedRateChanges(es.Options.ExecutionSegment), }, nil @@ -251,13 +251,13 @@ func (varc VariableArrivalRateConfig) NewScheduler( // specific period. //TODO: combine with the ConstantArrivalRate? type VariableArrivalRate struct { - *BaseScheduler + *BaseExecutor config VariableArrivalRateConfig plannedRateChanges []rateChange } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &VariableArrivalRate{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &VariableArrivalRate{} // streamRateChanges is a helper method that emits rate change events at their // proper time. @@ -286,7 +286,7 @@ func (varr VariableArrivalRate) streamRateChanges(ctx context.Context, startTime // Run executes a variable number of iterations per second. func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := varr.executorState.Options.ExecutionSegment + segment := varr.executionState.Options.ExecutionSegment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) preAllocatedVUs := varr.config.GetPreAllocatedVUs(segment) @@ -310,7 +310,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample varr.logger.WithFields(logrus.Fields{ "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, "numStages": len(varr.config.Stages), "startTickerPeriod": startTickerPeriod.Duration, "type": varr.config.GetType(), - }).Debug("Starting scheduler run...") + }).Debug("Starting executor run...") // Pre-allocate the VUs local shared buffer vus := make(chan lib.VU, maxVUs) @@ -321,13 +321,13 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} for i := uint64(0); i < initialisedVUs; i++ { - varr.executorState.ReturnVU(<-vus, true) + varr.executionState.ReturnVU(<-vus, true) } }() // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := varr.executorState.GetPlannedVU(varr.logger, true) + vu, err := varr.executionState.GetPlannedVU(varr.logger, true) if err != nil { return err } @@ -357,7 +357,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) regDurationDone := regDurationCtx.Done() - runIterationBasic := getIterationRunner(varr.executorState, varr.logger, out) + runIterationBasic := getIterationRunner(varr.executionState, varr.logger, out) runIteration := func(vu lib.VU) { runIterationBasic(maxDurationCtx, vu) vus <- vu @@ -386,7 +386,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } - vu, err := varr.executorState.GetUnplannedVU(maxDurationCtx, varr.logger) + vu, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) if err != nil { return err } diff --git a/lib/scheduler/variable_looping_vus.go b/lib/scheduler/variable_looping_vus.go index 1dc9b9b6868..acdbd21563a 100644 --- a/lib/scheduler/variable_looping_vus.go +++ b/lib/scheduler/variable_looping_vus.go @@ -43,9 +43,9 @@ const variableLoopingVUsType = "variable-looping-vus" const minIntervalBetweenVUAdjustments = 100 * time.Millisecond func init() { - lib.RegisterSchedulerConfigType( + lib.RegisterExecutorConfigType( variableLoopingVUsType, - func(name string, rawJSON []byte) (lib.SchedulerConfig, error) { + func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { config := NewVariableLoopingVUsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err @@ -60,7 +60,7 @@ type Stage struct { //TODO: add a progression function? } -// VariableLoopingVUsConfig stores the configuration for the stages scheduler +// VariableLoopingVUsConfig stores the configuration for the stages executor type VariableLoopingVUsConfig struct { BaseConfig StartVUs null.Int `json:"startVUs"` @@ -77,8 +77,8 @@ func NewVariableLoopingVUsConfig(name string) VariableLoopingVUsConfig { } } -// Make sure we implement the lib.SchedulerConfig interface -var _ lib.SchedulerConfig = &VariableLoopingVUsConfig{} +// Make sure we implement the lib.ExecutorConfig interface +var _ lib.ExecutorConfig = &VariableLoopingVUsConfig{} // GetStartVUs is just a helper method that returns the scaled starting VUs. func (vlvc VariableLoopingVUsConfig) GetStartVUs(es *lib.ExecutionSegment) int64 { @@ -91,7 +91,7 @@ func (vlvc VariableLoopingVUsConfig) GetGracefulRampDown() time.Duration { return time.Duration(vlvc.GracefulRampDown.Duration) } -// GetDescription returns a human-readable description of the scheduler options +// GetDescription returns a human-readable description of the executor options func (vlvc VariableLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { maxVUs := es.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", @@ -110,15 +110,15 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { } // getRawExecutionSteps calculates and returns as execution steps the number of -// actively running VUs the scheduler should have at every moment. +// actively running VUs the executor should have at every moment. // // It doesn't take into account graceful ramp-downs. It also doesn't deal with -// the end-of-scheduler drop to 0 VUs, whether graceful or not. These are +// the end-of-executor drop to 0 VUs, whether graceful or not. These are // handled by GetExecutionRequirements(), which internally uses this method and // reserveVUsForGracefulRampDowns(). // // The zeroEnd argument tells the method if we should artificially add a step -// with 0 VUs at offset sum(stages.duration), i.e. when the scheduler is +// with 0 VUs at offset sum(stages.duration), i.e. when the executor is // supposed to end. // // It's also important to note how scaling works. Say, we ramp up from 0 to 10 @@ -287,15 +287,15 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // If the graceful ramp-downs are enabled, we need to reserve any VUs that may // potentially have to finish running iterations when we're scaling their number -// down. This would prevent attempts from other schedulers to use them while the +// down. This would prevent attempts from other executors to use them while the // iterations are finishing up during their allotted gracefulRampDown periods. // // But we also need to be careful to not over-allocate more VUs than we actually // need. We should never have more PlannedVUs than the max(startVUs, // stage[n].target), even if we're quickly scaling VUs up and down multiple // times, one after the other. In those cases, any previously reserved VUs -// finishing up interrupted iterations should be reused by the scheduler, -// instead of new ones being requested from the executor. +// finishing up interrupted iterations should be reused by the executor, +// instead of new ones being requested from the execution state. // // Here's an example with graceful ramp-town (i.e. "uninterruptible" // iterations), where stars represent actively scheduled VUs and dots are used @@ -317,7 +317,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // We start with 4 VUs, scale to 6, scale down to 1, scale up to 5, scale down // to 1 again, scale up to 4, back to 1, and finally back down to 0. If our // gracefulStop timeout was 30s (the default), then we'll stay with 6 PlannedVUs -// until t=32 in the test above, and the actual scheduler could run until t=52. +// until t=32 in the test above, and the actual executor could run until t=52. // See TestVariableLoopingVUsConfigExecutionPlanExample() for the above example // as a unit test. // @@ -327,11 +327,11 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // VUs from faliing down for the configured gracefulRampDown period. // // Finishing up the test, i.e. making sure we have a step with 0 VUs at time -// schedulerEndOffset, is not handled here. Instead GetExecutionRequirements() +// executorEndOffset, is not handled here. Instead GetExecutionRequirements() // takes care of that. But to make its job easier, this method won't add any -// steps with an offset that's greater or equal to schedulerEndOffset. +// steps with an offset that's greater or equal to executorEndOffset. func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( - rawSteps []lib.ExecutionStep, schedulerEndOffset time.Duration) []lib.ExecutionStep { + rawSteps []lib.ExecutionStep, executorEndOffset time.Duration) []lib.ExecutionStep { rawStepsLen := len(rawSteps) gracefulRampDownPeriod := vlvc.GetGracefulRampDown() @@ -368,8 +368,8 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( // Both their planned VUs and their gracefulRampDown periods will // be lower than what we're going to set from that new rawStep - // we've basically found a new upward slope or equal value again. - // - We reach schedulerEndOffset, in which case we are done - we can't - // add any new steps, since those will be after the scheduler end + // - We reach executorEndOffset, in which case we are done - we can't + // add any new steps, since those will be after the executor end // offset. // - We reach the end of the rawSteps, or we don't find any higher or // equal steps to prevStep in the next gracefulRampDown period. So @@ -398,10 +398,10 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( continue } - // We've reached the absolute scheduler end offset, and we were already + // We've reached the absolute executor end offset, and we were already // on a downward "slope" (i.e. the previous planned VUs are more than // the current planned VUs), so nothing more we can do here. - if timeOffsetWithTimeout >= schedulerEndOffset { + if timeOffsetWithTimeout >= executorEndOffset { break } @@ -416,7 +416,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( } // GetExecutionRequirements very dynamically reserves exactly the number of -// required VUs for this scheduler at every moment of the test. +// required VUs for this executor at every moment of the test. // // If gracefulRampDown is specified, it will also be taken into account, and the // number of needed VUs to handle that will also be reserved. See the @@ -432,29 +432,29 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( // - If the user manually ramped down VUs at the end of the test (i.e. the // last stage's target is 0), then this will have no effect. // - If the last stage's target is more than 0, the VUs at the end of the -// scheduler's life will have more time to finish their last iterations. +// executor's life will have more time to finish their last iterations. func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { steps := vlvc.getRawExecutionSteps(es, false) - schedulerEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) + executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) // Handle graceful ramp-downs, if we have them if vlvc.GracefulRampDown.Duration > 0 { - steps = vlvc.reserveVUsForGracefulRampDowns(steps, schedulerEndOffset) + steps = vlvc.reserveVUsForGracefulRampDowns(steps, executorEndOffset) } // If the last PlannedVUs value wasn't 0, add a last step with 0 if steps[len(steps)-1].PlannedVUs != 0 { - steps = append(steps, lib.ExecutionStep{TimeOffset: schedulerEndOffset, PlannedVUs: 0}) + steps = append(steps, lib.ExecutionStep{TimeOffset: executorEndOffset, PlannedVUs: 0}) } return steps } -// NewScheduler creates a new VariableLoopingVUs scheduler -func (vlvc VariableLoopingVUsConfig) NewScheduler(es *lib.ExecutorState, logger *logrus.Entry) (lib.Scheduler, error) { +// NewExecutor creates a new VariableLoopingVUs executor +func (vlvc VariableLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return VariableLoopingVUs{ - BaseScheduler: NewBaseScheduler(vlvc, es, logger), - config: vlvc, + BaseExecutor: NewBaseExecutor(vlvc, es, logger), + config: vlvc, }, nil } @@ -462,12 +462,12 @@ func (vlvc VariableLoopingVUsConfig) NewScheduler(es *lib.ExecutorState, logger // loops iterations with a variable number of VUs for the sum of all of the // specified stages' duration. type VariableLoopingVUs struct { - *BaseScheduler + *BaseExecutor config VariableLoopingVUsConfig } -// Make sure we implement the lib.Scheduler interface. -var _ lib.Scheduler = &VariableLoopingVUs{} +// Make sure we implement the lib.Executor interface. +var _ lib.Executor = &VariableLoopingVUs{} // Run constantly loops through as many iterations as possible on a variable // number of VUs for the specified stages. @@ -476,7 +476,7 @@ var _ lib.Scheduler = &VariableLoopingVUs{} // of a less complex way to implement it (besides the old "increment by 100ms // and see what happens)... :/ so maybe see how it can be spit? func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := vlv.executorState.Options.ExecutionSegment + segment := vlv.executionState.Options.ExecutionSegment rawExecutionSteps := vlv.config.getRawExecutionSteps(segment, true) regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) if !isFinal { @@ -498,7 +498,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo vlv.logger.WithFields(logrus.Fields{ "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(segment), "maxVUs": maxVUs, "duration": regularDuration, "numStages": len(vlv.config.Stages)}, - ).Debug("Starting scheduler run...") + ).Debug("Starting executor run...") activeVUsCount := new(int64) vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) @@ -517,13 +517,13 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo go trackProgress(ctx, maxDurationCtx, regDurationCtx, vlv, progresFn) // Actually schedule the VUs and iterations, likely the most complicated - // scheduler among all of them... + // executor among all of them... activeVUs := &sync.WaitGroup{} defer activeVUs.Wait() - runIteration := getIterationRunner(vlv.executorState, vlv.logger, out) + runIteration := getIterationRunner(vlv.executionState, vlv.logger, out) getVU := func() (lib.VU, error) { - vu, err := vlv.executorState.GetPlannedVU(vlv.logger, true) + vu, err := vlv.executionState.GetPlannedVU(vlv.logger, true) if err != nil { cancel() } else { @@ -533,7 +533,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo return vu, err } returnVU := func(vu lib.VU) { - vlv.executorState.ReturnVU(vu, true) + vlv.executionState.ReturnVU(vu, true) atomic.AddInt64(activeVUsCount, -1) activeVUs.Done() } diff --git a/lib/scheduler/vu_handle.go b/lib/scheduler/vu_handle.go index 97f0a77eefe..6d3030c2c61 100644 --- a/lib/scheduler/vu_handle.go +++ b/lib/scheduler/vu_handle.go @@ -29,7 +29,7 @@ import ( "github.com/loadimpact/k6/lib" ) -// This is a helper type used in schedulers where we have to dynamically control +// This is a helper type used in executors where we have to dynamically control // the number of VUs that are simultaneously running. For the moment, it is used in the VariableLoopingVUs and // // TODO: something simpler? @@ -103,7 +103,7 @@ func (vh *vuHandle) hardStop() { //TODO: simplify this somehow - I feel like there should be a better way to //implement this logic... maybe with sync.Cond? func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.VU)) { - schedulerDone := vh.parentCtx.Done() + executorDone := vh.parentCtx.Done() var vu lib.VU defer func() { @@ -119,15 +119,15 @@ mainLoop: canStartIter, ctx := vh.canStartIter, vh.ctx vh.mutex.RUnlock() - // Wait for either the scheduler to be done, or for us to be unpaused + // Wait for either the executor to be done, or for us to be unpaused select { case <-canStartIter: // Best case, we're currently running, so we do nothing here, we // just continue straight ahead. - case <-schedulerDone: - return // The whole scheduler is done, nothing more to do. + case <-executorDone: + return // The whole executor is done, nothing more to do. default: - // We're not running, but the scheduler isn't done yet, so we wait + // We're not running, but the executor isn't done yet, so we wait // for either one of those conditions. But before that, we'll return // our VU to the pool, if we have it. if vu != nil { @@ -141,15 +141,15 @@ mainLoop: // hardStop was called, start a fresh iteration to get the new // context and signal channel continue mainLoop - case <-schedulerDone: - return // The whole scheduler is done, nothing more to do. + case <-executorDone: + return // The whole executor is done, nothing more to do. } } // Probably not needed, but just in case - if both running and - // schedulerDone were actice, check that the scheduler isn't done. + // executorDone were actice, check that the executor isn't done. select { - case <-schedulerDone: + case <-executorDone: return default: } diff --git a/lib/schedulers.go b/lib/schedulers.go index 450a0082d72..09ad012a226 100644 --- a/lib/schedulers.go +++ b/lib/schedulers.go @@ -38,30 +38,30 @@ import ( //TODO: remove globals and use some type of explicit dependency injection? //nolint:gochecknoglobals var ( - schedulerConfigTypesMutex sync.RWMutex - schedulerConfigConstructors = make(map[string]SchedulerConfigConstructor) + executorConfigTypesMutex sync.RWMutex + executorConfigConstructors = make(map[string]ExecutorConfigConstructor) ) -// ExecutionStep is used by different schedulers to specify the planned number +// ExecutionStep is used by different executors to specify the planned number // of VUs they will need at a particular time. The times are relative to their // StartTime, i.e. they don't take into account the specific starting time of -// the scheduler, as that will be considered by the external executor +// the executor, as that will be considered by the external execution executor // separately. // // A slice [{t1, v1}, {t2, v2}, {t3, v3}, ..., {tn, vn}] of execution steps -// means that a scheduler will need 0 VUs until t1, it will need v1 number of +// means that a executor will need 0 VUs until t1, it will need v1 number of // VUs from time t1 until t2, need v2 number of VUs from time t2 to t3, and so // on. t1 is usually 0, tn is usually the same as GetMaxDuration() and vn is // usually 0. // // Keep in mind that t(i) may be exactly equal to t(i+i), when there's an abrupt -// transition in the number of VUs required by a scheduler. For example, the -// variable-looping-vus scheduler may have 0-duration stages, or it may scale up +// transition in the number of VUs required by a executor. For example, the +// variable-looping-vus executor may have 0-duration stages, or it may scale up // VUs in its last stage right until the end. These // -// []ExecutionStep is also used by the SchedulerConfigMap, to represent the -// amount of needed VUs among all schedulers, during the whole execution of a -// test script. In that context, each scheduler's StartTime is accounted for and +// []ExecutionStep is also used by the ExecutorConfigMap, to represent the +// amount of needed VUs among all executors, during the whole execution of a +// test script. In that context, each executor's StartTime is accounted for and // included in the offsets. type ExecutionStep struct { TimeOffset time.Duration @@ -69,10 +69,10 @@ type ExecutionStep struct { MaxUnplannedVUs uint64 } -//TODO: make []ExecutionStep or []SchedulerConfig their own type? +//TODO: make []ExecutionStep or []ExecutorConfig their own type? -// SchedulerConfig is an interface that should be implemented by all scheduler config types -type SchedulerConfig interface { +// ExecutorConfig is an interface that should be implemented by all executor config types +type ExecutorConfig interface { Validate() []error GetName() string @@ -82,30 +82,30 @@ type SchedulerConfig interface { // This is used to validate whether a particular script can run in the cloud // or, in the future, in the native k6 distributed execution. Currently only - // the manual-execution scheduler should return false. + // the externally-controlled executor should return false. IsDistributable() bool GetEnv() map[string]string GetExec() null.String //TODO: use interface{} so plain http requests can be specified? - // Calculates the VU requirements in different stages of the scheduler's + // Calculates the VU requirements in different stages of the executor's // execution, including any extensions caused by waiting for iterations to // finish with graceful stops or ramp-downs. GetExecutionRequirements(*ExecutionSegment) []ExecutionStep - // Return a human-readable description of the scheduler + // Return a human-readable description of the executor GetDescription(es *ExecutionSegment) string - NewScheduler(*ExecutorState, *logrus.Entry) (Scheduler, error) + NewExecutor(*ExecutionState, *logrus.Entry) (Executor, error) } // InitVUFunc is just a shorthand so we don't have to type the function // signature every time. type InitVUFunc func(context.Context, *logrus.Entry) (VU, error) -// Scheduler is the interface all schedulers should implement -type Scheduler interface { - GetConfig() SchedulerConfig +// Executor is the interface all executors should implement +type Executor interface { + GetConfig() ExecutorConfig GetProgress() *pb.ProgressBar GetLogger() *logrus.Entry @@ -113,48 +113,48 @@ type Scheduler interface { Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error } -// PausableScheduler should be implemented by the schedulers that can be paused -// and resumend in the middle of the test execution. Currently, only the manual -// execution scheduler implements it. -type PausableScheduler interface { +// PausableExecutor should be implemented by the executors that can be paused +// and resumend in the middle of the test execution. Currently, only the +// externally controlled executor implements it. +type PausableExecutor interface { SetPaused(bool) error } -// LiveUpdatableScheduler should be implemented for the schedulers whoose +// LiveUpdatableExecutor should be implemented for the executors whoose // configuration can be modified in the middle of the test execution. Currently, -// only the manual execution scheduler implements it. -type LiveUpdatableScheduler interface { +// only the manual execution executor implements it. +type LiveUpdatableExecutor interface { UpdateConfig(ctx context.Context, newConfig interface{}) error } -// SchedulerConfigConstructor is a simple function that returns a concrete +// ExecutorConfigConstructor is a simple function that returns a concrete // Config instance with the specified name and all default values correctly // initialized -type SchedulerConfigConstructor func(name string, rawJSON []byte) (SchedulerConfig, error) +type ExecutorConfigConstructor func(name string, rawJSON []byte) (ExecutorConfig, error) -// RegisterSchedulerConfigType adds the supplied SchedulerConfigConstructor as +// RegisterExecutorConfigType adds the supplied ExecutorConfigConstructor as // the constructor for its type in the configConstructors map, in a thread-safe // manner -func RegisterSchedulerConfigType(configType string, constructor SchedulerConfigConstructor) { - schedulerConfigTypesMutex.Lock() - defer schedulerConfigTypesMutex.Unlock() +func RegisterExecutorConfigType(configType string, constructor ExecutorConfigConstructor) { + executorConfigTypesMutex.Lock() + defer executorConfigTypesMutex.Unlock() if constructor == nil { - panic("scheduler configs: constructor is nil") + panic("executor configs: constructor is nil") } - if _, configTypeExists := schedulerConfigConstructors[configType]; configTypeExists { - panic("scheduler configs: lib.RegisterSchedulerConfigType called twice for " + configType) + if _, configTypeExists := executorConfigConstructors[configType]; configTypeExists { + panic("executor configs: lib.RegisterExecutorConfigType called twice for " + configType) } - schedulerConfigConstructors[configType] = constructor + executorConfigConstructors[configType] = constructor } -// SchedulerConfigMap can contain mixed scheduler config types -type SchedulerConfigMap map[string]SchedulerConfig +// ExecutorConfigMap can contain mixed executor config types +type ExecutorConfigMap map[string]ExecutorConfig // UnmarshalJSON implements the json.Unmarshaler interface in a two-step manner, // creating the correct type of configs based on the `type` property. -func (scs *SchedulerConfigMap) UnmarshalJSON(data []byte) error { +func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { if len(data) == 0 { return nil } @@ -165,17 +165,17 @@ func (scs *SchedulerConfigMap) UnmarshalJSON(data []byte) error { //TODO: use a more sophisticated combination of dec.Token() and dec.More(), // which would allow us to support both arrays and maps for this config? - var protoConfigs map[string]protoSchedulerConfig + var protoConfigs map[string]protoExecutorConfig if err := StrictJSONUnmarshal(data, &protoConfigs); err != nil { return err } - result := make(SchedulerConfigMap, len(protoConfigs)) + result := make(ExecutorConfigMap, len(protoConfigs)) for k, v := range protoConfigs { if v.configType == "" { return fmt.Errorf("execution config '%s' doesn't have a type value", k) } - config, err := GetParsedSchedulerConfig(k, v.configType, v.rawJSON) + config, err := GetParsedExecutorConfig(k, v.configType, v.rawJSON) if err != nil { return err } @@ -187,18 +187,18 @@ func (scs *SchedulerConfigMap) UnmarshalJSON(data []byte) error { return nil } -// Validate checks if all of the specified scheduler options make sense -func (scs SchedulerConfigMap) Validate() (errors []error) { - for name, scheduler := range scs { - if schedErr := scheduler.Validate(); len(schedErr) != 0 { +// Validate checks if all of the specified executor options make sense +func (scs ExecutorConfigMap) Validate() (errors []error) { + for name, executor := range scs { + if schedErr := executor.Validate(); len(schedErr) != 0 { errors = append(errors, - fmt.Errorf("scheduler %s has errors: %s", name, ConcatErrors(schedErr, ", "))) + fmt.Errorf("executor %s has errors: %s", name, ConcatErrors(schedErr, ", "))) } } return errors } -// GetSortedSchedulerConfigs returns a slice with the scheduler configurations, +// GetSortedConfigs returns a slice with the executor configurations, // sorted in a consistent and predictable manner. It is useful when we want or // have to avoid using maps with string keys (and tons of string lookups in // them) and avoid the unpredictable iterations over Go maps. Slices allow us @@ -207,10 +207,10 @@ func (scs SchedulerConfigMap) Validate() (errors []error) { // The configs in the returned slice will be sorted by their start times in an // ascending order, and alphabetically by their names (which are unique) if // there are ties. -func (scs SchedulerConfigMap) GetSortedSchedulerConfigs() []SchedulerConfig { - configs := make([]SchedulerConfig, len(scs)) +func (scs ExecutorConfigMap) GetSortedConfigs() []ExecutorConfig { + configs := make([]ExecutorConfig, len(scs)) - // Populate the configs slice with sorted scheduler configs + // Populate the configs slice with sorted executor configs i := 0 for _, config := range scs { configs[i] = config // populate the slice in an unordered manner @@ -231,14 +231,14 @@ func (scs SchedulerConfigMap) GetSortedSchedulerConfigs() []SchedulerConfig { } // GetFullExecutionRequirements combines the execution requirements from all of -// the configured schedulers. It takes into account their start times and their +// the configured executors. It takes into account their start times and their // individual VU requirements and calculates the total VU requirements for each // moment in the test execution. -func (scs SchedulerConfigMap) GetFullExecutionRequirements(executionSegment *ExecutionSegment) []ExecutionStep { - sortedConfigs := scs.GetSortedSchedulerConfigs() +func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *ExecutionSegment) []ExecutionStep { + sortedConfigs := scs.GetSortedConfigs() - // Combine the steps and requirements from all different schedulers, and - // sort them by their time offset, counting the schedulers' startTimes as + // Combine the steps and requirements from all different executors, and + // sort them by their time offset, counting the executors' startTimes as // well. type trackedStep struct { ExecutionStep @@ -249,13 +249,13 @@ func (scs SchedulerConfigMap) GetFullExecutionRequirements(executionSegment *Exe configStartTime := config.GetStartTime() configSteps := config.GetExecutionRequirements(executionSegment) for _, cs := range configSteps { - cs.TimeOffset += configStartTime // add the scheduler start time to the step time offset + cs.TimeOffset += configStartTime // add the executor start time to the step time offset trackedSteps = append(trackedSteps, trackedStep{cs, configID}) } } // Sort by (time offset, config id). It's important that we use stable // sorting algorithm, since there are could be steps with the same time from - // the same scheduler and their order is important. + // the same executor and their order is important. sort.SliceStable(trackedSteps, func(a, b int) bool { switch { case trackedSteps[a].TimeOffset < trackedSteps[b].TimeOffset: @@ -267,9 +267,9 @@ func (scs SchedulerConfigMap) GetFullExecutionRequirements(executionSegment *Exe } }) - // Go through all of the sorted steps from all of the schedulers, and + // Go through all of the sorted steps from all of the executors, and // build a new list of execution steps that consolidates all of their - // requirements. If multiple schedulers have an execution step at exactly + // requirements. If multiple executors have an execution step at exactly // the same time offset, they will be combined into a single new execution // step with the sum of the values from the previous ones. currentTimeOffset := time.Duration(0) @@ -310,32 +310,32 @@ func (scs SchedulerConfigMap) GetFullExecutionRequirements(executionSegment *Exe return consolidatedSteps } -// GetParsedSchedulerConfig returns a struct instance corresponding to the supplied +// GetParsedExecutorConfig returns a struct instance corresponding to the supplied // config type. It will be fully initialized - with both the default values of // the type, as well as with whatever the user had specified in the JSON -func GetParsedSchedulerConfig(name, configType string, rawJSON []byte) (result SchedulerConfig, err error) { - schedulerConfigTypesMutex.Lock() - defer schedulerConfigTypesMutex.Unlock() +func GetParsedExecutorConfig(name, configType string, rawJSON []byte) (result ExecutorConfig, err error) { + executorConfigTypesMutex.Lock() + defer executorConfigTypesMutex.Unlock() - constructor, exists := schedulerConfigConstructors[configType] + constructor, exists := executorConfigConstructors[configType] if !exists { - return nil, fmt.Errorf("unknown execution scheduler type '%s'", configType) + return nil, fmt.Errorf("unknown executor type '%s'", configType) } return constructor(name, rawJSON) } -type protoSchedulerConfig struct { +type protoExecutorConfig struct { configType string rawJSON json.RawMessage } // UnmarshalJSON just reads unmarshals the base config (to get the type), but it also // stores the unprocessed JSON so we can parse the full config in the next step -func (pc *protoSchedulerConfig) UnmarshalJSON(b []byte) error { +func (pc *protoExecutorConfig) UnmarshalJSON(b []byte) error { var tmp struct { ConfigType string `json:"type"` } err := json.Unmarshal(b, &tmp) - *pc = protoSchedulerConfig{tmp.ConfigType, b} + *pc = protoExecutorConfig{tmp.ConfigType, b} return err } From a970beec64ab1c57e1b1f4e665abe88ddb43d107 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 16 Aug 2019 23:31:34 +0300 Subject: [PATCH 022/350] Rename files and package names, and clean up previous code renames --- api/v1/status_routes.go | 6 +-- cmd/config.go | 4 +- cmd/config_consolidation_test.go | 46 +++++++++---------- core/engine_test.go | 4 +- core/local/local.go | 16 +++---- core/local/local_test.go | 18 ++++---- js/runner_test.go | 2 +- lib/{executor.go => execution.go} | 0 lib/{executor_test.go => execution_test.go} | 0 lib/{scheduler => executor}/base_config.go | 2 +- .../base_executor.go} | 2 +- .../constant_arrival_rate.go | 2 +- .../constant_looping_vus.go | 2 +- .../execution_config_shortcuts.go | 2 +- .../executors_test.go} | 2 +- .../externally-controlled.go} | 2 +- lib/{scheduler => executor}/helpers.go | 2 +- .../per_vu_iterations.go | 2 +- .../shared_iterations.go | 2 +- .../variable_arrival_rate.go | 2 +- .../variable_looping_vus.go | 2 +- lib/{scheduler => executor}/vu_handle.go | 2 +- lib/{schedulers.go => executors.go} | 6 +-- 23 files changed, 64 insertions(+), 64 deletions(-) rename lib/{executor.go => execution.go} (100%) rename lib/{executor_test.go => execution_test.go} (100%) rename lib/{scheduler => executor}/base_config.go (99%) rename lib/{scheduler/base_scheduler.go => executor/base_executor.go} (99%) rename lib/{scheduler => executor}/constant_arrival_rate.go (99%) rename lib/{scheduler => executor}/constant_looping_vus.go (99%) rename lib/{scheduler => executor}/execution_config_shortcuts.go (99%) rename lib/{scheduler/schedulers_test.go => executor/executors_test.go} (99%) rename lib/{scheduler/manual.go => executor/externally-controlled.go} (99%) rename lib/{scheduler => executor}/helpers.go (99%) rename lib/{scheduler => executor}/per_vu_iterations.go (99%) rename lib/{scheduler => executor}/shared_iterations.go (99%) rename lib/{scheduler => executor}/variable_arrival_rate.go (99%) rename lib/{scheduler => executor}/variable_looping_vus.go (99%) rename lib/{scheduler => executor}/vu_handle.go (99%) rename lib/{schedulers.go => executors.go} (99%) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index 4b2b1993bfa..cf2acaad030 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -28,7 +28,7 @@ import ( "github.com/julienschmidt/httprouter" "github.com/loadimpact/k6/api/common" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/scheduler" + "github.com/loadimpact/k6/lib/executor" "github.com/manyminds/api2go/jsonapi" ) @@ -46,11 +46,11 @@ func HandleGetStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Param func getFirstExternallyControlledExecutor( execScheduler lib.ExecutionScheduler, -) (*scheduler.ExternallyControlled, error) { +) (*executor.ExternallyControlled, error) { executors := execScheduler.GetExecutors() for _, s := range executors { - if mex, ok := s.(*scheduler.ExternallyControlled); ok { + if mex, ok := s.(*executor.ExternallyControlled); ok { return mex, nil } } diff --git a/cmd/config.go b/cmd/config.go index 00c253a4fe6..d690bbd4dcf 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -34,7 +34,7 @@ import ( null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/scheduler" + "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/stats/cloud" "github.com/loadimpact/k6/stats/datadog" "github.com/loadimpact/k6/stats/influxdb" @@ -232,7 +232,7 @@ func applyDefault(conf Config) Config { func deriveAndValidateConfig(conf Config) (result Config, err error) { result = conf - result.Options, err = scheduler.DeriveExecutionFromShortcuts(conf.Options) + result.Options, err = executor.DeriveExecutionFromShortcuts(conf.Options) if err != nil { return result, err } diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 27621738fc1..e0dd220ef84 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -35,7 +35,7 @@ import ( null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/scheduler" + "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" ) @@ -74,10 +74,10 @@ func setEnv(t *testing.T, newEnv []string) (restoreEnv func()) { func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config - sched := c.Execution[lib.DefaultExecutorName] - require.NotEmpty(t, sched) - require.IsType(t, scheduler.PerVUIteationsConfig{}, sched) - perVuIters, ok := sched.(scheduler.PerVUIteationsConfig) + exec := c.Execution[lib.DefaultExecutorName] + require.NotEmpty(t, exec) + require.IsType(t, executor.PerVUIteationsConfig{}, exec) + perVuIters, ok := exec.(executor.PerVUIteationsConfig) require.True(t, ok) assert.Equal(t, null.NewInt(1, false), perVuIters.Iterations) assert.Equal(t, null.NewInt(1, false), perVuIters.VUs) @@ -85,10 +85,10 @@ func verifyOneIterPerOneVU(t *testing.T, c Config) { func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultExecutorName] - require.NotEmpty(t, sched) - require.IsType(t, scheduler.SharedIteationsConfig{}, sched) - sharedIterConfig, ok := sched.(scheduler.SharedIteationsConfig) + exec := c.Execution[lib.DefaultExecutorName] + require.NotEmpty(t, exec) + require.IsType(t, executor.SharedIteationsConfig{}, exec) + sharedIterConfig, ok := exec.(executor.SharedIteationsConfig) require.True(t, ok) assert.Equal(t, vus, sharedIterConfig.VUs) assert.Equal(t, iters, sharedIterConfig.Iterations) @@ -99,10 +99,10 @@ func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultExecutorName] - require.NotEmpty(t, sched) - require.IsType(t, scheduler.ConstantLoopingVUsConfig{}, sched) - clvc, ok := sched.(scheduler.ConstantLoopingVUsConfig) + exec := c.Execution[lib.DefaultExecutorName] + require.NotEmpty(t, exec) + require.IsType(t, executor.ConstantLoopingVUsConfig{}, exec) + clvc, ok := exec.(executor.ConstantLoopingVUsConfig) require.True(t, ok) assert.Equal(t, vus, clvc.VUs) assert.Equal(t, types.NullDurationFrom(duration), clvc.Duration) @@ -111,12 +111,12 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing } } -func verifyVarLoopingVUs(startVus null.Int, stages []scheduler.Stage) func(t *testing.T, c Config) { +func verifyVarLoopingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - sched := c.Execution[lib.DefaultExecutorName] - require.NotEmpty(t, sched) - require.IsType(t, scheduler.VariableLoopingVUsConfig{}, sched) - clvc, ok := sched.(scheduler.VariableLoopingVUsConfig) + exec := c.Execution[lib.DefaultExecutorName] + require.NotEmpty(t, exec) + require.IsType(t, executor.VariableLoopingVUsConfig{}, exec) + clvc, ok := exec.(executor.VariableLoopingVUsConfig) require.True(t, ok) assert.Equal(t, startVus, clvc.StartVUs) assert.Equal(t, startVus, c.VUs) @@ -131,14 +131,14 @@ func verifyVarLoopingVUs(startVus null.Int, stages []scheduler.Stage) func(t *te // A helper function that accepts (duration in second, VUs) pairs and returns // a valid slice of stage structs -func buildStages(durationsAndVUs ...int64) []scheduler.Stage { +func buildStages(durationsAndVUs ...int64) []executor.Stage { l := len(durationsAndVUs) if l%2 != 0 { panic("wrong len") } - result := make([]scheduler.Stage, 0, l/2) + result := make([]executor.Stage, 0, l/2) for i := 0; i < l; i += 2 { - result = append(result, scheduler.Stage{ + result = append(result, executor.Stage{ Duration: types.NullDurationFrom(time.Duration(durationsAndVUs[i]) * time.Second), Target: null.IntFrom(durationsAndVUs[i+1]), }) @@ -254,7 +254,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) - sharedIterConfig := c.Execution[lib.DefaultExecutorName].(scheduler.SharedIteationsConfig) + sharedIterConfig := c.Execution[lib.DefaultExecutorName].(executor.SharedIteationsConfig) assert.Equal(t, time.Duration(sharedIterConfig.MaxDuration.Duration), 10*time.Second) }}, // This should get a validation error since VUs are more than the shared iterations @@ -439,7 +439,7 @@ func runTestCase( require.NoError(t, err) derivedConfig := consolidatedConfig - derivedConfig.Options, err = scheduler.DeriveExecutionFromShortcuts(consolidatedConfig.Options) + derivedConfig.Options, err = executor.DeriveExecutionFromShortcuts(consolidatedConfig.Options) if testCase.expected.derivationError { require.Error(t, err) return diff --git a/core/engine_test.go b/core/engine_test.go index d1141fdb481..56fb1274bfd 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -33,8 +33,8 @@ import ( "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/lib/metrics" - "github.com/loadimpact/k6/lib/scheduler" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" @@ -58,7 +58,7 @@ func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts li ctx = context.Background() } - newOpts, err := scheduler.DeriveExecutionFromShortcuts(lib.Options{ + newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ MetricSamplesBufferSize: null.NewInt(200, false), }.Apply(runner.GetOptions()).Apply(opts)) require.NoError(t, err) diff --git a/core/local/local.go b/core/local/local.go index 844ed3da25f..7ed0be4a8ca 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -237,10 +237,10 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa }) logger.Debugf("Finished initializing needed VUs, start initializing executors...") - for _, executor := range e.executors { - executorConfig := executor.GetConfig() + for _, exec := range e.executors { + executorConfig := exec.GetConfig() - if err := executor.Init(ctx); err != nil { + if err := exec.Init(ctx); err != nil { return fmt.Errorf("error while initializing executor %s: %s", executorConfig.GetName(), err) } logger.Debugf("Initialized executor %s", executorConfig.GetName()) @@ -332,8 +332,8 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam // Start all executors at their particular startTime in a separate goroutine... logger.Debug("Start all executors...") - for _, executor := range e.executors { - go runExecutor(executor) + for _, exec := range e.executors { + go runExecutor(exec) } // Wait for all executors to finish @@ -370,12 +370,12 @@ func (e *ExecutionScheduler) SetPaused(pause bool) error { return e.state.Resume() } - for _, executor := range e.executors { - pausableExecutor, ok := executor.(lib.PausableExecutor) + for _, exec := range e.executors { + pausableExecutor, ok := exec.(lib.PausableExecutor) if !ok { return fmt.Errorf( "%s executor '%s' doesn't support pause and resume operations after its start", - executor.GetConfig().GetType(), executor.GetConfig().GetName(), + exec.GetConfig().GetType(), exec.GetConfig().GetName(), ) } if err := pausableExecutor.SetPaused(pause); err != nil { diff --git a/core/local/local_test.go b/core/local/local_test.go index eea635e8513..d9bc4ad44c5 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -32,9 +32,9 @@ import ( "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/netext" - "github.com/loadimpact/k6/lib/scheduler" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" @@ -53,7 +53,7 @@ func newTestExecutionScheduler( runner = &lib.MiniRunner{} } ctx, cancel = context.WithCancel(context.Background()) - newOpts, err := scheduler.DeriveExecutionFromShortcuts(lib.Options{ + newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ MetricSamplesBufferSize: null.NewInt(200, false), }.Apply(runner.GetOptions()).Apply(opts)) require.NoError(t, err) @@ -289,10 +289,10 @@ func TestExecutionSchedulerRuntimeErrors(t *testing.T) { func TestExecutionSchedulerEndErrors(t *testing.T) { t.Parallel() - scheduler := scheduler.NewConstantLoopingVUsConfig("we_need_hard_stop") - scheduler.VUs = null.IntFrom(10) - scheduler.Duration = types.NullDurationFrom(1 * time.Second) - scheduler.GracefulStop = types.NullDurationFrom(0 * time.Second) + exec := executor.NewConstantLoopingVUsConfig("we_need_hard_stop") + exec.VUs = null.IntFrom(10) + exec.Duration = types.NullDurationFrom(1 * time.Second) + exec.GracefulStop = types.NullDurationFrom(0 * time.Second) runner := &lib.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { @@ -300,7 +300,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { return errors.New("hi") }, Options: lib.Options{ - Execution: lib.ExecutorConfigMap{scheduler.GetName(): scheduler}, + Execution: lib.ExecutorConfigMap{exec.GetName(): exec}, }, } logger, hook := logtest.NewNullLogger() @@ -324,7 +324,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { t.Parallel() metric := &stats.Metric{Name: "test_metric"} - options, err := scheduler.DeriveExecutionFromShortcuts(lib.Options{ + options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ VUs: null.IntFrom(1), Iterations: null.IntFrom(100), }) @@ -540,7 +540,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { runner, err := js.New(&loader.SourceData{URL: &url.URL{Path: "/script.js"}, Data: script}, nil, lib.RuntimeOptions{}) require.NoError(t, err) - options, err := scheduler.DeriveExecutionFromShortcuts(lib.Options{ + options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ Iterations: null.IntFrom(2), VUs: null.IntFrom(1), SystemTags: lib.GetTagSet(lib.DefaultSystemTagList...), diff --git a/js/runner_test.go b/js/runner_test.go index 2b90d2b1a6b..b1f61a2e8e6 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -46,8 +46,8 @@ import ( k6metrics "github.com/loadimpact/k6/js/modules/k6/metrics" "github.com/loadimpact/k6/js/modules/k6/ws" "github.com/loadimpact/k6/lib" + _ "github.com/loadimpact/k6/lib/executor" //TODO: figure out something better "github.com/loadimpact/k6/lib/metrics" - _ "github.com/loadimpact/k6/lib/scheduler" //TODO: figure out something better "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" diff --git a/lib/executor.go b/lib/execution.go similarity index 100% rename from lib/executor.go rename to lib/execution.go diff --git a/lib/executor_test.go b/lib/execution_test.go similarity index 100% rename from lib/executor_test.go rename to lib/execution_test.go diff --git a/lib/scheduler/base_config.go b/lib/executor/base_config.go similarity index 99% rename from lib/scheduler/base_config.go rename to lib/executor/base_config.go index 6fe6aaa0114..f8ddf3edcf3 100644 --- a/lib/scheduler/base_config.go +++ b/lib/executor/base_config.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "fmt" diff --git a/lib/scheduler/base_scheduler.go b/lib/executor/base_executor.go similarity index 99% rename from lib/scheduler/base_scheduler.go rename to lib/executor/base_executor.go index d511ab1d3a5..43ad966aa13 100644 --- a/lib/scheduler/base_scheduler.go +++ b/lib/executor/base_executor.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go similarity index 99% rename from lib/scheduler/constant_arrival_rate.go rename to lib/executor/constant_arrival_rate.go index 8374e93bac3..946bb6b854e 100644 --- a/lib/scheduler/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/constant_looping_vus.go b/lib/executor/constant_looping_vus.go similarity index 99% rename from lib/scheduler/constant_looping_vus.go rename to lib/executor/constant_looping_vus.go index 8cfea79f6af..4d9a97b1543 100644 --- a/lib/scheduler/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go similarity index 99% rename from lib/scheduler/execution_config_shortcuts.go rename to lib/executor/execution_config_shortcuts.go index 2b4aa2f97cb..b0a63f24ddb 100644 --- a/lib/scheduler/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "github.com/loadimpact/k6/lib" diff --git a/lib/scheduler/schedulers_test.go b/lib/executor/executors_test.go similarity index 99% rename from lib/scheduler/schedulers_test.go rename to lib/executor/executors_test.go index f62964645f8..dcf2945b861 100644 --- a/lib/scheduler/schedulers_test.go +++ b/lib/executor/executors_test.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "encoding/json" diff --git a/lib/scheduler/manual.go b/lib/executor/externally-controlled.go similarity index 99% rename from lib/scheduler/manual.go rename to lib/executor/externally-controlled.go index 7068cab08cd..7c3a3643aea 100644 --- a/lib/scheduler/manual.go +++ b/lib/executor/externally-controlled.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/helpers.go b/lib/executor/helpers.go similarity index 99% rename from lib/scheduler/helpers.go rename to lib/executor/helpers.go index cbf968a23fa..99481b2deb5 100644 --- a/lib/scheduler/helpers.go +++ b/lib/executor/helpers.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/per_vu_iterations.go b/lib/executor/per_vu_iterations.go similarity index 99% rename from lib/scheduler/per_vu_iterations.go rename to lib/executor/per_vu_iterations.go index a6f325f31ba..63999a3d25e 100644 --- a/lib/scheduler/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/shared_iterations.go b/lib/executor/shared_iterations.go similarity index 99% rename from lib/scheduler/shared_iterations.go rename to lib/executor/shared_iterations.go index 3f7bdc7ebc6..6500a2edcda 100644 --- a/lib/scheduler/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go similarity index 99% rename from lib/scheduler/variable_arrival_rate.go rename to lib/executor/variable_arrival_rate.go index b7bdd204b90..2bad1795ea0 100644 --- a/lib/scheduler/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/variable_looping_vus.go b/lib/executor/variable_looping_vus.go similarity index 99% rename from lib/scheduler/variable_looping_vus.go rename to lib/executor/variable_looping_vus.go index acdbd21563a..2bb6883a016 100644 --- a/lib/scheduler/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/scheduler/vu_handle.go b/lib/executor/vu_handle.go similarity index 99% rename from lib/scheduler/vu_handle.go rename to lib/executor/vu_handle.go index 6d3030c2c61..2e0c13ee421 100644 --- a/lib/scheduler/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -18,7 +18,7 @@ * */ -package scheduler +package executor import ( "context" diff --git a/lib/schedulers.go b/lib/executors.go similarity index 99% rename from lib/schedulers.go rename to lib/executors.go index 09ad012a226..251831d8cbc 100644 --- a/lib/schedulers.go +++ b/lib/executors.go @@ -189,10 +189,10 @@ func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { // Validate checks if all of the specified executor options make sense func (scs ExecutorConfigMap) Validate() (errors []error) { - for name, executor := range scs { - if schedErr := executor.Validate(); len(schedErr) != 0 { + for name, exec := range scs { + if execErr := exec.Validate(); len(execErr) != 0 { errors = append(errors, - fmt.Errorf("executor %s has errors: %s", name, ConcatErrors(schedErr, ", "))) + fmt.Errorf("executor %s has errors: %s", name, ConcatErrors(execErr, ", "))) } } return errors From 96eb1b3ef357dbdf5de3b43bb034089284683402 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 16 Aug 2019 23:47:51 +0300 Subject: [PATCH 023/350] Rename partial iterations to interrupted iterations This change is to avoid calling the same thing with differen names, after I renamed the user-facing text as described in https://github.com/loadimpact/k6/pull/1007#issuecomment-497319205 --- lib/execution.go | 34 +++++++++++++++++----------------- lib/executor/helpers.go | 2 +- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/lib/execution.go b/lib/execution.go index 79bd138a57e..e15ed1cac94 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -199,7 +199,7 @@ type ExecutionState struct { // script `duration`, scaling down of VUs via `stages`, a user hitting // Ctrl+C, change of `vus` via the externally controlled executor's REST // API, etc. - partialIterationsCount *uint64 + interruptedIterationsCount *uint64 // A nanosecond UNIX timestamp that is set when the test is actually // started. The default 0 value is used to denote that the test hasn't @@ -255,18 +255,18 @@ func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *E Options: options, vus: make(chan VU, maxPossibleVUs), - currentVUIdentifier: new(uint64), - initializedVUs: new(int64), - uninitializedUnplannedVUs: &maxUnplannedUninitializedVUs, - activeVUs: new(int64), - fullIterationsCount: new(uint64), - partialIterationsCount: new(uint64), - startTime: new(int64), - endTime: new(int64), - currentPauseTime: new(int64), - pauseStateLock: sync.RWMutex{}, - totalPausedDuration: 0, // Accessed only behind the pauseStateLock - resumeNotify: resumeNotify, + currentVUIdentifier: new(uint64), + initializedVUs: new(int64), + uninitializedUnplannedVUs: &maxUnplannedUninitializedVUs, + activeVUs: new(int64), + fullIterationsCount: new(uint64), + interruptedIterationsCount: new(uint64), + startTime: new(int64), + endTime: new(int64), + currentPauseTime: new(int64), + pauseStateLock: sync.RWMutex{}, + totalPausedDuration: 0, // Accessed only behind the pauseStateLock + resumeNotify: resumeNotify, } } @@ -331,15 +331,15 @@ func (es *ExecutionState) AddFullIterations(count uint64) uint64 { // // IMPORTANT: for UI/information purposes only, don't use for synchronization. func (es *ExecutionState) GetPartialIterationCount() uint64 { - return atomic.LoadUint64(es.partialIterationsCount) + return atomic.LoadUint64(es.interruptedIterationsCount) } -// AddPartialIterations increments the number of partial (i.e interrupted) +// AddInterruptedIterations increments the number of partial (i.e interrupted) // iterations by the provided amount. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. -func (es *ExecutionState) AddPartialIterations(count uint64) uint64 { - return atomic.AddUint64(es.partialIterationsCount, count) +func (es *ExecutionState) AddInterruptedIterations(count uint64) uint64 { + return atomic.AddUint64(es.interruptedIterationsCount, count) } // MarkStarted saves the current timestamp as the test start time. diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 99481b2deb5..b59dd96af4e 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -91,7 +91,7 @@ func getIterationRunner(executionState *lib.ExecutionState, logger *logrus.Entry select { case <-ctx.Done(): // Don't log errors or emit iterations metrics from cancelled iterations - executionState.AddPartialIterations(1) + executionState.AddInterruptedIterations(1) default: if err != nil { if s, ok := err.(fmt.Stringer); ok { From 1ca5db7f5c8437dfd2fd1a5e1b231b1ea8cd386b Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Sat, 17 Aug 2019 00:02:01 +0300 Subject: [PATCH 024/350] Add some execution segment methods and tests --- lib/execution_segment.go | 43 +++++++++++++++++++ lib/execution_segment_test.go | 78 ++++++++++++++++++++++++++++++++++- 2 files changed, 120 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index a82bd10c26a..93c809d6a3e 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -185,6 +185,49 @@ func (es *ExecutionSegment) Split(numParts int64) ([]*ExecutionSegment, error) { //TODO: add a NewFromString() method +// Equal returns true only if the two execution segments have the same from and +// to values. +func (es *ExecutionSegment) Equal(other *ExecutionSegment) bool { + if es == other { + return true + } + thisFrom, otherFrom, thisTo, otherTo := zeroRat, zeroRat, oneRat, oneRat + if es != nil { + thisFrom, thisTo = es.from, es.to + } + if other != nil { + otherFrom, otherTo = other.from, other.to + } + return thisFrom.Cmp(otherFrom) == 0 && thisTo.Cmp(otherTo) == 0 +} + +// SubSegment returns a new execution sub-segment - if a is (1/2:1] and b is +// (0:1/2], then a.SubSegment(b) will return a new segment (1/2, 3/4]. +// +// The basic formula for c = a.SubSegment(b) is: +// c.from = a.from + b.from * (a.to - a.from) +// c.to = c.from + (b.to - b.from) * (a.to - a.from) +func (es *ExecutionSegment) SubSegment(child *ExecutionSegment) *ExecutionSegment { + if child == nil { + return es // 100% sub-segment is the original segment + } + + parentFrom, parentLength := zeroRat, oneRat + if es != nil { + parentFrom, parentLength = es.from, es.length + } + + resultFrom := new(big.Rat).Mul(parentLength, child.from) + resultFrom.Add(resultFrom, parentFrom) + + resultLength := new(big.Rat).Mul(parentLength, child.length) + return &ExecutionSegment{ + from: resultFrom, + length: resultLength, + to: new(big.Rat).Add(resultFrom, resultLength), + } +} + // helper function for rounding (up) of rational numbers to big.Int values func roundUp(rat *big.Rat) *big.Int { quo, rem := new(big.Int).QuoRem(rat.Num(), rat.Denom(), new(big.Int)) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index df4f6251fa1..b9c9db16184 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -20,4 +20,80 @@ package lib -//TODO +import ( + "math/big" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestExecutionSegments(t *testing.T) { + t.Parallel() + + var nilEs *ExecutionSegment + fullEs, err := NewExecutionSegment(big.NewRat(0, 1), big.NewRat(1, 1)) + require.NoError(t, err) + require.Equal(t, true, nilEs.Equal(fullEs)) + //TODO: more tests +} + +func TestExecutionSegmentSplit(t *testing.T) { + t.Parallel() + + var nilEs *ExecutionSegment + _, err := nilEs.Split(-1) + require.Error(t, err) + + _, err = nilEs.Split(0) + require.Error(t, err) + + segments, err := nilEs.Split(1) + require.NoError(t, err) + require.Len(t, segments, 1) + assert.Equal(t, "0:1", segments[0].String()) + + segments, err = nilEs.Split(2) + require.NoError(t, err) + require.Len(t, segments, 2) + assert.Equal(t, "0:1/2", segments[0].String()) + assert.Equal(t, "1/2:1", segments[1].String()) + + segments, err = nilEs.Split(3) + require.NoError(t, err) + require.Len(t, segments, 3) + assert.Equal(t, "0:1/3", segments[0].String()) + assert.Equal(t, "1/3:2/3", segments[1].String()) + assert.Equal(t, "2/3:1", segments[2].String()) + + secondQuarter, err := NewExecutionSegment(big.NewRat(1, 4), big.NewRat(2, 4)) + require.NoError(t, err) + + segments, err = secondQuarter.Split(1) + require.NoError(t, err) + require.Len(t, segments, 1) + assert.Equal(t, "1/4:1/2", segments[0].String()) + + segments, err = secondQuarter.Split(2) + require.NoError(t, err) + require.Len(t, segments, 2) + assert.Equal(t, "1/4:3/8", segments[0].String()) + assert.Equal(t, "3/8:1/2", segments[1].String()) + + segments, err = secondQuarter.Split(3) + require.NoError(t, err) + require.Len(t, segments, 3) + assert.Equal(t, "1/4:1/3", segments[0].String()) + assert.Equal(t, "1/3:5/12", segments[1].String()) + assert.Equal(t, "5/12:1/2", segments[2].String()) + + segments, err = secondQuarter.Split(4) + require.NoError(t, err) + require.Len(t, segments, 4) + assert.Equal(t, "1/4:5/16", segments[0].String()) + assert.Equal(t, "5/16:3/8", segments[1].String()) + assert.Equal(t, "3/8:7/16", segments[2].String()) + assert.Equal(t, "7/16:1/2", segments[3].String()) +} + +//TODO: a lot of other tests From 91b7a5f9176675ac39478931a2ac103614051ce4 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Sat, 17 Aug 2019 00:02:34 +0300 Subject: [PATCH 025/350] Remove an obsolete TODO --- cmd/cloud.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 59f4e2ad577..f2916c45b1c 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -104,7 +104,6 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return ExitCode{cerr, invalidConfigErrorCode} } - //TODO: warn about lack of support for --no-setup and --no-teardown in the cloud? //TODO: validate for usage of execution segment //TODO: validate for externally controlled executor (i.e. executors that aren't distributable) //TODO: move those validations to a separate function and reuse validateConfig()? From 1c6c78f9c88efec8230e359acc73bff0dbbc3594 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 13 Sep 2019 17:06:29 +0300 Subject: [PATCH 026/350] Export the archive CleanUpWrongMetadataJSON helper function --- lib/archive.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/lib/archive.go b/lib/archive.go index d9e482644d4..88a74f8eeb9 100644 --- a/lib/archive.go +++ b/lib/archive.go @@ -105,7 +105,7 @@ func (arc *Archive) getFs(name string) afero.Fs { return fs } -// cleanUpWrongMetadataJSON fixes issues with the metadata.json contents before +// CleanUpWrongMetadataJSON fixes issues with the metadata.json contents before // they are unmarshalled in the Archive struct. // // Currently, the only fix this function performs is the discarding of the @@ -114,7 +114,7 @@ func (arc *Archive) getFs(name string) afero.Fs { // 83193f8a96e06a190325b838b2cc451119d6b836. This basically means k6 v0.24.0 and // surrounding master commits. We filter these out by the value of the k6version // property, saved in the metadata.json since the previous to the above commit. -func cleanUpWrongMetadataJSON(data []byte) ([]byte, error) { +func CleanUpWrongMetadataJSON(data []byte) ([]byte, error) { var tmpArc map[string]interface{} if err := json.Unmarshal(data, &tmpArc); err != nil { return nil, err @@ -150,7 +150,7 @@ func cleanUpWrongMetadataJSON(data []byte) ([]byte, error) { } func (arc *Archive) loadMetadataJSON(data []byte) error { - data, err := cleanUpWrongMetadataJSON(data) + data, err := CleanUpWrongMetadataJSON(data) if err != nil { return err } From 5700ea66849ff84904bd22df8cba140935761de6 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 13 Sep 2019 17:07:21 +0300 Subject: [PATCH 027/350] Add text/JSON marshaling for execution segments --- lib/execution_segment.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 93c809d6a3e..bd0d124671e 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -140,6 +140,15 @@ func (es *ExecutionSegment) String() string { return es.from.RatString() + ":" + es.to.RatString() } +// MarshalText implements the encoding.TextMarshaler interface, so is used for +// text and JSON encoding of the execution segment. +func (es *ExecutionSegment) MarshalText() ([]byte, error) { + if es == nil { + return nil, nil + } + return []byte(es.String()), nil +} + // FloatLength is a helper method for getting some more human-readable // information about the execution segment. func (es *ExecutionSegment) FloatLength() float64 { From 99a114c1597ad9edef6ec0ff2bd7802ce955aa6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 2 Oct 2019 15:08:06 +0200 Subject: [PATCH 028/350] fix: typos --- api/v1/status_routes.go | 14 +++++++------- lib/execution.go | 18 +++++++++--------- lib/executor/externally-controlled.go | 4 ++-- lib/executors.go | 8 ++++---- 4 files changed, 22 insertions(+), 22 deletions(-) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index cf2acaad030..99b14014bb9 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -80,12 +80,12 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par } if status.VUsMax.Valid || status.VUs.Valid { - //TODO: add ability to specify the actual executor id? though thus should + //TODO: add ability to specify the actual executor id? Though this should //likely be in the v2 REST API, where we could implement it in a way that - //may allow us to eventually support other executor types - executor, uptateErr := getFirstExternallyControlledExecutor(engine.ExecutionScheduler) - if uptateErr != nil { - apiError(rw, "Execution config error", uptateErr.Error(), http.StatusInternalServerError) + //may allow us to eventually support other executor types. + executor, updateErr := getFirstExternallyControlledExecutor(engine.ExecutionScheduler) + if updateErr != nil { + apiError(rw, "Execution config error", updateErr.Error(), http.StatusInternalServerError) return } newConfig := executor.GetCurrentConfig().ExternallyControlledConfigParams @@ -95,8 +95,8 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par if status.VUs.Valid { newConfig.VUs = status.VUs } - if uptateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { - apiError(rw, "Config update error", uptateErr.Error(), http.StatusInternalServerError) + if updateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { + apiError(rw, "Config update error", updateErr.Error(), http.StatusInternalServerError) return } } diff --git a/lib/execution.go b/lib/execution.go index e15ed1cac94..f965898608a 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -68,7 +68,7 @@ type ExecutionScheduler interface { // // Currently, any executor, so any test, can be started in a paused state. // This will cause k6 to initialize all needed VUs, but it won't actually - // start the test. Later, the test can be started for real be + // start the test. Later, the test can be started for real by // resuming/unpausing it from the REST API. // // After a test is actually started, it may become impossible to pause it @@ -118,7 +118,7 @@ const MaxRetriesGetPlannedVU = 5 // involving atomics... // // The only functionality indended for synchronization is the one revolving -// around pausing, and uninitializedUnplannedVUs for restrictring the number of +// around pausing, and uninitializedUnplannedVUs for restricting the number of // unplanned VUs being initialized. type ExecutionState struct { // A copy of the options, so the different executors have access to them. @@ -206,9 +206,9 @@ type ExecutionState struct { // started yet... startTime *int64 - // A nanosecond UNIX timestamp that is set when the ends, either by an early - // context cancel or at its regularly scheduled time. The default 0 value is - // used to denote that the test hasn't ended yet... + // A nanosecond UNIX timestamp that is set when the test ends, either + // by an early context cancel or at its regularly scheduled time. + // The default 0 value is used to denote that the test hasn't ended yet. endTime *int64 // Stuff related to pausing follows. Read the docs in ExecutionScheduler for @@ -296,7 +296,7 @@ func (es *ExecutionState) ModInitializedVUsCount(mod int64) int64 { // GetCurrentlyActiveVUsCount returns the number of VUs that are currently // executing the test script. This also includes any VUs that are in the process -// of gracefullt winding down. +// of gracefully winding down. // // IMPORTANT: for UI/information purposes only, don't use for synchronization. func (es *ExecutionState) GetCurrentlyActiveVUsCount() int64 { @@ -365,7 +365,7 @@ func (es *ExecutionState) MarkEnded() { // HasStarted returns true if the test has actually started executing. // It will return false while a test is in the init phase, or if it has // been initially paused. But if will return true if a test is paused -// midway through its execution (see above for details regarind the +// midway through its execution (see above for details regarding the // feasibility of that pausing for normal executors). func (es *ExecutionState) HasStarted() bool { return atomic.LoadInt64(es.startTime) != 0 @@ -478,11 +478,11 @@ func (es *ExecutionState) ResumeNotify() <-chan struct{} { // we reach that timeout more than MaxRetriesGetPlannedVU number of times, this // function will return an error, since we either have a bug with some // executor, or the machine is very, very overloaded. -func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyAtiveVUCount bool) (VU, error) { +func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount bool) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { case vu := <-es.vus: - if modifyAtiveVUCount { + if modifyActiveVUCount { es.ModCurrentlyActiveVUsCount(+1) } //TODO: set environment and exec diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 7c3a3643aea..0363fcb2a36 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -72,7 +72,7 @@ func (mecc ExternallyControlledConfigParams) Validate() (errors []error) { if mecc.MaxVUs.Int64 < mecc.VUs.Int64 { errors = append(errors, fmt.Errorf( - "the specified maxVUs (%d) should more than or equal to the the number of active VUs (%d)", + "the specified maxVUs (%d) should be more than or equal to the the number of active VUs (%d)", mecc.MaxVUs.Int64, mecc.VUs.Int64, )) } @@ -264,7 +264,7 @@ func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf inter return errors.New("invalid config type") } if errs := newConfigParams.Validate(); len(errs) != 0 { - return fmt.Errorf("invalid confiuguration supplied: %s", lib.ConcatErrors(errs, ", ")) + return fmt.Errorf("invalid configuration supplied: %s", lib.ConcatErrors(errs, ", ")) } if newConfigParams.Duration != mex.startConfig.Duration { diff --git a/lib/executors.go b/lib/executors.go index 251831d8cbc..edf7fa3c5c0 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -114,13 +114,13 @@ type Executor interface { } // PausableExecutor should be implemented by the executors that can be paused -// and resumend in the middle of the test execution. Currently, only the +// and resumed in the middle of the test execution. Currently, only the // externally controlled executor implements it. type PausableExecutor interface { SetPaused(bool) error } -// LiveUpdatableExecutor should be implemented for the executors whoose +// LiveUpdatableExecutor should be implemented for the executors whose // configuration can be modified in the middle of the test execution. Currently, // only the manual execution executor implements it. type LiveUpdatableExecutor interface { @@ -254,7 +254,7 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec } } // Sort by (time offset, config id). It's important that we use stable - // sorting algorithm, since there are could be steps with the same time from + // sorting algorithm, since there could be steps with the same time from // the same executor and their order is important. sort.SliceStable(trackedSteps, func(a, b int) bool { switch { @@ -329,7 +329,7 @@ type protoExecutorConfig struct { rawJSON json.RawMessage } -// UnmarshalJSON just reads unmarshals the base config (to get the type), but it also +// UnmarshalJSON unmarshals the base config (to get the type), but it also // stores the unprocessed JSON so we can parse the full config in the next step func (pc *protoExecutorConfig) UnmarshalJSON(b []byte) error { var tmp struct { From e5e0058958a0d544a199cfbd29f98c9956fffa5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 3 Oct 2019 16:23:51 +0200 Subject: [PATCH 029/350] fix: typos --- core/engine.go | 2 +- core/engine_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/engine.go b/core/engine.go index 7a53f4e65e0..f4ef6b89b38 100644 --- a/core/engine.go +++ b/core/engine.go @@ -103,7 +103,7 @@ func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) return e, nil } -// Init is used to initialize the execuction scheduler. That's a costly operation, since it +// Init is used to initialize the execution scheduler. That's a costly operation, since it // initializes all of the planned VUs and could potentially take a long time. func (e *Engine) Init(ctx context.Context) error { return e.ExecutionScheduler.Init(ctx, e.Samples) diff --git a/core/engine_test.go b/core/engine_test.go index d4285654853..83aedbc29a6 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -726,7 +726,7 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { require.False(t, engine.IsTainted()) } - // The 3.1 sleep in the default function would cause the first VU to comlete 2 full iterations + // The 3.1 sleep in the default function would cause the first VU to complete 2 full iterations // and stat executing its third one, while the second VU will only fully complete 1 iteration // and will be canceled in the middle of its second one. assert.Equal(t, 3.0, getMetricSum(collector, metrics.Iterations.Name)) From 4110caf0ac208fc74e7df574aa0846dec838bb4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 7 Oct 2019 10:49:58 +0200 Subject: [PATCH 030/350] fix: typos --- core/local/local.go | 2 +- lib/helpers.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 7ed0be4a8ca..91a6c0059e5 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -246,7 +246,7 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa logger.Debugf("Initialized executor %s", executorConfig.GetName()) } - logger.Debugf("Initization completed") + logger.Debugf("Initialization completed") return nil } diff --git a/lib/helpers.go b/lib/helpers.go index 7953f288ad5..c3cbbd5f62c 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -70,9 +70,9 @@ func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { // called GetMaxPossibleVUs() and why there is no GetMaxUnplannedVUs() function. // // As an example, imagine that you have a executor with MaxPlannedVUs=20 and -// MaxUnaplannedVUs=0, followed immediately after by another executor with -// MaxPlannedVUs=10 and MaxUnaplannedVUs=10. The MaxPlannedVUs number for the -// whole test is 20, and MaxUnaplannedVUs is 10, but since those executors won't +// MaxUnplannedVUs=0, followed immediately after by another executor with +// MaxPlannedVUs=10 and MaxUnplannedVUs=10. The MaxPlannedVUs number for the +// whole test is 20, and MaxUnplannedVUs is 10, but since those executors won't // run concurrently, MaxVUs for the whole test is not 30, rather it's 20, since // 20 VUs will be sufficient to run the whole test. // From ec3411d01790f422ab151dbfd47ee679bad9c040 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 7 Oct 2019 18:34:32 +0200 Subject: [PATCH 031/350] fix: typos --- lib/execution_segment.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index bd0d124671e..ba6e3bb36c3 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -106,7 +106,7 @@ func stringToRat(s string) (*big.Rat, error) { // // The parsing logic is that values with a colon, i.e. ':', are full segments: // `1/2:3/4`, `0.5:0.75`, `50%:75%`, and even `2/4:75%` should be (1/2, 3/4] -// And values without a hyphen are the end of a first segment: +// And values without a colon are the end of a first segment: // `20%`, `0.2`, and `1/5` should be converted to (0, 1/5] // empty values should probably be treated as "1", i.e. the whole execution func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { @@ -159,7 +159,7 @@ func (es *ExecutionSegment) FloatLength() float64 { return res } -// Split evenly dividies the execution segment into the specified number of +// Split evenly divides the execution segment into the specified number of // equal consecutive execution sub-segments. func (es *ExecutionSegment) Split(numParts int64) ([]*ExecutionSegment, error) { if numParts < 1 { From 9012690d65794706b77ea5f64e43d1b48d4277c5 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 18:28:38 +0300 Subject: [PATCH 032/350] Fix some merge issues --- stats/cloud/collector.go | 16 ++++++++-------- stats/cloud/collector_test.go | 4 ++-- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index b1dcd6e423f..0922acecd31 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -135,14 +135,14 @@ func New( } return &Collector{ - config: conf, - thresholds: thresholds, - client: NewClient(conf.Token.String, conf.Host.String, version), - anonymous: !conf.Token.Valid, - executionPlan: executionPlan, - duration: int64(duration / time.Second), - opts: opts, - aggrBuckets: map[int64]aggregationBucket{}, + config: conf, + thresholds: thresholds, + client: NewClient(conf.Token.String, conf.Host.String, version), + anonymous: !conf.Token.Valid, + executionPlan: executionPlan, + duration: int64(duration / time.Second), + opts: opts, + aggrBuckets: map[int64]aggregationBucket{}, stopSendingMetricsCh: make(chan struct{}), }, nil } diff --git a/stats/cloud/collector_test.go b/stats/cloud/collector_test.go index 95bc67b7227..453391f1a68 100644 --- a/stats/cloud/collector_test.go +++ b/stats/cloud/collector_test.go @@ -387,7 +387,7 @@ func TestCloudCollectorStopSendingMetric(t *testing.T) { Host: null.StringFrom(tb.ServerHTTP.URL), NoCompress: null.BoolFrom(true), }) - collector, err := New(config, script, options, "1.0") + collector, err := New(config, script, options, []lib.ExecutionStep{}, "1.0") require.NoError(t, err) now := time.Now() tags := stats.IntoSampleTags(&map[string]string{"test": "mest", "a": "b"}) @@ -503,7 +503,7 @@ func TestCloudCollectorAggregationPeriodZeroNoBlock(t *testing.T) { Host: null.StringFrom(tb.ServerHTTP.URL), NoCompress: null.BoolFrom(true), }) - collector, err := New(config, script, options, "1.0") + collector, err := New(config, script, options, []lib.ExecutionStep{}, "1.0") require.NoError(t, err) assert.True(t, collector.config.Host.Valid) From 25ebe619787b828ad1965ddf59bf1f47bd3ca4ca Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 18:28:51 +0300 Subject: [PATCH 033/350] Fix typos and fix or silence new linter issues --- api/v1/status_routes.go | 1 - cmd/collectors.go | 1 - cmd/config_consolidation_test.go | 10 +++---- core/local/local.go | 1 - lib/executor/constant_arrival_rate.go | 6 ++-- lib/executor/externally-controlled.go | 3 +- lib/executor/helpers.go | 4 +-- lib/executor/per_vu_iterations.go | 42 +++++++++++++-------------- lib/executor/shared_iterations.go | 42 +++++++++++++-------------- lib/executor/variable_arrival_rate.go | 6 ++-- lib/executor/variable_looping_vus.go | 9 +++--- lib/executors.go | 1 - lib/helpers.go | 1 - 13 files changed, 61 insertions(+), 66 deletions(-) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index 99b14014bb9..a4706af5845 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -47,7 +47,6 @@ func HandleGetStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Param func getFirstExternallyControlledExecutor( execScheduler lib.ExecutionScheduler, ) (*executor.ExternallyControlled, error) { - executors := execScheduler.GetExecutors() for _, s := range executors { if mex, ok := s.(*executor.ExternallyControlled); ok { diff --git a/cmd/collectors.go b/cmd/collectors.go index 36aaadb81d9..e68151aa5cb 100644 --- a/cmd/collectors.go +++ b/cmd/collectors.go @@ -67,7 +67,6 @@ func parseCollector(s string) (t, arg string) { func newCollector( collectorName, arg string, src *loader.SourceData, conf Config, executionPlan []lib.ExecutionStep, ) (lib.Collector, error) { - getCollector := func() (lib.Collector, error) { switch collectorName { case collectorJSON: diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index e0dd220ef84..5f3dbe5a968 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -76,8 +76,8 @@ func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config exec := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, exec) - require.IsType(t, executor.PerVUIteationsConfig{}, exec) - perVuIters, ok := exec.(executor.PerVUIteationsConfig) + require.IsType(t, executor.PerVUIterationsConfig{}, exec) + perVuIters, ok := exec.(executor.PerVUIterationsConfig) require.True(t, ok) assert.Equal(t, null.NewInt(1, false), perVuIters.Iterations) assert.Equal(t, null.NewInt(1, false), perVuIters.VUs) @@ -87,8 +87,8 @@ func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { exec := c.Execution[lib.DefaultExecutorName] require.NotEmpty(t, exec) - require.IsType(t, executor.SharedIteationsConfig{}, exec) - sharedIterConfig, ok := exec.(executor.SharedIteationsConfig) + require.IsType(t, executor.SharedIterationsConfig{}, exec) + sharedIterConfig, ok := exec.(executor.SharedIterationsConfig) require.True(t, ok) assert.Equal(t, vus, sharedIterConfig.VUs) assert.Equal(t, iters, sharedIterConfig.Iterations) @@ -254,7 +254,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) - sharedIterConfig := c.Execution[lib.DefaultExecutorName].(executor.SharedIteationsConfig) + sharedIterConfig := c.Execution[lib.DefaultExecutorName].(executor.SharedIterationsConfig) assert.Equal(t, time.Duration(sharedIterConfig.MaxDuration.Duration), 10*time.Second) }}, // This should get a validation error since VUs are more than the shared iterations diff --git a/core/local/local.go b/core/local/local.go index 91a6c0059e5..2853b404de0 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -135,7 +135,6 @@ func (e *ExecutionScheduler) GetExecutionPlan() []lib.ExecutionStep { func (e *ExecutionScheduler) initVU( _ context.Context, logger *logrus.Entry, engineOut chan<- stats.SampleContainer, ) (lib.VU, error) { - vu, err := e.runner.NewVU(engineOut) if err != nil { return nil, fmt.Errorf("error while initializing a VU: '%s'", err) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 946bb6b854e..570503a031f 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -154,8 +154,8 @@ func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.Execution // NewExecutor creates a new ConstantArrivalRate executor func (carc ConstantArrivalRateConfig) NewExecutor( - es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - + es *lib.ExecutionState, logger *logrus.Entry, +) (lib.Executor, error) { return ConstantArrivalRate{ BaseExecutor: NewBaseExecutor(carc, es, logger), config: carc, @@ -175,7 +175,7 @@ var _ lib.Executor = &ConstantArrivalRate{} // Run executes a constant number of iterations per second. // // TODO: Reuse the variable arrival rate method? -func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen segment := car.executionState.Options.ExecutionSegment gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 0363fcb2a36..4de7b5a7153 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -337,7 +337,6 @@ type manualVUHandle struct { func newManualVUHandle( parentCtx context.Context, state *lib.ExecutionState, localActiveVUsCount *int64, vu lib.VU, logger *logrus.Entry, ) *manualVUHandle { - wg := sync.WaitGroup{} getVU := func() (lib.VU, error) { wg.Add(1) @@ -364,6 +363,7 @@ func newManualVUHandle( // until the test is manually stopped. // //TODO: split this up? somehow... :/ +//nolint:funlen func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { mex.configLock.RLock() // Safely get the current config - it's important that the close of the @@ -456,7 +456,6 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats if oldActiveVUs < newActiveVUs { for i := oldActiveVUs; i < newActiveVUs; i++ { - if !currentlyPaused { vuHandles[i].start() } diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index b59dd96af4e..c72284dc895 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -78,9 +78,9 @@ func validateStages(stages []Stage) []error { // getIterationRunner is a helper function that returns an iteration executor // closure. It takes care of updating metrics, execution state statistics, and // warning messages. -func getIterationRunner(executionState *lib.ExecutionState, logger *logrus.Entry, out chan<- stats.SampleContainer, +func getIterationRunner( + executionState *lib.ExecutionState, logger *logrus.Entry, out chan<- stats.SampleContainer, ) func(context.Context, lib.VU) { - return func(ctx context.Context, vu lib.VU) { err := vu.RunOnce(ctx) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 63999a3d25e..031984dc2f4 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -45,17 +45,17 @@ func init() { }) } -// PerVUIteationsConfig stores the number of VUs iterations, as well as maxDuration settings -type PerVUIteationsConfig struct { +// PerVUIterationsConfig stores the number of VUs iterations, as well as maxDuration settings +type PerVUIterationsConfig struct { BaseConfig VUs null.Int `json:"vus"` Iterations null.Int `json:"iterations"` MaxDuration types.NullDuration `json:"maxDuration"` } -// NewPerVUIterationsConfig returns a PerVUIteationsConfig with default values -func NewPerVUIterationsConfig(name string) PerVUIteationsConfig { - return PerVUIteationsConfig{ +// NewPerVUIterationsConfig returns a PerVUIterationsConfig with default values +func NewPerVUIterationsConfig(name string) PerVUIterationsConfig { + return PerVUIterationsConfig{ BaseConfig: NewBaseConfig(name, perVUIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), @@ -64,10 +64,10 @@ func NewPerVUIterationsConfig(name string) PerVUIteationsConfig { } // Make sure we implement the lib.ExecutorConfig interface -var _ lib.ExecutorConfig = &PerVUIteationsConfig{} +var _ lib.ExecutorConfig = &PerVUIterationsConfig{} // GetVUs returns the scaled VUs for the executor. -func (pvic PerVUIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { +func (pvic PerVUIterationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { return es.Scale(pvic.VUs.Int64) } @@ -75,19 +75,19 @@ func (pvic PerVUIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { // important to note that scaling per-VU iteration executor affects only the // number of VUs. If we also scaled the iterations, scaling would have quadratic // effects instead of just linear. -func (pvic PerVUIteationsConfig) GetIterations() int64 { +func (pvic PerVUIterationsConfig) GetIterations() int64 { return pvic.Iterations.Int64 } // GetDescription returns a human-readable description of the executor options -func (pvic PerVUIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { +func (pvic PerVUIterationsConfig) GetDescription(es *lib.ExecutionSegment) string { return fmt.Sprintf("%d iterations for each of %d VUs%s", pvic.GetIterations(), pvic.GetVUs(es), pvic.getBaseInfo(fmt.Sprintf("maxDuration: %s", pvic.MaxDuration.Duration))) } // Validate makes sure all options are configured and valid -func (pvic PerVUIteationsConfig) Validate() []error { +func (pvic PerVUIterationsConfig) Validate() []error { errors := pvic.BaseConfig.Validate() if pvic.VUs.Int64 <= 0 { errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) @@ -109,7 +109,7 @@ func (pvic PerVUIteationsConfig) Validate() []error { // GetExecutionRequirements just reserves the number of specified VUs for the // whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. -func (pvic PerVUIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (pvic PerVUIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, @@ -122,27 +122,27 @@ func (pvic PerVUIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegme } } -// NewExecutor creates a new PerVUIteations executor -func (pvic PerVUIteationsConfig) NewExecutor( - es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - - return PerVUIteations{ +// NewExecutor creates a new PerVUIterations executor +func (pvic PerVUIterationsConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry, +) (lib.Executor, error) { + return PerVUIterations{ BaseExecutor: NewBaseExecutor(pvic, es, logger), config: pvic, }, nil } -// PerVUIteations executes a specific number of iterations with each VU. -type PerVUIteations struct { +// PerVUIterations executes a specific number of iterations with each VU. +type PerVUIterations struct { *BaseExecutor - config PerVUIteationsConfig + config PerVUIterationsConfig } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &PerVUIteations{} +var _ lib.Executor = &PerVUIterations{} // Run executes a specific number of iterations with each confugured VU. -func (pvi PerVUIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := pvi.executionState.Options.ExecutionSegment numVUs := pvi.config.GetVUs(segment) iterations := pvi.config.GetIterations() diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 6500a2edcda..25604970746 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -48,17 +48,17 @@ func init() { ) } -// SharedIteationsConfig stores the number of VUs iterations, as well as maxDuration settings -type SharedIteationsConfig struct { +// SharedIterationsConfig stores the number of VUs iterations, as well as maxDuration settings +type SharedIterationsConfig struct { BaseConfig VUs null.Int `json:"vus"` Iterations null.Int `json:"iterations"` MaxDuration types.NullDuration `json:"maxDuration"` } -// NewSharedIterationsConfig returns a SharedIteationsConfig with default values -func NewSharedIterationsConfig(name string) SharedIteationsConfig { - return SharedIteationsConfig{ +// NewSharedIterationsConfig returns a SharedIterationsConfig with default values +func NewSharedIterationsConfig(name string) SharedIterationsConfig { + return SharedIterationsConfig{ BaseConfig: NewBaseConfig(name, sharedIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), @@ -67,27 +67,27 @@ func NewSharedIterationsConfig(name string) SharedIteationsConfig { } // Make sure we implement the lib.ExecutorConfig interface -var _ lib.ExecutorConfig = &SharedIteationsConfig{} +var _ lib.ExecutorConfig = &SharedIterationsConfig{} // GetVUs returns the scaled VUs for the executor. -func (sic SharedIteationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { +func (sic SharedIterationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { return es.Scale(sic.VUs.Int64) } // GetIterations returns the scaled iteration count for the executor. -func (sic SharedIteationsConfig) GetIterations(es *lib.ExecutionSegment) int64 { +func (sic SharedIterationsConfig) GetIterations(es *lib.ExecutionSegment) int64 { return es.Scale(sic.Iterations.Int64) } // GetDescription returns a human-readable description of the executor options -func (sic SharedIteationsConfig) GetDescription(es *lib.ExecutionSegment) string { +func (sic SharedIterationsConfig) GetDescription(es *lib.ExecutionSegment) string { return fmt.Sprintf("%d iterations shared among %d VUs%s", sic.GetIterations(es), sic.GetVUs(es), sic.getBaseInfo(fmt.Sprintf("maxDuration: %s", sic.MaxDuration.Duration))) } // Validate makes sure all options are configured and valid -func (sic SharedIteationsConfig) Validate() []error { +func (sic SharedIterationsConfig) Validate() []error { errors := sic.BaseConfig.Validate() if sic.VUs.Int64 <= 0 { errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) @@ -112,7 +112,7 @@ func (sic SharedIteationsConfig) Validate() []error { // GetExecutionRequirements just reserves the number of specified VUs for the // whole duration of the executor, including the maximum waiting time for // iterations to gracefully stop. -func (sic SharedIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (sic SharedIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, @@ -125,29 +125,29 @@ func (sic SharedIteationsConfig) GetExecutionRequirements(es *lib.ExecutionSegme } } -// NewExecutor creates a new SharedIteations executor -func (sic SharedIteationsConfig) NewExecutor( - es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - - return SharedIteations{ +// NewExecutor creates a new SharedIterations executor +func (sic SharedIterationsConfig) NewExecutor( + es *lib.ExecutionState, logger *logrus.Entry, +) (lib.Executor, error) { + return SharedIterations{ BaseExecutor: NewBaseExecutor(sic, es, logger), config: sic, }, nil } -// SharedIteations executes a specific total number of iterations, which are +// SharedIterations executes a specific total number of iterations, which are // all shared by the configured VUs. -type SharedIteations struct { +type SharedIterations struct { *BaseExecutor - config SharedIteationsConfig + config SharedIterationsConfig } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &PerVUIteations{} +var _ lib.Executor = &PerVUIterations{} // Run executes a specific total number of iterations, which are all shared by // the configured VUs. -func (si SharedIteations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := si.executionState.Options.ExecutionSegment numVUs := si.config.GetVUs(segment) iterations := si.config.GetIterations(segment) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 2bad1795ea0..2278036e034 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -238,8 +238,8 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi // NewExecutor creates a new VariableArrivalRate executor func (varc VariableArrivalRateConfig) NewExecutor( - es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - + es *lib.ExecutionState, logger *logrus.Entry, +) (lib.Executor, error) { return VariableArrivalRate{ BaseExecutor: NewBaseExecutor(varc, es, logger), config: varc, @@ -285,7 +285,7 @@ func (varr VariableArrivalRate) streamRateChanges(ctx context.Context, startTime } // Run executes a variable number of iterations per second. -func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen segment := varr.executionState.Options.ExecutionSegment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 2bb6883a016..8d1add2c061 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -185,6 +185,7 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // 00000000001111111111222 (t/10) // // More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 +//nolint:funlen func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegment, zeroEnd bool) []lib.ExecutionStep { // For accurate results, calculations are done with the unscaled values, and // the values are scaled only before we add them to the steps result slice @@ -330,9 +331,9 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // executorEndOffset, is not handled here. Instead GetExecutionRequirements() // takes care of that. But to make its job easier, this method won't add any // steps with an offset that's greater or equal to executorEndOffset. -func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( - rawSteps []lib.ExecutionStep, executorEndOffset time.Duration) []lib.ExecutionStep { - +func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:funlen + rawSteps []lib.ExecutionStep, executorEndOffset time.Duration, +) []lib.ExecutionStep { rawStepsLen := len(rawSteps) gracefulRampDownPeriod := vlvc.GetGracefulRampDown() newSteps := []lib.ExecutionStep{} @@ -475,7 +476,7 @@ var _ lib.Executor = &VariableLoopingVUs{} // TODO: split up? since this does a ton of things, unfortunately I can't think // of a less complex way to implement it (besides the old "increment by 100ms // and see what happens)... :/ so maybe see how it can be spit? -func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen segment := vlv.executionState.Options.ExecutionSegment rawExecutionSteps := vlv.config.getRawExecutionSteps(segment, true) regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) diff --git a/lib/executors.go b/lib/executors.go index edf7fa3c5c0..19c76fd2232 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -289,7 +289,6 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec if stepsLen == 0 || consolidatedSteps[stepsLen-1].PlannedVUs != newPlannedVUs || consolidatedSteps[stepsLen-1].MaxUnplannedVUs != newMaxUnplannedVUs { - consolidatedSteps = append(consolidatedSteps, ExecutionStep{ TimeOffset: currentTimeOffset, PlannedVUs: newPlannedVUs, diff --git a/lib/helpers.go b/lib/helpers.go index c3cbbd5f62c..927b972a849 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -126,7 +126,6 @@ func ConcatErrors(errors []error, separator string) string { func StreamExecutionSteps( ctx context.Context, startTime time.Time, steps []ExecutionStep, closeChanWhenDone bool, ) <-chan ExecutionStep { - ch := make(chan ExecutionStep) go func() { for _, step := range steps { From a3da2afd1ff4d8592f12040750e69498f5fc015a Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 19:30:30 +0300 Subject: [PATCH 034/350] Refactor the local ExecutionScheduler's Run() method --- core/local/local.go | 88 ++++++++++++++++++++++++--------------------- 1 file changed, 47 insertions(+), 41 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 2853b404de0..a28d086c49d 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -249,6 +249,51 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa return nil } +// runExecutor gets called by the public Run() method once per configured +// executor, each time in a new goroutine. It is responsible for waiting out the +// configured startTime for the specific executor and then running its Run() +// method. +func (e *ExecutionScheduler) runExecutor( + runCtx context.Context, runResults chan<- error, engineOut chan<- stats.SampleContainer, executor lib.Executor, +) { + executorConfig := executor.GetConfig() + executorStartTime := executorConfig.GetStartTime() + executorLogger := e.logger.WithFields(logrus.Fields{ + "executor": executorConfig.GetName(), + "type": executorConfig.GetType(), + "startTime": executorStartTime, + }) + executorProgress := executor.GetProgress() + + // Check if we have to wait before starting the actual executor execution + if executorStartTime > 0 { + startTime := time.Now() + executorProgress.Modify(pb.WithProgress(func() (float64, string) { + remWait := (executorStartTime - time.Since(startTime)) + return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) + })) + + executorLogger.Debugf("Waiting for executor start time...") + select { + case <-runCtx.Done(): + runResults <- nil // no error since executor hasn't started yet + return + case <-time.After(executorStartTime): + // continue + } + } + + executorProgress.Modify(pb.WithConstProgress(0, "started")) + executorLogger.Debugf("Starting executor") + err := executor.Run(runCtx, engineOut) // executor should handle context cancel itself + if err == nil { + executorLogger.Debugf("Executor finished successfully") + } else { + executorLogger.WithField("error", err).Errorf("Executor error") + } + runResults <- err +} + // Run the ExecutionScheduler, funneling all generated metric samples through the supplied // out channel. func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error { @@ -289,50 +334,10 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam } e.initProgress.Modify(pb.WithHijack(e.getRunStats)) - runCtxDone := runCtx.Done() - runExecutor := func(executor lib.Executor) { - executorConfig := executor.GetConfig() - executorStartTime := executorConfig.GetStartTime() - executorLogger := logger.WithFields(logrus.Fields{ - "executor": executorConfig.GetName(), - "type": executorConfig.GetType(), - "startTime": executorStartTime, - }) - executorProgress := executor.GetProgress() - - // Check if we have to wait before starting the actual executor execution - if executorStartTime > 0 { - startTime := time.Now() - executorProgress.Modify(pb.WithProgress(func() (float64, string) { - remWait := (executorStartTime - time.Since(startTime)) - return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) - })) - - executorLogger.Debugf("Waiting for executor start time...") - select { - case <-runCtxDone: - runResults <- nil // no error since executor hasn't started yet - return - case <-time.After(executorStartTime): - // continue - } - } - - executorProgress.Modify(pb.WithConstProgress(0, "started")) - executorLogger.Debugf("Starting executor") - err := executor.Run(runCtx, engineOut) // executor should handle context cancel itself - if err == nil { - executorLogger.Debugf("Executor finished successfully") - } else { - executorLogger.WithField("error", err).Errorf("Executor error") - } - runResults <- err - } - // Start all executors at their particular startTime in a separate goroutine... logger.Debug("Start all executors...") for _, exec := range e.executors { - go runExecutor(exec) + go e.runExecutor(runCtx, runResults, engineOut, exec) } // Wait for all executors to finish @@ -340,6 +345,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam for range e.executors { err := <-runResults if err != nil && firstErr == nil { + logger.WithError(err).Debug("Executor returned with an error, cancelling test run...") firstErr = err cancel() } From 9cb67ee789682f99f3d93d2052de0a6ed409470e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 19:31:00 +0300 Subject: [PATCH 035/350] Improve the end check and defers in the iteration-based executors The old way wasn't buggy, since nothing relies on the exact ordering, but this is better --- lib/executor/per_vu_iterations.go | 2 +- lib/executor/shared_iterations.go | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 031984dc2f4..085fc8c465d 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -177,8 +177,8 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta runIteration := getIterationRunner(pvi.executionState, pvi.logger, out) handleVU := func(vu lib.VU) { - defer pvi.executionState.ReturnVU(vu, true) defer activeVUs.Done() + defer pvi.executionState.ReturnVU(vu, true) for i := int64(0); i < iterations; i++ { select { diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 25604970746..6edf8d07d1d 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -183,10 +183,17 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) handleVU := func(vu lib.VU) { - defer si.executionState.ReturnVU(vu, true) defer activeVUs.Done() + defer si.executionState.ReturnVU(vu, true) for { + select { + case <-regDurationDone: + return // don't make more iterations + default: + // continue looping + } + attemptedIterNumber := atomic.AddUint64(attemptedIters, 1) if attemptedIterNumber > totalIters { return @@ -194,12 +201,6 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta runIteration(maxDurationCtx, vu) atomic.AddUint64(doneIters, 1) - select { - case <-regDurationDone: - return // don't make more iterations - default: - // continue looping - } } } From ab2fab6ded46ddfbffd9eebce6580d3b75307f2c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 19:49:41 +0300 Subject: [PATCH 036/350] Refactor the concurrent VU initialization This would reduce the number of spawned goroutines, slightly improving performance --- core/local/local.go | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index a28d086c49d..be0f34eb8ba 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -145,7 +145,6 @@ func (e *ExecutionScheduler) initVU( vuID := e.state.GetUniqueVUIdentifier() if err := vu.Reconfigure(int64(vuID)); err != nil { return nil, fmt.Errorf("error while reconfiguring VU #%d: '%s'", vuID, err) - } logger.Debugf("Initialized VU #%d", vuID) return vu, nil @@ -182,27 +181,31 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa "executorsCount": len(e.executors), }).Debugf("Start of initialization") + // Initialize VUs concurrently doneInits := make(chan error, vusToInitialize) // poor man's early-return waitgroup //TODO: make this an option? initConcurrency := runtime.NumCPU() - limiter := make(chan struct{}, initConcurrency) + limiter := make(chan struct{}) subctx, cancel := context.WithCancel(ctx) defer cancel() - initPlannedVU := func() { - newVU, err := e.initVU(ctx, logger, engineOut) - if err == nil { - e.state.AddInitializedVU(newVU) - <-limiter - } - doneInits <- err + for i := 0; i < initConcurrency; i++ { + go func() { + for range limiter { + newVU, err := e.initVU(ctx, logger, engineOut) + if err == nil { + e.state.AddInitializedVU(newVU) + } + doneInits <- err + } + }() } go func() { + defer close(limiter) for vuNum := uint64(0); vuNum < vusToInitialize; vuNum++ { select { case limiter <- struct{}{}: - go initPlannedVU() case <-subctx.Done(): return } From 157c05547ed8b6d89bf06f6b11d2cbc0ba931f0b Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 20:16:54 +0300 Subject: [PATCH 037/350] Log and comment when we abort due to a VU init error --- core/local/local.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/local/local.go b/core/local/local.go index be0f34eb8ba..d7cd3bc8c41 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -226,6 +226,9 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa select { case err := <-doneInits: if err != nil { + logger.WithError(err).Debug("VU initialization returned with an error, aborting...") + // the context's cancel() is called in a defer above and will + // abort any in-flight VU initializations return err } atomic.AddUint64(initializedVUs, 1) From 8aaed9069ae868e7de202f02eb97f66971a6cae5 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 20:19:49 +0300 Subject: [PATCH 038/350] Silence a linter error --- cmd/collectors.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmd/collectors.go b/cmd/collectors.go index e68151aa5cb..b4c74f7bf5e 100644 --- a/cmd/collectors.go +++ b/cmd/collectors.go @@ -64,6 +64,8 @@ func parseCollector(s string) (t, arg string) { } } +//TODO: totally refactor this... +//nolint:funlen func newCollector( collectorName, arg string, src *loader.SourceData, conf Config, executionPlan []lib.ExecutionStep, ) (lib.Collector, error) { From f6a82445dd897243dbb526e344ef5c5199b390cc Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 20:55:32 +0300 Subject: [PATCH 039/350] Fix new param check in the externally controlled executor --- lib/executor/externally-controlled.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 4de7b5a7153..1527a7b2ae9 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -267,10 +267,10 @@ func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf inter return fmt.Errorf("invalid configuration supplied: %s", lib.ConcatErrors(errs, ", ")) } - if newConfigParams.Duration != mex.startConfig.Duration { + if newConfigParams.Duration.Valid && newConfigParams.Duration != mex.startConfig.Duration { return fmt.Errorf("the externally controlled executor duration cannot be changed") } - if newConfigParams.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { + if newConfigParams.MaxVUs.Valid && newConfigParams.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { // This limitation is because the externally controlled executor is // still a executor that participates in the overall k6 scheduling. // Thus, any VUs that were explicitly specified by the user in the From 119fc185e6fc93521b60819d032613018a9d6dd6 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 21:03:20 +0300 Subject: [PATCH 040/350] Fix minor issues from PR comments --- api/v1/status_routes.go | 4 ++-- lib/executor/externally-controlled.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index a4706af5845..e1a9097d39e 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -21,7 +21,7 @@ package v1 import ( - "fmt" + "errors" "io/ioutil" "net/http" @@ -53,7 +53,7 @@ func getFirstExternallyControlledExecutor( return mex, nil } } - return nil, fmt.Errorf("a externally-controlled executor needs to be configured for live configuration updates") + return nil, errors.New("a externally-controlled executor needs to be configured for live configuration updates") } func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 1527a7b2ae9..614caf9a233 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -281,7 +281,7 @@ func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf inter ) } - mex.configLock.Lock() + mex.configLock.Lock() // guard against a simultaneous start of the test (which will close hasStarted) select { case <-mex.hasStarted: mex.configLock.Unlock() From 1a487dac2f03d72c2169dede7dcf8be3f399b8cf Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 22:06:41 +0300 Subject: [PATCH 041/350] Fix minor issues from PR comments --- cmd/ui.go | 9 +++++++-- lib/execution_segment.go | 3 +++ lib/executors.go | 6 +++++- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 0ed128135e2..5f98ca248ce 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -46,7 +46,8 @@ type consoleWriter struct { func (w *consoleWriter) Write(p []byte) (n int, err error) { origLen := len(p) if w.IsTTY { - //TODO: check how cross-platform this is... + // Add a TTY code to erase till the end of line with each new line + // TODO: check how cross-platform this is... p = bytes.Replace(p, []byte{'\n'}, []byte{'\x1b', '[', '0', 'K', '\n'}, -1) } @@ -66,7 +67,11 @@ func (w *consoleWriter) Write(p []byte) (n int, err error) { func printBar(bar *pb.ProgressBar, rightText string) { end := "\n" if stdout.IsTTY { - //TODO: check for cross platform support + // If we're in a TTY, instead of printing the bar and going to the next + // line, erase everything till the end of the line and return to the + // start, so that the next print will overwrite the same line. + // + // TODO: check for cross platform support end = "\x1b[0K\r" } fprintf(stdout, "%s %s%s", bar.String(), rightText, end) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index ba6e3bb36c3..c6e4cc9cf2f 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -112,6 +112,9 @@ func stringToRat(s string) (*big.Rat, error) { func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { from := zeroRat toStr := string(text) + if toStr == "" { + toStr = "1" // an empty string means a full 0:1 execution segment + } if strings.ContainsRune(toStr, ':') { fromToStr := strings.SplitN(toStr, ":", 2) toStr = fromToStr[1] diff --git a/lib/executors.go b/lib/executors.go index 19c76fd2232..046bd70e8da 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -86,7 +86,11 @@ type ExecutorConfig interface { IsDistributable() bool GetEnv() map[string]string - GetExec() null.String //TODO: use interface{} so plain http requests can be specified? + // Allows us to get the non-default function the executor should run, if it + // has been specified. + // + // TODO: use interface{} so plain http requests can be specified? + GetExec() null.String // Calculates the VU requirements in different stages of the executor's // execution, including any extensions caused by waiting for iterations to From 2ebe3dfdd6c3ef97decbd9ac80c6747dedb3aac1 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 23:13:55 +0300 Subject: [PATCH 042/350] Refactor the Run() method of the externally controlled executor This greatly simplifies the executor code by taking out some of the inline lambdas and functionality to a separate runState struct. --- lib/executor/externally-controlled.go | 251 +++++++++++++++----------- 1 file changed, 141 insertions(+), 110 deletions(-) diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 614caf9a233..38f6500082c 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -358,12 +358,119 @@ func newManualVUHandle( } } +// externallyControlledRunState is created and initialized by the Run() method +// of the externally controlled executor. It is used to track and modify various +// details of the execution, including handling of live config changes. +type externallyControlledRunState struct { + ctx context.Context + executor *ExternallyControlled + startMaxVUs int64 // the scaled number of initially configured MaxVUs + duration time.Duration // the total duration of the executor, could be 0 for infinite + activeVUsCount *int64 // the current number of active VUs, used only for the progress display + maxVUs *int64 // the current number of initialized VUs + vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs + currentlyPaused bool // whether the executor is currently paused + + runIteration func(context.Context, lib.VU) // a helper closure function that runs a single iteration +} + +// retrieveAndInitStartMaxVUs gets and initializes the (scaled) number of MaxVUs +// from the global VU buffer. These are the VUs that the user originally +// specified in the JS config, and that the ExecutionScheduler pre-initialized +// for us. +func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { + for i := int64(0); i < rs.startMaxVUs; i++ { // get the initial planned VUs from the common buffer + vu, vuGetErr := rs.executor.executionState.GetPlannedVU(rs.executor.logger, false) + if vuGetErr != nil { + return vuGetErr + } + vuHandle := newManualVUHandle( + rs.ctx, rs.executor.executionState, rs.activeVUsCount, vu, rs.executor.logger.WithField("vuNum", i), + ) + go vuHandle.runLoopsIfPossible(rs.runIteration) + rs.vuHandles[i] = vuHandle + } + return nil +} + +func (rs *externallyControlledRunState) progresFn() (float64, string) { + spent := rs.executor.executionState.GetCurrentTestRunDuration() + progress := 0.0 + if rs.duration > 0 { + progress = math.Min(1, float64(spent)/float64(rs.duration)) + } + //TODO: simulate spinner for the other case or cycle 0-100? + currentActiveVUs := atomic.LoadInt64(rs.activeVUsCount) + currentMaxVUs := atomic.LoadInt64(rs.maxVUs) + vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) + return progress, fmt.Sprintf( + "currently "+vusFmt+" out of "+vusFmt+" active looping VUs, %s/%s", currentActiveVUs, currentMaxVUs, + pb.GetFixedLengthDuration(spent, rs.duration), rs.duration, + ) +} + +func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg ExternallyControlledConfigParams) error { + executionState := rs.executor.executionState + segment := executionState.Options.ExecutionSegment + oldActiveVUs := segment.Scale(oldCfg.VUs.Int64) + oldMaxVUs := segment.Scale(oldCfg.MaxVUs.Int64) + newActiveVUs := segment.Scale(newCfg.VUs.Int64) + newMaxVUs := segment.Scale(newCfg.MaxVUs.Int64) + + rs.executor.logger.WithFields(logrus.Fields{ + "oldActiveVUs": oldActiveVUs, "oldMaxVUs": oldMaxVUs, + "newActiveVUs": newActiveVUs, "newMaxVUs": newMaxVUs, + }).Debug("Updating execution configuration...") + + for i := oldMaxVUs; i < newMaxVUs; i++ { + vu, vuInitErr := executionState.InitializeNewVU(rs.ctx, rs.executor.logger) + if vuInitErr != nil { + return vuInitErr + } + vuHandle := newManualVUHandle( + rs.ctx, executionState, rs.activeVUsCount, vu, rs.executor.logger.WithField("vuNum", i), + ) + go vuHandle.runLoopsIfPossible(rs.runIteration) + rs.vuHandles = append(rs.vuHandles, vuHandle) + } + + if oldActiveVUs < newActiveVUs { + for i := oldActiveVUs; i < newActiveVUs; i++ { + if !rs.currentlyPaused { + rs.vuHandles[i].start() + } + } + } else { + for i := newActiveVUs; i < oldActiveVUs; i++ { + rs.vuHandles[i].hardStop() + } + for i := newActiveVUs; i < oldActiveVUs; i++ { + rs.vuHandles[i].wg.Wait() + } + } + + if oldMaxVUs > newMaxVUs { + for i := newMaxVUs; i < oldMaxVUs; i++ { + rs.vuHandles[i].cancelVU() + if i < rs.startMaxVUs { + // return the initial planned VUs to the common buffer + executionState.ReturnVU(rs.vuHandles[i].vu, false) + } else { + executionState.ModInitializedVUsCount(-1) + } + rs.vuHandles[i] = nil + } + rs.vuHandles = rs.vuHandles[:newMaxVUs] + } + + atomic.StoreInt64(rs.maxVUs, newMaxVUs) + return nil +} + // Run constantly loops through as many iterations as possible on a variable // dynamically controlled number of VUs either for the specified duration, or // until the test is manually stopped. -// -//TODO: split this up? somehow... :/ -//nolint:funlen +// nolint:funlen func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { mex.configLock.RLock() // Safely get the current config - it's important that the close of the @@ -373,11 +480,10 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats close(mex.hasStarted) mex.configLock.RUnlock() - segment := mex.executionState.Options.ExecutionSegment - duration := time.Duration(currentControlConfig.Duration.Duration) - ctx, cancel := context.WithCancel(parentCtx) defer cancel() + + duration := time.Duration(currentControlConfig.Duration.Duration) if duration > 0 { // Only keep track of duration if it's not infinite go mex.stopWhenDurationIsReached(ctx, duration, cancel) } @@ -386,111 +492,34 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats logrus.Fields{"type": externallyControlledType, "duration": duration}, ).Debug("Starting executor run...") - // Retrieve and initialize the (scaled) number of MaxVUs from the global VU - // buffer that the user originally specified in the JS config. - startMaxVUs := segment.Scale(mex.startConfig.MaxVUs.Int64) - vuHandles := make([]*manualVUHandle, startMaxVUs) - activeVUsCount := new(int64) - runIteration := getIterationRunner(mex.executionState, mex.logger, out) - for i := int64(0); i < startMaxVUs; i++ { // get the initial planned VUs from the common buffer - vu, vuGetErr := mex.executionState.GetPlannedVU(mex.logger, false) - if vuGetErr != nil { - return vuGetErr - } - vuHandle := newManualVUHandle( - parentCtx, mex.executionState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), - ) - go vuHandle.runLoopsIfPossible(runIteration) - vuHandles[i] = vuHandle + startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.startConfig.MaxVUs.Int64) + runState := &externallyControlledRunState{ + ctx: ctx, + executor: mex, + startMaxVUs: startMaxVUs, + duration: duration, + vuHandles: make([]*manualVUHandle, startMaxVUs), + currentlyPaused: false, + activeVUsCount: new(int64), + maxVUs: new(int64), + runIteration: getIterationRunner(mex.executionState, mex.logger, out), } - - // Keep track of the progress - maxVUs := new(int64) - *maxVUs = startMaxVUs - progresFn := func() (float64, string) { - spent := mex.executionState.GetCurrentTestRunDuration() - progress := 0.0 - if duration > 0 { - progress = math.Min(1, float64(spent)/float64(duration)) - } - //TODO: simulate spinner for the other case or cycle 0-100? - currentActiveVUs := atomic.LoadInt64(activeVUsCount) - currentMaxVUs := atomic.LoadInt64(maxVUs) - vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) - return progress, fmt.Sprintf( - "currently "+vusFmt+" out of "+vusFmt+" active looping VUs, %s/%s", currentActiveVUs, currentMaxVUs, - pb.GetFixedLengthDuration(spent, duration), duration, - ) + *runState.maxVUs = startMaxVUs + if err = runState.retrieveStartMaxVUs(); err != nil { + return err } - mex.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, ctx, ctx, mex, progresFn) - currentlyPaused := false - waitVUs := func(from, to int64) { - for i := from; i < to; i++ { - vuHandles[i].wg.Wait() - } - } - handleConfigChange := func(oldControlConfig, newControlConfig ExternallyControlledConfigParams) error { - oldActiveVUs := segment.Scale(oldControlConfig.VUs.Int64) - oldMaxVUs := segment.Scale(oldControlConfig.MaxVUs.Int64) - newActiveVUs := segment.Scale(newControlConfig.VUs.Int64) - newMaxVUs := segment.Scale(newControlConfig.MaxVUs.Int64) - - mex.logger.WithFields(logrus.Fields{ - "oldActiveVUs": oldActiveVUs, "oldMaxVUs": oldMaxVUs, - "newActiveVUs": newActiveVUs, "newMaxVUs": newMaxVUs, - }).Debug("Updating execution configuration...") - - for i := oldMaxVUs; i < newMaxVUs; i++ { - vu, vuInitErr := mex.executionState.InitializeNewVU(ctx, mex.logger) - if vuInitErr != nil { - return vuInitErr - } - vuHandle := newManualVUHandle( - ctx, mex.executionState, activeVUsCount, vu, mex.logger.WithField("vuNum", i), - ) - go vuHandle.runLoopsIfPossible(runIteration) - vuHandles = append(vuHandles, vuHandle) - } - - if oldActiveVUs < newActiveVUs { - for i := oldActiveVUs; i < newActiveVUs; i++ { - if !currentlyPaused { - vuHandles[i].start() - } - } - } else { - for i := newActiveVUs; i < oldActiveVUs; i++ { - vuHandles[i].hardStop() - } - waitVUs(newActiveVUs, oldActiveVUs) - } - - if oldMaxVUs > newMaxVUs { - for i := newMaxVUs; i < oldMaxVUs; i++ { - vuHandles[i].cancelVU() - if i < startMaxVUs { - // return the initial planned VUs to the common buffer - mex.executionState.ReturnVU(vuHandles[i].vu, false) - } else { - mex.executionState.ModInitializedVUsCount(-1) - } - vuHandles[i] = nil - } - vuHandles = vuHandles[:newMaxVUs] - } - - atomic.StoreInt64(maxVUs, newMaxVUs) - return nil - } + mex.progress.Modify(pb.WithProgress(runState.progresFn)) // Keep track of the progress + go trackProgress(parentCtx, ctx, ctx, mex, runState.progresFn) - err = handleConfigChange(ExternallyControlledConfigParams{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig) + err = runState.handleConfigChange( // Start by setting MaxVUs to the starting MaxVUs + ExternallyControlledConfigParams{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig, + ) if err != nil { return err } - defer func() { - err = handleConfigChange(currentControlConfig, ExternallyControlledConfigParams{}) + defer func() { // Make sure we release the VUs at the end + err = runState.handleConfigChange(currentControlConfig, ExternallyControlledConfigParams{}) }() for { @@ -498,7 +527,7 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats case <-ctx.Done(): return nil case updateConfigEvent := <-mex.newControlConfigs: - err := handleConfigChange(currentControlConfig, updateConfigEvent.newConfig) + err := runState.handleConfigChange(currentControlConfig, updateConfigEvent.newConfig) if err != nil { updateConfigEvent.err <- err return err @@ -510,22 +539,24 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats updateConfigEvent.err <- nil case pauseEvent := <-mex.pauseEvents: - if pauseEvent.isPaused == currentlyPaused { + if pauseEvent.isPaused == runState.currentlyPaused { pauseEvent.err <- nil continue } activeVUs := currentControlConfig.VUs.Int64 if pauseEvent.isPaused { for i := int64(0); i < activeVUs; i++ { - vuHandles[i].gracefulStop() + runState.vuHandles[i].gracefulStop() + } + for i := int64(0); i < activeVUs; i++ { + runState.vuHandles[i].wg.Wait() } - waitVUs(0, activeVUs) } else { for i := int64(0); i < activeVUs; i++ { - vuHandles[i].start() + runState.vuHandles[i].start() } } - currentlyPaused = pauseEvent.isPaused + runState.currentlyPaused = pauseEvent.isPaused pauseEvent.err <- nil } } From edd03a37360c7efbe05fe98f835050016fb7f06e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 10 Oct 2019 23:21:56 +0300 Subject: [PATCH 043/350] Handle interrupts during manual VU initializations gracefully --- lib/executor/externally-controlled.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally-controlled.go index 38f6500082c..a081642a892 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally-controlled.go @@ -423,6 +423,11 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern }).Debug("Updating execution configuration...") for i := oldMaxVUs; i < newMaxVUs; i++ { + select { // check if the user didn't try to abort k6 while we're scaling up the VUs + case <-rs.ctx.Done(): + return rs.ctx.Err() + default: // do nothing + } vu, vuInitErr := executionState.InitializeNewVU(rs.ctx, rs.executor.logger) if vuInitErr != nil { return vuInitErr @@ -530,6 +535,9 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats err := runState.handleConfigChange(currentControlConfig, updateConfigEvent.newConfig) if err != nil { updateConfigEvent.err <- err + if ctx.Err() == err { + return nil // we've already returned an error to the API client, but k6 should stop normally + } return err } currentControlConfig = updateConfigEvent.newConfig From 3421078d34e9eb2dce507c2687eeb3981d1afc87 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 14 Oct 2019 15:37:54 +0300 Subject: [PATCH 044/350] Fix various minor issues to address PR comments --- cmd/run.go | 2 +- lib/executor/constant_arrival_rate.go | 8 +++++--- lib/executor/constant_looping_vus.go | 8 +++++--- ...{externally-controlled.go => externally_controlled.go} | 4 ++-- lib/executor/per_vu_iterations.go | 8 +++++--- lib/executor/shared_iterations.go | 8 +++++--- lib/executor/variable_arrival_rate.go | 8 +++++--- 7 files changed, 28 insertions(+), 18 deletions(-) rename lib/executor/{externally-controlled.go => externally_controlled.go} (99%) diff --git a/cmd/run.go b/cmd/run.go index 287cf893054..c2e7ee1c4ea 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -97,7 +97,7 @@ a commandline interface for interacting with it.`, initBar := pb.New(pb.WithConstLeft(" init")) // Create the Runner. - fprintf(stdout, "%s runner\r", initBar.String()) //TODO + fprintf(stdout, "%s runner\r", initBar.String()) //TODO use printBar() pwd, err := os.Getwd() if err != nil { return err diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 570503a031f..565f5410029 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -135,9 +135,11 @@ func (carc ConstantArrivalRateConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the executor, including the maximum waiting time for -// iterations to gracefully stop. +// GetExecutionRequirements returns the number of required VUs to run the +// executor for its whole duration (disregarding any startTime), including the +// maximum waiting time for any iterations to gracefully stop. This is used by +// the execution scheduler in its VU reservation calculations, so it knows how +// many VUs to pre-initialize. func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 4d9a97b1543..4ab79b2b7ef 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -98,9 +98,11 @@ func (clvc ConstantLoopingVUsConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the executor, including the maximum waiting time for -// iterations to gracefully stop. +// GetExecutionRequirements returns the number of required VUs to run the +// executor for its whole duration (disregarding any startTime), including the +// maximum waiting time for any iterations to gracefully stop. This is used by +// the execution scheduler in its VU reservation calculations, so it knows how +// many VUs to pre-initialize. func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { diff --git a/lib/executor/externally-controlled.go b/lib/executor/externally_controlled.go similarity index 99% rename from lib/executor/externally-controlled.go rename to lib/executor/externally_controlled.go index a081642a892..e97acfaf46d 100644 --- a/lib/executor/externally-controlled.go +++ b/lib/executor/externally_controlled.go @@ -60,7 +60,7 @@ func init() { // determine the scheduling of VUs in the externally controlled executor. type ExternallyControlledConfigParams struct { VUs null.Int `json:"vus"` - Duration types.NullDuration `json:"duration"` + Duration types.NullDuration `json:"duration"` // 0 is a valid value, meaning infinite duration MaxVUs null.Int `json:"maxVUs"` } @@ -142,7 +142,7 @@ func (mec ExternallyControlledConfig) Validate() []error { func (mec ExternallyControlledConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { startVUs := lib.ExecutionStep{ TimeOffset: 0, - PlannedVUs: uint64(es.Scale(mec.MaxVUs.Int64)), // use + PlannedVUs: uint64(es.Scale(mec.MaxVUs.Int64)), // user-configured, VUs to be pre-initialized MaxUnplannedVUs: 0, // intentional, see function comment } diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 085fc8c465d..c5f983a9f90 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -106,9 +106,11 @@ func (pvic PerVUIterationsConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the executor, including the maximum waiting time for -// iterations to gracefully stop. +// GetExecutionRequirements returns the number of required VUs to run the +// executor for its whole duration (disregarding any startTime), including the +// maximum waiting time for any iterations to gracefully stop. This is used by +// the execution scheduler in its VU reservation calculations, so it knows how +// many VUs to pre-initialize. func (pvic PerVUIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 6edf8d07d1d..a4029b0f842 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -109,9 +109,11 @@ func (sic SharedIterationsConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the executor, including the maximum waiting time for -// iterations to gracefully stop. +// GetExecutionRequirements returns the number of required VUs to run the +// executor for its whole duration (disregarding any startTime), including the +// maximum waiting time for any iterations to gracefully stop. This is used by +// the execution scheduler in its VU reservation calculations, so it knows how +// many VUs to pre-initialize. func (sic SharedIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 2278036e034..8f388468929 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -134,9 +134,11 @@ func (varc VariableArrivalRateConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the number of specified VUs for the -// whole duration of the executor, including the maximum waiting time for -// iterations to gracefully stop. +// GetExecutionRequirements returns the number of required VUs to run the +// executor for its whole duration (disregarding any startTime), including the +// maximum waiting time for any iterations to gracefully stop. This is used by +// the execution scheduler in its VU reservation calculations, so it knows how +// many VUs to pre-initialize. func (varc VariableArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { return []lib.ExecutionStep{ { From 71a4aff2791e445f5a5e40c7df1e4a8dbfc871e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 11 Oct 2019 18:15:50 +0200 Subject: [PATCH 045/350] fix: typos --- lib/executor/externally_controlled.go | 4 ++-- lib/executor/vu_handle.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index e97acfaf46d..d08fd3c36a8 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -135,7 +135,7 @@ func (mec ExternallyControlledConfig) Validate() []error { // k6 is running on. // // This is not a problem, because the MaxUnplannedVUs are mostly meant to be -// used for calculating the maximum possble number of initialized VUs at any +// used for calculating the maximum possible number of initialized VUs at any // point during a test run. That's used for sizing purposes and for user qouta // checking in the cloud execution, where the externally controlled executor // isn't supported. @@ -374,7 +374,7 @@ type externallyControlledRunState struct { runIteration func(context.Context, lib.VU) // a helper closure function that runs a single iteration } -// retrieveAndInitStartMaxVUs gets and initializes the (scaled) number of MaxVUs +// retrieveStartMaxVUs gets and initializes the (scaled) number of MaxVUs // from the global VU buffer. These are the VUs that the user originally // specified in the JS config, and that the ExecutionScheduler pre-initialized // for us. diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 2e0c13ee421..71f82ddad33 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -147,7 +147,7 @@ mainLoop: } // Probably not needed, but just in case - if both running and - // executorDone were actice, check that the executor isn't done. + // executorDone were active, check that the executor isn't done. select { case <-executorDone: return From 0058481616f0a2c1bd8ed803432dd1d0e770ff1e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 4 Nov 2019 11:34:08 +0100 Subject: [PATCH 046/350] fix: typos --- core/local/local.go | 2 +- lib/execution_segment.go | 2 +- lib/executor/base_config.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index d7cd3bc8c41..58b513b7f40 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -115,7 +115,7 @@ func (e *ExecutionScheduler) GetExecutors() []lib.Executor { return e.executors } -// GetInitProgressBar returns a the progress bar assotiated with the Init +// GetInitProgressBar returns the progress bar associated with the Init // function. After the Init is done, it is "hijacked" to display real-time // execution statistics as a text bar. func (e *ExecutionScheduler) GetInitProgressBar() *pb.ProgressBar { diff --git a/lib/execution_segment.go b/lib/execution_segment.go index c6e4cc9cf2f..dea79b41b81 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -37,7 +37,7 @@ import ( // that we'll lose a VU along the way... // // The most important part is that if work is split between multiple k6 instances, -// each k6 instance can precisely and reproducably calculate its share of the work, +// each k6 instance can precisely and reproducibly calculate its share of the work, // just by knowing its own segment. There won't be a need to schedule the // execution from a master node, or to even know how many other k6 instances are // running! diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index f8ddf3edcf3..32547b3bb8a 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -123,7 +123,7 @@ func (bc BaseConfig) IsDistributable() bool { return true } -// getBaseInfo is a helper method for the the "parent" String methods. +// getBaseInfo is a helper method for the "parent" String methods. func (bc BaseConfig) getBaseInfo(facts ...string) string { if bc.Exec.Valid { facts = append(facts, fmt.Sprintf("exec: %s", bc.Exec.String)) From f35926202479e826379e77ad47b6568238099fb5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 5 Nov 2019 13:34:26 +0100 Subject: [PATCH 047/350] fix: typos --- lib/executor/constant_arrival_rate.go | 2 +- lib/executor/helpers.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 565f5410029..4e7857fc6c7 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -202,7 +202,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC vus := make(chan lib.VU, maxVUs) initialisedVUs := uint64(0) - // Make sure we put back planned and unplanned VUs back in the global + // Make sure we put planned and unplanned VUs back in the global // buffer, and as an extra incentive, this replaces a waitgroup. defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index c72284dc895..86c3491c535 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -153,7 +153,7 @@ func getDurationContexts(parentCtx context.Context, regularDuration, gracefulSto } // trackProgress is a helper function that monitors certain end-events in a -// executor and updates it's progressbar accordingly. +// executor and updates its progressbar accordingly. func trackProgress( parentCtx, maxDurationCtx, regDurationCtx context.Context, sched lib.Executor, snapshot func() (float64, string), @@ -189,7 +189,7 @@ func getScaledArrivalRate(es *lib.ExecutionSegment, rate int64, period time.Dura return es.InPlaceScaleRat(big.NewRat(rate, int64(period))) } -// just a cached value to avoid allocationg it every getTickerPeriod() call +// just a cached value to avoid allocating it every getTickerPeriod() call var zero = big.NewInt(0) //nolint:gochecknoglobals // getTickerPeriod is just a helper function that returns the ticker interval* From d4742c7fd902e9f0dd991199f0e8b934cf61560f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 6 Nov 2019 12:14:51 +0100 Subject: [PATCH 048/350] fix: typos --- lib/execution.go | 2 +- lib/executor/externally_controlled.go | 2 +- lib/executor/helpers.go | 8 ++++---- lib/executor/per_vu_iterations.go | 2 +- lib/executors.go | 4 ++-- lib/helpers.go | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/lib/execution.go b/lib/execution.go index f965898608a..32e6195151c 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -85,7 +85,7 @@ type ExecutionScheduler interface { SetPaused(paused bool) error } -// MaxTimeToWaitForPlannedVU specifies the maximum allowable time for a executor +// MaxTimeToWaitForPlannedVU specifies the maximum allowable time for an executor // to wait for a planned VU to be retrieved from the ExecutionState.PlannedVUs // buffer. If it's exceeded, k6 will emit a warning log message, since it either // means that there's a bug in the k6 scheduling code, or that the machine is diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index d08fd3c36a8..e6d94731581 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -272,7 +272,7 @@ func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf inter } if newConfigParams.MaxVUs.Valid && newConfigParams.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { // This limitation is because the externally controlled executor is - // still a executor that participates in the overall k6 scheduling. + // still an executor that participates in the overall k6 scheduling. // Thus, any VUs that were explicitly specified by the user in the // config may be reused from or by other executors. return fmt.Errorf( diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 86c3491c535..54b64c0c05e 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -113,12 +113,12 @@ func getIterationRunner( } } -// getDurationContexts is used to create sub-contexts that can restrict a +// getDurationContexts is used to create sub-contexts that can restrict an // executor to only run for its allotted time. // // If the executor doesn't have a graceful stop period for iterations, then // both returned sub-contexts will be the same one, with a timeout equal to -// supplied regular executor duration. +// the supplied regular executor duration. // // But if a graceful stop is enabled, then the first returned context (and the // cancel func) will be for the "outer" sub-context. Its timeout will include @@ -152,7 +152,7 @@ func getDurationContexts(parentCtx context.Context, regularDuration, gracefulSto return startTime, maxDurationCtx, regDurationCtx, maxDurationCancel } -// trackProgress is a helper function that monitors certain end-events in a +// trackProgress is a helper function that monitors certain end-events in an // executor and updates its progressbar accordingly. func trackProgress( parentCtx, maxDurationCtx, regDurationCtx context.Context, @@ -192,7 +192,7 @@ func getScaledArrivalRate(es *lib.ExecutionSegment, rate int64, period time.Dura // just a cached value to avoid allocating it every getTickerPeriod() call var zero = big.NewInt(0) //nolint:gochecknoglobals -// getTickerPeriod is just a helper function that returns the ticker interval* +// getTickerPeriod is just a helper function that returns the ticker interval // we need for given arrival-rate parameters. // // It's possible for this function to return a zero duration (i.e. valid=false) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index c5f983a9f90..e13c1bc40a2 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -143,7 +143,7 @@ type PerVUIterations struct { // Make sure we implement the lib.Executor interface. var _ lib.Executor = &PerVUIterations{} -// Run executes a specific number of iterations with each confugured VU. +// Run executes a specific number of iterations with each configured VU. func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := pvi.executionState.Options.ExecutionSegment numVUs := pvi.config.GetVUs(segment) diff --git a/lib/executors.go b/lib/executors.go index 046bd70e8da..c512c89fe8a 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -49,13 +49,13 @@ var ( // separately. // // A slice [{t1, v1}, {t2, v2}, {t3, v3}, ..., {tn, vn}] of execution steps -// means that a executor will need 0 VUs until t1, it will need v1 number of +// means that an executor will need 0 VUs until t1, it will need v1 number of // VUs from time t1 until t2, need v2 number of VUs from time t2 to t3, and so // on. t1 is usually 0, tn is usually the same as GetMaxDuration() and vn is // usually 0. // // Keep in mind that t(i) may be exactly equal to t(i+i), when there's an abrupt -// transition in the number of VUs required by a executor. For example, the +// transition in the number of VUs required by an executor. For example, the // variable-looping-vus executor may have 0-duration stages, or it may scale up // VUs in its last stage right until the end. These // diff --git a/lib/helpers.go b/lib/helpers.go index 927b972a849..cedcbe0fa7e 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -69,7 +69,7 @@ func GetMaxPlannedVUs(steps []ExecutionStep) (result uint64) { // duration of a test can often lead to mistakes. That's why this function is // called GetMaxPossibleVUs() and why there is no GetMaxUnplannedVUs() function. // -// As an example, imagine that you have a executor with MaxPlannedVUs=20 and +// As an example, imagine that you have an executor with MaxPlannedVUs=20 and // MaxUnplannedVUs=0, followed immediately after by another executor with // MaxPlannedVUs=10 and MaxUnplannedVUs=10. The MaxPlannedVUs number for the // whole test is 20, and MaxUnplannedVUs is 10, but since those executors won't From 0b55bbadb9c28ef08ab599a07f045912466b92c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 7 Nov 2019 18:02:54 +0100 Subject: [PATCH 049/350] fix: typos --- lib/executor/variable_looping_vus.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 8d1add2c061..fe2c259d73e 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -156,7 +156,7 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // And the chart for the other execution segment would look like this: // // VUs ^ -// 5| y +// 5| Y // 4| YYYYY // 3| YYYYYYYYY // 2| YYYYYYYYYYYYY From 61069240e34c89dcfa5464648e1caab2cd857046 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 19 Nov 2019 15:09:58 +0200 Subject: [PATCH 050/350] Address PR comments by @imiric --- lib/execution.go | 15 +++++++++++---- lib/executor/externally_controlled.go | 10 +++++----- lib/executor/vu_handle.go | 3 ++- lib/executors.go | 8 +++++--- 4 files changed, 23 insertions(+), 13 deletions(-) diff --git a/lib/execution.go b/lib/execution.go index 32e6195151c..b8a3016d184 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -474,10 +474,17 @@ func (es *ExecutionState) ResumeNotify() <-chan struct{} { // GetPlannedVU tries to get a pre-initialized VU from the buffer channel. This // shouldn't fail and should generally be an instantaneous action, but if it -// doesn't happen for MaxTimeToWaitForPlannedVU, a warning will be printed. If -// we reach that timeout more than MaxRetriesGetPlannedVU number of times, this -// function will return an error, since we either have a bug with some -// executor, or the machine is very, very overloaded. +// doesn't happen for MaxTimeToWaitForPlannedVU (for example, because the system +// is overloaded), a warning will be printed. If we reach that timeout more than +// MaxRetriesGetPlannedVU number of times, this function will return an error, +// since we either have a bug with some executor, or the machine is very, very +// overloaded. +// +// If modifyActiveVUCount is true, the method would also increment the counter +// for active VUs. In most cases, that's the desired behavior, but some +// executors might have to retrieve their reserved VUs without using them +// immediately - for example, the the externally-controlled executor when the +// configured maxVUs number is greater than the configured starting VUs. func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount bool) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 649cab3232c..baa3c216a71 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -122,17 +122,17 @@ func (mec ExternallyControlledConfig) Validate() []error { return errors } -// GetExecutionRequirements just reserves the specified number of max VUs for -// the whole duration of the executor, so these VUs can be initialized in the -// beginning of the test. +// GetExecutionRequirements reserves the configured number of max VUs for the +// whole duration of the executor, so these VUs can be externally initialized in +// the beginning of the test. // // Importantly, if 0 (i.e. infinite) duration is configured, this executor // doesn't emit the last step to relinquish these VUs. // // Also, the externally controlled executor doesn't set MaxUnplannedVUs in the // returned steps, since their initialization and usage is directly controlled -// by the user and is effectively bounded only by the resources of the machine -// k6 is running on. +// by the user, can be changed during the test runtime, and is effectively +// bounded only by the resources of the machine k6 is running on. // // This is not a problem, because the MaxUnplannedVUs are mostly meant to be // used for calculating the maximum possible number of initialized VUs at any diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 71f82ddad33..42376b5c86a 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -30,7 +30,8 @@ import ( ) // This is a helper type used in executors where we have to dynamically control -// the number of VUs that are simultaneously running. For the moment, it is used in the VariableLoopingVUs and +// the number of VUs that are simultaneously running. For the moment, it is used +// in the VariableLoopingVUs and the ExternallyControlled executors. // // TODO: something simpler? type vuHandle struct { diff --git a/lib/executors.go b/lib/executors.go index c512c89fe8a..77c0450b950 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -42,8 +42,8 @@ var ( executorConfigConstructors = make(map[string]ExecutorConfigConstructor) ) -// ExecutionStep is used by different executors to specify the planned number -// of VUs they will need at a particular time. The times are relative to their +// ExecutionStep is used by different executors to specify the planned number of +// VUs they will need at a particular time. The times are relative to their // StartTime, i.e. they don't take into account the specific starting time of // the executor, as that will be considered by the external execution executor // separately. @@ -57,7 +57,9 @@ var ( // Keep in mind that t(i) may be exactly equal to t(i+i), when there's an abrupt // transition in the number of VUs required by an executor. For example, the // variable-looping-vus executor may have 0-duration stages, or it may scale up -// VUs in its last stage right until the end. These +// VUs in its last stage right until the end. These immediate transitions cannot +// be ignored, since the gracefulStop/gracefulRampDown options potentially allow +// any started iterations to finish. // // []ExecutionStep is also used by the ExecutorConfigMap, to represent the // amount of needed VUs among all executors, during the whole execution of a From dff6b0f628e6bd10ca7a4529890b7be037a0c189 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 22 Aug 2019 15:36:34 +0300 Subject: [PATCH 051/350] Test everything function of lib.ExecutionSegment This doesn't include a single line that I am not sure it reachable at all. More tests can be added with better corner cases in the future --- lib/execution_segment_test.go | 237 +++++++++++++++++++++++++++++++++- 1 file changed, 230 insertions(+), 7 deletions(-) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index b9c9db16184..4d050756e6d 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -28,14 +28,102 @@ import ( "github.com/stretchr/testify/require" ) -func TestExecutionSegments(t *testing.T) { +func stringToES(t *testing.T, str string) *ExecutionSegment { + var es = new(ExecutionSegment) + require.NoError(t, es.UnmarshalText([]byte(str))) + return es +} +func TestExecutionSegmentEquals(t *testing.T) { t.Parallel() - var nilEs *ExecutionSegment - fullEs, err := NewExecutionSegment(big.NewRat(0, 1), big.NewRat(1, 1)) - require.NoError(t, err) - require.Equal(t, true, nilEs.Equal(fullEs)) - //TODO: more tests + t.Run("nil segment to full", func(t *testing.T) { + var nilEs *ExecutionSegment + fullEs := stringToES(t, "0:1") + require.True(t, nilEs.Equal(fullEs)) + require.True(t, fullEs.Equal(nilEs)) + }) + + t.Run("To it's self", func(t *testing.T) { + var es = stringToES(t, "1/2:2/3") + require.True(t, es.Equal(es)) + }) +} + +func TestExecutionSegmentNew(t *testing.T) { + t.Parallel() + t.Run("from is below zero", func(t *testing.T) { + _, err := NewExecutionSegment(big.NewRat(-1, 1), big.NewRat(1, 1)) + require.Error(t, err) + }) + t.Run("to is more than 1", func(t *testing.T) { + _, err := NewExecutionSegment(big.NewRat(0, 1), big.NewRat(2, 1)) + require.Error(t, err) + }) + t.Run("from is smaller than to", func(t *testing.T) { + _, err := NewExecutionSegment(big.NewRat(1, 2), big.NewRat(1, 3)) + require.Error(t, err) + }) + + t.Run("from is equal to 'to'", func(t *testing.T) { + _, err := NewExecutionSegment(big.NewRat(1, 2), big.NewRat(1, 2)) + require.Error(t, err) + }) + t.Run("ok", func(t *testing.T) { + _, err := NewExecutionSegment(big.NewRat(0, 1), big.NewRat(1, 1)) + require.NoError(t, err) + }) +} + +func TestExecutionSegmentUnmarshalText(t *testing.T) { + t.Parallel() + var testCases = []struct { + input string + output *ExecutionSegment + isErr bool + }{ + {input: "0:1", output: &ExecutionSegment{from: zeroRat, to: oneRat}}, + {input: "0.5:0.75", output: &ExecutionSegment{from: big.NewRat(1, 2), to: big.NewRat(3, 4)}}, + {input: "1/2:3/4", output: &ExecutionSegment{from: big.NewRat(1, 2), to: big.NewRat(3, 4)}}, + {input: "50%:75%", output: &ExecutionSegment{from: big.NewRat(1, 2), to: big.NewRat(3, 4)}}, + {input: "2/4:75%", output: &ExecutionSegment{from: big.NewRat(1, 2), to: big.NewRat(3, 4)}}, + {input: "75%", output: &ExecutionSegment{from: zeroRat, to: big.NewRat(3, 4)}}, + {input: "125%", isErr: true}, + {input: "1a5%", isErr: true}, + {input: "1a5", isErr: true}, + {input: "1a5%:2/3", isErr: true}, + {input: "125%:250%", isErr: true}, + {input: "55%:50%", isErr: true}, + // TODO add more strange or not so strange cases + } + for _, testCase := range testCases { + testCase := testCase + t.Run(testCase.input, func(t *testing.T) { + var es = new(ExecutionSegment) + err := es.UnmarshalText([]byte(testCase.input)) + if testCase.isErr { + require.Error(t, err) + return + } + require.NoError(t, err) + require.True(t, es.Equal(testCase.output)) + + // see if unmarshalling a stringified segment gets you back the same segment + err = es.UnmarshalText([]byte(es.String())) + require.NoError(t, err) + require.True(t, es.Equal(testCase.output)) + }) + } + + t.Run("Unmarshal nilSegment.String", func(t *testing.T) { + var nilEs *ExecutionSegment + var nilEsStr = nilEs.String() + require.Equal(t, "0:1", nilEsStr) + + var es = new(ExecutionSegment) + err := es.UnmarshalText([]byte(nilEsStr)) + require.NoError(t, err) + require.True(t, es.Equal(nilEs)) + }) } func TestExecutionSegmentSplit(t *testing.T) { @@ -96,4 +184,139 @@ func TestExecutionSegmentSplit(t *testing.T) { assert.Equal(t, "7/16:1/2", segments[3].String()) } -//TODO: a lot of other tests +func TestExecutionSegmentScale(t *testing.T) { + t.Parallel() + var es = new(ExecutionSegment) + require.NoError(t, es.UnmarshalText([]byte("0.5"))) + require.Equal(t, int64(1), es.Scale(2)) + require.Equal(t, int64(2), es.Scale(3)) + + require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) + require.Equal(t, int64(1), es.Scale(2)) + require.Equal(t, int64(1), es.Scale(3)) +} + +func TestExecutionSegmentCopyScaleRat(t *testing.T) { + t.Parallel() + var es = new(ExecutionSegment) + var twoRat = big.NewRat(2, 1) + var threeRat = big.NewRat(3, 1) + require.NoError(t, es.UnmarshalText([]byte("0.5"))) + require.Equal(t, oneRat, es.CopyScaleRat(twoRat)) + require.Equal(t, big.NewRat(3, 2), es.CopyScaleRat(threeRat)) + + require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) + require.Equal(t, oneRat, es.CopyScaleRat(twoRat)) + require.Equal(t, big.NewRat(3, 2), es.CopyScaleRat(threeRat)) + + var nilEs *ExecutionSegment + require.Equal(t, twoRat, nilEs.CopyScaleRat(twoRat)) + require.Equal(t, threeRat, nilEs.CopyScaleRat(threeRat)) +} + +func TestExecutionSegmentInPlaceScaleRat(t *testing.T) { + t.Parallel() + var es = new(ExecutionSegment) + var twoRat = big.NewRat(2, 1) + var threeRat = big.NewRat(3, 1) + var threeSecondsRat = big.NewRat(3, 2) + require.NoError(t, es.UnmarshalText([]byte("0.5"))) + require.Equal(t, oneRat, es.InPlaceScaleRat(twoRat)) + require.Equal(t, oneRat, twoRat) + require.Equal(t, threeSecondsRat, es.InPlaceScaleRat(threeRat)) + require.Equal(t, threeSecondsRat, threeRat) + + es = stringToES(t, "0.5:1.0") + twoRat = big.NewRat(2, 1) + threeRat = big.NewRat(3, 1) + require.Equal(t, oneRat, es.InPlaceScaleRat(twoRat)) + require.Equal(t, oneRat, twoRat) + require.Equal(t, threeSecondsRat, es.InPlaceScaleRat(threeRat)) + require.Equal(t, threeSecondsRat, threeRat) + + var nilEs *ExecutionSegment + twoRat = big.NewRat(2, 1) + threeRat = big.NewRat(3, 1) + require.Equal(t, big.NewRat(2, 1), nilEs.InPlaceScaleRat(twoRat)) + require.Equal(t, big.NewRat(2, 1), twoRat) + require.Equal(t, big.NewRat(3, 1), nilEs.InPlaceScaleRat(threeRat)) + require.Equal(t, big.NewRat(3, 1), threeRat) +} + +func TestExecutionSegmentSubSegment(t *testing.T) { + t.Parallel() + var testCases = []struct { + name string + base, sub, result *ExecutionSegment + }{ + // TODO add more strange or not so strange cases + { + name: "nil base", + base: (*ExecutionSegment)(nil), + sub: stringToES(t, "0.2:0.3"), + result: stringToES(t, "0.2:0.3"), + }, + + { + name: "nil sub", + base: stringToES(t, "0.2:0.3"), + sub: (*ExecutionSegment)(nil), + result: stringToES(t, "0.2:0.3"), + }, + { + name: "doc example", + base: stringToES(t, "1/2:1"), + sub: stringToES(t, "0:1/2"), + result: stringToES(t, "1/2:3/4"), + }, + } + + for _, testCase := range testCases { + testCase := testCase + t.Run(testCase.name, func(t *testing.T) { + require.Equal(t, testCase.result, testCase.base.SubSegment(testCase.sub)) + }) + } +} + +func TestSplitBadSegment(t *testing.T) { + t.Parallel() + var es = &ExecutionSegment{from: oneRat, to: zeroRat} + _, err := es.Split(5) + require.Error(t, err) +} + +func TestSegmentExecutionFloatLength(t *testing.T) { + t.Parallel() + t.Run("nil has 1.0", func(t *testing.T) { + var nilEs *ExecutionSegment + require.Equal(t, 1.0, nilEs.FloatLength()) + }) + + var testCases = []struct { + es *ExecutionSegment + expected float64 + }{ + // TODO add more strange or not so strange cases + { + es: stringToES(t, "1/2:1"), + expected: 0.5, + }, + { + es: stringToES(t, "1/3:1"), + expected: 0.66666, + }, + + { + es: stringToES(t, "0:1/2"), + expected: 0.5, + }, + } + + for _, testCase := range testCases { + testCase := testCase + t.Run(testCase.es.String(), func(t *testing.T) { + require.InEpsilon(t, testCase.expected, testCase.es.FloatLength(), 0.001) + }) + } +} From 5990c895f7f4d94d883f2c4cca7cb5d3ef331b4c Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 22 Aug 2019 16:29:50 +0300 Subject: [PATCH 052/350] Enable a long time ago disabled test that works ? --- api/v1/group_routes_test.go | 55 ++++++++++++++++++------------------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index 4d4e6c40bc5..cf8d8487c3d 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -65,34 +65,33 @@ func TestGetGroups(t *testing.T) { } }) - // t.Run("groups", func(t *testing.T) { - // var groups []Group - // assert.NoError(t, jsonapi.Unmarshal(body, &groups)) - // if assert.Len(t, groups, 3) { - // for _, g := range groups { - // switch g.ID { - // case g0.ID: - // assert.Equal(t, "", g.Name) - // assert.Nil(t, g.Parent) - // assert.Equal(t, "", g.ParentID) - // assert.Len(t, g.GroupIDs, 1) - // assert.EqualValues(t, []string{g1.ID}, g.GroupIDs) - // case g1.ID: - // assert.Equal(t, "group 1", g.Name) - // assert.Nil(t, g.Parent) - // assert.Equal(t, g0.ID, g.ParentID) - // assert.EqualValues(t, []string{g2.ID}, g.GroupIDs) - // case g2.ID: - // assert.Equal(t, "group 2", g.Name) - // assert.Nil(t, g.Parent) - // assert.Equal(t, g1.ID, g.ParentID) - // assert.EqualValues(t, []string{}, g.GroupIDs) - // default: - // assert.Fail(t, "Unknown ID: "+g.ID) - // } - // } - // } - // }) + t.Run("groups", func(t *testing.T) { + var groups []Group + require.NoError(t, jsonapi.Unmarshal(body, &groups)) + require.Len(t, groups, 3) + for _, g := range groups { + switch g.ID { + case g0.ID: + assert.Equal(t, "", g.Name) + assert.Nil(t, g.Parent) + assert.Equal(t, "", g.ParentID) + assert.Len(t, g.GroupIDs, 1) + assert.EqualValues(t, []string{g1.ID}, g.GroupIDs) + case g1.ID: + assert.Equal(t, "group 1", g.Name) + assert.Nil(t, g.Parent) + assert.Equal(t, g0.ID, g.ParentID) + assert.EqualValues(t, []string{g2.ID}, g.GroupIDs) + case g2.ID: + assert.Equal(t, "group 2", g.Name) + assert.Nil(t, g.Parent) + assert.Equal(t, g1.ID, g.ParentID) + assert.EqualValues(t, []string{}, g.GroupIDs) + default: + assert.Fail(t, "Unknown ID: "+g.ID) + } + } + }) }) for _, gp := range []*lib.Group{g0, g1, g2} { t.Run(gp.Name, func(t *testing.T) { From d2e9d58b019707e11aea807fbb973bf72f491873 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 3 Sep 2019 09:58:27 +0300 Subject: [PATCH 053/350] Add more tests for lib.ExecutionState also fixed a bug --- lib/execution.go | 2 +- lib/execution_test.go | 114 ++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 112 insertions(+), 4 deletions(-) diff --git a/lib/execution.go b/lib/execution.go index b8a3016d184..79f6d5e90b0 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -532,7 +532,7 @@ func (es *ExecutionState) GetUnplannedVU(ctx context.Context, logger *logrus.Ent logger.Debug("Initializing an unplanned VU, this may affect test results") vu, err := es.InitializeNewVU(ctx, logger) - if err != nil { + if err == nil { es.ModCurrentlyActiveVUsCount(+1) } return vu, err diff --git a/lib/execution_test.go b/lib/execution_test.go index fbc9840ac0a..147f9008488 100644 --- a/lib/execution_test.go +++ b/lib/execution_test.go @@ -21,22 +21,28 @@ package lib import ( + "context" + "io/ioutil" "math/rand" "sync" "testing" "time" + "github.com/loadimpact/k6/lib/testutils" + "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestExecutionStateVUIDs(t *testing.T) { - es := NewExecutionState(Options{}, 0, 0) //TODO: fix + t.Parallel() + es := NewExecutionState(Options{}, 0, 0) assert.Equal(t, uint64(1), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(2), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(3), es.GetUniqueVUIdentifier()) wg := sync.WaitGroup{} rand.Seed(time.Now().UnixNano()) - count := rand.Intn(50) + count := 100 + rand.Intn(50) wg.Add(count) for i := 0; i < count; i++ { go func() { @@ -48,4 +54,106 @@ func TestExecutionStateVUIDs(t *testing.T) { assert.Equal(t, uint64(4+count), es.GetUniqueVUIdentifier()) } -//TODO: way more tests... +func TestExecutionStateGettingVUsWhenNonAreAvailable(t *testing.T) { + t.Parallel() + es := NewExecutionState(Options{}, 0, 0) + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(ioutil.Discard) + vu, err := es.GetPlannedVU(logrus.NewEntry(testLog), true) + require.Nil(t, vu) + require.Error(t, err) + require.Contains(t, err.Error(), "could not get a VU from the buffer in") + entries := logHook.Drain() + require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + for _, entry := range entries { + require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") + } +} + +func TestExecutionStateGettingVUs(t *testing.T) { + t.Parallel() + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel, logrus.DebugLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(ioutil.Discard) + logEntry := logrus.NewEntry(testLog) + + es := NewExecutionState(Options{}, 10, 20) + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (VU, error) { + return &MiniRunnerVU{}, nil + }) + + for i := 0; i < 10; i++ { + require.EqualValues(t, i, es.GetInitializedVUsCount()) + vu, err := es.InitializeNewVU(context.Background(), logEntry) + require.NoError(t, err) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + es.ReturnVU(vu, false) + require.EqualValues(t, 0, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + } + + // Test getting initialized VUs is okay :) + for i := 0; i < 10; i++ { + require.EqualValues(t, i, es.GetCurrentlyActiveVUsCount()) + vu, err := es.GetPlannedVU(logEntry, true) + require.NoError(t, err) + require.Empty(t, logHook.Drain()) + require.NotNil(t, vu) + require.EqualValues(t, i+1, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, 10, es.GetInitializedVUsCount()) + } + + // Check that getting 1 more planned VU will error out + vu, err := es.GetPlannedVU(logEntry, true) + require.Nil(t, vu) + require.Error(t, err) + require.Contains(t, err.Error(), "could not get a VU from the buffer in") + entries := logHook.Drain() + require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + for _, entry := range entries { + require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") + } + + // Test getting uninitiazed vus will work + for i := 0; i < 10; i++ { + require.EqualValues(t, 10+i, es.GetCurrentlyActiveVUsCount()) + vu, err = es.GetUnplannedVU(context.Background(), logEntry) + require.NoError(t, err) + require.Empty(t, logHook.Drain()) + require.NotNil(t, vu) + require.EqualValues(t, 10+i+1, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, 10+i+1, es.GetInitializedVUsCount()) + } + + // Check that getting 1 more unplanned VU will error out + vu, err = es.GetUnplannedVU(context.Background(), logEntry) + require.Nil(t, vu) + require.Error(t, err) + require.Contains(t, err.Error(), "could not get a VU from the buffer in") + entries = logHook.Drain() + require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + for _, entry := range entries { + require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") + } +} + +func TestMarkStartedPanicsOnSecondRun(t *testing.T) { + t.Parallel() + es := NewExecutionState(Options{}, 0, 0) + require.False(t, es.HasStarted()) + es.MarkStarted() + require.True(t, es.HasStarted()) + require.Panics(t, es.MarkStarted) +} + +func TestMarkEnded(t *testing.T) { + t.Parallel() + es := NewExecutionState(Options{}, 0, 0) + require.False(t, es.HasEnded()) + es.MarkEnded() + require.True(t, es.HasEnded()) + require.Panics(t, es.MarkEnded) +} From 5f15268193f7f3fd89f4ff383746fe9c5aaa1634 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 2 Oct 2019 18:32:49 +0300 Subject: [PATCH 054/350] Test the VariableArrivalRate Executor Also fix it to not emit two rate changes when there is 0 duration stage --- lib/executor/variable_arrival_rate.go | 13 +- lib/executor/variable_arrival_rate_test.go | 328 +++++++++++++++++++++ 2 files changed, 337 insertions(+), 4 deletions(-) create mode 100644 lib/executor/variable_arrival_rate_test.go diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 8f388468929..86bea0230db 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -196,10 +196,15 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi // Handle 0-duration stages, i.e. instant rate jumps if stageDuration == 0 { - rateChanges = append(rateChanges, rateChange{ - timeOffset: timeFromStart, - tickerPeriod: getTickerPeriod(stageTargetRate), - }) + // check if the last set change is for the same time and overwrite it + if len(rateChanges) > 0 && rateChanges[len(rateChanges)-1].timeOffset == timeFromStart { + rateChanges[len(rateChanges)-1].tickerPeriod = getTickerPeriod(stageTargetRate) + } else { + rateChanges = append(rateChanges, rateChange{ + timeOffset: timeFromStart, + tickerPeriod: getTickerPeriod(stageTargetRate), + }) + } currentRate = stageTargetRate continue } diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go new file mode 100644 index 00000000000..24bc486f21d --- /dev/null +++ b/lib/executor/variable_arrival_rate_test.go @@ -0,0 +1,328 @@ +package executor + +import ( + "context" + "io/ioutil" + "math/big" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" +) + +func TestGetPlannedRateChanges0DurationStage(t *testing.T) { + t.Parallel() + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(100), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(100), + }, + }, + } + var es *lib.ExecutionSegment + changes := config.getPlannedRateChanges(es) + require.Equal(t, 2, len(changes)) + require.Equal(t, time.Duration(0), changes[0].timeOffset) + require.Equal(t, types.NullDurationFrom(time.Millisecond*20), changes[0].tickerPeriod) + + require.Equal(t, time.Minute, changes[1].timeOffset) + require.Equal(t, types.NullDurationFrom(time.Millisecond*10), changes[1].tickerPeriod) +} + +// helper function to calculate the expected rate change at a given time +func calculateTickerPeriod(current, start, duration time.Duration, from, to int64) types.Duration { + var coef = big.NewRat( + (current - start).Nanoseconds(), + duration.Nanoseconds(), + ) + + var oneRat = new(big.Rat).Mul(big.NewRat(from-to, 1), coef) + oneRat = new(big.Rat).Sub(big.NewRat(from, 1), oneRat) + oneRat = new(big.Rat).Mul(big.NewRat(int64(time.Second), 1), new(big.Rat).Inv(oneRat)) + return types.Duration(new(big.Int).Div(oneRat.Num(), oneRat.Denom()).Int64()) +} + +func TestGetPlannedRateChangesZeroDurationStart(t *testing.T) { + // TODO: Make multiple of those tests + t.Parallel() + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(100), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(100), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(0), + }, + }, + } + + var es *lib.ExecutionSegment + changes := config.getPlannedRateChanges(es) + var expectedTickerPeriod types.Duration + for i, change := range changes { + switch { + case change.timeOffset == 0: + expectedTickerPeriod = types.Duration(20 * time.Millisecond) + case change.timeOffset == time.Minute*1: + expectedTickerPeriod = types.Duration(10 * time.Millisecond) + case change.timeOffset < time.Minute*3: + expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 2*time.Minute, time.Minute, 100, 0) + case change.timeOffset == time.Minute*3: + expectedTickerPeriod = 0 + default: + t.Fatalf("this shouldn't happen %d index %+v", i, change) + } + require.Equal(t, time.Duration(0), + change.timeOffset%minIntervalBetweenRateAdjustments, "%d index %+v", i, change) + require.Equal(t, change.tickerPeriod.Duration, expectedTickerPeriod, "%d index %+v", i, change) + } +} + +func TestGetPlannedRateChanges(t *testing.T) { + // TODO: Make multiple of those tests + t.Parallel() + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(2 * time.Minute), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(time.Minute), + Target: null.IntFrom(100), + }, + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(200), + }, + + { + Duration: types.NullDurationFrom(time.Second * 23), + Target: null.IntFrom(50), + }, + }, + } + + var es *lib.ExecutionSegment + changes := config.getPlannedRateChanges(es) + var expectedTickerPeriod types.Duration + for i, change := range changes { + switch { + case change.timeOffset <= time.Minute*2: + expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 0, time.Minute*2, 0, 50) + case change.timeOffset < time.Minute*4: + expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, time.Minute*3, time.Minute, 50, 100) + case change.timeOffset == time.Minute*4: + expectedTickerPeriod = types.Duration(5 * time.Millisecond) + default: + expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 4*time.Minute, 23*time.Second, 200, 50) + } + require.Equal(t, time.Duration(0), + change.timeOffset%minIntervalBetweenRateAdjustments, "%d index %+v", i, change) + require.Equal(t, change.tickerPeriod.Duration, expectedTickerPeriod, "%d index %+v", i, change) + } +} + +func initializeVUs( + ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, +) { + for i := 0; i < number; i++ { + require.EqualValues(t, i, es.GetInitializedVUsCount()) + vu, err := es.InitializeNewVU(ctx, logEntry) + require.NoError(t, err) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + es.ReturnVU(vu, false) + require.EqualValues(t, 0, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + } +} + +func testVariableArrivalRateSetup(t *testing.T, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { + ctx, cancel := context.WithCancel(context.Background()) + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(10), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(time.Second * 1), + Target: null.IntFrom(10), + }, + { + Duration: types.NullDurationFrom(time.Second * 1), + Target: null.IntFrom(50), + }, + { + Duration: types.NullDurationFrom(time.Second * 1), + Target: null.IntFrom(50), + }, + }, + PreAllocatedVUs: null.IntFrom(10), + MaxVUs: null.IntFrom(20), + } + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(ioutil.Discard) + logEntry := logrus.NewEntry(testLog) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + runner := lib.MiniRunner{ + Fn: vuFn, + } + + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { + return &lib.MiniRunnerVU{R: runner}, nil + }) + + initializeVUs(ctx, t, logEntry, es, 10) + + executor, err := config.NewExecutor(es, logEntry) + require.NoError(t, err) + err = executor.Init(ctx) + require.NoError(t, err) + return ctx, cancel, executor, logHook +} + +func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { + t.Parallel() + var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(time.Second) + return nil + }) + defer cancel() + var engineOut = make(chan stats.SampleContainer, 1000) + err := executor.Run(ctx, engineOut) + require.NoError(t, err) + entries := logHook.Drain() + require.NotEmpty(t, entries) + for _, entry := range entries { + require.Equal(t, + "Insufficient VUs, reached 20 active VUs and cannot allocate more", + entry.Message) + require.Equal(t, logrus.WarnLevel, entry.Level) + } +} + +func TestVariableArrivalRateRunCorrectRate(t *testing.T) { + t.Parallel() + var count int64 + var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + atomic.AddInt64(&count, 1) + return nil + }) + defer cancel() + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + // check that we got around the amount of VU iterations as we would expect + var currentCount int64 + + time.Sleep(time.Second) + currentCount = atomic.SwapInt64(&count, 0) + require.InDelta(t, 10, currentCount, 1) + + time.Sleep(time.Second) + currentCount = atomic.SwapInt64(&count, 0) + // this is highly dependant on minIntervalBetweenRateAdjustments + // TODO find out why this isn't 30 and fix it + require.InDelta(t, 23, currentCount, 2) + + time.Sleep(time.Second) + currentCount = atomic.SwapInt64(&count, 0) + require.InDelta(t, 50, currentCount, 2) + }() + var engineOut = make(chan stats.SampleContainer, 1000) + err := executor.Run(ctx, engineOut) + wg.Wait() + require.NoError(t, err) + require.Empty(t, logHook.Drain()) +} + +func TestVariableArrivalRateCancel(t *testing.T) { + t.Parallel() + var ch = make(chan struct{}) + var errCh = make(chan error, 1) + var weAreDoneCh = make(chan struct{}) + var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + select { + case <-ch: + <-ch + default: + } + return nil + }) + defer cancel() + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + + var engineOut = make(chan stats.SampleContainer, 1000) + errCh <- executor.Run(ctx, engineOut) + close(weAreDoneCh) + }() + + time.Sleep(time.Second) + ch <- struct{}{} + cancel() + time.Sleep(time.Second) + select { + case <-weAreDoneCh: + t.Fatal("Run raturned before all VU iterations were finished") + default: + } + close(ch) + <-weAreDoneCh + wg.Wait() + require.NoError(t, <-errCh) + require.Empty(t, logHook.Drain()) +} From 002f27befde971ce002bf079e342d2c3a436b68d Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 4 Oct 2019 10:03:57 +0300 Subject: [PATCH 055/350] Test ConstantArrivalRate executor --- lib/executor/constant_arrival_rate_test.go | 154 +++++++++++++++++++++ lib/executor/variable_arrival_rate_test.go | 41 ------ 2 files changed, 154 insertions(+), 41 deletions(-) create mode 100644 lib/executor/constant_arrival_rate_test.go diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go new file mode 100644 index 00000000000..964828a201d --- /dev/null +++ b/lib/executor/constant_arrival_rate_test.go @@ -0,0 +1,154 @@ +package executor + +import ( + "context" + "io/ioutil" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" +) + +func testConstantArrivalRateSetup(t *testing.T, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { + ctx, cancel := context.WithCancel(context.Background()) + var config = ConstantArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + Rate: null.IntFrom(50), + Duration: types.NullDurationFrom(5 * time.Second), + PreAllocatedVUs: null.IntFrom(10), + MaxVUs: null.IntFrom(20), + } + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(ioutil.Discard) + logEntry := logrus.NewEntry(testLog) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + runner := lib.MiniRunner{ + Fn: vuFn, + } + + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { + return &lib.MiniRunnerVU{R: runner}, nil + }) + + initializeVUs(ctx, t, logEntry, es, 10) + + executor, err := config.NewExecutor(es, logEntry) + require.NoError(t, err) + err = executor.Init(ctx) + require.NoError(t, err) + return ctx, cancel, executor, logHook +} + +func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { + t.Parallel() + var ctx, cancel, executor, logHook = testConstantArrivalRateSetup( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + time.Sleep(time.Second) + return nil + }) + defer cancel() + var engineOut = make(chan stats.SampleContainer, 1000) + err := executor.Run(ctx, engineOut) + require.NoError(t, err) + entries := logHook.Drain() + require.NotEmpty(t, entries) + for _, entry := range entries { + require.Equal(t, + "Insufficient VUs, reached 20 active VUs and cannot allocate more", + entry.Message) + require.Equal(t, logrus.WarnLevel, entry.Level) + } +} + +func TestConstantArrivalRateRunCorrectRate(t *testing.T) { + t.Parallel() + var count int64 + var ctx, cancel, executor, logHook = testConstantArrivalRateSetup( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + atomic.AddInt64(&count, 1) + return nil + }) + defer cancel() + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + // check that we got around the amount of VU iterations as we would expect + var currentCount int64 + + for i := 0; i < 5; i++ { + time.Sleep(time.Second) + currentCount = atomic.SwapInt64(&count, 0) + require.InDelta(t, 50, currentCount, 1) + } + }() + var engineOut = make(chan stats.SampleContainer, 1000) + err := executor.Run(ctx, engineOut) + wg.Wait() + require.NoError(t, err) + require.Empty(t, logHook.Drain()) +} + +func TestArrivalRateCancel(t *testing.T) { + t.Parallel() + var mat = map[string]func( + t *testing.T, + vuFn func(context.Context, chan<- stats.SampleContainer) error, + ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook){ + "constant": testConstantArrivalRateSetup, + "variable": testVariableArrivalRateSetup, + } + for name, fn := range mat { + fn := fn + t.Run(name, func(t *testing.T) { + var ch = make(chan struct{}) + var errCh = make(chan error, 1) + var weAreDoneCh = make(chan struct{}) + var ctx, cancel, executor, logHook = fn( + t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + select { + case <-ch: + <-ch + default: + } + return nil + }) + defer cancel() + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + + var engineOut = make(chan stats.SampleContainer, 1000) + errCh <- executor.Run(ctx, engineOut) + close(weAreDoneCh) + }() + + time.Sleep(time.Second) + ch <- struct{}{} + cancel() + time.Sleep(time.Second) + select { + case <-weAreDoneCh: + t.Fatal("Run returned before all VU iterations were finished") + default: + } + close(ch) + <-weAreDoneCh + wg.Wait() + require.NoError(t, <-errCh) + require.Empty(t, logHook.Drain()) + }) + } +} diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 24bc486f21d..e82c25942e8 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -285,44 +285,3 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { require.NoError(t, err) require.Empty(t, logHook.Drain()) } - -func TestVariableArrivalRateCancel(t *testing.T) { - t.Parallel() - var ch = make(chan struct{}) - var errCh = make(chan error, 1) - var weAreDoneCh = make(chan struct{}) - var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { - select { - case <-ch: - <-ch - default: - } - return nil - }) - defer cancel() - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - - var engineOut = make(chan stats.SampleContainer, 1000) - errCh <- executor.Run(ctx, engineOut) - close(weAreDoneCh) - }() - - time.Sleep(time.Second) - ch <- struct{}{} - cancel() - time.Sleep(time.Second) - select { - case <-weAreDoneCh: - t.Fatal("Run raturned before all VU iterations were finished") - default: - } - close(ch) - <-weAreDoneCh - wg.Wait() - require.NoError(t, <-errCh) - require.Empty(t, logHook.Drain()) -} From 9740c92ca96e1a31f7e7e9c9c674904db7a1c6f7 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 8 Oct 2019 11:43:23 +0300 Subject: [PATCH 056/350] Test the local execution scheduler's SetPaused --- core/local/local_test.go | 83 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/core/local/local_test.go b/core/local/local_test.go index bb1cb8313c0..fe72f72d405 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -645,3 +645,86 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { } } } + +// Just a lib.PausableExecutor implementation that can return an error +type pausableExecutor struct { + lib.Executor + err error +} + +func (p pausableExecutor) SetPaused(bool) error { + return p.err +} + +func TestSetPaused(t *testing.T) { + t.Run("second pause is an error", func(t *testing.T) { + var runner = &lib.MiniRunner{} + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + var sched, err = NewExecutionScheduler(runner, logger) + require.NoError(t, err) + sched.executors = []lib.Executor{pausableExecutor{err: nil}} + + require.NoError(t, sched.SetPaused(true)) + err = sched.SetPaused(true) + require.Error(t, err) + require.Contains(t, err.Error(), "execution is already paused") + }) + + t.Run("unpause at the start is an error", func(t *testing.T) { + var runner = &lib.MiniRunner{} + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + var sched, err = NewExecutionScheduler(runner, logger) + require.NoError(t, err) + sched.executors = []lib.Executor{pausableExecutor{err: nil}} + err = sched.SetPaused(false) + require.Error(t, err) + require.Contains(t, err.Error(), "execution wasn't paused") + }) + + t.Run("second unpause is an error", func(t *testing.T) { + var runner = &lib.MiniRunner{} + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + var sched, err = NewExecutionScheduler(runner, logger) + require.NoError(t, err) + sched.executors = []lib.Executor{pausableExecutor{err: nil}} + require.NoError(t, sched.SetPaused(true)) + require.NoError(t, sched.SetPaused(false)) + err = sched.SetPaused(false) + require.Error(t, err) + require.Contains(t, err.Error(), "execution wasn't paused") + }) + + t.Run("an error on pausing is propagated", func(t *testing.T) { + var runner = &lib.MiniRunner{} + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + var sched, err = NewExecutionScheduler(runner, logger) + require.NoError(t, err) + var expectedErr = errors.New("testing pausable executor error") + sched.executors = []lib.Executor{pausableExecutor{err: expectedErr}} + err = sched.SetPaused(true) + require.Error(t, err) + require.Equal(t, err, expectedErr) + }) + + t.Run("can't pause unpausable executor", func(t *testing.T) { + var runner = &lib.MiniRunner{} + options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ + Iterations: null.IntFrom(2), + VUs: null.IntFrom(1), + }.Apply(runner.GetOptions())) + require.NoError(t, err) + require.NoError(t, runner.SetOptions(options)) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + sched, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + err = sched.SetPaused(true) + require.Error(t, err) + require.Contains(t, err.Error(), "doesn't support pause and resume operations after its start") + }) +} From 9c2b340bed394a233d3fa7b4c7b4777f0bb1273b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 9 Oct 2019 11:00:26 +0300 Subject: [PATCH 057/350] Add benchmark for getPlannedRateChanges and perf improvement The results are: benchmark old ns/op new ns/op delta BenchmarkGetPlannedRateChanges-4 4140227 2710011 -34.54% BenchmarkGetPlannedRateChanges-4 3531805 2397757 -32.11% BenchmarkGetPlannedRateChanges-4 3337196 2277776 -31.75% BenchmarkGetPlannedRateChanges-4 2984674 2192676 -26.54% BenchmarkGetPlannedRateChanges-4 2900849 2492187 -14.09% benchmark old allocs new allocs delta BenchmarkGetPlannedRateChanges-4 72977 56205 -22.98% BenchmarkGetPlannedRateChanges-4 72977 56205 -22.98% BenchmarkGetPlannedRateChanges-4 72977 56205 -22.98% BenchmarkGetPlannedRateChanges-4 72977 56205 -22.98% BenchmarkGetPlannedRateChanges-4 72977 56205 -22.98% benchmark old bytes new bytes delta BenchmarkGetPlannedRateChanges-4 1503784 1254588 -16.57% BenchmarkGetPlannedRateChanges-4 1503780 1254585 -16.57% BenchmarkGetPlannedRateChanges-4 1503780 1254588 -16.57% BenchmarkGetPlannedRateChanges-4 1503771 1254584 -16.57% BenchmarkGetPlannedRateChanges-4 1503775 1254588 -16.57% --- lib/executor/variable_arrival_rate.go | 4 +-- lib/executor/variable_arrival_rate_test.go | 31 ++++++++++++++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 86bea0230db..adf6747a7b6 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -214,16 +214,16 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi // nanosecond precision, so there isn't any actual loss of precision... stepNumber := (stageDuration / minIntervalBetweenRateAdjustments) if stepNumber > 1 { + rateDiff := new(big.Rat).Sub(stageTargetRate, currentRate) stepInterval := stageDuration / stepNumber for t := stepInterval; ; t += stepInterval { if stageDuration-t < minIntervalBetweenRateAdjustments { break } - rateDiff := new(big.Rat).Sub(stageTargetRate, currentRate) tArrivalRate := new(big.Rat).Add( currentRate, - rateDiff.Mul(rateDiff, big.NewRat(int64(t), int64(stageDuration))), + new(big.Rat).Mul(rateDiff, big.NewRat(int64(t), int64(stageDuration))), ) rateChanges = append(rateChanges, rateChange{ diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index e82c25942e8..d6f9eb214ca 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -168,6 +168,37 @@ func TestGetPlannedRateChanges(t *testing.T) { } } +func BenchmarkGetPlannedRateChanges(b *testing.B) { + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(5 * time.Minute), + Target: null.IntFrom(5000), + }, + { + Duration: types.NullDurationFrom(50 * time.Minute), + Target: null.IntFrom(5000), + }, + { + Duration: types.NullDurationFrom(5 * time.Minute), + Target: null.IntFrom(0), + }, + }, + } + + var es *lib.ExecutionSegment + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + changes := config.getPlannedRateChanges(es) + + require.Equal(b, time.Duration(0), + changes[0].timeOffset%minIntervalBetweenRateAdjustments, "%+v", changes[0]) + } + }) +} + func initializeVUs( ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, ) { From 769f811e2feab36f57618ee3777455a553f35f3e Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 11 Oct 2019 13:13:23 +0300 Subject: [PATCH 058/350] Limit appveyor test run to using 1 process --- appveyor.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/appveyor.yml b/appveyor.yml index 5b1700e4718..55f20a0e2e6 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -60,7 +60,7 @@ build_script: test_script: - cd %APPVEYOR_BUILD_FOLDER% - go version - - go test ./... + - go test -p 1 ./... deploy_script: - cd %APPVEYOR_BUILD_FOLDER%\packaging From d7be3a7e00458f902d5b045b2e24f79a1cdcf261 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 14 Oct 2019 15:46:19 +0300 Subject: [PATCH 059/350] Use big.Rat.Inv() in a helper function MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This also gives some performance improvements: name old time/op new time/op delta GetPlannedRateChanges-4 1.72ms ± 6% 1.66ms ± 7% ~ (p=0.222 n=5+5) name old alloc/op new alloc/op delta GetPlannedRateChanges-4 1.25MB ± 0% 1.22MB ± 0% -2.93% (p=0.008 n=5+5) name old allocs/op new allocs/op delta GetPlannedRateChanges-4 56.2k ± 0% 51.6k ± 0% -8.16% (p=0.008 n=5+5) --- lib/executor/helpers.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index d44bd7564a2..af0d3e3826d 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -201,7 +201,7 @@ func getTickerPeriod(scaledArrivalRate *big.Rat) types.NullDuration { // Basically, the ticker rate is time.Duration(1/arrivalRate). Considering // that time.Duration is represented as int64 nanoseconds, no meaningful // precision is likely to be lost here... - result, _ := new(big.Rat).SetFrac(scaledArrivalRate.Denom(), scaledArrivalRate.Num()).Float64() + result, _ := new(big.Rat).Inv(scaledArrivalRate).Float64() return types.NewNullDuration(time.Duration(result), true) } From a885b96618a94c24dafd8c086c8cd02c57383aaa Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 15 Oct 2019 11:52:59 +0300 Subject: [PATCH 060/350] Use big.Rat#Sign() and drop a global var --- lib/executor/helpers.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index af0d3e3826d..5b927184ee7 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -185,9 +185,6 @@ func getScaledArrivalRate(es *lib.ExecutionSegment, rate int64, period time.Dura return es.InPlaceScaleRat(big.NewRat(rate, int64(period))) } -// just a cached value to avoid allocating it every getTickerPeriod() call -var zero = big.NewInt(0) //nolint:gochecknoglobals - // getTickerPeriod is just a helper function that returns the ticker interval // we need for given arrival-rate parameters. // @@ -195,7 +192,7 @@ var zero = big.NewInt(0) //nolint:gochecknoglobals // and 0 isn't a valid ticker period. This happens so we don't divide by 0 when // the arrival-rate period is 0. This case has to be handled separately. func getTickerPeriod(scaledArrivalRate *big.Rat) types.NullDuration { - if scaledArrivalRate.Num().Cmp(zero) == 0 { + if scaledArrivalRate.Sign() == 0 { return types.NewNullDuration(0, false) } // Basically, the ticker rate is time.Duration(1/arrivalRate). Considering From d77c5dd07a26e4f759dde7aa75465231b2d7c896 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 15 Oct 2019 12:56:39 +0300 Subject: [PATCH 061/350] Reuse some big.Rat in getPlannedRateChanges MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetPlannedRateChanges-4 1.66ms ± 7% 1.34ms ± 2% -19.42% (p=0.008 n=5+5) name old alloc/op new alloc/op delta GetPlannedRateChanges-4 1.22MB ± 0% 0.86MB ± 0% -29.13% (p=0.008 n=5+5) name old allocs/op new allocs/op delta GetPlannedRateChanges-4 51.6k ± 0% 37.2k ± 0% -27.86% (p=0.008 n=5+5) --- lib/executor/variable_arrival_rate.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index adf6747a7b6..75938aadb88 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -183,6 +183,9 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi rateChanges := []rateChange{} timeFromStart := time.Duration(0) + var tArrivalRate = new(big.Rat) + var tArrivalRateStep = new(big.Rat) + var stepCoef = new(big.Rat) for _, stage := range varc.Stages { stageTargetRate := getScaledArrivalRate(segment, stage.Target.Int64, timeUnit) stageDuration := time.Duration(stage.Duration.Duration) @@ -221,9 +224,12 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi break } - tArrivalRate := new(big.Rat).Add( + tArrivalRate.Add( currentRate, - new(big.Rat).Mul(rateDiff, big.NewRat(int64(t), int64(stageDuration))), + tArrivalRateStep.Mul( + rateDiff, + stepCoef.SetFrac64(int64(t), int64(stageDuration)), + ), ) rateChanges = append(rateChanges, rateChange{ From 345ded631afd9e70029568d571082df270abaf28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 26 Nov 2019 15:21:32 +0100 Subject: [PATCH 062/350] test(executor): make executor setup generic This makes the setup function reusable for most (all?) executor tests. --- lib/executor/common_test.go | 53 ++++++++++++++++++++++ lib/executor/constant_arrival_rate_test.go | 52 ++++++++------------- lib/executor/variable_arrival_rate_test.go | 53 ++++------------------ 3 files changed, 79 insertions(+), 79 deletions(-) create mode 100644 lib/executor/common_test.go diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go new file mode 100644 index 00000000000..1d8f30836f7 --- /dev/null +++ b/lib/executor/common_test.go @@ -0,0 +1,53 @@ +package executor + +import ( + "context" + "io/ioutil" + "testing" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" +) + +func setupExecutor(t *testing.T, config lib.ExecutorConfig, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { + ctx, cancel := context.WithCancel(context.Background()) + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(ioutil.Discard) + logEntry := logrus.NewEntry(testLog) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + runner := lib.MiniRunner{ + Fn: vuFn, + } + + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { + return &lib.MiniRunnerVU{R: runner}, nil + }) + + initializeVUs(ctx, t, logEntry, es, 10) + + executor, err := config.NewExecutor(es, logEntry) + require.NoError(t, err) + err = executor.Init(ctx) + require.NoError(t, err) + return ctx, cancel, executor, logHook +} + +func initializeVUs( + ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, +) { + for i := 0; i < number; i++ { + require.EqualValues(t, i, es.GetInitializedVUsCount()) + vu, err := es.InitializeNewVU(ctx, logEntry) + require.NoError(t, err) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + es.ReturnVU(vu, false) + require.EqualValues(t, 0, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + } +} diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 964828a201d..74eeb0dfd68 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -2,7 +2,6 @@ package executor import ( "context" - "io/ioutil" "sync" "sync/atomic" "testing" @@ -17,43 +16,21 @@ import ( null "gopkg.in/guregu/null.v3" ) -func testConstantArrivalRateSetup(t *testing.T, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( - context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - ctx, cancel := context.WithCancel(context.Background()) - var config = ConstantArrivalRateConfig{ +func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { + return ConstantArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Rate: null.IntFrom(50), Duration: types.NullDurationFrom(5 * time.Second), PreAllocatedVUs: null.IntFrom(10), MaxVUs: null.IntFrom(20), } - logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} - testLog := logrus.New() - testLog.AddHook(logHook) - testLog.SetOutput(ioutil.Discard) - logEntry := logrus.NewEntry(testLog) - es := lib.NewExecutionState(lib.Options{}, 10, 50) - runner := lib.MiniRunner{ - Fn: vuFn, - } - - es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { - return &lib.MiniRunnerVU{R: runner}, nil - }) - - initializeVUs(ctx, t, logEntry, es, 10) - - executor, err := config.NewExecutor(es, logEntry) - require.NoError(t, err) - err = executor.Init(ctx) - require.NoError(t, err) - return ctx, cancel, executor, logHook } func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() - var ctx, cancel, executor, logHook = testConstantArrivalRateSetup( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestConstantArrivalRateConfig(), + func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(time.Second) return nil }) @@ -74,8 +51,9 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestConstantArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 - var ctx, cancel, executor, logHook = testConstantArrivalRateSetup( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestConstantArrivalRateConfig(), + func(ctx context.Context, out chan<- stats.SampleContainer) error { atomic.AddInt64(&count, 1) return nil }) @@ -103,11 +81,17 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { func TestArrivalRateCancel(t *testing.T) { t.Parallel() var mat = map[string]func( - t *testing.T, - vuFn func(context.Context, chan<- stats.SampleContainer) error, + *testing.T, + func(context.Context, chan<- stats.SampleContainer) error, ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook){ - "constant": testConstantArrivalRateSetup, - "variable": testVariableArrivalRateSetup, + "constant": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { + return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn) + }, + "variable": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { + return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn) + }, } for name, fn := range mat { fn := fn diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index d6f9eb214ca..9a1286d5690 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -2,7 +2,6 @@ package executor import ( "context" - "io/ioutil" "math/big" "sync" "sync/atomic" @@ -10,7 +9,6 @@ import ( "time" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" @@ -199,24 +197,8 @@ func BenchmarkGetPlannedRateChanges(b *testing.B) { }) } -func initializeVUs( - ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, -) { - for i := 0; i < number; i++ { - require.EqualValues(t, i, es.GetInitializedVUsCount()) - vu, err := es.InitializeNewVU(ctx, logEntry) - require.NoError(t, err) - require.EqualValues(t, i+1, es.GetInitializedVUsCount()) - es.ReturnVU(vu, false) - require.EqualValues(t, 0, es.GetCurrentlyActiveVUsCount()) - require.EqualValues(t, i+1, es.GetInitializedVUsCount()) - } -} - -func testVariableArrivalRateSetup(t *testing.T, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( - context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - ctx, cancel := context.WithCancel(context.Background()) - var config = VariableArrivalRateConfig{ +func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { + return VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(10), Stages: []Stage{ @@ -236,33 +218,13 @@ func testVariableArrivalRateSetup(t *testing.T, vuFn func(context.Context, chan< PreAllocatedVUs: null.IntFrom(10), MaxVUs: null.IntFrom(20), } - logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} - testLog := logrus.New() - testLog.AddHook(logHook) - testLog.SetOutput(ioutil.Discard) - logEntry := logrus.NewEntry(testLog) - es := lib.NewExecutionState(lib.Options{}, 10, 50) - runner := lib.MiniRunner{ - Fn: vuFn, - } - - es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { - return &lib.MiniRunnerVU{R: runner}, nil - }) - - initializeVUs(ctx, t, logEntry, es, 10) - - executor, err := config.NewExecutor(es, logEntry) - require.NoError(t, err) - err = executor.Init(ctx) - require.NoError(t, err) - return ctx, cancel, executor, logHook } func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() - var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestVariableArrivalRateConfig(), + func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(time.Second) return nil }) @@ -283,8 +245,9 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestVariableArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 - var ctx, cancel, executor, logHook = testVariableArrivalRateSetup( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestVariableArrivalRateConfig(), + func(ctx context.Context, out chan<- stats.SampleContainer) error { atomic.AddInt64(&count, 1) return nil }) From 5348dcc185c4503feaf8bb376cebeeb887006472 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 26 Nov 2019 18:18:16 +0100 Subject: [PATCH 063/350] test(executor): WIP TestPerVUIterations This also uses a logger to test functionality, which is a hack at best. We need a more reliable / testable approach... --- lib/executor/common_test.go | 14 +++--- lib/executor/constant_arrival_rate_test.go | 12 +++-- lib/executor/per_vu_iterations_test.go | 52 ++++++++++++++++++++++ lib/executor/variable_arrival_rate_test.go | 8 +++- lib/runner.go | 20 +++++++-- 5 files changed, 91 insertions(+), 15 deletions(-) create mode 100644 lib/executor/per_vu_iterations_test.go diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 1d8f30836f7..e735b25567b 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -3,6 +3,7 @@ package executor import ( "context" "io/ioutil" + "math/rand" "testing" "github.com/loadimpact/k6/lib" @@ -12,10 +13,13 @@ import ( "github.com/stretchr/testify/require" ) -func setupExecutor(t *testing.T, config lib.ExecutorConfig, vuFn func(context.Context, chan<- stats.SampleContainer) error) ( - context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { +func setupExecutor( + t *testing.T, config lib.ExecutorConfig, + vuFn func(context.Context, chan<- stats.SampleContainer) error, + logLevels []logrus.Level, +) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { ctx, cancel := context.WithCancel(context.Background()) - logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} + logHook := &testutils.SimpleLogrusHook{HookedLevels: logLevels} testLog := logrus.New() testLog.AddHook(logHook) testLog.SetOutput(ioutil.Discard) @@ -25,8 +29,8 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, vuFn func(context.Co Fn: vuFn, } - es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { - return &lib.MiniRunnerVU{R: runner}, nil + es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.VU, error) { + return &lib.MiniRunnerVU{R: runner, ID: rand.Int63(), Logger: logger}, nil }) initializeVUs(ctx, t, logEntry, es, 10) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 74eeb0dfd68..47ded84e0f8 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -33,7 +33,9 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(time.Second) return nil - }) + }, + []logrus.Level{logrus.WarnLevel}, + ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) err := executor.Run(ctx, engineOut) @@ -56,7 +58,9 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { func(ctx context.Context, out chan<- stats.SampleContainer) error { atomic.AddInt64(&count, 1) return nil - }) + }, + []logrus.Level{logrus.WarnLevel}, + ) defer cancel() var wg sync.WaitGroup wg.Add(1) @@ -86,11 +90,11 @@ func TestArrivalRateCancel(t *testing.T) { ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook){ "constant": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn) + return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn, []logrus.Level{logrus.WarnLevel}) }, "variable": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn) + return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn, []logrus.Level{logrus.WarnLevel}) }, } for name, fn := range mat { diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go new file mode 100644 index 00000000000..f2e2c4a312c --- /dev/null +++ b/lib/executor/per_vu_iterations_test.go @@ -0,0 +1,52 @@ +package executor + +import ( + "context" + "sync/atomic" + "testing" + "time" + + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" +) + +func getTestPerVUIterationsConfig() PerVUIterationsConfig { + return PerVUIterationsConfig{ + VUs: null.IntFrom(10), + Iterations: null.IntFrom(100), + MaxDuration: types.NullDurationFrom(5 * time.Second), + } +} + +func TestPerVUIterations(t *testing.T) { + t.Parallel() + doneIters := uint64(0) + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestPerVUIterationsConfig(), + func(ctx context.Context, out chan<- stats.SampleContainer) error { + atomic.AddUint64(&doneIters, 1) + return nil + }, + []logrus.Level{logrus.InfoLevel}, + ) + defer cancel() + err := executor.Run(ctx, nil) + require.NoError(t, err) + assert.Equal(t, uint64(1000), doneIters) + + entries := logHook.Drain() + require.NotEmpty(t, entries) + result := map[int64]uint64{} + for _, entry := range entries { + vuID := entry.Data["vu_id"].(int64) + result[vuID]++ + } + assert.Equal(t, 10, len(result)) + for _, vuIterCount := range result { + assert.Equal(t, uint64(100), vuIterCount) + } +} diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 9a1286d5690..aae4eea6ab2 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -227,7 +227,9 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(time.Second) return nil - }) + }, + []logrus.Level{logrus.WarnLevel}, + ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) err := executor.Run(ctx, engineOut) @@ -250,7 +252,9 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { func(ctx context.Context, out chan<- stats.SampleContainer) error { atomic.AddInt64(&count, 1) return nil - }) + }, + []logrus.Level{logrus.WarnLevel}, + ) defer cancel() var wg sync.WaitGroup wg.Add(1) diff --git a/lib/runner.go b/lib/runner.go index c7f4a62bd09..80f021f55ca 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -22,8 +22,10 @@ package lib import ( "context" + "math/rand" "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" ) // Ensure mock implementations conform to the interfaces. @@ -97,7 +99,7 @@ type MiniRunner struct { } func (r MiniRunner) VU(out chan<- stats.SampleContainer) *MiniRunnerVU { - return &MiniRunnerVU{R: r, Out: out} + return &MiniRunnerVU{R: r, Out: out, ID: rand.Int63()} } func (r MiniRunner) MakeArchive() *Archive { @@ -105,6 +107,9 @@ func (r MiniRunner) MakeArchive() *Archive { } func (r MiniRunner) NewVU(out chan<- stats.SampleContainer) (VU, error) { + // XXX: This method isn't called by the new executors. + // Confirm whether this was intentional and if other methods of + // the Runner interface are unused. return r.VU(out), nil } @@ -150,15 +155,22 @@ func (r *MiniRunner) SetOptions(opts Options) error { // A VU spawned by a MiniRunner. type MiniRunnerVU struct { - R MiniRunner - Out chan<- stats.SampleContainer - ID int64 + R MiniRunner + Out chan<- stats.SampleContainer + ID int64 + Logger *logrus.Entry } func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { if vu.R.Fn == nil { return nil } + // HACK: fixme, we shouldn't rely on logging for testing + if vu.Logger != nil { + vu.Logger.WithFields( + logrus.Fields{"vu_id": vu.ID}, + ).Info("running function") + } return vu.R.Fn(ctx, vu.Out) } From eda42f568cda135300adcf8a25e38ae937ec5899 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 28 Nov 2019 11:41:48 +0100 Subject: [PATCH 064/350] test(executor): set lib.State for MiniRunnerVU, improve TestPerVUIterations This models MiniRunnerVU closer to the way js.VU works, and is enough to cleanup TestPerVUIterations to avoid abuse of logging. There is no need to emit metrics in this case, since this is a lower level baseline test, and for higher level functional tests involving js.Runner we can use `__VU` or `__ITER` to track the result. --- lib/executor/common_test.go | 5 ++-- lib/executor/constant_arrival_rate_test.go | 6 ++-- lib/executor/per_vu_iterations_test.go | 33 ++++++++++------------ lib/executor/variable_arrival_rate_test.go | 2 -- lib/runner.go | 24 ++++++++-------- 5 files changed, 32 insertions(+), 38 deletions(-) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index e735b25567b..d062fe8eddb 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -16,10 +16,9 @@ import ( func setupExecutor( t *testing.T, config lib.ExecutorConfig, vuFn func(context.Context, chan<- stats.SampleContainer) error, - logLevels []logrus.Level, ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { ctx, cancel := context.WithCancel(context.Background()) - logHook := &testutils.SimpleLogrusHook{HookedLevels: logLevels} + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} testLog := logrus.New() testLog.AddHook(logHook) testLog.SetOutput(ioutil.Discard) @@ -30,7 +29,7 @@ func setupExecutor( } es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.VU, error) { - return &lib.MiniRunnerVU{R: runner, ID: rand.Int63(), Logger: logger}, nil + return &lib.MiniRunnerVU{R: runner, ID: rand.Int63()}, nil }) initializeVUs(ctx, t, logEntry, es, 10) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 47ded84e0f8..1f9ecc57585 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -34,7 +34,6 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { time.Sleep(time.Second) return nil }, - []logrus.Level{logrus.WarnLevel}, ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) @@ -59,7 +58,6 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { atomic.AddInt64(&count, 1) return nil }, - []logrus.Level{logrus.WarnLevel}, ) defer cancel() var wg sync.WaitGroup @@ -90,11 +88,11 @@ func TestArrivalRateCancel(t *testing.T) { ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook){ "constant": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn, []logrus.Level{logrus.WarnLevel}) + return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn) }, "variable": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn, []logrus.Level{logrus.WarnLevel}) + return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn) }, } for name, fn := range mat { diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index f2e2c4a312c..83800474f84 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -2,13 +2,13 @@ package executor import ( "context" - "sync/atomic" + "sync" "testing" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" @@ -24,29 +24,26 @@ func getTestPerVUIterationsConfig() PerVUIterationsConfig { func TestPerVUIterations(t *testing.T) { t.Parallel() - doneIters := uint64(0) - var ctx, cancel, executor, logHook = setupExecutor( + var result sync.Map + var ctx, cancel, executor, _ = setupExecutor( t, getTestPerVUIterationsConfig(), func(ctx context.Context, out chan<- stats.SampleContainer) error { - atomic.AddUint64(&doneIters, 1) + state := lib.GetState(ctx) + currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) + result.Store(state.Vu, currIter.(uint64)+1) return nil }, - []logrus.Level{logrus.InfoLevel}, ) defer cancel() err := executor.Run(ctx, nil) require.NoError(t, err) - assert.Equal(t, uint64(1000), doneIters) - entries := logHook.Drain() - require.NotEmpty(t, entries) - result := map[int64]uint64{} - for _, entry := range entries { - vuID := entry.Data["vu_id"].(int64) - result[vuID]++ - } - assert.Equal(t, 10, len(result)) - for _, vuIterCount := range result { - assert.Equal(t, uint64(100), vuIterCount) - } + var totalIters uint64 + result.Range(func(key, value interface{}) bool { + vuIters := value.(uint64) + assert.Equal(t, uint64(100), vuIters) + totalIters += vuIters + return true + }) + assert.Equal(t, uint64(1000), totalIters) } diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index aae4eea6ab2..1055482d992 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -228,7 +228,6 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { time.Sleep(time.Second) return nil }, - []logrus.Level{logrus.WarnLevel}, ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) @@ -253,7 +252,6 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { atomic.AddInt64(&count, 1) return nil }, - []logrus.Level{logrus.WarnLevel}, ) defer cancel() var wg sync.WaitGroup diff --git a/lib/runner.go b/lib/runner.go index 80f021f55ca..e0e5bdd3f8f 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -25,7 +25,6 @@ import ( "math/rand" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" ) // Ensure mock implementations conform to the interfaces. @@ -155,23 +154,26 @@ func (r *MiniRunner) SetOptions(opts Options) error { // A VU spawned by a MiniRunner. type MiniRunnerVU struct { - R MiniRunner - Out chan<- stats.SampleContainer - ID int64 - Logger *logrus.Entry + R MiniRunner + Out chan<- stats.SampleContainer + ID int64 + Iteration int64 } func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { if vu.R.Fn == nil { return nil } - // HACK: fixme, we shouldn't rely on logging for testing - if vu.Logger != nil { - vu.Logger.WithFields( - logrus.Fields{"vu_id": vu.ID}, - ).Info("running function") + + state := &State{ + Vu: vu.ID, + Iteration: vu.Iteration, } - return vu.R.Fn(ctx, vu.Out) + newctx := WithState(ctx, state) + + vu.Iteration++ + + return vu.R.Fn(newctx, vu.Out) } func (vu *MiniRunnerVU) Reconfigure(id int64) error { From 75db195a4af42671cc6176775e37d1342d03d72e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 22 Nov 2019 16:12:48 +0100 Subject: [PATCH 065/350] Fix typos --- lib/executor/variable_looping_vus.go | 14 +++++++------- lib/types/types.go | 2 +- ui/pb/helpers.go | 4 ++-- ui/pb/progressbar.go | 2 +- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 67e334be08e..884b53240cb 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -242,10 +242,10 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // // IMPORTANT: we have to be very careful of rounding errors, // both from the step duration and from the VUs. It's especially - // importatnt that the scaling via the execution segment should + // important that the scaling via the execution segment should // happen AFTER the rest of the calculations have been done and // we've rounded the global "global" number of VUs. - for t := stepInterval; ; t += stepInterval { // Skip step the first step, since we've already added that + for t := stepInterval; ; t += stepInterval { // Skip the first step, since we've already added that if time.Duration(abs(int64(stageDuration-t))) < minIntervalBetweenVUAdjustments { // Skip the last step of the stage, add it below to correct any minor clock skew break @@ -298,7 +298,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // finishing up interrupted iterations should be reused by the executor, // instead of new ones being requested from the execution state. // -// Here's an example with graceful ramp-town (i.e. "uninterruptible" +// Here's an example with graceful ramp-down (i.e. "uninterruptible" // iterations), where stars represent actively scheduled VUs and dots are used // for VUs that are potentially finishing up iterations: // @@ -325,7 +325,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // The algorithm we use below to reserve VUs so that ramping-down VUs can finish // their last iterations is pretty simple. It just traverses the raw execution // steps and whenever there's a scaling down of VUs, it prevents the number of -// VUs from faliing down for the configured gracefulRampDown period. +// VUs from decreasing for the configured gracefulRampDown period. // // Finishing up the test, i.e. making sure we have a step with 0 VUs at time // executorEndOffset, is not handled here. Instead GetExecutionRequirements() @@ -341,7 +341,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu lastPlannedVUs := uint64(0) for rawStepNum := 0; rawStepNum < rawStepsLen; rawStepNum++ { rawStep := rawSteps[rawStepNum] - // Add the first step or any step where the number of planned VUs us + // Add the first step or any step where the number of planned VUs is // greater than the ones in the previous step. We don't need to worry // about reserving time for ramping-down VUs when the number of planned // VUs is growing. That's because the gracefulRampDown period is a fixed @@ -358,7 +358,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu continue } - // If we're here, we have a downward "slope" - thelastPlannedVUs are + // If we're here, we have a downward "slope" - the lastPlannedVUs are // more than the current rawStep's planned VUs. We're going to look // forward in time (up to gracefulRampDown) and inspect the rawSteps. // There are a 3 possibilities: @@ -475,7 +475,7 @@ var _ lib.Executor = &VariableLoopingVUs{} // // TODO: split up? since this does a ton of things, unfortunately I can't think // of a less complex way to implement it (besides the old "increment by 100ms -// and see what happens)... :/ so maybe see how it can be spit? +// and see what happens)... :/ so maybe see how it can be split? // nolint:funlen,gocognit func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := vlv.executionState.Options.ExecutionSegment diff --git a/lib/types/types.go b/lib/types/types.go index d0a82746b03..158ea2744b0 100644 --- a/lib/types/types.go +++ b/lib/types/types.go @@ -85,7 +85,7 @@ func NullDecoder(f reflect.Type, t reflect.Type, data interface{}) (interface{}, return data, nil } -//TODO: something better that won't reuire so much boilerplate and casts for NullDuration values... +//TODO: something better that won't require so much boilerplate and casts for NullDuration values... // Duration is an alias for time.Duration that de/serialises to JSON as human-readable strings. type Duration time.Duration diff --git a/ui/pb/helpers.go b/ui/pb/helpers.go index 751d635d7ad..af2ea23b2a1 100644 --- a/ui/pb/helpers.go +++ b/ui/pb/helpers.go @@ -43,7 +43,7 @@ func GetFixedLengthIntFormat(maxValue int64) (formatStr string) { } // GetFixedLengthFloatFormat returns "%0__.__f" format argument for fmt -// functions that will produce a base-10 right-aligned zero-padded string +// functions that will produce a base-10 right-aligned zero-padded string // representation of the supplied float value, with the specified decimal // precision. The number of characters (i.e. the actual number + maybe dot and // precision + how many zeros it will be padded on the left with) in the @@ -67,7 +67,7 @@ func GetFixedLengthFloatFormat(maxValue float64, precision uint) (formatStr stri // GetFixedLengthDuration takes a *positive* duration and its max value and // returns a string with a fixed width so we can prevent UI elements jumping // around. The format is "___d__h__m__s.s", but leading values can be omitted -// based on the maxDuration value, the results can be: "___h__m__s.s" +// based on the maxDuration value, the results can be: "___h__m__s.s". // // This is code was inspired by the Go stdlib's time.Duration.String() code. // TODO: more flexibility - negative values or variable precision? diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 457edf0ea1f..ebbcad13051 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -99,7 +99,7 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // String locks the progressbar struct for reading and calls all of its methods // to assemble the progress bar and return it as a string. -//TODO: something prettier? paddings, right-alighment of the left column, line trimming by terminal size +//TODO: something prettier? paddings, right-alignment of the left column, line trimming by terminal size func (pb *ProgressBar) String() string { pb.mutex.RLock() defer pb.mutex.RUnlock() From 46cba7b51b67524f1eb53b0ca1eba05ddcb177c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 13 Dec 2019 16:20:55 +0100 Subject: [PATCH 066/350] Fix minor comment issues Resolves: - https://github.com/loadimpact/k6/pull/1007#discussion_r349524994 - https://github.com/loadimpact/k6/pull/1007#discussion_r349544387 --- lib/executor/variable_looping_vus.go | 7 ++++--- lib/executor/vu_handle.go | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 884b53240cb..8fe851030c5 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -28,12 +28,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const variableLoopingVUsType = "variable-looping-vus" @@ -429,7 +430,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu // added after it will be ignored. Thus: // - gracefulStop can be less than gracefulRampDown and can cut the graceful // ramp-down periods of the last VUs short. -// - gracefulRampDown can be more than gracefulRampDown: +// - gracefulRampDown can be more than gracefulStop: // - If the user manually ramped down VUs at the end of the test (i.e. the // last stage's target is 0), then this will have no effect. // - If the last stage's target is more than 0, the VUs at the end of the diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 42376b5c86a..dafb9c89658 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -91,7 +91,7 @@ func (vh *vuHandle) hardStop() { vh.logger.Debugf("Hard stop") vh.cancel() // cancel the previous context vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) // create a new context - select { // if needed, + select { case <-vh.canStartIter: vh.canStartIter = make(chan struct{}) default: From a397344b9aeafa30b731998c489ab715be30ae56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 13 Dec 2019 16:22:40 +0100 Subject: [PATCH 067/350] Fix lib.Executor interface check for SharedIterations Resolves https://github.com/loadimpact/k6/pull/1007#discussion_r350311364 --- lib/executor/shared_iterations.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index a4029b0f842..9ca50f6a98c 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -27,12 +27,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const sharedIterationsType = "shared-iterations" @@ -145,7 +146,7 @@ type SharedIterations struct { } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &PerVUIterations{} +var _ lib.Executor = &SharedIterations{} // Run executes a specific total number of iterations, which are all shared by // the configured VUs. From 15df310252edc8798a0e379c81f5e5429e5ffd7b Mon Sep 17 00:00:00 2001 From: Cuong Manh Le Date: Thu, 12 Dec 2019 15:40:33 +0700 Subject: [PATCH 068/350] Fixing minor comments in vu handle --- lib/executor/vu_handle.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index dafb9c89658..c97e221096a 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -68,7 +68,7 @@ func newStoppedVUHandle( func (vh *vuHandle) start() { vh.mutex.Lock() - vh.logger.Debugf("Start") + vh.logger.Debug("Start") close(vh.canStartIter) vh.mutex.Unlock() } @@ -78,25 +78,23 @@ func (vh *vuHandle) gracefulStop() { select { case <-vh.canStartIter: vh.canStartIter = make(chan struct{}) - vh.logger.Debugf("Graceful stop") + vh.logger.Debug("Graceful stop") default: - // do nothing, the signalling channel was already closed by either - // hardStop() or gracefulStop() + // do nothing, the signalling channel was already initialized by hardStop() } vh.mutex.Unlock() } func (vh *vuHandle) hardStop() { vh.mutex.Lock() - vh.logger.Debugf("Hard stop") + vh.logger.Debug("Hard stop") vh.cancel() // cancel the previous context vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) // create a new context select { case <-vh.canStartIter: vh.canStartIter = make(chan struct{}) default: - // do nothing, the signalling channel was already closed by either - // hardStop() or gracefulStop() + // do nothing, the signalling channel was already initialized by gracefulStop() } vh.mutex.Unlock() } @@ -110,7 +108,6 @@ func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.VU)) { defer func() { if vu != nil { vh.returnVU(vu) - vu = nil } }() @@ -120,13 +117,14 @@ mainLoop: canStartIter, ctx := vh.canStartIter, vh.ctx vh.mutex.RUnlock() - // Wait for either the executor to be done, or for us to be unpaused + // Wait for either the executor to be done, or for us to be un-paused select { case <-canStartIter: // Best case, we're currently running, so we do nothing here, we // just continue straight ahead. case <-executorDone: - return // The whole executor is done, nothing more to do. + // The whole executor is done, nothing more to do. + return default: // We're not running, but the executor isn't done yet, so we wait // for either one of those conditions. But before that, we'll return @@ -143,7 +141,8 @@ mainLoop: // context and signal channel continue mainLoop case <-executorDone: - return // The whole executor is done, nothing more to do. + // The whole executor is done, nothing more to do. + return } } @@ -155,7 +154,8 @@ mainLoop: default: } - if vu == nil { // Ensure we have a VU + // Ensure we have a VU + if vu == nil { freshVU, err := vh.getVU() if err != nil { return From f6be35cfa006c56954d6821e0f94f4863f063e45 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 13 Dec 2019 14:23:26 +0200 Subject: [PATCH 069/350] Move MiniRunner to testutils and simplify executor test setup MiniRunner was also slightly improved to not use random VU IDs, so it matches the JS runner more closely. --- api/server_test.go | 3 +- api/v1/group_routes_test.go | 3 +- api/v1/metric_routes_test.go | 5 +- api/v1/status_routes_test.go | 3 +- cmd/config_consolidation_test.go | 2 +- core/engine_test.go | 6 +- core/local/local_test.go | 50 ++++---- lib/executor/common_test.go | 24 ++-- lib/executor/constant_arrival_rate_test.go | 47 +++---- lib/{ => executor}/execution_test.go | 23 ++-- lib/executor/per_vu_iterations_test.go | 5 +- lib/executor/variable_arrival_rate_test.go | 8 +- lib/runner.go | 102 --------------- lib/testutils/mini_runner.go | 137 +++++++++++++++++++++ 14 files changed, 227 insertions(+), 191 deletions(-) rename lib/{ => executor}/execution_test.go (88%) create mode 100644 lib/testutils/mini_runner.go diff --git a/api/server_test.go b/api/server_test.go index adade23f848..25128a1de8d 100644 --- a/api/server_test.go +++ b/api/server_test.go @@ -36,6 +36,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" ) func testHTTPHandler(rw http.ResponseWriter, r *http.Request) { @@ -76,7 +77,7 @@ func TestLogger(t *testing.T) { } func TestWithEngine(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index cf8d8487c3d..7b9241556a1 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -29,6 +29,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -43,7 +44,7 @@ func TestGetGroups(t *testing.T) { g2, err := g1.Group("group 2") assert.NoError(t, err) - execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{Group: g0}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{Group: g0}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/metric_routes_test.go b/api/v1/metric_routes_test.go index 7d18a2117c6..b276567ba99 100644 --- a/api/v1/metric_routes_test.go +++ b/api/v1/metric_routes_test.go @@ -33,6 +33,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/stats" "github.com/manyminds/api2go/jsonapi" "github.com/stretchr/testify/assert" @@ -40,7 +41,7 @@ import ( ) func TestGetMetrics(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) @@ -81,7 +82,7 @@ func TestGetMetrics(t *testing.T) { } func TestGetMetric(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index 25163ae25b6..a4dae2dfbc0 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -29,6 +29,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -36,7 +37,7 @@ import ( ) func TestGetStatus(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&lib.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 053c0731e84..fd8105159a2 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -452,7 +452,7 @@ func runTestCase( var runner lib.Runner if testCase.options.runner != nil { - runner = &lib.MiniRunner{Options: *testCase.options.runner} + runner = &testutils.MiniRunner{Options: *testCase.options.runner} } if testCase.options.fs == nil { t.Logf("Creating an empty FS for this test") diff --git a/core/engine_test.go b/core/engine_test.go index 960c79813f2..f0e2c911252 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -51,7 +51,7 @@ const isWindows = runtime.GOOS == "windows" // Wrapper around NewEngine that applies a logger and manages the options. func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts lib.Options) *Engine { //nolint: golint if runner == nil { - runner = &lib.MiniRunner{} + runner = &testutils.MiniRunner{} } if ctx == nil { ctx = context.Background() @@ -109,7 +109,7 @@ func TestEngineRun(t *testing.T) { signalChan := make(chan interface{}) - runner := &lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + runner := &testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { stats.PushIfNotDone(ctx, out, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1}) close(signalChan) <-ctx.Done() @@ -158,7 +158,7 @@ func TestEngineAtTime(t *testing.T) { func TestEngineCollector(t *testing.T) { testMetric := stats.New("test_metric", stats.Trend) - runner := &lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + runner := &testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { out <- stats.Sample{Metric: testMetric} return nil }} diff --git a/core/local/local_test.go b/core/local/local_test.go index fe72f72d405..1a06d1f2214 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -50,7 +50,7 @@ func newTestExecutionScheduler( t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, //nolint: golint ) (ctx context.Context, cancel func(), execScheduler *ExecutionScheduler, samples chan stats.SampleContainer) { if runner == nil { - runner = &lib.MiniRunner{} + runner = &testutils.MiniRunner{} } ctx, cancel = context.WithCancel(context.Background()) newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ @@ -100,7 +100,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Run("Normal", func(t *testing.T) { setupC := make(chan struct{}) teardownC := make(chan struct{}) - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { close(setupC) return nil, nil @@ -120,7 +120,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.NoError(t, <-err) }) t.Run("Setup Error", func(t *testing.T) { - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, errors.New("setup error") }, @@ -130,7 +130,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.EqualError(t, execScheduler.Run(ctx, samples), "setup error") }) t.Run("Don't Run Setup", func(t *testing.T) { - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, errors.New("setup error") }, @@ -148,7 +148,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { }) t.Run("Teardown Error", func(t *testing.T) { - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, nil }, @@ -165,7 +165,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") }) t.Run("Don't Run Teardown", func(t *testing.T) { - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, nil }, @@ -212,7 +212,7 @@ func TestExecutionSchedulerStages(t *testing.T) { data := data t.Run(name, func(t *testing.T) { t.Parallel() - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(100 * time.Millisecond) return nil @@ -231,7 +231,7 @@ func TestExecutionSchedulerStages(t *testing.T) { func TestExecutionSchedulerEndTime(t *testing.T) { t.Parallel() - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(100 * time.Millisecond) return nil @@ -256,7 +256,7 @@ func TestExecutionSchedulerEndTime(t *testing.T) { func TestExecutionSchedulerRuntimeErrors(t *testing.T) { t.Parallel() - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(10 * time.Millisecond) return errors.New("hi") @@ -294,7 +294,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { exec.Duration = types.NullDurationFrom(1 * time.Second) exec.GracefulStop = types.NullDurationFrom(0 * time.Second) - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { <-ctx.Done() return errors.New("hi") @@ -332,7 +332,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { require.Empty(t, options.Validate()) var i int64 - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { select { case <-ctx.Done(): @@ -371,7 +371,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { func TestExecutionSchedulerIsRunning(t *testing.T) { t.Parallel() - runner := &lib.MiniRunner{ + runner := &testutils.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { <-ctx.Done() return nil @@ -404,7 +404,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { }) t.Run("Raise", func(t *testing.T) { - e := New(&lib.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + e := New(&testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return nil }}) e.ctx = context.Background() @@ -416,7 +416,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { num++ if assert.NotNil(t, handle.vu, "vu %d lacks impl", i) { - assert.Equal(t, int64(0), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(0), handle.vu.(*testutils.MiniRunnerVU).ID) } assert.Nil(t, handle.ctx, "vu %d has ctx", i) assert.Nil(t, handle.cancel, "vu %d has cancel", i) @@ -431,11 +431,11 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { if i < 50 { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) num++ } else { assert.Nil(t, handle.cancel, "vu %d has cancel", i) - assert.Equal(t, int64(0), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(0), handle.vu.(*testutils.MiniRunnerVU).ID) } } assert.Equal(t, 50, num) @@ -447,7 +447,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { num := 0 for i, handle := range e.vus { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) num++ } assert.Equal(t, 100, num) @@ -465,7 +465,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { } else { assert.Nil(t, handle.cancel, "vu %d has cancel", i) } - assert.Equal(t, int64(i+1), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) } assert.Equal(t, 50, num) } @@ -477,9 +477,9 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) if i < 50 { - assert.Equal(t, int64(i+1), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) } else { - assert.Equal(t, int64(50+i+1), handle.vu.(*lib.MiniRunnerVU).ID) + assert.Equal(t, int64(50+i+1), handle.vu.(*testutils.MiniRunnerVU).ID) } } } @@ -658,7 +658,7 @@ func (p pausableExecutor) SetPaused(bool) error { func TestSetPaused(t *testing.T) { t.Run("second pause is an error", func(t *testing.T) { - var runner = &lib.MiniRunner{} + var runner = &testutils.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) var sched, err = NewExecutionScheduler(runner, logger) @@ -672,7 +672,7 @@ func TestSetPaused(t *testing.T) { }) t.Run("unpause at the start is an error", func(t *testing.T) { - var runner = &lib.MiniRunner{} + var runner = &testutils.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) var sched, err = NewExecutionScheduler(runner, logger) @@ -684,7 +684,7 @@ func TestSetPaused(t *testing.T) { }) t.Run("second unpause is an error", func(t *testing.T) { - var runner = &lib.MiniRunner{} + var runner = &testutils.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) var sched, err = NewExecutionScheduler(runner, logger) @@ -698,7 +698,7 @@ func TestSetPaused(t *testing.T) { }) t.Run("an error on pausing is propagated", func(t *testing.T) { - var runner = &lib.MiniRunner{} + var runner = &testutils.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) var sched, err = NewExecutionScheduler(runner, logger) @@ -711,7 +711,7 @@ func TestSetPaused(t *testing.T) { }) t.Run("can't pause unpausable executor", func(t *testing.T) { - var runner = &lib.MiniRunner{} + var runner = &testutils.MiniRunner{} options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ Iterations: null.IntFrom(2), VUs: null.IntFrom(1), diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index d062fe8eddb..eb606cd7f2e 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -3,7 +3,6 @@ package executor import ( "context" "io/ioutil" - "math/rand" "testing" "github.com/loadimpact/k6/lib" @@ -13,23 +12,29 @@ import ( "github.com/stretchr/testify/require" ) -func setupExecutor( - t *testing.T, config lib.ExecutorConfig, - vuFn func(context.Context, chan<- stats.SampleContainer) error, -) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { +func simpleRunner(vuFn func(context.Context) error) lib.Runner { + return &testutils.MiniRunner{ + Fn: func(ctx context.Context, _ chan<- stats.SampleContainer) error { + return vuFn(ctx) + }, + } +} + +func setupExecutor(t *testing.T, config lib.ExecutorConfig, runner lib.Runner) ( + context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook, +) { ctx, cancel := context.WithCancel(context.Background()) + engineOut := make(chan stats.SampleContainer, 100) //TODO: return this for more complicated tests? + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} testLog := logrus.New() testLog.AddHook(logHook) testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) es := lib.NewExecutionState(lib.Options{}, 10, 50) - runner := lib.MiniRunner{ - Fn: vuFn, - } es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.VU, error) { - return &lib.MiniRunnerVU{R: runner, ID: rand.Int63()}, nil + return runner.NewVU(engineOut) }) initializeVUs(ctx, t, logEntry, es, 10) @@ -44,6 +49,7 @@ func setupExecutor( func initializeVUs( ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, ) { + // This is not how the local ExecutionScheduler initializes VUs, but should do the same job for i := 0; i < number; i++ { require.EqualValues(t, i, es.GetInitializedVUsCount()) vu, err := es.InitializeNewVU(ctx, logEntry) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 1f9ecc57585..d7f98a00427 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -8,7 +8,6 @@ import ( "time" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" @@ -30,10 +29,10 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() var ctx, cancel, executor, logHook = setupExecutor( t, getTestConstantArrivalRateConfig(), - func(ctx context.Context, out chan<- stats.SampleContainer) error { + simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) return nil - }, + }), ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) @@ -54,10 +53,10 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { var count int64 var ctx, cancel, executor, logHook = setupExecutor( t, getTestConstantArrivalRateConfig(), - func(ctx context.Context, out chan<- stats.SampleContainer) error { + simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) return nil - }, + }), ) defer cancel() var wg sync.WaitGroup @@ -82,34 +81,26 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { func TestArrivalRateCancel(t *testing.T) { t.Parallel() - var mat = map[string]func( - *testing.T, - func(context.Context, chan<- stats.SampleContainer) error, - ) (context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook){ - "constant": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( - context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestConstantArrivalRateConfig(), vuFn) - }, - "variable": func(t *testing.T, vuFn func(ctx context.Context, out chan<- stats.SampleContainer) error) ( - context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook) { - return setupExecutor(t, getTestVariableArrivalRateConfig(), vuFn) - }, + + var testCases = map[string]lib.ExecutorConfig{ + "constant": getTestConstantArrivalRateConfig(), + "variable": getTestVariableArrivalRateConfig(), } - for name, fn := range mat { - fn := fn + for name, config := range testCases { + config := config t.Run(name, func(t *testing.T) { + t.Parallel() var ch = make(chan struct{}) var errCh = make(chan error, 1) var weAreDoneCh = make(chan struct{}) - var ctx, cancel, executor, logHook = fn( - t, func(ctx context.Context, out chan<- stats.SampleContainer) error { - select { - case <-ch: - <-ch - default: - } - return nil - }) + var ctx, cancel, executor, logHook = setupExecutor(t, config, simpleRunner(func(ctx context.Context) error { + select { + case <-ch: + <-ch + default: + } + return nil + })) defer cancel() var wg sync.WaitGroup wg.Add(1) diff --git a/lib/execution_test.go b/lib/executor/execution_test.go similarity index 88% rename from lib/execution_test.go rename to lib/executor/execution_test.go index 147f9008488..37770b57be5 100644 --- a/lib/execution_test.go +++ b/lib/executor/execution_test.go @@ -18,7 +18,7 @@ * */ -package lib +package executor import ( "context" @@ -28,6 +28,7 @@ import ( "testing" "time" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -36,7 +37,7 @@ import ( func TestExecutionStateVUIDs(t *testing.T) { t.Parallel() - es := NewExecutionState(Options{}, 0, 0) + es := lib.NewExecutionState(lib.Options{}, 0, 0) assert.Equal(t, uint64(1), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(2), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(3), es.GetUniqueVUIdentifier()) @@ -56,7 +57,7 @@ func TestExecutionStateVUIDs(t *testing.T) { func TestExecutionStateGettingVUsWhenNonAreAvailable(t *testing.T) { t.Parallel() - es := NewExecutionState(Options{}, 0, 0) + es := lib.NewExecutionState(lib.Options{}, 0, 0) logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} testLog := logrus.New() testLog.AddHook(logHook) @@ -66,7 +67,7 @@ func TestExecutionStateGettingVUsWhenNonAreAvailable(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "could not get a VU from the buffer in") entries := logHook.Drain() - require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + require.Equal(t, lib.MaxRetriesGetPlannedVU, len(entries)) for _, entry := range entries { require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") } @@ -80,9 +81,9 @@ func TestExecutionStateGettingVUs(t *testing.T) { testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) - es := NewExecutionState(Options{}, 10, 20) - es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (VU, error) { - return &MiniRunnerVU{}, nil + es := lib.NewExecutionState(lib.Options{}, 10, 20) + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { + return &testutils.MiniRunnerVU{}, nil }) for i := 0; i < 10; i++ { @@ -112,7 +113,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "could not get a VU from the buffer in") entries := logHook.Drain() - require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + require.Equal(t, lib.MaxRetriesGetPlannedVU, len(entries)) for _, entry := range entries { require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") } @@ -134,7 +135,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "could not get a VU from the buffer in") entries = logHook.Drain() - require.Equal(t, MaxRetriesGetPlannedVU, len(entries)) + require.Equal(t, lib.MaxRetriesGetPlannedVU, len(entries)) for _, entry := range entries { require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") } @@ -142,7 +143,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { func TestMarkStartedPanicsOnSecondRun(t *testing.T) { t.Parallel() - es := NewExecutionState(Options{}, 0, 0) + es := lib.NewExecutionState(lib.Options{}, 0, 0) require.False(t, es.HasStarted()) es.MarkStarted() require.True(t, es.HasStarted()) @@ -151,7 +152,7 @@ func TestMarkStartedPanicsOnSecondRun(t *testing.T) { func TestMarkEnded(t *testing.T) { t.Parallel() - es := NewExecutionState(Options{}, 0, 0) + es := lib.NewExecutionState(lib.Options{}, 0, 0) require.False(t, es.HasEnded()) es.MarkEnded() require.True(t, es.HasEnded()) diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 83800474f84..2a4393140d3 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -8,7 +8,6 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" @@ -27,12 +26,12 @@ func TestPerVUIterations(t *testing.T) { var result sync.Map var ctx, cancel, executor, _ = setupExecutor( t, getTestPerVUIterationsConfig(), - func(ctx context.Context, out chan<- stats.SampleContainer) error { + simpleRunner(func(ctx context.Context) error { state := lib.GetState(ctx) currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) result.Store(state.Vu, currIter.(uint64)+1) return nil - }, + }), ) defer cancel() err := executor.Run(ctx, nil) diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 1055482d992..b0e4f4b6a52 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -224,10 +224,10 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() var ctx, cancel, executor, logHook = setupExecutor( t, getTestVariableArrivalRateConfig(), - func(ctx context.Context, out chan<- stats.SampleContainer) error { + simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) return nil - }, + }), ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) @@ -248,10 +248,10 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { var count int64 var ctx, cancel, executor, logHook = setupExecutor( t, getTestVariableArrivalRateConfig(), - func(ctx context.Context, out chan<- stats.SampleContainer) error { + simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) return nil - }, + }), ) defer cancel() var wg sync.WaitGroup diff --git a/lib/runner.go b/lib/runner.go index e0e5bdd3f8f..41b68285198 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -22,15 +22,10 @@ package lib import ( "context" - "math/rand" "github.com/loadimpact/k6/stats" ) -// 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. The Runner doesn't actually *do* anything in @@ -83,100 +78,3 @@ type VU interface { //TODO: support reconfiguring of env vars, tags and exec Reconfigure(id int64) error } - -// MiniRunner wraps a function in a runner whose VUs will simply call that function. -//TODO: move to testutils, or somewhere else that's not lib... -type MiniRunner struct { - Fn func(ctx context.Context, out chan<- stats.SampleContainer) error - SetupFn func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) - TeardownFn func(ctx context.Context, out chan<- stats.SampleContainer) error - - setupData []byte - - Group *Group - Options Options -} - -func (r MiniRunner) VU(out chan<- stats.SampleContainer) *MiniRunnerVU { - return &MiniRunnerVU{R: r, Out: out, ID: rand.Int63()} -} - -func (r MiniRunner) MakeArchive() *Archive { - return nil -} - -func (r MiniRunner) NewVU(out chan<- stats.SampleContainer) (VU, error) { - // XXX: This method isn't called by the new executors. - // Confirm whether this was intentional and if other methods of - // the Runner interface are unused. - return r.VU(out), nil -} - -func (r *MiniRunner) Setup(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - if fn := r.SetupFn; fn != nil { - r.setupData, err = fn(ctx, out) - } - return -} - -// GetSetupData returns json representation of the setup data if setup() is specified and run, nil otherwise -func (r MiniRunner) GetSetupData() []byte { - return r.setupData -} - -// SetSetupData saves the externally supplied setup data as json in the runner -func (r *MiniRunner) SetSetupData(data []byte) { - r.setupData = data -} - -func (r MiniRunner) Teardown(ctx context.Context, out chan<- stats.SampleContainer) error { - if fn := r.TeardownFn; fn != nil { - return fn(ctx, out) - } - return nil -} - -func (r MiniRunner) GetDefaultGroup() *Group { - if r.Group == nil { - r.Group = &Group{} - } - return r.Group -} - -func (r MiniRunner) GetOptions() Options { - return r.Options -} - -func (r *MiniRunner) SetOptions(opts Options) error { - r.Options = opts - return nil -} - -// A VU spawned by a MiniRunner. -type MiniRunnerVU struct { - R MiniRunner - Out chan<- stats.SampleContainer - ID int64 - Iteration int64 -} - -func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { - if vu.R.Fn == nil { - return nil - } - - state := &State{ - Vu: vu.ID, - Iteration: vu.Iteration, - } - newctx := WithState(ctx, state) - - vu.Iteration++ - - return vu.R.Fn(newctx, vu.Out) -} - -func (vu *MiniRunnerVU) Reconfigure(id int64) error { - vu.ID = id - return nil -} diff --git a/lib/testutils/mini_runner.go b/lib/testutils/mini_runner.go new file mode 100644 index 00000000000..11f6681f499 --- /dev/null +++ b/lib/testutils/mini_runner.go @@ -0,0 +1,137 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package testutils + +import ( + "context" + "sync/atomic" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" +) + +// Ensure mock implementations conform to the interfaces. +var _ lib.Runner = &MiniRunner{} +var _ lib.VU = &MiniRunnerVU{} + +// MiniRunner partially implements the lib.Runner interface, but instead of +// using a real JS runtime, it allows us to directly specify the options and +// functions with Go code. +type MiniRunner struct { + Fn func(ctx context.Context, out chan<- stats.SampleContainer) error + SetupFn func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) + TeardownFn func(ctx context.Context, out chan<- stats.SampleContainer) error + + SetupData []byte + + NextVUID int64 + Group *lib.Group + Options lib.Options +} + +// MakeArchive isn't implemented, it always returns nil and is just here to +// satisfy the lib.Runner interface. +func (r MiniRunner) MakeArchive() *lib.Archive { + return nil +} + +// NewVU returns a new MiniRunnerVU with an incremental ID. +func (r *MiniRunner) NewVU(out chan<- stats.SampleContainer) (lib.VU, error) { + nextVUNum := atomic.AddInt64(&r.NextVUID, 1) + return &MiniRunnerVU{R: r, Out: out, ID: nextVUNum - 1}, nil +} + +// Setup calls the supplied mock setup() function, if present. +func (r *MiniRunner) Setup(ctx context.Context, out chan<- stats.SampleContainer) (err error) { + if fn := r.SetupFn; fn != nil { + r.SetupData, err = fn(ctx, out) + } + return +} + +// GetSetupData returns json representation of the setup data if setup() is +// specified and was ran, nil otherwise. +func (r MiniRunner) GetSetupData() []byte { + return r.SetupData +} + +// SetSetupData saves the externally supplied setup data as JSON in the runner. +func (r *MiniRunner) SetSetupData(data []byte) { + r.SetupData = data +} + +// Teardown calls the supplied mock teardown() function, if present. +func (r MiniRunner) Teardown(ctx context.Context, out chan<- stats.SampleContainer) error { + if fn := r.TeardownFn; fn != nil { + return fn(ctx, out) + } + return nil +} + +// GetDefaultGroup returns the default group. +func (r MiniRunner) GetDefaultGroup() *lib.Group { + if r.Group == nil { + r.Group = &lib.Group{} + } + return r.Group +} + +// GetOptions returns the supplied options struct. +func (r MiniRunner) GetOptions() lib.Options { + return r.Options +} + +// SetOptions allows you to override the runner options. +func (r *MiniRunner) SetOptions(opts lib.Options) error { + r.Options = opts + return nil +} + +// MiniRunnerVU is a mock VU, spawned by a MiniRunner. +type MiniRunnerVU struct { + R *MiniRunner + Out chan<- stats.SampleContainer + ID int64 + Iteration int64 +} + +// RunOnce runs the mock default function once, incrementing its iteration. +func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { + if vu.R.Fn == nil { + return nil + } + + state := &lib.State{ + Vu: vu.ID, + Iteration: vu.Iteration, + } + newctx := lib.WithState(ctx, state) + + vu.Iteration++ + + return vu.R.Fn(newctx, vu.Out) +} + +// Reconfigure changes the VU ID. +func (vu *MiniRunnerVU) Reconfigure(id int64) error { + vu.ID = id + return nil +} From 80429f21c334f80a929025cc8980807840232869 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 17 Dec 2019 15:56:35 +0100 Subject: [PATCH 070/350] Fix typos --- lib/execution.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/lib/execution.go b/lib/execution.go index 79f6d5e90b0..28d879f52cb 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -29,8 +29,9 @@ import ( "github.com/sirupsen/logrus" - "github.com/loadimpact/k6/stats" "github.com/pkg/errors" + + "github.com/loadimpact/k6/stats" ) // An ExecutionScheduler is in charge of initializing executors and using them @@ -117,7 +118,7 @@ const MaxRetriesGetPlannedVU = 5 // races, because the Go data race detector can't detect any data races // involving atomics... // -// The only functionality indended for synchronization is the one revolving +// The only functionality intended for synchronization is the one revolving // around pausing, and uninitializedUnplannedVUs for restricting the number of // unplanned VUs being initialized. type ExecutionState struct { From f47d04d27fe9c1686c08027596dde3e8f9dc9cfd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 18 Dec 2019 16:51:35 +0100 Subject: [PATCH 071/350] Rename sched -> exec This seems leftover from when the feature was referred to as new-schedulers, so I found "sched" confusing at first glance. --- lib/executor/helpers.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 5b927184ee7..59cf6a219c1 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -152,13 +152,13 @@ func getDurationContexts(parentCtx context.Context, regularDuration, gracefulSto // executor and updates its progressbar accordingly. func trackProgress( parentCtx, maxDurationCtx, regDurationCtx context.Context, - sched lib.Executor, snapshot func() (float64, string), + exec lib.Executor, snapshot func() (float64, string), ) { - progressBar := sched.GetProgress() - logger := sched.GetLogger() + progressBar := exec.GetProgress() + logger := exec.GetLogger() <-regDurationCtx.Done() // Wait for the regular context to be over - gracefulStop := sched.GetConfig().GetGracefulStop() + gracefulStop := exec.GetConfig().GetGracefulStop() if parentCtx.Err() == nil && gracefulStop > 0 { p, right := snapshot() logger.WithField("gracefulStop", gracefulStop).Debug( From 4597f7ab128c5d900d6bf7d61e91b6a45a974377 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 18 Dec 2019 18:23:08 +0100 Subject: [PATCH 072/350] Refactor ExternallyControlled executor to embed BaseExecutor This aligns it with all other executors, and centralizes where progress bars are created, for example. --- lib/executor/externally_controlled.go | 31 +++++++++++---------------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index baa3c216a71..0cb2c03e44c 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -29,12 +29,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const externallyControlledType = "externally-controlled" @@ -167,16 +168,13 @@ func (ExternallyControlledConfig) IsDistributable() bool { // NewExecutor creates a new ExternallyControlled executor func (mec ExternallyControlledConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return &ExternallyControlled{ - startConfig: mec, + BaseExecutor: NewBaseExecutor(mec, es, logger), + config: mec, currentControlConfig: mec.ExternallyControlledConfigParams, configLock: &sync.RWMutex{}, newControlConfigs: make(chan updateConfigEvent), pauseEvents: make(chan pauseEvent), hasStarted: make(chan struct{}), - - executionState: es, - logger: logger, - progress: pb.New(pb.WithLeft(mec.GetName)), }, nil } @@ -194,16 +192,13 @@ type updateConfigEvent struct { // controlled externally, via the k6 REST API. It implements both the // lib.PausableExecutor and the lib.LiveUpdatableExecutor interfaces. type ExternallyControlled struct { - startConfig ExternallyControlledConfig + *BaseExecutor + config ExternallyControlledConfig currentControlConfig ExternallyControlledConfigParams configLock *sync.RWMutex newControlConfigs chan updateConfigEvent pauseEvents chan pauseEvent hasStarted chan struct{} - - executionState *lib.ExecutionState - logger *logrus.Entry - progress *pb.ProgressBar } // Make sure we implement all the interfaces @@ -216,7 +211,7 @@ func (mex *ExternallyControlled) GetCurrentConfig() ExternallyControlledConfig { mex.configLock.RLock() defer mex.configLock.RUnlock() return ExternallyControlledConfig{ - BaseConfig: mex.startConfig.BaseConfig, + BaseConfig: mex.config.BaseConfig, ExternallyControlledConfigParams: mex.currentControlConfig, } } @@ -267,17 +262,17 @@ func (mex *ExternallyControlled) UpdateConfig(ctx context.Context, newConf inter return fmt.Errorf("invalid configuration supplied: %s", lib.ConcatErrors(errs, ", ")) } - if newConfigParams.Duration.Valid && newConfigParams.Duration != mex.startConfig.Duration { + if newConfigParams.Duration.Valid && newConfigParams.Duration != mex.config.Duration { return fmt.Errorf("the externally controlled executor duration cannot be changed") } - if newConfigParams.MaxVUs.Valid && newConfigParams.MaxVUs.Int64 < mex.startConfig.MaxVUs.Int64 { + if newConfigParams.MaxVUs.Valid && newConfigParams.MaxVUs.Int64 < mex.config.MaxVUs.Int64 { // This limitation is because the externally controlled executor is // still an executor that participates in the overall k6 scheduling. // Thus, any VUs that were explicitly specified by the user in the // config may be reused from or by other executors. return fmt.Errorf( "the new number of max VUs cannot be lower than the starting number of max VUs (%d)", - mex.startConfig.MaxVUs.Int64, + mex.config.MaxVUs.Int64, ) } @@ -497,7 +492,7 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats logrus.Fields{"type": externallyControlledType, "duration": duration}, ).Debug("Starting executor run...") - startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.startConfig.MaxVUs.Int64) + startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.config.MaxVUs.Int64) runState := &externallyControlledRunState{ ctx: ctx, executor: mex, @@ -518,7 +513,7 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats go trackProgress(parentCtx, ctx, ctx, mex, runState.progresFn) err = runState.handleConfigChange( // Start by setting MaxVUs to the starting MaxVUs - ExternallyControlledConfigParams{MaxVUs: mex.startConfig.MaxVUs}, currentControlConfig, + ExternallyControlledConfigParams{MaxVUs: mex.config.MaxVUs}, currentControlConfig, ) if err != nil { return err From 01e7c2ad3142f7545320188be9bad4eaffa28188 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 19 Dec 2019 19:12:12 +0100 Subject: [PATCH 073/350] Refactor run cmd to use printBar() --- cmd/run.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 8dae15a52ee..a14c31e1876 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -97,7 +97,7 @@ a commandline interface for interacting with it.`, initBar := pb.New(pb.WithConstLeft(" init")) // Create the Runner. - fprintf(stdout, "%s runner\r", initBar.String()) //TODO use printBar() + printBar(initBar, "runner") pwd, err := os.Getwd() if err != nil { return err @@ -119,7 +119,7 @@ a commandline interface for interacting with it.`, return err } - fprintf(stdout, "%s options\r", initBar.String()) + printBar(initBar, "options") cliConf, err := getConfig(cmd.Flags()) if err != nil { @@ -147,7 +147,7 @@ a commandline interface for interacting with it.`, defer cancel() // Create a local execution scheduler wrapping the runner. - fprintf(stdout, "%s execution scheduler\r", initBar.String()) + printBar(initBar, "execution scheduler") execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err From 4add1b6060cac32ad3be8a17330ef7a2bd041299 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 19 Dec 2019 19:06:42 +0100 Subject: [PATCH 074/350] Fix progressbar left-side alignment This breaks the Stringer interface, so we should either abandon it or have String() wrap the rendering methods. I'm leaning towards abandoning it in favor of the current String() being renamed to Render(), since it's unlikely we'll use progressbars as a Stringer. --- cmd/ui.go | 12 ++++++++++-- ui/pb/progressbar.go | 31 +++++++++++++++++++++++-------- 2 files changed, 33 insertions(+), 10 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 17163d139eb..1e163438f52 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -74,7 +74,7 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.String(), rightText, end) + fprintf(stdout, "%s %s%s", bar.String(0), rightText, end) } func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { @@ -84,11 +84,19 @@ func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { lineEnd = "\x1b[K\n" // erase till end of line } + var leftPad int + for _, pb := range pbs { + l := pb.Left() + if len(l) > leftPad { + leftPad = len(l) + } + } + pbsCount := len(pbs) result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.String() + lineEnd + result[i+1] = pb.String(leftPad) + lineEnd } if isTTY && goBack { // Go back to the beginning diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index ebbcad13051..6390aa3f161 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -88,6 +88,23 @@ func New(options ...ProgressBarOption) *ProgressBar { return pb } +// Left returns the left part of the progressbar in a thread-safe way. +func (pb *ProgressBar) Left() string { + pb.mutex.RLock() + defer pb.mutex.RUnlock() + + return pb.renderLeft(0) +} + +func (pb *ProgressBar) renderLeft(pad int) string { + var left string + if pb.left != nil { + padFmt := fmt.Sprintf("%%-%ds", pad) + left = fmt.Sprintf(padFmt, pb.left()) + } + return left +} + // Modify changes the progressbar options in a thread-safe way. func (pb *ProgressBar) Modify(options ...ProgressBarOption) { pb.mutex.Lock() @@ -100,7 +117,7 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // String locks the progressbar struct for reading and calls all of its methods // to assemble the progress bar and return it as a string. //TODO: something prettier? paddings, right-alignment of the left column, line trimming by terminal size -func (pb *ProgressBar) String() string { +func (pb *ProgressBar) String(leftPad int) string { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -108,12 +125,10 @@ func (pb *ProgressBar) String() string { return pb.hijack() } - var left, right string - if pb.left != nil { - left = pb.left() + " " - } - - var progress float64 + var ( + progress float64 + right string + ) if pb.progress != nil { progress, right = pb.progress() right = " " + right @@ -138,5 +153,5 @@ func (pb *ProgressBar) String() string { padding = pb.color.Sprint(strings.Repeat("-", space-filled)) } - return fmt.Sprintf("%s[%s%s%s]%s", left, filling, caret, padding, right) + return fmt.Sprintf("%s [%s%s%s]%s", pb.renderLeft(leftPad), filling, caret, padding, right) } From 9dba77e4a62825a4bcb07069b4562a4302bb3557 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 20 Dec 2019 12:12:20 +0100 Subject: [PATCH 075/350] Add progressbar tests --- ui/pb/progressbar_test.go | 107 +++++++++++++++++++++++++++++++++++++- 1 file changed, 106 insertions(+), 1 deletion(-) diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index d36af9b0d3d..77c242d5ad2 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -20,4 +20,109 @@ package pb -//TODO +import ( + "strings" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestProgressBarRender(t *testing.T) { + t.Parallel() + + testCases := []struct { + options []ProgressBarOption + expected string + }{ + {[]ProgressBarOption{WithLeft(func() string { return "left" })}, + "left [--------------------------------------]"}, + {[]ProgressBarOption{WithConstLeft("constLeft")}, + "constLeft [--------------------------------------]"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, string) { return 0, "right" }), + }, + "left [--------------------------------------] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, string) { return 0.5, "right" }), + }, + "left [==================>-------------------] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, string) { return 1.0, "right" }), + }, + "left [======================================] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, string) { return -1, "right" }), + }, + "left [" + strings.Repeat("-", 76) + "] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, string) { return 2, "right" }), + }, + "left [" + strings.Repeat("=", 76) + "] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithConstProgress(0.2, "constProgress"), + }, + "left [======>-------------------------------] constProgress"}, + {[]ProgressBarOption{ + WithHijack(func() string { return "progressbar hijack!" }), + }, + "progressbar hijack!"}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.expected, func(t *testing.T) { + pbar := New(tc.options...) + assert.NotNil(t, pbar) + assert.Equal(t, tc.expected, pbar.String(0)) + }) + } +} + +func TestProgressBarRenderPaddingLeft(t *testing.T) { + t.Parallel() + + testCases := []struct { + padding int + expected string + }{ + {-1, "left [--------------------------------------]"}, + {0, "left [--------------------------------------]"}, + {10, "left [--------------------------------------]"}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.expected, func(t *testing.T) { + pbar := New(WithLeft(func() string { return "left" })) + assert.NotNil(t, pbar) + assert.Equal(t, tc.expected, pbar.String(tc.padding)) + }) + } +} + +func TestProgressBarLeft(t *testing.T) { + t.Parallel() + + testCases := []struct { + left func() string + expected string + }{ + {nil, ""}, + {func() string { return " left " }, " left "}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.expected, func(t *testing.T) { + pbar := New(WithLeft(tc.left)) + assert.NotNil(t, pbar) + assert.Equal(t, tc.expected, pbar.Left()) + }) + } +} From 00268a847a5942bc2ebe8bb1d9fca06a228c5cd7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 20 Dec 2019 14:39:33 +0100 Subject: [PATCH 076/350] Rename ProgressBar.String -> ProgressBar.Render It doesn't make sense to keep calling this String() since it breaks the fmt.Stringer interface because of the padding argument, and more arguments will need to be added to control the rendering. I suppose we could still implement Stringer and have it call Render() with some sane defaults, but I don't think we have a use case for that API, and we can add it if needed. --- cmd/ui.go | 4 ++-- ui/pb/progressbar.go | 6 +++--- ui/pb/progressbar_test.go | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 1e163438f52..9eec1884579 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -74,7 +74,7 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.String(0), rightText, end) + fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) } func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { @@ -96,7 +96,7 @@ func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.String(leftPad) + lineEnd + result[i+1] = pb.Render(leftPad) + lineEnd } if isTTY && goBack { // Go back to the beginning diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 6390aa3f161..0a8627d5ab3 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -114,10 +114,10 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { } } -// String locks the progressbar struct for reading and calls all of its methods +// Render locks the progressbar struct for reading and calls all of its methods // to assemble the progress bar and return it as a string. -//TODO: something prettier? paddings, right-alignment of the left column, line trimming by terminal size -func (pb *ProgressBar) String(leftPad int) string { +// - leftPad sets the padding between the left text and the opening square bracket. +func (pb *ProgressBar) Render(leftPad int) string { pb.mutex.RLock() defer pb.mutex.RUnlock() diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index 77c242d5ad2..a0d7d71eabf 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -79,7 +79,7 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.String(0)) + assert.Equal(t, tc.expected, pbar.Render(0)) }) } } @@ -101,7 +101,7 @@ func TestProgressBarRenderPaddingLeft(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(WithLeft(func() string { return "left" })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.String(tc.padding)) + assert.Equal(t, tc.expected, pbar.Render(tc.padding)) }) } } From a1c894e1210b67d80c301b9fecbdbc173b283c92 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 20 Dec 2019 15:54:20 +0100 Subject: [PATCH 077/350] Trim left progressbar text if exceeds max. length The global value will need to be configurable, but that's something to be tested in cmd/ui_test.go, and is not related to these progressbar changes. --- cmd/ui.go | 10 ++++++++-- ui/pb/progressbar.go | 24 ++++++++++++++++++------ ui/pb/progressbar_test.go | 23 ++++++++++++++--------- 3 files changed, 40 insertions(+), 17 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 9eec1884579..372931ee888 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -30,9 +30,13 @@ import ( "time" "github.com/loadimpact/k6/core/local" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/ui/pb" ) +// TODO: Make configurable +const maxLeftLength = 30 + // A writer that syncs writes with a mutex and, if the output is a TTY, clears before newlines. type consoleWriter struct { Writer io.Writer @@ -74,7 +78,7 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) + fprintf(stdout, "%s %s%s", bar.Render(0, 0), rightText, end) } func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { @@ -91,12 +95,14 @@ func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { leftPad = len(l) } } + // Floor padding to maximum left text length + leftPad = int(lib.Min(int64(leftPad), maxLeftLength)) pbsCount := len(pbs) result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.Render(leftPad) + lineEnd + result[i+1] = pb.Render(leftPad, maxLeftLength) + lineEnd } if isTTY && goBack { // Go back to the beginning diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 0a8627d5ab3..ebeec38feae 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -93,14 +93,21 @@ func (pb *ProgressBar) Left() string { pb.mutex.RLock() defer pb.mutex.RUnlock() - return pb.renderLeft(0) + return pb.renderLeft(0, 0) } -func (pb *ProgressBar) renderLeft(pad int) string { +// renderLeft renders the left part of the progressbar, applying the +// given padding and trimming text exceeding maxLen length, +// replacing it with an ellipsis. +func (pb *ProgressBar) renderLeft(pad, maxLen int) string { var left string if pb.left != nil { + l := pb.left() + if maxLen > 0 && len(l) > maxLen { + l = l[:maxLen-3] + "..." + } padFmt := fmt.Sprintf("%%-%ds", pad) - left = fmt.Sprintf(padFmt, pb.left()) + left = fmt.Sprintf(padFmt, l) } return left } @@ -116,8 +123,12 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // Render locks the progressbar struct for reading and calls all of its methods // to assemble the progress bar and return it as a string. -// - leftPad sets the padding between the left text and the opening square bracket. -func (pb *ProgressBar) Render(leftPad int) string { +// - leftPad sets the padding between the left text and the opening +// square bracket. +// - leftMax sets the maximum character length of the left text. +// Characters exceeding this length will be replaced with a single ellipsis. +// Passing <=0 disables trimming. +func (pb *ProgressBar) Render(leftPad, leftMax int) string { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -153,5 +164,6 @@ func (pb *ProgressBar) Render(leftPad int) string { padding = pb.color.Sprint(strings.Repeat("-", space-filled)) } - return fmt.Sprintf("%s [%s%s%s]%s", pb.renderLeft(leftPad), filling, caret, padding, right) + return fmt.Sprintf("%s [%s%s%s]%s", + pb.renderLeft(leftPad, leftMax), filling, caret, padding, right) } diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index a0d7d71eabf..a55a5396253 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -79,29 +79,34 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0)) + assert.Equal(t, tc.expected, pbar.Render(0, 0)) }) } } -func TestProgressBarRenderPaddingLeft(t *testing.T) { +func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Parallel() - testCases := []struct { padding int + maxLen int + left string expected string }{ - {-1, "left [--------------------------------------]"}, - {0, "left [--------------------------------------]"}, - {10, "left [--------------------------------------]"}, + {-1, 0, "left", "left [--------------------------------------]"}, + {0, 0, "left", "left [--------------------------------------]"}, + {10, 0, "left", "left [--------------------------------------]"}, + {0, 10, "left_truncated", + "left_tr... [--------------------------------------]"}, + {20, 10, "left_truncated_padding", + "left_tr... [--------------------------------------]"}, } for _, tc := range testCases { tc := tc - t.Run(tc.expected, func(t *testing.T) { - pbar := New(WithLeft(func() string { return "left" })) + t.Run(tc.left, func(t *testing.T) { + pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.padding)) + assert.Equal(t, tc.expected, pbar.Render(tc.padding, tc.maxLen)) }) } } From eece282be18eba0d45f98ee54d21d8e8f63ef0aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 10 Jan 2020 17:34:07 +0100 Subject: [PATCH 078/350] Calculate max left length only once, reuse padding argument Resolves: - https://github.com/loadimpact/k6/pull/1286#pullrequestreview-339549503 - https://github.com/loadimpact/k6/pull/1286#discussion_r363983577 --- cmd/ui.go | 35 +++++++++++++++++++---------------- ui/pb/progressbar.go | 19 +++++++++---------- ui/pb/progressbar_test.go | 15 ++++++--------- 3 files changed, 34 insertions(+), 35 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 372931ee888..324c2fb45c1 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -78,31 +78,21 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.Render(0, 0), rightText, end) + fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) } -func renderMultipleBars(isTTY, goBack bool, pbs []*pb.ProgressBar) string { +func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) string { lineEnd := "\n" if isTTY { //TODO: check for cross platform support lineEnd = "\x1b[K\n" // erase till end of line } - var leftPad int - for _, pb := range pbs { - l := pb.Left() - if len(l) > leftPad { - leftPad = len(l) - } - } - // Floor padding to maximum left text length - leftPad = int(lib.Min(int64(leftPad), maxLeftLength)) - pbsCount := len(pbs) result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.Render(leftPad, maxLeftLength) + lineEnd + result[i+1] = pb.Render(leftMax) + lineEnd } if isTTY && goBack { // Go back to the beginning @@ -127,8 +117,21 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi pbs = append(pbs, s.GetProgress()) } + // Get the longest left side string length, to align progress bars + // horizontally and trim excess text. + var leftLen int + for _, pb := range pbs { + l := pb.Left() + if len(l) > leftLen { + leftLen = len(l) + } + } + + // Limit to maximum left text length + leftLen = int(lib.Min(int64(leftLen), maxLeftLength)) + // For flicker-free progressbars! - progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, pbs)) + progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs)) progressBarsPrint := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } @@ -149,7 +152,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi stderr.PersistentText = nil if ctx.Err() != nil { // Render a last plain-text progressbar in an error - progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, pbs)) + progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs)) progressBarsPrint() } outMutex.Unlock() @@ -161,7 +164,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi for { select { case <-ticker.C: - barText := renderMultipleBars(stdoutTTY, true, pbs) + barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs) outMutex.Lock() progressBarsLastRender = []byte(barText) progressBarsPrint() diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index ebeec38feae..225c592700e 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -93,20 +93,20 @@ func (pb *ProgressBar) Left() string { pb.mutex.RLock() defer pb.mutex.RUnlock() - return pb.renderLeft(0, 0) + return pb.renderLeft(0) } // renderLeft renders the left part of the progressbar, applying the // given padding and trimming text exceeding maxLen length, // replacing it with an ellipsis. -func (pb *ProgressBar) renderLeft(pad, maxLen int) string { +func (pb *ProgressBar) renderLeft(maxLen int) string { var left string if pb.left != nil { l := pb.left() if maxLen > 0 && len(l) > maxLen { l = l[:maxLen-3] + "..." } - padFmt := fmt.Sprintf("%%-%ds", pad) + padFmt := fmt.Sprintf("%%-%ds", maxLen) left = fmt.Sprintf(padFmt, l) } return left @@ -123,12 +123,11 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // Render locks the progressbar struct for reading and calls all of its methods // to assemble the progress bar and return it as a string. -// - leftPad sets the padding between the left text and the opening -// square bracket. -// - leftMax sets the maximum character length of the left text. -// Characters exceeding this length will be replaced with a single ellipsis. -// Passing <=0 disables trimming. -func (pb *ProgressBar) Render(leftPad, leftMax int) string { +// - leftMax defines the maximum character length of the left-side +// text, as well as the padding between the text and the opening +// square bracket. Characters exceeding this length will be replaced +// with a single ellipsis. Passing <=0 disables this. +func (pb *ProgressBar) Render(leftMax int) string { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -165,5 +164,5 @@ func (pb *ProgressBar) Render(leftPad, leftMax int) string { } return fmt.Sprintf("%s [%s%s%s]%s", - pb.renderLeft(leftPad, leftMax), filling, caret, padding, right) + pb.renderLeft(leftMax), filling, caret, padding, right) } diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index a55a5396253..bc88cc6f1b8 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -79,7 +79,7 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0, 0)) + assert.Equal(t, tc.expected, pbar.Render(0)) }) } } @@ -87,18 +87,15 @@ func TestProgressBarRender(t *testing.T) { func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Parallel() testCases := []struct { - padding int maxLen int left string expected string }{ - {-1, 0, "left", "left [--------------------------------------]"}, - {0, 0, "left", "left [--------------------------------------]"}, - {10, 0, "left", "left [--------------------------------------]"}, - {0, 10, "left_truncated", + {-1, "left", "left [--------------------------------------]"}, + {0, "left", "left [--------------------------------------]"}, + {10, "left", "left [--------------------------------------]"}, + {10, "left_truncated", "left_tr... [--------------------------------------]"}, - {20, 10, "left_truncated_padding", - "left_tr... [--------------------------------------]"}, } for _, tc := range testCases { @@ -106,7 +103,7 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Run(tc.left, func(t *testing.T) { pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.padding, tc.maxLen)) + assert.Equal(t, tc.expected, pbar.Render(tc.maxLen)) }) } } From f9d9d611d756c41c5565967f62b402dd96bf4478 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 10 Jan 2020 18:54:58 +0100 Subject: [PATCH 079/350] Clamp progress value between 0 and 1, log warning Resolves https://github.com/loadimpact/k6/pull/1286#discussion_r360812402 --- cmd/cloud.go | 20 +++++++++++--------- cmd/run.go | 14 +++++++------- cmd/ui.go | 22 ++++++++++++++-------- ui/pb/helpers.go | 13 +++++++++++++ ui/pb/progressbar.go | 10 +++++++++- ui/pb/progressbar_test.go | 22 ++++++++++++++++------ 6 files changed, 70 insertions(+), 31 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index d80dd204612..039c4230331 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -65,11 +65,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud k6 cloud script.js`[1:], Args: exactArgsWithMsg(1, "arg should either be \"-\", if reading script from stdin, or a path to a script file"), RunE: func(cmd *cobra.Command, args []string) error { + logger := logrus.StandardLogger() + //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) progressBar := pb.New(pb.WithConstLeft(" Init")) - printBar(progressBar, "Parsing script") + printBar(progressBar, "Parsing script", logger) // Runner pwd, err := os.Getwd() @@ -89,13 +91,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - printBar(progressBar, "Getting script options") + printBar(progressBar, "Getting script options", logger) r, err := newRunner(src, runType, filesystems, runtimeOptions) if err != nil { return err } - printBar(progressBar, "Consolidating options") + printBar(progressBar, "Consolidating options", logger) cliOpts, err := getOptions(cmd.Flags()) if err != nil { return err @@ -128,7 +130,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return errors.New("Not logged in, please use `k6 login cloud`.") } - printBar(progressBar, "Building the archive") + printBar(progressBar, "Building the archive", logger) arc := r.MakeArchive() // TODO: Fix this // We reuse cloud.Config for parsing options.ext.loadimpact, but this probably shouldn't be @@ -177,19 +179,19 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud } // Start cloud test run - printBar(progressBar, "Validating script options") + printBar(progressBar, "Validating script options", logger) client := cloud.NewClient(cloudConfig.Token.String, cloudConfig.Host.String, consts.Version) if err := client.ValidateOptions(arc.Options); err != nil { return err } - printBar(progressBar, "Uploading archive") + printBar(progressBar, "Uploading archive", logger) refID, err := client.StartCloudTestRun(name, cloudConfig.ProjectID.Int64, arc) if err != nil { return err } progressBar.Modify(pb.WithConstLeft(" Run")) - printBar(progressBar, "Initializing the cloud test") + printBar(progressBar, "Initializing the cloud test", logger) testURL := cloud.URLForResults(refID, cloudConfig) fprintf(stdout, "\n\n") @@ -198,7 +200,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) //TODO: print executors information fprintf(stdout, "\n") - printBar(progressBar, "Initializing the cloud test") + printBar(progressBar, "Initializing the cloud test", logger) // The quiet option hides the progress bar and disallow aborting the test if quiet { @@ -234,7 +236,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } - printBar(progressBar, "") + printBar(progressBar, "", logger) } else { logrus.WithError(progressErr).Error("Test progress error") } diff --git a/cmd/run.go b/cmd/run.go index a14c31e1876..73ab968e574 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -91,13 +91,16 @@ a commandline interface for interacting with it.`, k6 run -o influxdb=http://1.2.3.4:8086/k6`[1:], Args: exactArgsWithMsg(1, "arg should either be \"-\", if reading script from stdin, or a path to a script file"), RunE: func(cmd *cobra.Command, args []string) error { + //TODO: don't use a global... or maybe change the logger? + logger := logrus.StandardLogger() + //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) initBar := pb.New(pb.WithConstLeft(" init")) // Create the Runner. - printBar(initBar, "runner") + printBar(initBar, "runner", logger) pwd, err := os.Getwd() if err != nil { return err @@ -119,7 +122,7 @@ a commandline interface for interacting with it.`, return err } - printBar(initBar, "options") + printBar(initBar, "options", logger) cliConf, err := getConfig(cmd.Flags()) if err != nil { @@ -140,14 +143,11 @@ a commandline interface for interacting with it.`, return err } - //TODO: don't use a global... or maybe change the logger? - logger := logrus.StandardLogger() - ctx, cancel := context.WithCancel(context.Background()) //TODO: move even earlier? defer cancel() // Create a local execution scheduler wrapping the runner. - printBar(initBar, "execution scheduler") + printBar(initBar, "execution scheduler", logger) execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err @@ -158,7 +158,7 @@ a commandline interface for interacting with it.`, progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) go func() { - showProgress(ctx, conf, execScheduler) + showProgress(ctx, conf, execScheduler, logger) progressBarWG.Done() }() diff --git a/cmd/ui.go b/cmd/ui.go index 324c2fb45c1..c4fdaa008d1 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -29,6 +29,8 @@ import ( "sync" "time" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/ui/pb" @@ -68,7 +70,7 @@ func (w *consoleWriter) Write(p []byte) (n int, err error) { return origLen, err } -func printBar(bar *pb.ProgressBar, rightText string) { +func printBar(bar *pb.ProgressBar, rightText string, logger *logrus.Logger) { end := "\n" if stdout.IsTTY { // If we're in a TTY, instead of printing the bar and going to the next @@ -78,10 +80,11 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) + fprintf(stdout, "%s %s%s", bar.Render(0, logger), rightText, end) } -func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) string { +func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar, + logger *logrus.Logger) string { lineEnd := "\n" if isTTY { //TODO: check for cross platform support @@ -92,7 +95,7 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.Render(leftMax) + lineEnd + result[i+1] = pb.Render(leftMax, logger) + lineEnd } if isTTY && goBack { // Go back to the beginning @@ -107,7 +110,10 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) //TODO: show other information here? //TODO: add a no-progress option that will disable these //TODO: don't use global variables... -func showProgress(ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler) { +func showProgress( + ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler, + logger *logrus.Logger, +) { if quiet || conf.HTTPDebug.Valid && conf.HTTPDebug.String != "" { return } @@ -131,7 +137,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi leftLen = int(lib.Min(int64(leftLen), maxLeftLength)) // For flicker-free progressbars! - progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs)) + progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs, logger)) progressBarsPrint := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } @@ -152,7 +158,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi stderr.PersistentText = nil if ctx.Err() != nil { // Render a last plain-text progressbar in an error - progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs)) + progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs, logger)) progressBarsPrint() } outMutex.Unlock() @@ -164,7 +170,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi for { select { case <-ticker.C: - barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs) + barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs, logger) outMutex.Lock() progressBarsLastRender = []byte(barText) progressBarsPrint() diff --git a/ui/pb/helpers.go b/ui/pb/helpers.go index af2ea23b2a1..58edadae094 100644 --- a/ui/pb/helpers.go +++ b/ui/pb/helpers.go @@ -129,3 +129,16 @@ func GetFixedLengthDuration(d, maxDuration time.Duration) (result string) { return string(buf[i:]) } + +// Clampf returns the given value, "clamped" to the range [min, max]. +// This is copied from lib/util.go to avoid circular imports. +func Clampf(val, min, max float64) float64 { + switch { + case val < min: + return min + case val > max: + return max + default: + return val + } +} diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 225c592700e..78a84777c1e 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -26,6 +26,7 @@ import ( "sync" "github.com/fatih/color" + "github.com/sirupsen/logrus" ) const defaultWidth = 40 @@ -127,7 +128,7 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // text, as well as the padding between the text and the opening // square bracket. Characters exceeding this length will be replaced // with a single ellipsis. Passing <=0 disables this. -func (pb *ProgressBar) Render(leftMax int) string { +func (pb *ProgressBar) Render(leftMax int, logger *logrus.Logger) string { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -142,6 +143,13 @@ func (pb *ProgressBar) Render(leftMax int) string { if pb.progress != nil { progress, right = pb.progress() right = " " + right + progressClamped := Clampf(progress, 0, 1) + if progress != progressClamped { + if logger != nil { + logger.Warnf("progress value %.2f exceeds valid range, clamped between 0 and 1", progress) + } + progress = progressClamped + } } space := pb.width - 2 diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index bc88cc6f1b8..9d4d4a59892 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -21,12 +21,20 @@ package pb import ( - "strings" "testing" "github.com/stretchr/testify/assert" ) +// XXX: This introduces an import cycle: pb -> lib -> pb +// func getTestLogger() *logger.Entry { +// logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} +// testLog := logrus.New() +// testLog.AddHook(logHook) +// testLog.SetOutput(ioutil.Discard) +// return logrus.NewEntry(testLog) +// } + func TestProgressBarRender(t *testing.T) { t.Parallel() @@ -57,12 +65,12 @@ func TestProgressBarRender(t *testing.T) { WithLeft(func() string { return "left" }), WithProgress(func() (float64, string) { return -1, "right" }), }, - "left [" + strings.Repeat("-", 76) + "] right"}, + "left [--------------------------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, string) { return 2, "right" }), }, - "left [" + strings.Repeat("=", 76) + "] right"}, + "left [======================================] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithConstProgress(0.2, "constProgress"), @@ -79,13 +87,14 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0)) + assert.Equal(t, tc.expected, pbar.Render(0, nil)) }) } } func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Parallel() + testCases := []struct { maxLen int left string @@ -93,7 +102,8 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { }{ {-1, "left", "left [--------------------------------------]"}, {0, "left", "left [--------------------------------------]"}, - {10, "left", "left [--------------------------------------]"}, + {15, "left_pad", + "left_pad [--------------------------------------]"}, {10, "left_truncated", "left_tr... [--------------------------------------]"}, } @@ -103,7 +113,7 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Run(tc.left, func(t *testing.T) { pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.maxLen)) + assert.Equal(t, tc.expected, pbar.Render(tc.maxLen, nil)) }) } } From e3ac2e0393fa662743d72cef8269bc2d0f1f1f4d Mon Sep 17 00:00:00 2001 From: Cuong Manh Le Date: Tue, 14 Jan 2020 17:00:35 +0700 Subject: [PATCH 080/350] Exclude executors without work (#1307) This fixes https://github.com/loadimpact/k6/issues/1295 by not initializing any of the executors that would have no work for the particular execution segment used in the run. --- cmd/run.go | 8 ++-- core/local/local.go | 50 ++++++++++++++++--------- core/local/local_test.go | 54 +++++++++++++++++++++++++++ lib/executor/constant_arrival_rate.go | 5 +++ lib/executor/constant_looping_vus.go | 5 +++ lib/executor/externally_controlled.go | 6 +++ lib/executor/per_vu_iterations.go | 5 +++ lib/executor/shared_iterations.go | 5 +++ lib/executor/variable_arrival_rate.go | 5 +++ lib/executor/variable_looping_vus.go | 5 +++ lib/executors.go | 3 ++ 11 files changed, 130 insertions(+), 21 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 8dae15a52ee..9d271986276 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -228,17 +228,17 @@ a commandline interface for interacting with it.`, fprintf(stdout, "\n") plan := execScheduler.GetExecutionPlan() - executors := execScheduler.GetExecutors() + executorConfigs := execScheduler.GetExecutorConfigs() maxDuration, _ := lib.GetEndOffset(plan) fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprintf( "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", - conf.ExecutionSegment.FloatLength()*100, len(executors), + conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), lib.GetMaxPossibleVUs(plan), maxDuration), ) - for _, sched := range executors { + for _, ec := range executorConfigs { fprintf(stdout, " * %s: %s\n", - sched.GetConfig().GetName(), sched.GetConfig().GetDescription(conf.ExecutionSegment)) + ec.GetName(), ec.GetDescription(conf.ExecutionSegment)) } fprintf(stdout, "\n") } diff --git a/core/local/local.go b/core/local/local.go index 58b513b7f40..154644989bb 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -40,12 +40,13 @@ type ExecutionScheduler struct { options lib.Options logger *logrus.Logger - initProgress *pb.ProgressBar - executors []lib.Executor // sorted by (startTime, ID) - executionPlan []lib.ExecutionStep - maxDuration time.Duration // cached value derived from the execution plan - maxPossibleVUs uint64 // cached value derived from the execution plan - state *lib.ExecutionState + initProgress *pb.ProgressBar + executorConfigs []lib.ExecutorConfig // sorted by (startTime, ID) + executors []lib.Executor // sorted by (startTime, ID), excludes executors with no work + executionPlan []lib.ExecutionStep + maxDuration time.Duration // cached value derived from the execution plan + maxPossibleVUs uint64 // cached value derived from the execution plan + state *lib.ExecutionState } // Check to see if we implement the lib.ExecutionScheduler interface @@ -66,13 +67,21 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution maxDuration, _ := lib.GetEndOffset(executionPlan) // we don't care if the end offset is final executorConfigs := options.Execution.GetSortedConfigs() - executors := make([]lib.Executor, len(executorConfigs)) - for i, sc := range executorConfigs { + executors := make([]lib.Executor, 0, len(executorConfigs)) + // Only take executors which have work. + for _, sc := range executorConfigs { + if !sc.HasWork(options.ExecutionSegment) { + logger.Warnf( + "Executor '%s' is disabled for segment %s due to lack of work!", + sc.GetName(), options.ExecutionSegment, + ) + continue + } s, err := sc.NewExecutor(executionState, logger.WithField("executor", sc.GetName())) if err != nil { return nil, err } - executors[i] = s + executors = append(executors, s) } if options.Paused.Bool { @@ -86,12 +95,13 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution logger: logger, options: options, - initProgress: pb.New(pb.WithConstLeft("Init")), - executors: executors, - executionPlan: executionPlan, - maxDuration: maxDuration, - maxPossibleVUs: maxPossibleVUs, - state: executionState, + initProgress: pb.New(pb.WithConstLeft("Init")), + executors: executors, + executorConfigs: executorConfigs, + executionPlan: executionPlan, + maxDuration: maxDuration, + maxPossibleVUs: maxPossibleVUs, + state: executionState, }, nil } @@ -109,12 +119,18 @@ func (e *ExecutionScheduler) GetState() *lib.ExecutionState { return e.state } -// GetExecutors returns the slice of configured executor instances, sorted by -// their (startTime, name) in an ascending order. +// GetExecutors returns the slice of configured executor instances which +// have work, sorted by their (startTime, name) in an ascending order. func (e *ExecutionScheduler) GetExecutors() []lib.Executor { return e.executors } +// GetExecutorConfigs returns the slice of all executor configs, sorted by +// their (startTime, name) in an ascending order. +func (e *ExecutionScheduler) GetExecutorConfigs() []lib.ExecutorConfig { + return e.executorConfigs +} + // GetInitProgressBar returns the progress bar associated with the Init // function. After the Init is done, it is "hijacked" to display real-time // execution statistics as a text bar. diff --git a/core/local/local_test.go b/core/local/local_test.go index 1a06d1f2214..78d77b7bba3 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -728,3 +728,57 @@ func TestSetPaused(t *testing.T) { require.Contains(t, err.Error(), "doesn't support pause and resume operations after its start") }) } + +func TestNewExecutionSchedulerHasWork(t *testing.T) { + t.Parallel() + script := []byte(` + import http from 'k6/http'; + + export let options = { + executionSegment: "2/4:3/4", + execution: { + shared_iters1: { + type: "shared-iterations", + vus: 3, + iterations: 3, + }, + shared_iters2: { + type: "shared-iterations", + vus: 4, + iterations: 4, + }, + constant_arr_rate: { + type: "constant-arrival-rate", + rate: 3, + timeUnit: "1s", + duration: "20s", + preAllocatedVUs: 4, + maxVUs: 4, + }, + }, + }; + + export default function() { + const response = http.get("http://test.loadimpact.com"); + }; +`) + + runner, err := js.New( + &loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, + Data: script, + }, + nil, + lib.RuntimeOptions{}, + ) + require.NoError(t, err) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + assert.Len(t, execScheduler.executors, 2) + assert.Len(t, execScheduler.executorConfigs, 3) +} diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 4e7857fc6c7..56bdecd3f60 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -164,6 +164,11 @@ func (carc ConstantArrivalRateConfig) NewExecutor( }, nil } +// HasWork reports whether there is any work to be done for the given execution segment. +func (carc ConstantArrivalRateConfig) HasWork(es *lib.ExecutionSegment) bool { + return carc.GetMaxVUs(es) > 0 +} + // ConstantArrivalRate tries to execute a specific number of iterations for a // specific period. type ConstantArrivalRate struct { diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 4ab79b2b7ef..28f301b7acc 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -116,6 +116,11 @@ func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionS } } +// HasWork reports whether there is any work to be done for the given execution segment. +func (clvc ConstantLoopingVUsConfig) HasWork(es *lib.ExecutionSegment) bool { + return clvc.GetVUs(es) > 0 +} + // NewExecutor creates a new ConstantLoopingVUs executor func (clvc ConstantLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { return ConstantLoopingVUs{ diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index baa3c216a71..7db1ba30b76 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -180,6 +180,12 @@ func (mec ExternallyControlledConfig) NewExecutor(es *lib.ExecutionState, logger }, nil } +// HasWork reports whether there is any work to be done for the given execution segment. +func (mec ExternallyControlledConfig) HasWork(es *lib.ExecutionSegment) bool { + // We can always initialize new VUs via the REST API, so return true. + return true +} + type pauseEvent struct { isPaused bool err chan error diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index e13c1bc40a2..23f8ed551b6 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -134,6 +134,11 @@ func (pvic PerVUIterationsConfig) NewExecutor( }, nil } +// HasWork reports whether there is any work to be done for the given execution segment. +func (pvic PerVUIterationsConfig) HasWork(es *lib.ExecutionSegment) bool { + return pvic.GetVUs(es) > 0 && pvic.GetIterations() > 0 +} + // PerVUIterations executes a specific number of iterations with each VU. type PerVUIterations struct { *BaseExecutor diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 9ca50f6a98c..167a9e64387 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -148,6 +148,11 @@ type SharedIterations struct { // Make sure we implement the lib.Executor interface. var _ lib.Executor = &SharedIterations{} +// HasWork reports whether there is any work to be done for the given execution segment. +func (sic SharedIterationsConfig) HasWork(es *lib.ExecutionSegment) bool { + return sic.GetVUs(es) > 0 && sic.GetIterations(es) > 0 +} + // Run executes a specific total number of iterations, which are all shared by // the configured VUs. func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 75938aadb88..09a956f2724 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -260,6 +260,11 @@ func (varc VariableArrivalRateConfig) NewExecutor( }, nil } +// HasWork reports whether there is any work to be done for the given execution segment. +func (varc VariableArrivalRateConfig) HasWork(es *lib.ExecutionSegment) bool { + return varc.GetMaxVUs(es) > 0 +} + // VariableArrivalRate tries to execute a specific number of iterations for a // specific period. //TODO: combine with the ConstantArrivalRate? diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 8fe851030c5..6bfc40b6796 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -460,6 +460,11 @@ func (vlvc VariableLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger }, nil } +// HasWork reports whether there is any work to be done for the given execution segment. +func (vlvc VariableLoopingVUsConfig) HasWork(es *lib.ExecutionSegment) bool { + return lib.GetMaxPlannedVUs(vlvc.GetExecutionRequirements(es)) > 0 +} + // VariableLoopingVUs handles the old "stages" execution configuration - it // loops iterations with a variable number of VUs for the sum of all of the // specified stages' duration. diff --git a/lib/executors.go b/lib/executors.go index 77c0450b950..ea914688131 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -103,6 +103,9 @@ type ExecutorConfig interface { GetDescription(es *ExecutionSegment) string NewExecutor(*ExecutionState, *logrus.Entry) (Executor, error) + + // HasWork reports whether there is any work for the executor to do with a given segment. + HasWork(*ExecutionSegment) bool } // InitVUFunc is just a shorthand so we don't have to type the function From 84bf065c38b94d6eaf6b9afa0c43591e9981c1d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 14 Jan 2020 13:05:30 +0100 Subject: [PATCH 081/350] Add pb.WithLogger function, cleanup printBar Resolves https://github.com/loadimpact/k6/pull/1286#discussion_r365805409 --- cmd/cloud.go | 20 +++++++++----------- cmd/run.go | 14 +++++++------- cmd/ui.go | 22 ++++++++-------------- lib/executor/base_executor.go | 6 ++++-- ui/pb/progressbar.go | 20 +++++++++++++------- ui/pb/progressbar_test.go | 7 ++++--- 6 files changed, 45 insertions(+), 44 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 039c4230331..d80dd204612 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -65,13 +65,11 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud k6 cloud script.js`[1:], Args: exactArgsWithMsg(1, "arg should either be \"-\", if reading script from stdin, or a path to a script file"), RunE: func(cmd *cobra.Command, args []string) error { - logger := logrus.StandardLogger() - //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) progressBar := pb.New(pb.WithConstLeft(" Init")) - printBar(progressBar, "Parsing script", logger) + printBar(progressBar, "Parsing script") // Runner pwd, err := os.Getwd() @@ -91,13 +89,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - printBar(progressBar, "Getting script options", logger) + printBar(progressBar, "Getting script options") r, err := newRunner(src, runType, filesystems, runtimeOptions) if err != nil { return err } - printBar(progressBar, "Consolidating options", logger) + printBar(progressBar, "Consolidating options") cliOpts, err := getOptions(cmd.Flags()) if err != nil { return err @@ -130,7 +128,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return errors.New("Not logged in, please use `k6 login cloud`.") } - printBar(progressBar, "Building the archive", logger) + printBar(progressBar, "Building the archive") arc := r.MakeArchive() // TODO: Fix this // We reuse cloud.Config for parsing options.ext.loadimpact, but this probably shouldn't be @@ -179,19 +177,19 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud } // Start cloud test run - printBar(progressBar, "Validating script options", logger) + printBar(progressBar, "Validating script options") client := cloud.NewClient(cloudConfig.Token.String, cloudConfig.Host.String, consts.Version) if err := client.ValidateOptions(arc.Options); err != nil { return err } - printBar(progressBar, "Uploading archive", logger) + printBar(progressBar, "Uploading archive") refID, err := client.StartCloudTestRun(name, cloudConfig.ProjectID.Int64, arc) if err != nil { return err } progressBar.Modify(pb.WithConstLeft(" Run")) - printBar(progressBar, "Initializing the cloud test", logger) + printBar(progressBar, "Initializing the cloud test") testURL := cloud.URLForResults(refID, cloudConfig) fprintf(stdout, "\n\n") @@ -200,7 +198,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) //TODO: print executors information fprintf(stdout, "\n") - printBar(progressBar, "Initializing the cloud test", logger) + printBar(progressBar, "Initializing the cloud test") // The quiet option hides the progress bar and disallow aborting the test if quiet { @@ -236,7 +234,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } - printBar(progressBar, "", logger) + printBar(progressBar, "") } else { logrus.WithError(progressErr).Error("Test progress error") } diff --git a/cmd/run.go b/cmd/run.go index 73ab968e574..a14c31e1876 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -91,16 +91,13 @@ a commandline interface for interacting with it.`, k6 run -o influxdb=http://1.2.3.4:8086/k6`[1:], Args: exactArgsWithMsg(1, "arg should either be \"-\", if reading script from stdin, or a path to a script file"), RunE: func(cmd *cobra.Command, args []string) error { - //TODO: don't use a global... or maybe change the logger? - logger := logrus.StandardLogger() - //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) initBar := pb.New(pb.WithConstLeft(" init")) // Create the Runner. - printBar(initBar, "runner", logger) + printBar(initBar, "runner") pwd, err := os.Getwd() if err != nil { return err @@ -122,7 +119,7 @@ a commandline interface for interacting with it.`, return err } - printBar(initBar, "options", logger) + printBar(initBar, "options") cliConf, err := getConfig(cmd.Flags()) if err != nil { @@ -143,11 +140,14 @@ a commandline interface for interacting with it.`, return err } + //TODO: don't use a global... or maybe change the logger? + logger := logrus.StandardLogger() + ctx, cancel := context.WithCancel(context.Background()) //TODO: move even earlier? defer cancel() // Create a local execution scheduler wrapping the runner. - printBar(initBar, "execution scheduler", logger) + printBar(initBar, "execution scheduler") execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err @@ -158,7 +158,7 @@ a commandline interface for interacting with it.`, progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) go func() { - showProgress(ctx, conf, execScheduler, logger) + showProgress(ctx, conf, execScheduler) progressBarWG.Done() }() diff --git a/cmd/ui.go b/cmd/ui.go index c4fdaa008d1..324c2fb45c1 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -29,8 +29,6 @@ import ( "sync" "time" - "github.com/sirupsen/logrus" - "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/ui/pb" @@ -70,7 +68,7 @@ func (w *consoleWriter) Write(p []byte) (n int, err error) { return origLen, err } -func printBar(bar *pb.ProgressBar, rightText string, logger *logrus.Logger) { +func printBar(bar *pb.ProgressBar, rightText string) { end := "\n" if stdout.IsTTY { // If we're in a TTY, instead of printing the bar and going to the next @@ -80,11 +78,10 @@ func printBar(bar *pb.ProgressBar, rightText string, logger *logrus.Logger) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.Render(0, logger), rightText, end) + fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) } -func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar, - logger *logrus.Logger) string { +func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) string { lineEnd := "\n" if isTTY { //TODO: check for cross platform support @@ -95,7 +92,7 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar, result := make([]string, pbsCount+2) result[0] = lineEnd // start with an empty line for i, pb := range pbs { - result[i+1] = pb.Render(leftMax, logger) + lineEnd + result[i+1] = pb.Render(leftMax) + lineEnd } if isTTY && goBack { // Go back to the beginning @@ -110,10 +107,7 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar, //TODO: show other information here? //TODO: add a no-progress option that will disable these //TODO: don't use global variables... -func showProgress( - ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler, - logger *logrus.Logger, -) { +func showProgress(ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler) { if quiet || conf.HTTPDebug.Valid && conf.HTTPDebug.String != "" { return } @@ -137,7 +131,7 @@ func showProgress( leftLen = int(lib.Min(int64(leftLen), maxLeftLength)) // For flicker-free progressbars! - progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs, logger)) + progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs)) progressBarsPrint := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } @@ -158,7 +152,7 @@ func showProgress( stderr.PersistentText = nil if ctx.Err() != nil { // Render a last plain-text progressbar in an error - progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs, logger)) + progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs)) progressBarsPrint() } outMutex.Unlock() @@ -170,7 +164,7 @@ func showProgress( for { select { case <-ticker.C: - barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs, logger) + barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs) outMutex.Lock() progressBarsLastRender = []byte(barText) progressBarsPrint() diff --git a/lib/executor/base_executor.go b/lib/executor/base_executor.go index 43ad966aa13..95a7b04b286 100644 --- a/lib/executor/base_executor.go +++ b/lib/executor/base_executor.go @@ -23,9 +23,10 @@ package executor import ( "context" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" ) // BaseExecutor is a helper struct that contains common properties and methods @@ -39,7 +40,7 @@ type BaseExecutor struct { progress *pb.ProgressBar } -// NewBaseExecutor just returns an initialized BaseExecutor +// NewBaseExecutor returns an initialized BaseExecutor func NewBaseExecutor(config lib.ExecutorConfig, es *lib.ExecutionState, logger *logrus.Entry) *BaseExecutor { return &BaseExecutor{ config: config, @@ -47,6 +48,7 @@ func NewBaseExecutor(config lib.ExecutorConfig, es *lib.ExecutionState, logger * logger: logger, progress: pb.New( pb.WithLeft(config.GetName), + pb.WithLogger(logger), ), } } diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 78a84777c1e..9dc40c2d059 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -35,9 +35,10 @@ const defaultBarColor = color.Faint // ProgressBar is just a simple thread-safe progressbar implementation with // callbacks. type ProgressBar struct { - mutex sync.RWMutex - width int - color *color.Color + mutex sync.RWMutex + width int + color *color.Color + logger *logrus.Entry left func() string progress func() (progress float64, right string) @@ -60,6 +61,11 @@ func WithConstLeft(left string) ProgressBarOption { } } +// WithLogger modifies the logger instance +func WithLogger(logger *logrus.Entry) ProgressBarOption { + return func(pb *ProgressBar) { pb.logger = logger } +} + // WithProgress modifies the progress calculation function. func WithProgress(progress func() (float64, string)) ProgressBarOption { return func(pb *ProgressBar) { pb.progress = progress } @@ -128,7 +134,7 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // text, as well as the padding between the text and the opening // square bracket. Characters exceeding this length will be replaced // with a single ellipsis. Passing <=0 disables this. -func (pb *ProgressBar) Render(leftMax int, logger *logrus.Logger) string { +func (pb *ProgressBar) Render(leftMax int) string { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -145,10 +151,10 @@ func (pb *ProgressBar) Render(leftMax int, logger *logrus.Logger) string { right = " " + right progressClamped := Clampf(progress, 0, 1) if progress != progressClamped { - if logger != nil { - logger.Warnf("progress value %.2f exceeds valid range, clamped between 0 and 1", progress) - } progress = progressClamped + if pb.logger != nil { + pb.logger.Warnf("progress value %.2f exceeds valid range, clamped between 0 and 1", progress) + } } } diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index 9d4d4a59892..1e427d45d81 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -26,7 +26,8 @@ import ( "github.com/stretchr/testify/assert" ) -// XXX: This introduces an import cycle: pb -> lib -> pb +// TODO(imiric): Consider adding logging tests for 100% pb coverage. +// Unfortunately the following introduces an import cycle: pb -> lib -> pb // func getTestLogger() *logger.Entry { // logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} // testLog := logrus.New() @@ -87,7 +88,7 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0, nil)) + assert.Equal(t, tc.expected, pbar.Render(0)) }) } } @@ -113,7 +114,7 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Run(tc.left, func(t *testing.T) { pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.maxLen, nil)) + assert.Equal(t, tc.expected, pbar.Render(tc.maxLen)) }) } } From 1108470829cef2632f1c9d21c94c0d5f91d443ab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 13 Jan 2020 14:34:07 +0100 Subject: [PATCH 082/350] Make right side progress bar text more compact, better aligned --- lib/executor/constant_arrival_rate.go | 10 +++++----- lib/executor/constant_looping_vus.go | 9 +++++---- lib/executor/externally_controlled.go | 2 +- lib/executor/per_vu_iterations.go | 12 ++++++++---- lib/executor/shared_iterations.go | 6 ++++-- lib/executor/variable_arrival_rate.go | 12 +++++++----- lib/executor/variable_looping_vus.go | 6 +++--- 7 files changed, 33 insertions(+), 24 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 56bdecd3f60..9a30f204f60 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -27,12 +27,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const constantArrivalRateType = "constant-arrival-rate" @@ -227,15 +228,14 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } vusFmt := pb.GetFixedLengthIntFormat(maxVUs) - fmtStr := pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 2) + - " iters/s, " + vusFmt + " out of " + vusFmt + " VUs active" + fmtStr := vusFmt + "/" + vusFmt + " VUs\t" + pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0) + " iters/s" progresFn := func() (float64, string) { spent := time.Since(startTime) currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) vusInBuffer := uint64(len(vus)) return math.Min(1, float64(spent)/float64(duration)), fmt.Sprintf(fmtStr, - arrivalRatePerSec, currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, + currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, arrivalRatePerSec, ) } car.progress.Modify(pb.WithProgress(progresFn)) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 28f301b7acc..d119938a63c 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -26,12 +26,13 @@ import ( "sync" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const constantLoopingVUsType = "constant-looping-vus" @@ -158,10 +159,10 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo progresFn := func() (float64, string) { spent := time.Since(startTime) if spent > duration { - return 1, fmt.Sprintf("constant looping %d VUs for %s", numVUs, duration) + return 1, fmt.Sprintf("%d VUs\t%s", numVUs, duration) } return float64(spent) / float64(duration), fmt.Sprintf( - "constant looping %d VUs, %s/%s", numVUs, pb.GetFixedLengthDuration(spent, duration), duration, + "%d VUs\t%s/%s", numVUs, pb.GetFixedLengthDuration(spent, duration), duration, ) } clv.progress.Modify(pb.WithProgress(progresFn)) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index b35e3c1c9e0..808954f7211 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -405,7 +405,7 @@ func (rs *externallyControlledRunState) progresFn() (float64, string) { currentMaxVUs := atomic.LoadInt64(rs.maxVUs) vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) return progress, fmt.Sprintf( - "currently "+vusFmt+" out of "+vusFmt+" active looping VUs, %s/%s", currentActiveVUs, currentMaxVUs, + vusFmt+"/"+vusFmt+" VUs\t%s/%s", currentActiveVUs, currentMaxVUs, pb.GetFixedLengthDuration(spent, rs.duration), rs.duration, ) } diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 23f8ed551b6..04aaa4bc275 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -27,12 +27,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const perVUIterationsType = "per-vu-iterations" @@ -166,11 +167,14 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta totalIters := uint64(numVUs * iterations) doneIters := new(uint64) - fmtStr := pb.GetFixedLengthIntFormat(int64(totalIters)) + "/%d iters, %d from each of %d VUs" + + vusFmt := pb.GetFixedLengthIntFormat(int64(numVUs)) + itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) + fmtStr := vusFmt + " VUs\t" + itersFmt + "/" + itersFmt + " iters, %d per VU" progresFn := func() (float64, string) { currentDoneIters := atomic.LoadUint64(doneIters) return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( - fmtStr, currentDoneIters, totalIters, iterations, numVUs, + fmtStr, numVUs, currentDoneIters, totalIters, iterations, ) } pvi.progress.Modify(pb.WithProgress(progresFn)) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 167a9e64387..50f5ed7da7d 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -172,11 +172,13 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta totalIters := uint64(iterations) doneIters := new(uint64) - fmtStr := pb.GetFixedLengthIntFormat(int64(totalIters)) + "/%d shared iters among %d VUs" + vusFmt := pb.GetFixedLengthIntFormat(int64(numVUs)) + itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) + fmtStr := vusFmt + " VUs\t" + itersFmt + "/" + itersFmt + " shared iters" progresFn := func() (float64, string) { currentDoneIters := atomic.LoadUint64(doneIters) return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( - fmtStr, currentDoneIters, totalIters, numVUs, + fmtStr, numVUs, currentDoneIters, totalIters, ) } si.progress.Modify(pb.WithProgress(progresFn)) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 09a956f2724..ea7033b5ffa 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -28,12 +28,13 @@ import ( "sync/atomic" "time" + "github.com/sirupsen/logrus" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const variableArrivalRateType = "variable-arrival-rate" @@ -356,8 +357,9 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample tickerPeriod := new(int64) *tickerPeriod = int64(startTickerPeriod.Duration) - fmtStr := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 2) + " iters/s, " + - pb.GetFixedLengthIntFormat(maxVUs) + " out of " + pb.GetFixedLengthIntFormat(maxVUs) + " VUs active" + vusFmt := pb.GetFixedLengthIntFormat(maxVUs) + fmtStr := vusFmt + "/" + vusFmt + " VUs\t" + pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" + progresFn := func() (float64, string) { currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) currentTickerPeriod := atomic.LoadInt64(tickerPeriod) @@ -368,7 +370,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample itersPerSec = float64(time.Second) / float64(currentTickerPeriod) } return math.Min(1, float64(time.Since(startTime))/float64(duration)), fmt.Sprintf(fmtStr, - itersPerSec, currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, + currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, itersPerSec, ) } varr.progress.Modify(pb.WithProgress(progresFn)) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 6bfc40b6796..8cd5a3fece3 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -512,12 +512,12 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) progresFn := func() (float64, string) { spent := time.Since(startTime) + currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) if spent > regularDuration { - return 1, fmt.Sprintf("variable looping VUs for %s", regularDuration) + return 1, fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs\t%s", currentlyActiveVUs, maxVUs, regularDuration) } - currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) return float64(spent) / float64(regularDuration), fmt.Sprintf( - "currently "+vusFmt+" active looping VUs, %s/%s", currentlyActiveVUs, + vusFmt+"/"+vusFmt+" VUs\t%s/%s", currentlyActiveVUs, maxVUs, pb.GetFixedLengthDuration(spent, regularDuration), regularDuration, ) } From eb7073548e1d033df12a333aa8c153bfd942b550 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 27 Jan 2020 09:52:28 +0200 Subject: [PATCH 083/350] Implement a basic version of a sequence of execution segments --- .golangci.yml | 1 + lib/execution_segment.go | 131 ++++++++++++++++++++++++++++++---- lib/execution_segment_test.go | 83 ++++++++++++++++----- 3 files changed, 183 insertions(+), 32 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index fdd148f1819..83e8eb0f7f2 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -54,6 +54,7 @@ linters: - gochecknoinits - godox - wsl + - gomnd fast: false service: diff --git a/lib/execution_segment.go b/lib/execution_segment.go index dea79b41b81..07f64576aa9 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -51,12 +51,16 @@ type ExecutionSegment struct { } // Ensure we implement those interfaces -var _ encoding.TextUnmarshaler = &ExecutionSegment{} -var _ fmt.Stringer = &ExecutionSegment{} +var ( + _ encoding.TextUnmarshaler = &ExecutionSegment{} + _ fmt.Stringer = &ExecutionSegment{} +) // Helpful "constants" so we don't initialize them in every function call -var zeroRat, oneRat = big.NewRat(0, 1), big.NewRat(1, 1) //nolint:gochecknoglobals -var oneBigInt, twoBigInt = big.NewInt(1), big.NewInt(2) //nolint:gochecknoglobals +var ( + zeroRat, oneRat = big.NewRat(0, 1), big.NewRat(1, 1) //nolint:gochecknoglobals + oneBigInt, twoBigInt = big.NewInt(1), big.NewInt(2) //nolint:gochecknoglobals +) // NewExecutionSegment validates the supplied arguments (basically, that 0 <= // from < to <= 1) and either returns an error, or it returns a @@ -95,12 +99,11 @@ func stringToRat(s string) (*big.Rat, error) { return rat, nil } -// UnmarshalText implements the encoding.TextUnmarshaler interface, so that -// execution segments can be specified as CLI flags, environment variables, and -// JSON strings. +// NewExecutionSegmentFromString validates the supplied string value and returns +// the newly created ExecutionSegment or and error from it. // // We are able to parse both single percentage/float/fraction values, and actual -// (from; to] segments. For the single values, we just treat them as the +// (from: to] segments. For the single values, we just treat them as the // beginning segment - thus the execution segment can be used as a shortcut for // quickly running an arbitrarily scaled-down version of a test. // @@ -109,9 +112,8 @@ func stringToRat(s string) (*big.Rat, error) { // And values without a colon are the end of a first segment: // `20%`, `0.2`, and `1/5` should be converted to (0, 1/5] // empty values should probably be treated as "1", i.e. the whole execution -func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { +func NewExecutionSegmentFromString(toStr string) (result *ExecutionSegment, err error) { from := zeroRat - toStr := string(text) if toStr == "" { toStr = "1" // an empty string means a full 0:1 execution segment } @@ -119,16 +121,23 @@ func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { fromToStr := strings.SplitN(toStr, ":", 2) toStr = fromToStr[1] if from, err = stringToRat(fromToStr[0]); err != nil { - return err + return nil, err } } to, err := stringToRat(toStr) if err != nil { - return err + return nil, err } - segment, err := NewExecutionSegment(from, to) + return NewExecutionSegment(from, to) +} + +// UnmarshalText implements the encoding.TextUnmarshaler interface, so that +// execution segments can be specified as CLI flags, environment variables, and +// JSON strings. It is a wrapper for the NewExecutionFromString() constructor. +func (es *ExecutionSegment) UnmarshalText(text []byte) (err error) { + segment, err := NewExecutionSegmentFromString(string(text)) if err != nil { return err } @@ -195,8 +204,6 @@ func (es *ExecutionSegment) Split(numParts int64) ([]*ExecutionSegment, error) { return results, nil } -//TODO: add a NewFromString() method - // Equal returns true only if the two execution segments have the same from and // to values. func (es *ExecutionSegment) Equal(other *ExecutionSegment) bool { @@ -293,3 +300,97 @@ func (es *ExecutionSegment) CopyScaleRat(value *big.Rat) *big.Rat { } return new(big.Rat).Mul(value, es.length) } + +// ExecutionSegmentSequence represents an ordered chain of execution segments, +// where the end of one segment is the beginning of the next. It can serialized +// as a comma-separated string of rational numbers "r1,r2,r3,...,rn", which +// represents the sequence (r1, r2], (r2, r3], (r3, r4], ..., (r{n-1}, rn]. +// The empty value should be treated as if there is a single (0, 1] segment. +type ExecutionSegmentSequence []*ExecutionSegment + +// NewExecutionSegmentSequence validates the that the supplied execution +// segments are non-overlapping and without gaps. It will return a new execution +// segment sequence if that is true, and an error if it's not. +func NewExecutionSegmentSequence(segments ...*ExecutionSegment) (ExecutionSegmentSequence, error) { + if len(segments) > 2 { + to := segments[0].to + for i, segment := range segments[1:] { + if segment.from.Cmp(to) != 0 { + return nil, fmt.Errorf( + "the start value %s of segment #%d should be equal to the end value of the previous one, but it is %s", + segment.from, i+1, to, + ) + } + to = segment.to + } + } + return ExecutionSegmentSequence(segments), nil +} + +// NewExecutionSegmentSequenceFromString parses strings of the format +// "r1,r2,r3,...,rn", which represents the sequences like (r1, r2], (r2, r3], +// (r3, r4], ..., (r{n-1}, rn]. +func NewExecutionSegmentSequenceFromString(strSeq string) (ExecutionSegmentSequence, error) { + var segments []*ExecutionSegment + if len(strSeq) != 0 { + points := strings.Split(strSeq, ",") + if len(points) < 2 { + return nil, fmt.Errorf("at least 2 points are needed for an execution segment sequence, %d given", len(points)) + } + start := points[0] + + segments = make([]*ExecutionSegment, 0, len(points)-1) + for _, point := range points[1:] { + segment, errl := NewExecutionSegmentFromString(start + ":" + point) + if errl != nil { + return nil, errl + } + segments = append(segments, segment) + start = point + } + } + + return NewExecutionSegmentSequence(segments...) +} + +// UnmarshalText implements the encoding.TextUnmarshaler interface, so that +// execution segment sequences can be specified as CLI flags, environment +// variables, and JSON strings. +func (ess *ExecutionSegmentSequence) UnmarshalText(text []byte) (err error) { + seq, err := NewExecutionSegmentSequenceFromString(string(text)) + if err != nil { + return err + } + *ess = seq + return nil +} + +// MarshalText implements the encoding.TextMarshaler interface, so is used for +// text and JSON encoding of the execution segment sequences. +func (ess ExecutionSegmentSequence) MarshalText() ([]byte, error) { + return []byte(ess.String()), nil +} + +// String just implements the fmt.Stringer interface, encoding the sequence of +// segments as "start1,end1,end2,end3,...,endn". +func (ess ExecutionSegmentSequence) String() string { + result := make([]string, 0, len(ess)+1) + for i, s := range ess { + if i == 0 { + result = append(result, s.from.RatString()) + } + result = append(result, s.to.RatString()) + } + return strings.Join(result, ",") +} + +// GetStripedOffsets returns everything that you need in order to execute only +// the iterations that belong to the supplied segment... +// +// TODO: add a more detailed algorithm description +func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int, []int, error) { + start := 0 + offsets := []int{} + // TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit + return start, offsets, fmt.Errorf("not implemented") +} diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 4d050756e6d..ea3d7e0e95c 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -29,10 +29,11 @@ import ( ) func stringToES(t *testing.T, str string) *ExecutionSegment { - var es = new(ExecutionSegment) + es := new(ExecutionSegment) require.NoError(t, es.UnmarshalText([]byte(str))) return es } + func TestExecutionSegmentEquals(t *testing.T) { t.Parallel() @@ -44,7 +45,7 @@ func TestExecutionSegmentEquals(t *testing.T) { }) t.Run("To it's self", func(t *testing.T) { - var es = stringToES(t, "1/2:2/3") + es := stringToES(t, "1/2:2/3") require.True(t, es.Equal(es)) }) } @@ -76,7 +77,7 @@ func TestExecutionSegmentNew(t *testing.T) { func TestExecutionSegmentUnmarshalText(t *testing.T) { t.Parallel() - var testCases = []struct { + testCases := []struct { input string output *ExecutionSegment isErr bool @@ -98,7 +99,7 @@ func TestExecutionSegmentUnmarshalText(t *testing.T) { for _, testCase := range testCases { testCase := testCase t.Run(testCase.input, func(t *testing.T) { - var es = new(ExecutionSegment) + es := new(ExecutionSegment) err := es.UnmarshalText([]byte(testCase.input)) if testCase.isErr { require.Error(t, err) @@ -116,10 +117,10 @@ func TestExecutionSegmentUnmarshalText(t *testing.T) { t.Run("Unmarshal nilSegment.String", func(t *testing.T) { var nilEs *ExecutionSegment - var nilEsStr = nilEs.String() + nilEsStr := nilEs.String() require.Equal(t, "0:1", nilEsStr) - var es = new(ExecutionSegment) + es := new(ExecutionSegment) err := es.UnmarshalText([]byte(nilEsStr)) require.NoError(t, err) require.True(t, es.Equal(nilEs)) @@ -186,7 +187,7 @@ func TestExecutionSegmentSplit(t *testing.T) { func TestExecutionSegmentScale(t *testing.T) { t.Parallel() - var es = new(ExecutionSegment) + es := new(ExecutionSegment) require.NoError(t, es.UnmarshalText([]byte("0.5"))) require.Equal(t, int64(1), es.Scale(2)) require.Equal(t, int64(2), es.Scale(3)) @@ -198,9 +199,9 @@ func TestExecutionSegmentScale(t *testing.T) { func TestExecutionSegmentCopyScaleRat(t *testing.T) { t.Parallel() - var es = new(ExecutionSegment) - var twoRat = big.NewRat(2, 1) - var threeRat = big.NewRat(3, 1) + es := new(ExecutionSegment) + twoRat := big.NewRat(2, 1) + threeRat := big.NewRat(3, 1) require.NoError(t, es.UnmarshalText([]byte("0.5"))) require.Equal(t, oneRat, es.CopyScaleRat(twoRat)) require.Equal(t, big.NewRat(3, 2), es.CopyScaleRat(threeRat)) @@ -216,10 +217,10 @@ func TestExecutionSegmentCopyScaleRat(t *testing.T) { func TestExecutionSegmentInPlaceScaleRat(t *testing.T) { t.Parallel() - var es = new(ExecutionSegment) - var twoRat = big.NewRat(2, 1) - var threeRat = big.NewRat(3, 1) - var threeSecondsRat = big.NewRat(3, 2) + es := new(ExecutionSegment) + twoRat := big.NewRat(2, 1) + threeRat := big.NewRat(3, 1) + threeSecondsRat := big.NewRat(3, 2) require.NoError(t, es.UnmarshalText([]byte("0.5"))) require.Equal(t, oneRat, es.InPlaceScaleRat(twoRat)) require.Equal(t, oneRat, twoRat) @@ -245,7 +246,7 @@ func TestExecutionSegmentInPlaceScaleRat(t *testing.T) { func TestExecutionSegmentSubSegment(t *testing.T) { t.Parallel() - var testCases = []struct { + testCases := []struct { name string base, sub, result *ExecutionSegment }{ @@ -281,7 +282,7 @@ func TestExecutionSegmentSubSegment(t *testing.T) { func TestSplitBadSegment(t *testing.T) { t.Parallel() - var es = &ExecutionSegment{from: oneRat, to: zeroRat} + es := &ExecutionSegment{from: oneRat, to: zeroRat} _, err := es.Split(5) require.Error(t, err) } @@ -293,7 +294,7 @@ func TestSegmentExecutionFloatLength(t *testing.T) { require.Equal(t, 1.0, nilEs.FloatLength()) }) - var testCases = []struct { + testCases := []struct { es *ExecutionSegment expected float64 }{ @@ -320,3 +321,51 @@ func TestSegmentExecutionFloatLength(t *testing.T) { }) } } + +func TestExecutionSegmentSequences(t *testing.T) { + t.Parallel() + testCases := []struct { + seq string + expSegments []string + expError bool + canReverse bool + // TODO: checks for least common denominator and maybe striped partitioning + }{ + {seq: "", expSegments: nil}, + {seq: "0.5", expError: true}, + {seq: "1,1", expError: true}, + {seq: "-0.5,1", expError: true}, + {seq: "1/2,1/2", expError: true}, + {seq: "1/2,1/3", expError: true}, + {seq: "0.5,1", expSegments: []string{"1/2:1"}}, + {seq: "1/2,1", expSegments: []string{"1/2:1"}, canReverse: true}, + {seq: "1/3,2/3", expSegments: []string{"1/3:2/3"}, canReverse: true}, + {seq: "0,1/3,2/3", expSegments: []string{"0:1/3", "1/3:2/3"}, canReverse: true}, + {seq: "0,1/3,2/3,1", expSegments: []string{"0:1/3", "1/3:2/3", "2/3:1"}, canReverse: true}, + {seq: "0.5,0.7", expSegments: []string{"1/2:7/10"}}, + {seq: "0.5,0.7,1", expSegments: []string{"1/2:7/10", "7/10:1"}}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.seq, func(t *testing.T) { + result, err := NewExecutionSegmentSequenceFromString(tc.seq) + if tc.expError { + require.Error(t, err) + return + } + require.NoError(t, err) + require.Equal(t, len(tc.expSegments), len(result)) + for i, expStrSeg := range tc.expSegments { + expSeg, errl := NewExecutionSegmentFromString(expStrSeg) + require.NoError(t, errl) + assert.Truef(t, expSeg.Equal(result[i]), "Segment %d (%s) should be equal to %s", i, result[i], expSeg) + } + if tc.canReverse { + assert.Equal(t, result.String(), tc.seq) + } + }) + } +} + +// TODO: test with randomized things From cd62c9cd3e0c42cc134dbdbe3406757065dff306 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 14 Jan 2020 18:32:22 +0100 Subject: [PATCH 084/350] Add ProgressBar.status, render status symbol --- core/local/local.go | 19 +++++++++++++------ lib/executor/helpers.go | 15 ++++++++++----- ui/pb/progressbar.go | 30 +++++++++++++++++++++++++++--- 3 files changed, 50 insertions(+), 14 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 154644989bb..d525eb25c67 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -29,9 +29,10 @@ import ( "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" ) // ExecutionScheduler is the local implementation of lib.ExecutionScheduler @@ -290,10 +291,13 @@ func (e *ExecutionScheduler) runExecutor( // Check if we have to wait before starting the actual executor execution if executorStartTime > 0 { startTime := time.Now() - executorProgress.Modify(pb.WithProgress(func() (float64, string) { - remWait := (executorStartTime - time.Since(startTime)) - return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) - })) + executorProgress.Modify( + pb.WithStatus(pb.Waiting), + pb.WithProgress(func() (float64, string) { + remWait := (executorStartTime - time.Since(startTime)) + return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) + }), + ) executorLogger.Debugf("Waiting for executor start time...") select { @@ -305,7 +309,10 @@ func (e *ExecutionScheduler) runExecutor( } } - executorProgress.Modify(pb.WithConstProgress(0, "started")) + executorProgress.Modify( + pb.WithStatus(pb.Running), + pb.WithConstProgress(0, "started"), + ) executorLogger.Debugf("Starting executor") err := executor.Run(runCtx, engineOut) // executor should handle context cancel itself if err == nil { diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 59cf6a219c1..3f4a4074ba3 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -28,10 +28,11 @@ import ( "github.com/loadimpact/k6/ui/pb" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" ) func sumStagesDuration(stages []Stage) (result time.Duration) { @@ -152,7 +153,7 @@ func getDurationContexts(parentCtx context.Context, regularDuration, gracefulSto // executor and updates its progressbar accordingly. func trackProgress( parentCtx, maxDurationCtx, regDurationCtx context.Context, - exec lib.Executor, snapshot func() (float64, string), + exec lib.Executor, snapshot func() (float64, []string), ) { progressBar := exec.GetProgress() logger := exec.GetLogger() @@ -164,16 +165,20 @@ func trackProgress( logger.WithField("gracefulStop", gracefulStop).Debug( "Regular duration is done, waiting for iterations to gracefully finish", ) - progressBar.Modify(pb.WithConstProgress(p, right+", gracefully stopping...")) + progressBar.Modify( + pb.WithStatus(pb.Stopping), + pb.WithConstProgress(p, right...), + ) } <-maxDurationCtx.Done() p, right := snapshot() + constProg := pb.WithConstProgress(p, right...) select { case <-parentCtx.Done(): - progressBar.Modify(pb.WithConstProgress(p, right+" interrupted!")) + progressBar.Modify(pb.WithStatus(pb.Interrupted), constProg) default: - progressBar.Modify(pb.WithConstProgress(p, right+" done!")) + progressBar.Modify(pb.WithStatus(pb.Done), constProg) } } diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 9dc40c2d059..eb5dae2a4d0 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -32,13 +32,32 @@ import ( const defaultWidth = 40 const defaultBarColor = color.Faint -// ProgressBar is just a simple thread-safe progressbar implementation with +// Status of the progress bar +type Status string + +// Progress bar status symbols +const ( + Running Status = " " + Waiting Status = "•" + Stopping Status = "↓" + Interrupted Status = "✗" + Done Status = "✓" +) + +//nolint:gochecknoglobals +var statusColors = map[Status]*color.Color{ + Interrupted: color.New(color.FgRed), + Done: color.New(color.FgGreen), +} + +// ProgressBar is a simple thread-safe progressbar implementation with // callbacks. type ProgressBar struct { mutex sync.RWMutex width int color *color.Color logger *logrus.Entry + status Status left func() string progress func() (progress float64, right string) @@ -71,6 +90,11 @@ func WithProgress(progress func() (float64, string)) ProgressBarOption { return func(pb *ProgressBar) { pb.progress = progress } } +// WithStatus modifies the progressbar status +func WithStatus(status Status) ProgressBarOption { + return func(pb *ProgressBar) { pb.status = status } +} + // WithConstProgress sets the progress and right padding to the supplied consts. func WithConstProgress(progress float64, right string) ProgressBarOption { return func(pb *ProgressBar) { @@ -177,6 +201,6 @@ func (pb *ProgressBar) Render(leftMax int) string { padding = pb.color.Sprint(strings.Repeat("-", space-filled)) } - return fmt.Sprintf("%s [%s%s%s]%s", - pb.renderLeft(leftMax), filling, caret, padding, right) + return fmt.Sprintf("%s%2s [%s%s%s]%s", + pb.renderLeft(leftMax), pb.status, filling, caret, padding, right) } From 30634cf3ab638e06b1131cd45ef96a4d95c9898a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 15 Jan 2020 12:00:01 +0100 Subject: [PATCH 085/350] Split right-side progress bar text into columns I tried using tabwriter for this, but it still produced "jumping" to align with the next column if text on the left was too long. I didn't find a more reliable approach than calculating the offset manually and applying fmt string padding like this, but it's not pretty... --- cmd/cloud.go | 7 ++- cmd/ui.go | 67 +++++++++++++++++++---- core/local/local.go | 10 ++-- lib/executor/constant_arrival_rate.go | 12 ++-- lib/executor/constant_looping_vus.go | 12 ++-- lib/executor/externally_controlled.go | 9 ++- lib/executor/per_vu_iterations.go | 13 +++-- lib/executor/shared_iterations.go | 13 +++-- lib/executor/variable_arrival_rate.go | 12 ++-- lib/executor/variable_looping_vus.go | 12 ++-- ui/pb/progressbar.go | 79 +++++++++++++++++---------- 11 files changed, 159 insertions(+), 87 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index d80dd204612..970a2754a81 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -214,11 +214,12 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud testProgress := &cloud.TestProgressResponse{} percentageFmt := "[" + pb.GetFixedLengthFloatFormat(100, 2) + "%%] %s" progressBar.Modify( - pb.WithProgress(func() (float64, string) { + pb.WithProgress(func() (float64, []string) { if testProgress.RunStatus < lib.RunStatusRunning { - return 0, testProgress.RunStatusText + return 0, []string{testProgress.RunStatusText} } - return testProgress.Progress, fmt.Sprintf(percentageFmt, testProgress.Progress*100, testProgress.RunStatusText) + return testProgress.Progress, []string{ + fmt.Sprintf(percentageFmt, testProgress.Progress*100, testProgress.RunStatusText)} }), ) diff --git a/cmd/ui.go b/cmd/ui.go index 324c2fb45c1..88f6226e98d 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -78,7 +78,9 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - fprintf(stdout, "%s %s%s", bar.Render(0), rightText, end) + rendered := bar.Render(0) + // Only output the left and middle part of the progress bar + fprintf(stdout, "%s %s %s%s", rendered.Left, rendered.Progress, rightText, end) } func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) string { @@ -88,12 +90,55 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) lineEnd = "\x1b[K\n" // erase till end of line } - pbsCount := len(pbs) - result := make([]string, pbsCount+2) + var ( + // Maximum length of each right side column except last, + // used to calculate the padding between columns. + maxRColumnLen = make([]int, 1) + pbsCount = len(pbs) + rendered = make([]pb.ProgressBarRender, pbsCount) + result = make([]string, pbsCount+2) + ) + result[0] = lineEnd // start with an empty line + + // First pass to render all progressbars and get the maximum + // lengths of right-side columns. for i, pb := range pbs { - result[i+1] = pb.Render(leftMax) + lineEnd + rend := pb.Render(leftMax) + for i := range rend.Right { + // Don't calculate for last column, since there's nothing to align + // after it (yet?). + if i == len(rend.Right)-1 { + break + } + if len(rend.Right[i]) > maxRColumnLen[i] { + maxRColumnLen[i] = len(rend.Right[i]) + } + } + rendered[i] = rend + } + + // Second pass to render final output, applying padding where needed + for i := range rendered { + rend := rendered[i] + if rend.Hijack != "" { + result[i+1] = rend.Hijack + lineEnd + continue + } + var leftText, rightText string + leftPadFmt := fmt.Sprintf("%%-%ds %%s ", leftMax) + leftText = fmt.Sprintf(leftPadFmt, rend.Left, rend.Status) + for i := range rend.Right { + rpad := 0 + if len(maxRColumnLen) > i { + rpad = maxRColumnLen[i] + } + rightPadFmt := fmt.Sprintf(" %%-%ds", rpad+1) + rightText += fmt.Sprintf(rightPadFmt, rend.Right[i]) + } + result[i+1] = leftText + rend.Progress + rightText + lineEnd } + if isTTY && goBack { // Go back to the beginning //TODO: check for cross platform support @@ -119,19 +164,17 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi // Get the longest left side string length, to align progress bars // horizontally and trim excess text. - var leftLen int + var leftLen int64 for _, pb := range pbs { l := pb.Left() - if len(l) > leftLen { - leftLen = len(l) - } + leftLen = lib.Max(int64(len(l)), leftLen) } // Limit to maximum left text length - leftLen = int(lib.Min(int64(leftLen), maxLeftLength)) + leftMax := int(lib.Min(leftLen, maxLeftLength)) // For flicker-free progressbars! - progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftLen, pbs)) + progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftMax, pbs)) progressBarsPrint := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } @@ -152,7 +195,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi stderr.PersistentText = nil if ctx.Err() != nil { // Render a last plain-text progressbar in an error - progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftLen, pbs)) + progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftMax, pbs)) progressBarsPrint() } outMutex.Unlock() @@ -164,7 +207,7 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi for { select { case <-ticker.C: - barText := renderMultipleBars(stdoutTTY, true, leftLen, pbs) + barText := renderMultipleBars(stdoutTTY, true, leftMax, pbs) outMutex.Lock() progressBarsLastRender = []byte(barText) progressBarsPrint() diff --git a/core/local/local.go b/core/local/local.go index d525eb25c67..506ac9a6c61 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -232,10 +232,10 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa initializedVUs := new(uint64) vusFmt := pb.GetFixedLengthIntFormat(int64(vusToInitialize)) e.initProgress.Modify( - pb.WithProgress(func() (float64, string) { + pb.WithProgress(func() (float64, []string) { doneVUs := atomic.LoadUint64(initializedVUs) - return float64(doneVUs) / float64(vusToInitialize), - fmt.Sprintf(vusFmt+"/%d VUs initialized", doneVUs, vusToInitialize) + right := fmt.Sprintf(vusFmt+"/%d VUs initialized", doneVUs, vusToInitialize) + return float64(doneVUs) / float64(vusToInitialize), []string{right} }), ) @@ -293,9 +293,9 @@ func (e *ExecutionScheduler) runExecutor( startTime := time.Now() executorProgress.Modify( pb.WithStatus(pb.Waiting), - pb.WithProgress(func() (float64, string) { + pb.WithProgress(func() (float64, []string) { remWait := (executorStartTime - time.Since(startTime)) - return 0, fmt.Sprintf("waiting %s", pb.GetFixedLengthDuration(remWait, executorStartTime)) + return 0, []string{"waiting", pb.GetFixedLengthDuration(remWait, executorStartTime)} }), ) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 9a30f204f60..284dce044a4 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -228,15 +228,15 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } vusFmt := pb.GetFixedLengthIntFormat(maxVUs) - fmtStr := vusFmt + "/" + vusFmt + " VUs\t" + pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0) + " iters/s" - - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { spent := time.Since(startTime) currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) vusInBuffer := uint64(len(vus)) - return math.Min(1, float64(spent)/float64(duration)), fmt.Sprintf(fmtStr, - currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, arrivalRatePerSec, - ) + progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", + currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) + progIters := fmt.Sprintf( + pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0)+" iters/s", arrivalRatePerSec) + return math.Min(1, float64(spent)/float64(duration)), []string{progVUs, progIters} } car.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, car, progresFn) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index d119938a63c..a96e971aa0f 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -156,14 +156,16 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo logrus.Fields{"vus": numVUs, "duration": duration, "type": clv.config.GetType()}, ).Debug("Starting executor run...") - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { spent := time.Since(startTime) + right := []string{fmt.Sprintf("%d VUs", numVUs)} if spent > duration { - return 1, fmt.Sprintf("%d VUs\t%s", numVUs, duration) + right = append(right, duration.String()) + return 1, right } - return float64(spent) / float64(duration), fmt.Sprintf( - "%d VUs\t%s/%s", numVUs, pb.GetFixedLengthDuration(spent, duration), duration, - ) + right = append(right, fmt.Sprintf("%s/%s", + pb.GetFixedLengthDuration(spent, duration), duration)) + return float64(spent) / float64(duration), right } clv.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, clv, progresFn) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 808954f7211..f28bae31fb8 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -394,7 +394,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { return nil } -func (rs *externallyControlledRunState) progresFn() (float64, string) { +func (rs *externallyControlledRunState) progresFn() (float64, []string) { spent := rs.executor.executionState.GetCurrentTestRunDuration() progress := 0.0 if rs.duration > 0 { @@ -404,10 +404,9 @@ func (rs *externallyControlledRunState) progresFn() (float64, string) { currentActiveVUs := atomic.LoadInt64(rs.activeVUsCount) currentMaxVUs := atomic.LoadInt64(rs.maxVUs) vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) - return progress, fmt.Sprintf( - vusFmt+"/"+vusFmt+" VUs\t%s/%s", currentActiveVUs, currentMaxVUs, - pb.GetFixedLengthDuration(spent, rs.duration), rs.duration, - ) + progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentActiveVUs, currentMaxVUs) + progDur := fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, rs.duration), rs.duration) + return progress, []string{progVUs, progDur} } func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg ExternallyControlledConfigParams) error { diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 04aaa4bc275..0ce68c305a9 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -168,14 +168,15 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta totalIters := uint64(numVUs * iterations) doneIters := new(uint64) - vusFmt := pb.GetFixedLengthIntFormat(int64(numVUs)) + vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) - fmtStr := vusFmt + " VUs\t" + itersFmt + "/" + itersFmt + " iters, %d per VU" - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { currentDoneIters := atomic.LoadUint64(doneIters) - return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( - fmtStr, numVUs, currentDoneIters, totalIters, iterations, - ) + return float64(currentDoneIters) / float64(totalIters), []string{ + fmt.Sprintf(vusFmt+" VUs", numVUs), + fmt.Sprintf(itersFmt+"/"+itersFmt+" iters, %d per VU", + currentDoneIters, totalIters, iterations), + } } pvi.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, pvi, progresFn) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 50f5ed7da7d..163757835b9 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -172,14 +172,15 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta totalIters := uint64(iterations) doneIters := new(uint64) - vusFmt := pb.GetFixedLengthIntFormat(int64(numVUs)) + vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) - fmtStr := vusFmt + " VUs\t" + itersFmt + "/" + itersFmt + " shared iters" - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { currentDoneIters := atomic.LoadUint64(doneIters) - return float64(currentDoneIters) / float64(totalIters), fmt.Sprintf( - fmtStr, numVUs, currentDoneIters, totalIters, - ) + return float64(currentDoneIters) / float64(totalIters), []string{ + fmt.Sprintf(vusFmt+" VUs", numVUs), + fmt.Sprintf(itersFmt+"/"+itersFmt+" shared iters", + currentDoneIters, totalIters), + } } si.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, si, progresFn) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index ea7033b5ffa..e7beac0f787 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -358,9 +358,9 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample *tickerPeriod = int64(startTickerPeriod.Duration) vusFmt := pb.GetFixedLengthIntFormat(maxVUs) - fmtStr := vusFmt + "/" + vusFmt + " VUs\t" + pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" + itersFmt := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) currentTickerPeriod := atomic.LoadInt64(tickerPeriod) vusInBuffer := uint64(len(vus)) @@ -369,9 +369,11 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample if currentTickerPeriod > 0 { itersPerSec = float64(time.Second) / float64(currentTickerPeriod) } - return math.Min(1, float64(time.Since(startTime))/float64(duration)), fmt.Sprintf(fmtStr, - currentInitialisedVUs-vusInBuffer, currentInitialisedVUs, itersPerSec, - ) + return math.Min(1, float64(time.Since(startTime))/float64(duration)), []string{ + fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", + currentInitialisedVUs-vusInBuffer, currentInitialisedVUs), + fmt.Sprintf(itersFmt, itersPerSec), + } } varr.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 8cd5a3fece3..17726fe3c24 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -510,16 +510,16 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo activeVUsCount := new(int64) vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) - progresFn := func() (float64, string) { + progresFn := func() (float64, []string) { spent := time.Since(startTime) currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) + vus := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentlyActiveVUs, maxVUs) if spent > regularDuration { - return 1, fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs\t%s", currentlyActiveVUs, maxVUs, regularDuration) + return 1, []string{vus, regularDuration.String()} } - return float64(spent) / float64(regularDuration), fmt.Sprintf( - vusFmt+"/"+vusFmt+" VUs\t%s/%s", currentlyActiveVUs, maxVUs, - pb.GetFixedLengthDuration(spent, regularDuration), regularDuration, - ) + progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentlyActiveVUs, maxVUs) + progDur := fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, regularDuration), regularDuration) + return float64(spent) / float64(regularDuration), []string{progVUs, progDur} } vlv.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, vlv, progresFn) diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index eb5dae2a4d0..9738ee110c4 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -60,7 +60,7 @@ type ProgressBar struct { status Status left func() string - progress func() (progress float64, right string) + progress func() (progress float64, right []string) hijack func() string } @@ -68,12 +68,12 @@ type ProgressBar struct { // parameters, either in the constructor or via the Modify() method. type ProgressBarOption func(*ProgressBar) -// WithLeft modifies the function that returns the left progressbar padding. +// WithLeft modifies the function that returns the left progressbar value. func WithLeft(left func() string) ProgressBarOption { return func(pb *ProgressBar) { pb.left = left } } -// WithConstLeft sets the left progressbar padding to the supplied const. +// WithConstLeft sets the left progressbar value to the supplied const. func WithConstLeft(left string) ProgressBarOption { return func(pb *ProgressBar) { pb.left = func() string { return left } @@ -86,7 +86,7 @@ func WithLogger(logger *logrus.Entry) ProgressBarOption { } // WithProgress modifies the progress calculation function. -func WithProgress(progress func() (float64, string)) ProgressBarOption { +func WithProgress(progress func() (float64, []string)) ProgressBarOption { return func(pb *ProgressBar) { pb.progress = progress } } @@ -95,14 +95,14 @@ func WithStatus(status Status) ProgressBarOption { return func(pb *ProgressBar) { pb.status = status } } -// WithConstProgress sets the progress and right padding to the supplied consts. -func WithConstProgress(progress float64, right string) ProgressBarOption { +// WithConstProgress sets the progress and right values to the supplied consts. +func WithConstProgress(progress float64, right ...string) ProgressBarOption { return func(pb *ProgressBar) { - pb.progress = func() (float64, string) { return progress, right } + pb.progress = func() (float64, []string) { return progress, right } } } -// WithHijack replaces the progressbar String function with the argument. +// WithHijack replaces the progressbar Render function with the argument. func WithHijack(hijack func() string) ProgressBarOption { return func(pb *ProgressBar) { pb.hijack = hijack } } @@ -127,9 +127,8 @@ func (pb *ProgressBar) Left() string { return pb.renderLeft(0) } -// renderLeft renders the left part of the progressbar, applying the -// given padding and trimming text exceeding maxLen length, -// replacing it with an ellipsis. +// renderLeft renders the left part of the progressbar, replacing text +// exceeding maxLen with an ellipsis. func (pb *ProgressBar) renderLeft(maxLen int) string { var left string if pb.left != nil { @@ -137,8 +136,7 @@ func (pb *ProgressBar) renderLeft(maxLen int) string { if maxLen > 0 && len(l) > maxLen { l = l[:maxLen-3] + "..." } - padFmt := fmt.Sprintf("%%-%ds", maxLen) - left = fmt.Sprintf(padFmt, l) + left = l } return left } @@ -152,27 +150,45 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { } } -// Render locks the progressbar struct for reading and calls all of its methods -// to assemble the progress bar and return it as a string. +// ProgressBarRender stores the different rendered parts of the +// progress bar UI. +type ProgressBarRender struct { + Left, Status, Progress, Hijack string + Right []string +} + +func (pbr ProgressBarRender) String() string { + if pbr.Hijack != "" { + return pbr.Hijack + } + var right string + if len(pbr.Right) > 0 { + right = " " + strings.Join(pbr.Right, " ") + } + return fmt.Sprintf("%s %-1s %s%s", + pbr.Left, pbr.Status, pbr.Progress, right) +} + +// Render locks the progressbar struct for reading and calls all of +// its methods to return the final output. A struct is returned over a +// plain string to allow dynamic padding and positioning of elements +// depending on other elements on the screen. // - leftMax defines the maximum character length of the left-side -// text, as well as the padding between the text and the opening -// square bracket. Characters exceeding this length will be replaced -// with a single ellipsis. Passing <=0 disables this. -func (pb *ProgressBar) Render(leftMax int) string { +// text. Characters exceeding this length will be replaced with a +// single ellipsis. Passing <=0 disables this. +func (pb *ProgressBar) Render(leftMax int) ProgressBarRender { pb.mutex.RLock() defer pb.mutex.RUnlock() + var out ProgressBarRender if pb.hijack != nil { - return pb.hijack() + out.Hijack = pb.hijack() + return out } - var ( - progress float64 - right string - ) + var progress float64 if pb.progress != nil { - progress, right = pb.progress() - right = " " + right + progress, out.Right = pb.progress() progressClamped := Clampf(progress, 0, 1) if progress != progressClamped { progress = progressClamped @@ -201,6 +217,13 @@ func (pb *ProgressBar) Render(leftMax int) string { padding = pb.color.Sprint(strings.Repeat("-", space-filled)) } - return fmt.Sprintf("%s%2s [%s%s%s]%s", - pb.renderLeft(leftMax), pb.status, filling, caret, padding, right) + out.Left = pb.renderLeft(leftMax) + status := string(pb.status) + if c, ok := statusColors[pb.status]; ok { + status = c.Sprint(pb.status) + } + out.Status = fmt.Sprintf("%-1s", status) + out.Progress = fmt.Sprintf("[%s%s%s]", filling, caret, padding) + + return out } From 81d01cfcd67b02347e25654b0756d27d0ed3ae2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 16 Jan 2020 19:17:37 +0100 Subject: [PATCH 086/350] Fix progress bar tests, add cmd.renderMultipleBars test --- cmd/ui_test.go | 70 +++++++++++++++++++++++++++++++++++++++ ui/pb/progressbar_test.go | 43 +++++++++++++----------- 2 files changed, 93 insertions(+), 20 deletions(-) create mode 100644 cmd/ui_test.go diff --git a/cmd/ui_test.go b/cmd/ui_test.go new file mode 100644 index 00000000000..95c7df02ade --- /dev/null +++ b/cmd/ui_test.go @@ -0,0 +1,70 @@ +package cmd + +import ( + "fmt" + "strings" + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/ui/pb" +) + +// Return progressbars with different content lengths, to test for +// padding. +func createTestProgressBars(num, padding, colIdx int) []*pb.ProgressBar { + pbs := make([]*pb.ProgressBar, num) + for i := 0; i < num; i++ { + left := fmt.Sprintf("left %d", i) + rightCol1 := fmt.Sprintf("right %d", i) + progress := 0.0 + status := pb.Running + if i == colIdx { + pad := strings.Repeat("+", padding) + left += pad + rightCol1 += pad + progress = 1.0 + status = pb.Done + } + pbs[i] = pb.New( + pb.WithLeft(func() string { return left }), + pb.WithStatus(status), + pb.WithProgress(func() (float64, []string) { + return progress, []string{rightCol1, "000"} + }), + ) + } + return pbs +} + +func TestRenderMultipleBars(t *testing.T) { + t.Parallel() + + testCases := []struct { + name string + padding int + expected string + }{ + {"nopad", 0, ` +left 0 [--------------------------------------] right 0 000 +left 1 ✓ [======================================] right 1 000 +left 2 [--------------------------------------] right 2 000 + +`}, + {"pad2", 2, ` +left 0 [--------------------------------------] right 0 000 +left 1++ ✓ [======================================] right 1++ 000 +left 2 [--------------------------------------] right 2 000 + +`}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + pbs := createTestProgressBars(3, tc.padding, 1) + out := renderMultipleBars(false, false, 6+tc.padding, pbs) + assert.Equal(t, tc.expected, out) + }) + } +} diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index 1e427d45d81..a27a230c37b 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -44,39 +44,44 @@ func TestProgressBarRender(t *testing.T) { expected string }{ {[]ProgressBarOption{WithLeft(func() string { return "left" })}, - "left [--------------------------------------]"}, + "left [--------------------------------------]"}, {[]ProgressBarOption{WithConstLeft("constLeft")}, - "constLeft [--------------------------------------]"}, + "constLeft [--------------------------------------]"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), - WithProgress(func() (float64, string) { return 0, "right" }), + WithStatus(Done), }, - "left [--------------------------------------] right"}, + "left ✓ [--------------------------------------]"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), - WithProgress(func() (float64, string) { return 0.5, "right" }), + WithProgress(func() (float64, []string) { return 0, []string{"right"} }), }, - "left [==================>-------------------] right"}, + "left [--------------------------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), - WithProgress(func() (float64, string) { return 1.0, "right" }), + WithProgress(func() (float64, []string) { return 0.5, []string{"right"} }), }, - "left [======================================] right"}, + "left [==================>-------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), - WithProgress(func() (float64, string) { return -1, "right" }), + WithProgress(func() (float64, []string) { return 1.0, []string{"right"} }), }, - "left [--------------------------------------] right"}, + "left [======================================] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), - WithProgress(func() (float64, string) { return 2, "right" }), + WithProgress(func() (float64, []string) { return -1, []string{"right"} }), }, - "left [======================================] right"}, + "left [--------------------------------------] right"}, + {[]ProgressBarOption{ + WithLeft(func() string { return "left" }), + WithProgress(func() (float64, []string) { return 2, []string{"right"} }), + }, + "left [======================================] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithConstProgress(0.2, "constProgress"), }, - "left [======>-------------------------------] constProgress"}, + "left [======>-------------------------------] constProgress"}, {[]ProgressBarOption{ WithHijack(func() string { return "progressbar hijack!" }), }, @@ -88,7 +93,7 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0)) + assert.Equal(t, tc.expected, pbar.Render(0).String()) }) } } @@ -101,12 +106,10 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { left string expected string }{ - {-1, "left", "left [--------------------------------------]"}, - {0, "left", "left [--------------------------------------]"}, - {15, "left_pad", - "left_pad [--------------------------------------]"}, + {-1, "left", "left [--------------------------------------]"}, + {0, "left", "left [--------------------------------------]"}, {10, "left_truncated", - "left_tr... [--------------------------------------]"}, + "left_tr... [--------------------------------------]"}, } for _, tc := range testCases { @@ -114,7 +117,7 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Run(tc.left, func(t *testing.T) { pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.maxLen)) + assert.Equal(t, tc.expected, pbar.Render(tc.maxLen).String()) }) } } From d340d1d0fdb2370b9128c8c0f0d8747e34621105 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 31 Jan 2020 11:19:47 +0100 Subject: [PATCH 087/350] Apply faint color to Waiting progress bar status Resolves https://github.com/loadimpact/k6/pull/1310#pullrequestreview-350685906 --- ui/pb/progressbar.go | 1 + 1 file changed, 1 insertion(+) diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 9738ee110c4..3bc3409d0b5 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -48,6 +48,7 @@ const ( var statusColors = map[Status]*color.Color{ Interrupted: color.New(color.FgRed), Done: color.New(color.FgGreen), + Waiting: color.New(defaultBarColor), } // ProgressBar is a simple thread-safe progressbar implementation with From 686f32b9204b02f4214be812ad2ec1966b4463fc Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 5 Feb 2020 09:00:58 +0200 Subject: [PATCH 088/350] Fix an off-by-one error and optimize ESS constructor --- .golangci.yml | 2 +- lib/execution_segment.go | 40 ++++++++++++++++++++++------------- lib/execution_segment_test.go | 11 ++++++++++ 3 files changed, 37 insertions(+), 16 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index 83e8eb0f7f2..d15ec1dcfef 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -58,4 +58,4 @@ linters: fast: false service: - golangci-lint-version: 1.20.x + golangci-lint-version: 1.23.x diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 07f64576aa9..94ff0dd8b9e 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -312,7 +312,7 @@ type ExecutionSegmentSequence []*ExecutionSegment // segments are non-overlapping and without gaps. It will return a new execution // segment sequence if that is true, and an error if it's not. func NewExecutionSegmentSequence(segments ...*ExecutionSegment) (ExecutionSegmentSequence, error) { - if len(segments) > 2 { + if len(segments) > 1 { to := segments[0].to for i, segment := range segments[1:] { if segment.from.Cmp(to) != 0 { @@ -331,23 +331,33 @@ func NewExecutionSegmentSequence(segments ...*ExecutionSegment) (ExecutionSegmen // "r1,r2,r3,...,rn", which represents the sequences like (r1, r2], (r2, r3], // (r3, r4], ..., (r{n-1}, rn]. func NewExecutionSegmentSequenceFromString(strSeq string) (ExecutionSegmentSequence, error) { - var segments []*ExecutionSegment - if len(strSeq) != 0 { - points := strings.Split(strSeq, ",") - if len(points) < 2 { - return nil, fmt.Errorf("at least 2 points are needed for an execution segment sequence, %d given", len(points)) + if len(strSeq) == 0 { + return nil, nil + } + + points := strings.Split(strSeq, ",") + if len(points) < 2 { + return nil, fmt.Errorf("at least 2 points are needed for an execution segment sequence, %d given", len(points)) + } + var start *big.Rat + + segments := make([]*ExecutionSegment, 0, len(points)-1) + for i, point := range points { + rat, err := stringToRat(point) + if err != nil { + return nil, err + } + if i == 0 { + start = rat + continue } - start := points[0] - segments = make([]*ExecutionSegment, 0, len(points)-1) - for _, point := range points[1:] { - segment, errl := NewExecutionSegmentFromString(start + ":" + point) - if errl != nil { - return nil, errl - } - segments = append(segments, segment) - start = point + segment, err := NewExecutionSegment(start, rat) + if err != nil { + return nil, err } + segments = append(segments, segment) + start = rat } return NewExecutionSegmentSequence(segments...) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index ea3d7e0e95c..386a3f63062 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -324,6 +324,13 @@ func TestSegmentExecutionFloatLength(t *testing.T) { func TestExecutionSegmentSequences(t *testing.T) { t.Parallel() + + _, err := NewExecutionSegmentSequence(stringToES(t, "0:1/3"), stringToES(t, "1/2:1")) + assert.Error(t, err) +} + +func TestExecutionSegmentStringSequences(t *testing.T) { + t.Parallel() testCases := []struct { seq string expSegments []string @@ -337,6 +344,7 @@ func TestExecutionSegmentSequences(t *testing.T) { {seq: "-0.5,1", expError: true}, {seq: "1/2,1/2", expError: true}, {seq: "1/2,1/3", expError: true}, + {seq: "0,1,1/2", expError: true}, {seq: "0.5,1", expSegments: []string{"1/2:1"}}, {seq: "1/2,1", expSegments: []string{"1/2:1"}, canReverse: true}, {seq: "1/3,2/3", expSegments: []string{"1/3:2/3"}, canReverse: true}, @@ -344,6 +352,9 @@ func TestExecutionSegmentSequences(t *testing.T) { {seq: "0,1/3,2/3,1", expSegments: []string{"0:1/3", "1/3:2/3", "2/3:1"}, canReverse: true}, {seq: "0.5,0.7", expSegments: []string{"1/2:7/10"}}, {seq: "0.5,0.7,1", expSegments: []string{"1/2:7/10", "7/10:1"}}, + {seq: "0,1/13,2/13,1/3,1/2,3/4,1", expSegments: []string{ + "0:1/13", "1/13:2/13", "2/13:1/3", "1/3:1/2", "1/2:3/4", "3/4:1", + }, canReverse: true}, } for _, tc := range testCases { From 5047368410a3733f2f021b0e0894a870043b0fbb Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 7 Feb 2020 09:24:09 +0200 Subject: [PATCH 089/350] Move the CompatibilityMode type to the lib package This was needed because if the `lib` package depended on the `js/compiler` package, any Go project imports lib would also import goja and babel... --- js/bundle.go | 6 +- js/bundle_test.go | 72 +++++++++++-------- js/compiler/compiler.go | 19 ++--- js/compiler/compiler_test.go | 13 ++-- js/initcontext.go | 7 +- .../compatibility_mode_gen.go | 2 +- lib/runtime_options.go | 22 ++++-- 7 files changed, 80 insertions(+), 61 deletions(-) rename {js/compiler => lib}/compatibility_mode_gen.go (98%) diff --git a/js/bundle.go b/js/bundle.go index 9cb550f337b..b473b33b400 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -49,7 +49,7 @@ type Bundle struct { BaseInitContext *InitContext Env map[string]string - CompatibilityMode compiler.CompatibilityMode + CompatibilityMode lib.CompatibilityMode } // A BundleInstance is a self-contained instance of a Bundle. @@ -198,7 +198,7 @@ func (b *Bundle) makeArchive() *lib.Archive { // Instantiate creates a new runtime from this bundle. func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { - //TODO: actually use a real context here, so that the instantiation can be killed + // TODO: actually use a real context here, so that the instantiation can be killed // Placeholder for a real context. ctxPtr := new(context.Context) @@ -244,7 +244,7 @@ func (b *Bundle) instantiate(rt *goja.Runtime, init *InitContext) error { rt.SetFieldNameMapper(common.FieldNameMapper{}) rt.SetRandSource(common.NewRandSource()) - if init.compatibilityMode == compiler.CompatibilityModeExtended { + if init.compatibilityMode == lib.CompatibilityModeExtended { if _, err := rt.RunProgram(jslib.GetCoreJS()); err != nil { return err } diff --git a/js/bundle_test.go b/js/bundle_test.go index 7e8017c43f5..0b48ae160f5 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -33,7 +33,6 @@ import ( "time" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/compiler" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/fsext" @@ -114,14 +113,16 @@ func TestNewBundle(t *testing.T) { t.Run("CompatibilityMode", func(t *testing.T) { t.Run("Extended/ok/CoreJS", func(t *testing.T) { rtOpts := lib.RuntimeOptions{ - CompatibilityMode: null.StringFrom(compiler.CompatibilityModeExtended.String())} + CompatibilityMode: null.StringFrom(lib.CompatibilityModeExtended.String()), + } _, err := getSimpleBundle("/script.js", `export default function() {}; new Set([1, 2, 3, 2, 1]);`, rtOpts) assert.NoError(t, err) }) t.Run("Base/ok/Minimal", func(t *testing.T) { rtOpts := lib.RuntimeOptions{ - CompatibilityMode: null.StringFrom(compiler.CompatibilityModeBase.String())} + CompatibilityMode: null.StringFrom(lib.CompatibilityModeBase.String()), + } _, err := getSimpleBundle("/script.js", `module.exports.default = function() {};`, rtOpts) assert.NoError(t, err) @@ -133,19 +134,27 @@ func TestNewBundle(t *testing.T) { code string expErr string }{ - {"InvalidCompat", "es1", `export default function() {};`, - `invalid compatibility mode "es1". Use: "extended", "base"`}, + { + "InvalidCompat", "es1", `export default function() {};`, + `invalid compatibility mode "es1". Use: "extended", "base"`, + }, // ES2015 modules are not supported - {"Modules", "base", `export default function() {};`, - "file:///script.js: Line 1:1 Unexpected reserved word"}, + { + "Modules", "base", `export default function() {};`, + "file:///script.js: Line 1:1 Unexpected reserved word", + }, // Arrow functions are not supported - {"ArrowFuncs", "base", + { + "ArrowFuncs", "base", `module.exports.default = function() {}; () => {};`, - "file:///script.js: Line 1:42 Unexpected token ) (and 1 more errors)"}, + "file:///script.js: Line 1:42 Unexpected token ) (and 1 more errors)", + }, // ES2015 objects polyfilled by core.js are not supported - {"CoreJS", "base", + { + "CoreJS", "base", `module.exports.default = function() {}; new Set([1, 2, 3, 2, 1]);`, - "ReferenceError: Set is not defined at file:///script.js:1:45(5)"}, + "ReferenceError: Set is not defined at file:///script.js:1:45(5)", + }, } for _, tc := range testCases { @@ -386,7 +395,6 @@ func TestNewBundle(t *testing.T) { assert.Equal(t, b.Options.TLSVersion.Min, lib.TLSVersion(tls.VersionSSL30)) assert.Equal(t, b.Options.TLSVersion.Max, lib.TLSVersion(tls.VersionSSL30)) } - }) }) t.Run("Thresholds", func(t *testing.T) { @@ -424,10 +432,10 @@ func TestNewBundleFromArchive(t *testing.T) { {"", ` export let options = { vus: 12345 }; export default function() { return "hi!"; };`}, - {compiler.CompatibilityModeExtended.String(), ` + {lib.CompatibilityModeExtended.String(), ` export let options = { vus: 12345 }; export default function() { return "hi!"; };`}, - {compiler.CompatibilityModeBase.String(), ` + {lib.CompatibilityModeBase.String(), ` module.exports.options = { vus: 12345 }; module.exports.default = function() { return "hi!" };`}, } @@ -445,7 +453,7 @@ func TestNewBundleFromArchive(t *testing.T) { assert.Equal(t, lib.Options{VUs: null.IntFrom(12345)}, b.Options) expCM := tc.compatMode if expCM == "" { - expCM = compiler.CompatibilityModeExtended.String() + expCM = lib.CompatibilityModeExtended.String() } assert.Equal(t, expCM, b.CompatibilityMode.String()) @@ -466,14 +474,18 @@ func TestNewBundleFromArchive(t *testing.T) { compatMode, code, expErr string }{ // Incompatible mode - {compiler.CompatibilityModeBase.String(), ` + { + lib.CompatibilityModeBase.String(), ` export let options = { vus: 12345 }; export default function() { return "hi!"; };`, - "file://script.js: Line 2:5 Unexpected reserved word (and 2 more errors)"}, - {"wrongcompat", ` + "file://script.js: Line 2:5 Unexpected reserved word (and 2 more errors)", + }, + { + "wrongcompat", ` export let options = { vus: 12345 }; export default function() { return "hi!"; };`, - `invalid compatibility mode "wrongcompat". Use: "extended", "base"`}, + `invalid compatibility mode "wrongcompat". Use: "extended", "base"`, + }, } for _, tc := range testCases { @@ -488,7 +500,7 @@ func TestNewBundleFromArchive(t *testing.T) { } func TestOpen(t *testing.T) { - var testCases = [...]struct { + testCases := [...]struct { name string openPath string pwd string @@ -588,8 +600,8 @@ func TestOpen(t *testing.T) { for _, tCase := range testCases { tCase := tCase - var testFunc = func(t *testing.T) { - var openPath = tCase.openPath + testFunc := func(t *testing.T) { + openPath := tCase.openPath // if fullpath prepend prefix if openPath != "" && (openPath[0] == '/' || openPath[0] == '\\') { openPath = filepath.Join(prefix, openPath) @@ -597,7 +609,7 @@ func TestOpen(t *testing.T) { if isWindows { openPath = strings.Replace(openPath, `\`, `\\`, -1) } - var pwd = tCase.pwd + pwd := tCase.pwd if pwd == "" { pwd = "/path/to/" } @@ -731,22 +743,26 @@ func TestBundleEnv(t *testing.T) { func TestBundleMakeArchive(t *testing.T) { testCases := []struct { - cm compiler.CompatibilityMode + cm lib.CompatibilityMode script string exclaim string }{ - {compiler.CompatibilityModeExtended, ` + { + lib.CompatibilityModeExtended, ` import exclaim from "./exclaim.js"; export let options = { vus: 12345 }; export let file = open("./file.txt"); export default function() { return exclaim(file); };`, - `export default function(s) { return s + "!" };`}, - {compiler.CompatibilityModeBase, ` + `export default function(s) { return s + "!" };`, + }, + { + lib.CompatibilityModeBase, ` var exclaim = require("./exclaim.js"); module.exports.options = { vus: 12345 }; module.exports.file = open("./file.txt"); module.exports.default = function() { return exclaim(module.exports.file); };`, - `module.exports.default = function(s) { return s + "!" };`}, + `module.exports.default = function(s) { return s + "!" };`, + }, } for _, tc := range testCases { diff --git a/js/compiler/compiler.go b/js/compiler/compiler.go index 3f7b5313012..2e5b35f7290 100644 --- a/js/compiler/compiler.go +++ b/js/compiler/compiler.go @@ -29,6 +29,7 @@ import ( rice "github.com/GeertJohan/go.rice" "github.com/dop251/goja" "github.com/dop251/goja/parser" + "github.com/loadimpact/k6/lib" "github.com/mitchellh/mapstructure" "github.com/sirupsen/logrus" ) @@ -48,18 +49,6 @@ var ( globalBabel *babel // nolint:gochecknoglobals ) -// CompatibilityMode specifies the JS compatibility mode -// nolint:lll -//go:generate enumer -type=CompatibilityMode -transform=snake -trimprefix CompatibilityMode -output compatibility_mode_gen.go -type CompatibilityMode uint8 - -const ( - // CompatibilityModeExtended achieves ES6+ compatibility with Babel and core.js - CompatibilityModeExtended CompatibilityMode = iota + 1 - // CompatibilityModeBase is standard goja ES5.1+ - CompatibilityModeBase -) - // A Compiler compiles JavaScript source code (ES5.1 or ES6) into a goja.Program type Compiler struct{} @@ -80,11 +69,11 @@ func (c *Compiler) Transform(src, filename string) (code string, srcmap *SourceM // Compile the program in the given CompatibilityMode, optionally running pre and post code. func (c *Compiler) Compile(src, filename, pre, post string, - strict bool, compatMode CompatibilityMode) (*goja.Program, string, error) { + strict bool, compatMode lib.CompatibilityMode) (*goja.Program, string, error) { code := pre + src + post ast, err := parser.ParseFile(nil, filename, code, 0) if err != nil { - if compatMode == CompatibilityModeExtended { + if compatMode == lib.CompatibilityModeExtended { code, _, err = c.Transform(src, filename) if err != nil { return nil, code, err @@ -101,7 +90,7 @@ type babel struct { vm *goja.Runtime this goja.Value transform goja.Callable - mutex sync.Mutex //TODO: cache goja.CompileAST() in an init() function? + mutex sync.Mutex // TODO: cache goja.CompileAST() in an init() function? } func newBabel() (*babel, error) { diff --git a/js/compiler/compiler_test.go b/js/compiler/compiler_test.go index add1625cbea..534703d3191 100644 --- a/js/compiler/compiler_test.go +++ b/js/compiler/compiler_test.go @@ -24,6 +24,7 @@ import ( "testing" "github.com/dop251/goja" + "github.com/loadimpact/k6/lib" "github.com/stretchr/testify/assert" ) @@ -71,7 +72,7 @@ func TestCompile(t *testing.T) { c := New() t.Run("ES5", func(t *testing.T) { src := `1+(function() { return 2; })()` - pgm, code, err := c.Compile(src, "script.js", "", "", true, CompatibilityModeBase) + pgm, code, err := c.Compile(src, "script.js", "", "", true, lib.CompatibilityModeBase) if !assert.NoError(t, err) { return } @@ -83,7 +84,7 @@ func TestCompile(t *testing.T) { t.Run("Wrap", func(t *testing.T) { pgm, code, err := c.Compile(src, "script.js", - "(function(){return ", "})", true, CompatibilityModeBase) + "(function(){return ", "})", true, lib.CompatibilityModeBase) if !assert.NoError(t, err) { return } @@ -102,14 +103,14 @@ func TestCompile(t *testing.T) { t.Run("Invalid", func(t *testing.T) { src := `1+(function() { return 2; )()` - _, _, err := c.Compile(src, "script.js", "", "", true, CompatibilityModeExtended) + _, _, err := c.Compile(src, "script.js", "", "", true, lib.CompatibilityModeExtended) assert.IsType(t, &goja.Exception{}, err) assert.Contains(t, err.Error(), `SyntaxError: script.js: Unexpected token (1:26) > 1 | 1+(function() { return 2; )()`) }) }) t.Run("ES6", func(t *testing.T) { - pgm, code, err := c.Compile(`1+(()=>2)()`, "script.js", "", "", true, CompatibilityModeExtended) + pgm, code, err := c.Compile(`1+(()=>2)()`, "script.js", "", "", true, lib.CompatibilityModeExtended) if !assert.NoError(t, err) { return } @@ -120,7 +121,7 @@ func TestCompile(t *testing.T) { } t.Run("Wrap", func(t *testing.T) { - pgm, code, err := c.Compile(`fn(1+(()=>2)())`, "script.js", "(function(fn){", "})", true, CompatibilityModeExtended) + pgm, code, err := c.Compile(`fn(1+(()=>2)())`, "script.js", "(function(fn){", "})", true, lib.CompatibilityModeExtended) if !assert.NoError(t, err) { return } @@ -141,7 +142,7 @@ func TestCompile(t *testing.T) { }) t.Run("Invalid", func(t *testing.T) { - _, _, err := c.Compile(`1+(=>2)()`, "script.js", "", "", true, CompatibilityModeExtended) + _, _, err := c.Compile(`1+(=>2)()`, "script.js", "", "", true, lib.CompatibilityModeExtended) assert.IsType(t, &goja.Exception{}, err) assert.Contains(t, err.Error(), `SyntaxError: script.js: Unexpected token (1:3) > 1 | 1+(=>2)()`) diff --git a/js/initcontext.go b/js/initcontext.go index f78620e099c..47c175a19f3 100644 --- a/js/initcontext.go +++ b/js/initcontext.go @@ -31,6 +31,7 @@ import ( "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/js/compiler" "github.com/loadimpact/k6/js/modules" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" "github.com/pkg/errors" "github.com/spf13/afero" @@ -58,12 +59,12 @@ type InitContext struct { // Cache of loaded programs and files. programs map[string]programWithSource - compatibilityMode compiler.CompatibilityMode + compatibilityMode lib.CompatibilityMode } // NewInitContext creates a new initcontext with the provided arguments func NewInitContext( - rt *goja.Runtime, c *compiler.Compiler, compatMode compiler.CompatibilityMode, + rt *goja.Runtime, c *compiler.Compiler, compatMode lib.CompatibilityMode, ctxPtr *context.Context, filesystems map[string]afero.Fs, pwd *url.URL, ) *InitContext { return &InitContext{ @@ -81,7 +82,7 @@ func newBoundInitContext(base *InitContext, ctxPtr *context.Context, rt *goja.Ru // we don't copy the exports as otherwise they will be shared and we don't want this. // this means that all the files will be executed again but once again only once per compilation // of the main file. - var programs = make(map[string]programWithSource, len(base.programs)) + programs := make(map[string]programWithSource, len(base.programs)) for key, program := range base.programs { programs[key] = programWithSource{ src: program.src, diff --git a/js/compiler/compatibility_mode_gen.go b/lib/compatibility_mode_gen.go similarity index 98% rename from js/compiler/compatibility_mode_gen.go rename to lib/compatibility_mode_gen.go index 19d46d1876f..75b62e6de4b 100644 --- a/js/compiler/compatibility_mode_gen.go +++ b/lib/compatibility_mode_gen.go @@ -1,7 +1,7 @@ // Code generated by "enumer -type=CompatibilityMode -transform=snake -trimprefix CompatibilityMode -output compatibility_mode_gen.go"; DO NOT EDIT. // -package compiler +package lib import ( "fmt" diff --git a/lib/runtime_options.go b/lib/runtime_options.go index e1245d9161c..566b94d9c76 100644 --- a/lib/runtime_options.go +++ b/lib/runtime_options.go @@ -24,16 +24,28 @@ import ( "fmt" "strings" - "github.com/loadimpact/k6/js/compiler" null "gopkg.in/guregu/null.v3" ) +// CompatibilityMode specifies the JS compatibility mode +// nolint:lll +//go:generate enumer -type=CompatibilityMode -transform=snake -trimprefix CompatibilityMode -output compatibility_mode_gen.go +type CompatibilityMode uint8 + +const ( + // CompatibilityModeExtended achieves ES6+ compatibility with Babel and core.js + CompatibilityModeExtended CompatibilityMode = iota + 1 + // CompatibilityModeBase is standard goja ES5.1+ + CompatibilityModeBase +) + // RuntimeOptions are settings passed onto the goja JS runtime type RuntimeOptions struct { // Whether to pass the actual system environment variables to the JS runtime IncludeSystemEnvVars null.Bool `json:"includeSystemEnvVars" envconfig:"K6_INCLUDE_SYSTEM_ENV_VARS"` // JS compatibility mode: "extended" (Goja+Babel+core.js) or "base" (plain Goja) + // TODO: maybe use CompatibilityMode directly? this seems strange... CompatibilityMode null.String `json:"compatibilityMode"` // Environment variables passed onto the runner @@ -56,13 +68,13 @@ func (o RuntimeOptions) Apply(opts RuntimeOptions) RuntimeOptions { } // ValidateCompatibilityMode checks if the provided val is a valid compatibility mode -func ValidateCompatibilityMode(val string) (cm compiler.CompatibilityMode, err error) { +func ValidateCompatibilityMode(val string) (cm CompatibilityMode, err error) { if val == "" { - return compiler.CompatibilityModeExtended, nil + return CompatibilityModeExtended, nil } - if cm, err = compiler.CompatibilityModeString(val); err != nil { + if cm, err = CompatibilityModeString(val); err != nil { var compatValues []string - for _, v := range compiler.CompatibilityModeValues() { + for _, v := range CompatibilityModeValues() { compatValues = append(compatValues, v.String()) } err = fmt.Errorf(`invalid compatibility mode "%s". Use: "%s"`, From c33423d1aa7d077ef64854193aac70f6a04ae724 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 12 Feb 2020 08:18:30 +0200 Subject: [PATCH 090/350] Fix a few minor bugs in runtime options and improve the tests The fixed bugs include: - K6_COMPATIBILITY_MODE was used only when RuntimeOptions.IncludeSystemEnvVars was true (which, by default, it isn't for `k6 cloud`) - K6_INCLUDE_SYSTEM_ENV_VARS was not used in any way, despite the wrong envconfig (which we don't use for the runtime options) struct tags - RuntimeOptions.Apply() wasn't used _or_ tested... So I just removed it... - `k6 run --compatibility-mode=base es6_archive.tar` wouldn't have failed, it would have just silently ignored the CLI flag... I also refactored the way RuntimeOptions are assembled in cmd/, so they receive their flags and env vars as a parameter, making them more decoupled and much easier to test. --- cmd/archive.go | 2 +- cmd/cloud.go | 2 +- cmd/inspect.go | 2 +- cmd/run.go | 2 +- cmd/runtime_options.go | 46 ++-- cmd/runtime_options_test.go | 409 ++++++++++++++++++------------------ js/bundle.go | 8 +- js/bundle_test.go | 160 +++++++------- lib/runtime_options.go | 25 +-- 9 files changed, 345 insertions(+), 311 deletions(-) diff --git a/cmd/archive.go b/cmd/archive.go index 0af47b7f597..7c0466f7ce8 100644 --- a/cmd/archive.go +++ b/cmd/archive.go @@ -58,7 +58,7 @@ An archive is a fully self-contained test run, and can be executed identically e return err } - runtimeOptions, err := getRuntimeOptions(cmd.Flags()) + runtimeOptions, err := getRuntimeOptions(cmd.Flags(), buildEnvMap(os.Environ())) if err != nil { return err } diff --git a/cmd/cloud.go b/cmd/cloud.go index d80dd204612..407f99e7fbf 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -84,7 +84,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - runtimeOptions, err := getRuntimeOptions(cmd.Flags()) + runtimeOptions, err := getRuntimeOptions(cmd.Flags(), buildEnvMap(os.Environ())) if err != nil { return err } diff --git a/cmd/inspect.go b/cmd/inspect.go index c48221d5cf1..8a53f1c5d88 100644 --- a/cmd/inspect.go +++ b/cmd/inspect.go @@ -54,7 +54,7 @@ var inspectCmd = &cobra.Command{ typ = detectType(src.Data) } - runtimeOptions, err := getRuntimeOptions(cmd.Flags()) + runtimeOptions, err := getRuntimeOptions(cmd.Flags(), buildEnvMap(os.Environ())) if err != nil { return err } diff --git a/cmd/run.go b/cmd/run.go index f4f26686d56..e874113cd76 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -109,7 +109,7 @@ a commandline interface for interacting with it.`, return err } - runtimeOptions, err := getRuntimeOptions(cmd.Flags()) + runtimeOptions, err := getRuntimeOptions(cmd.Flags(), buildEnvMap(os.Environ())) if err != nil { return err } diff --git a/cmd/runtime_options.go b/cmd/runtime_options.go index a594f581aa0..ec36b37d366 100644 --- a/cmd/runtime_options.go +++ b/cmd/runtime_options.go @@ -21,8 +21,8 @@ package cmd import ( - "os" "regexp" + "strconv" "strings" "github.com/loadimpact/k6/lib" @@ -31,6 +31,10 @@ import ( "gopkg.in/guregu/null.v3" ) +// TODO: move this whole file out of the cmd package? maybe when fixing +// https://github.com/loadimpact/k6/issues/883, since this code is fairly +// self-contained and easily testable now, without any global dependencies... + var userEnvVarName = regexp.MustCompile(`^[a-zA-Z_][a-zA-Z0-9_]*$`) func parseEnvKeyValue(kv string) (string, string) { @@ -40,9 +44,9 @@ func parseEnvKeyValue(kv string) (string, string) { return kv, "" } -func collectEnv() map[string]string { - env := make(map[string]string) - for _, kv := range os.Environ() { +func buildEnvMap(environ []string) map[string]string { + env := make(map[string]string, len(environ)) + for _, kv := range environ { k, v := parseEnvKeyValue(kv) env[k] = v } @@ -63,16 +67,35 @@ extended: base + Babel with ES2015 preset + core.js v2, return flags } -func getRuntimeOptions(flags *pflag.FlagSet) (lib.RuntimeOptions, error) { +func getRuntimeOptions(flags *pflag.FlagSet, environment map[string]string) (lib.RuntimeOptions, error) { opts := lib.RuntimeOptions{ IncludeSystemEnvVars: getNullBool(flags, "include-system-env-vars"), CompatibilityMode: getNullString(flags, "compatibility-mode"), Env: make(map[string]string), } - // If enabled, gather the actual system environment variables - if opts.IncludeSystemEnvVars.Bool { - opts.Env = collectEnv() + if !opts.CompatibilityMode.Valid { // If not explicitly set via CLI flags, look for an environment variable + if envVar, ok := environment["K6_COMPATIBILITY_MODE"]; ok { + opts.CompatibilityMode = null.StringFrom(envVar) + } + } + if _, err := lib.ValidateCompatibilityMode(opts.CompatibilityMode.String); err != nil { + // some early validation + return opts, err + } + + if !opts.IncludeSystemEnvVars.Valid { // If not explicitly set via CLI flags, look for an environment variable + if envVar, ok := environment["K6_INCLUDE_SYSTEM_ENV_VARS"]; ok { + val, err := strconv.ParseBool(envVar) + if err != nil { + return opts, err + } + opts.IncludeSystemEnvVars = null.BoolFrom(val) + } + } + + if opts.IncludeSystemEnvVars.Bool { // If enabled, gather the actual system environment variables + opts.Env = environment } // Set/overwrite environment variables with custom user-supplied values @@ -80,7 +103,6 @@ func getRuntimeOptions(flags *pflag.FlagSet) (lib.RuntimeOptions, error) { if err != nil { return opts, err } - for _, kv := range envVars { k, v := parseEnvKeyValue(kv) // Allow only alphanumeric ASCII variable names for now @@ -90,11 +112,5 @@ func getRuntimeOptions(flags *pflag.FlagSet) (lib.RuntimeOptions, error) { opts.Env[k] = v } - // Fallback to env - compatMode := opts.Env["K6_COMPATIBILITY_MODE"] - if !opts.CompatibilityMode.Valid && compatMode != "" { - opts.CompatibilityMode = null.StringFrom(compatMode) - } - return opts, nil } diff --git a/cmd/runtime_options_test.go b/cmd/runtime_options_test.go index 2321f691cb6..ceb7005f0a3 100644 --- a/cmd/runtime_options_test.go +++ b/cmd/runtime_options_test.go @@ -24,11 +24,11 @@ import ( "bytes" "fmt" "net/url" - "os" - "runtime" "strings" "testing" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" "github.com/spf13/afero" @@ -36,234 +36,243 @@ import ( "github.com/stretchr/testify/require" ) -var envVars []string - -func init() { - envVars = os.Environ() +type runtimeOptionsTestCase struct { + useSysEnv bool // Whether to include the system env vars by default (run) or not (cloud/archive/inspect) + expErr bool + cliFlags []string + systemEnv map[string]string + expEnv map[string]string + expCompatMode null.String } -type EnvVarTest struct { - name string - useSysEnv bool // Whether to include the system env vars by default (run) or not (cloud/archive/inspect) - systemEnv map[string]string - cliOpts []string - expErr bool - expEnv map[string]string -} +//nolint:gochecknoglobals +var ( + defaultCompatMode = null.NewString("extended", false) + baseCompatMode = null.NewString("base", true) + extendedCompatMode = null.NewString("extended", true) +) -var envVarTestCases = []EnvVarTest{ - { - "empty env", - true, - map[string]string{}, - []string{}, - false, - map[string]string{}, +var runtimeOptionsTestCases = map[string]runtimeOptionsTestCase{ //nolint:gochecknoglobals + "empty env": { + useSysEnv: true, + // everything else is empty + expCompatMode: defaultCompatMode, + }, + "disabled sys env by default": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1"}, + expEnv: map[string]string{}, + expCompatMode: defaultCompatMode, + }, + "disabled sys env by default with ext compat mode": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1", "K6_COMPATIBILITY_MODE": "extended"}, + expEnv: map[string]string{}, + expCompatMode: extendedCompatMode, + }, + "disabled sys env by cli 1": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "val1", "K6_COMPATIBILITY_MODE": "base"}, + cliFlags: []string{"--include-system-env-vars=false"}, + expEnv: map[string]string{}, + expCompatMode: baseCompatMode, + }, + "disabled sys env by cli 2": { + useSysEnv: true, + systemEnv: map[string]string{"K6_INCLUDE_SYSTEM_ENV_VARS": "true", "K6_COMPATIBILITY_MODE": "extended"}, + cliFlags: []string{"--include-system-env-vars=0", "--compatibility-mode=base"}, + expEnv: map[string]string{}, + expCompatMode: baseCompatMode, }, - { - "disabled sys env by default", - false, - map[string]string{"test1": "val1"}, - []string{}, - false, - map[string]string{}, + "disabled sys env by env": { + useSysEnv: true, + systemEnv: map[string]string{"K6_INCLUDE_SYSTEM_ENV_VARS": "false", "K6_COMPATIBILITY_MODE": "extended"}, + expEnv: map[string]string{}, + expCompatMode: extendedCompatMode, }, - { - "disabled sys env by cli 1", - true, - map[string]string{"test1": "val1"}, - []string{"--include-system-env-vars=false"}, - false, - map[string]string{}, + "enabled sys env by env": { + useSysEnv: false, + systemEnv: map[string]string{"K6_INCLUDE_SYSTEM_ENV_VARS": "true", "K6_COMPATIBILITY_MODE": "extended"}, + expEnv: map[string]string{"K6_INCLUDE_SYSTEM_ENV_VARS": "true", "K6_COMPATIBILITY_MODE": "extended"}, + expCompatMode: extendedCompatMode, }, - { - "disabled sys env by cli 2", - true, - map[string]string{"test1": "val1"}, - []string{"--include-system-env-vars=0"}, - false, - map[string]string{}, + "enabled sys env by default": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "val1"}, + cliFlags: []string{}, + expEnv: map[string]string{"test1": "val1"}, + expCompatMode: defaultCompatMode, }, - { - "enabled sys env by default", - true, - map[string]string{"test1": "val1"}, - []string{}, - false, - map[string]string{"test1": "val1"}, + "enabled sys env by cli 1": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1"}, + cliFlags: []string{"--include-system-env-vars"}, + expEnv: map[string]string{"test1": "val1"}, + expCompatMode: defaultCompatMode, }, - { - "enabled sys env by cli 1", - false, - map[string]string{"test1": "val1"}, - []string{"--include-system-env-vars"}, - false, - map[string]string{"test1": "val1"}, + "enabled sys env by cli 2": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1"}, + cliFlags: []string{"--include-system-env-vars=true"}, + expEnv: map[string]string{"test1": "val1"}, + expCompatMode: defaultCompatMode, }, - { - "enabled sys env by cli 2", - false, - map[string]string{"test1": "val1"}, - []string{"--include-system-env-vars=true"}, - false, - map[string]string{"test1": "val1"}, + "run only system env": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "val1"}, + cliFlags: []string{}, + expEnv: map[string]string{"test1": "val1"}, + expCompatMode: defaultCompatMode, }, - { - "run only system env", - true, - map[string]string{"test1": "val1"}, - []string{}, - false, - map[string]string{"test1": "val1"}, + "mixed system and cli env": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "val1", "test2": ""}, + cliFlags: []string{"--env", "test3=val3", "-e", "test4", "-e", "test5="}, + expEnv: map[string]string{"test1": "val1", "test2": "", "test3": "val3", "test4": "", "test5": ""}, + expCompatMode: defaultCompatMode, }, - { - "mixed system and cli env", - true, - map[string]string{"test1": "val1", "test2": ""}, - []string{"--env", "test3=val3", "-e", "test4", "-e", "test5="}, - false, - map[string]string{"test1": "val1", "test2": "", "test3": "val3", "test4": "", "test5": ""}, + "mixed system and cli env 2": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1", "test2": ""}, + cliFlags: []string{"--env", "test3=val3", "-e", "test4", "-e", "test5=", "--include-system-env-vars=1"}, + expEnv: map[string]string{"test1": "val1", "test2": "", "test3": "val3", "test4": "", "test5": ""}, + expCompatMode: defaultCompatMode, }, - { - "mixed system and cli env 2", - false, - map[string]string{"test1": "val1", "test2": ""}, - []string{"--env", "test3=val3", "-e", "test4", "-e", "test5=", "--include-system-env-vars=1"}, - false, - map[string]string{"test1": "val1", "test2": "", "test3": "val3", "test4": "", "test5": ""}, + "disabled system env with cli params": { + useSysEnv: false, + systemEnv: map[string]string{"test1": "val1"}, + cliFlags: []string{"-e", "test2=overwriten", "-e", "test2=val2"}, + expEnv: map[string]string{"test2": "val2"}, + expCompatMode: defaultCompatMode, }, - { - "disabled system env with cli params", - false, - map[string]string{"test1": "val1"}, - []string{"-e", "test2=overwriten", "-e", "test2=val2"}, - false, - map[string]string{"test2": "val2"}, + "overwriting system env with cli param": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "val1sys"}, + cliFlags: []string{"--env", "test1=val1cli"}, + expEnv: map[string]string{"test1": "val1cli"}, + expCompatMode: defaultCompatMode, }, - { - "overwriting system env with cli param", - true, - map[string]string{"test1": "val1sys"}, - []string{"--env", "test1=val1cli"}, - false, - map[string]string{"test1": "val1cli"}, + "error wrong compat mode env var value": { + systemEnv: map[string]string{"K6_COMPATIBILITY_MODE": "asdf"}, + expErr: true, }, - { - "error invalid cli var name 1", - true, - map[string]string{}, - []string{"--env", "test a=error"}, - true, - map[string]string{}, + "error wrong compat mode cli flag value": { + cliFlags: []string{"--compatibility-mode", "whatever"}, + expErr: true, }, - { - "error invalid cli var name 2", - true, - map[string]string{}, - []string{"--env", "1var=error"}, - true, - map[string]string{}, + "error invalid cli var name 1": { + useSysEnv: true, + systemEnv: map[string]string{}, + cliFlags: []string{"--env", "test a=error"}, + expErr: true, + expEnv: map[string]string{}, + expCompatMode: defaultCompatMode, }, - { - "error invalid cli var name 3", - true, - map[string]string{}, - []string{"--env", "уникод=unicode-disabled"}, - true, - map[string]string{}, + "error invalid cli var name 2": { + useSysEnv: true, + systemEnv: map[string]string{}, + cliFlags: []string{"--env", "1var=error"}, + expErr: true, + expEnv: map[string]string{}, + expCompatMode: defaultCompatMode, }, - { - "valid env vars with spaces", - true, - map[string]string{"test1": "value 1"}, - []string{"--env", "test2=value 2"}, - false, - map[string]string{"test1": "value 1", "test2": "value 2"}, + "error invalid cli var name 3": { + useSysEnv: true, + systemEnv: map[string]string{}, + cliFlags: []string{"--env", "уникод=unicode-disabled"}, + expErr: true, + expEnv: map[string]string{}, + expCompatMode: defaultCompatMode, }, - { - "valid env vars with special chars", - true, - map[string]string{"test1": "value 1"}, - []string{"--env", "test2=value,2", "-e", `test3= , ,,, value, ,, 2!'@#,"`}, - false, - map[string]string{"test1": "value 1", "test2": "value,2", "test3": ` , ,,, value, ,, 2!'@#,"`}, + "valid env vars with spaces": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "value 1"}, + cliFlags: []string{"--env", "test2=value 2"}, + expEnv: map[string]string{"test1": "value 1", "test2": "value 2"}, + expCompatMode: defaultCompatMode, + }, + "valid env vars with special chars": { + useSysEnv: true, + systemEnv: map[string]string{"test1": "value 1"}, + cliFlags: []string{"--env", "test2=value,2", "-e", `test3= , ,,, value, ,, 2!'@#,"`}, + expEnv: map[string]string{"test1": "value 1", "test2": "value,2", "test3": ` , ,,, value, ,, 2!'@#,"`}, + expCompatMode: defaultCompatMode, }, } -func TestEnvVars(t *testing.T) { - for _, tc := range envVarTestCases { - t.Run(fmt.Sprintf("EnvVar test '%s'", tc.name), func(t *testing.T) { - os.Clearenv() - for key, val := range tc.systemEnv { - require.NoError(t, os.Setenv(key, val)) - } - flags := runtimeOptionFlagSet(tc.useSysEnv) - require.NoError(t, flags.Parse(tc.cliOpts)) +func testRuntimeOptionsCase(t *testing.T, tc runtimeOptionsTestCase) { + flags := runtimeOptionFlagSet(tc.useSysEnv) + require.NoError(t, flags.Parse(tc.cliFlags)) + + rtOpts, err := getRuntimeOptions(flags, tc.systemEnv) + if tc.expErr { + require.Error(t, err) + return + } + require.NoError(t, err) + require.EqualValues(t, tc.expEnv, rtOpts.Env) + assert.Equal(t, tc.expCompatMode, rtOpts.CompatibilityMode) - rtOpts, err := getRuntimeOptions(flags) - if tc.expErr { - require.Error(t, err) - return - } - require.NoError(t, err) - require.EqualValues(t, tc.expEnv, rtOpts.Env) + compatMode, err := lib.ValidateCompatibilityMode(rtOpts.CompatibilityMode.String) + require.NoError(t, err) - // Clear the env again so real system values don't accidentally pollute the end-to-end test - os.Clearenv() + jsCodeLines := make([]string, 0, len(tc.expEnv)+2) + if compatMode == lib.CompatibilityModeExtended { + jsCodeLines = append(jsCodeLines, "export default function() {") + } else { + jsCodeLines = append(jsCodeLines, "module.exports.default = function() {") + } - jsCode := "export default function() {\n" - for key, val := range tc.expEnv { - jsCode += fmt.Sprintf( - "if (__ENV.%s !== `%s`) { throw new Error('Invalid %s: ' + __ENV.%s); }\n", - key, val, key, key, - ) - } - jsCode += "}" + for key, val := range tc.expEnv { + jsCodeLines = append(jsCodeLines, fmt.Sprintf( + "if (__ENV.%s !== `%s`) { throw new Error('Invalid %s: ' + __ENV.%s); }", + key, val, key, key, + )) + } + jsCodeLines = append(jsCodeLines, "}") + jsCode := []byte(strings.Join(jsCodeLines, "\n")) - // windows requires the environment variables to be loaded to gerenate the rand source - if runtime.GOOS == "windows" { - for _, e := range envVars { - parts := strings.Split(e, "=") - os.Setenv(parts[0], parts[1]) - } - } + fs := afero.NewMemMapFs() + require.NoError(t, afero.WriteFile(fs, "/script.js", jsCode, 0644)) + runner, err := newRunner( + &loader.SourceData{Data: jsCode, URL: &url.URL{Path: "/script.js", Scheme: "file"}}, + typeJS, + map[string]afero.Fs{"file": fs}, + rtOpts, + ) + require.NoError(t, err) - fs := afero.NewMemMapFs() - require.NoError(t, afero.WriteFile(fs, "/script.js", []byte(jsCode), 0644)) - runner, err := newRunner( - &loader.SourceData{ - Data: []byte(jsCode), - URL: &url.URL{Path: "/script.js", Scheme: "file"}, - }, - typeJS, - map[string]afero.Fs{"file": fs}, - rtOpts, - ) - require.NoError(t, err) + archive := runner.MakeArchive() + archiveBuf := &bytes.Buffer{} + assert.NoError(t, archive.Write(archiveBuf)) - archive := runner.MakeArchive() - archiveBuf := &bytes.Buffer{} - assert.NoError(t, archive.Write(archiveBuf)) + getRunnerErr := func(rtOpts lib.RuntimeOptions) (lib.Runner, error) { + return newRunner( + &loader.SourceData{ + Data: archiveBuf.Bytes(), + URL: &url.URL{Path: "/script.js"}, + }, + typeArchive, + nil, + rtOpts, + ) + } - getRunnerErr := func(rtOpts lib.RuntimeOptions) (lib.Runner, error) { - return newRunner( - &loader.SourceData{ - Data: archiveBuf.Bytes(), - URL: &url.URL{Path: "/script.js"}, - }, - typeArchive, - nil, - rtOpts, - ) - } + _, err = getRunnerErr(lib.RuntimeOptions{}) + require.NoError(t, err) + for key, val := range tc.expEnv { + r, err := getRunnerErr(lib.RuntimeOptions{Env: map[string]string{key: "almost " + val}}) + assert.NoError(t, err) + assert.Equal(t, r.MakeArchive().Env[key], "almost "+val) + } +} - _, err = getRunnerErr(lib.RuntimeOptions{}) - require.NoError(t, err) - for key, val := range tc.expEnv { - r, err := getRunnerErr(lib.RuntimeOptions{Env: map[string]string{key: "almost " + val}}) - assert.NoError(t, err) - assert.Equal(t, r.MakeArchive().Env[key], "almost "+val) - } +func TestRuntimeOptions(t *testing.T) { + for name, tc := range runtimeOptionsTestCases { + tc := tc + t.Run(fmt.Sprintf("RuntimeOptions test '%s'", name), func(t *testing.T) { + t.Parallel() + testRuntimeOptionsCase(t, tc) }) } } diff --git a/js/bundle.go b/js/bundle.go index b473b33b400..526c6a96e67 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -137,7 +137,13 @@ func NewBundleFromArchive(arc *lib.Archive, rtOpts lib.RuntimeOptions) (*Bundle, return nil, errors.Errorf("expected bundle type 'js', got '%s'", arc.Type) } - compatMode, err := lib.ValidateCompatibilityMode(arc.CompatibilityMode) + compatModeStr := arc.CompatibilityMode + if rtOpts.CompatibilityMode.Valid { + // `k6 run --compatibility-mode=whatever archive.tar` should override + // whatever value is in the archive + compatModeStr = rtOpts.CompatibilityMode.String + } + compatMode, err := lib.ValidateCompatibilityMode(compatModeStr) if err != nil { return nil, err } diff --git a/js/bundle_test.go b/js/bundle_test.go index 0b48ae160f5..dc3c50a8188 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -415,87 +415,101 @@ func TestNewBundle(t *testing.T) { }) } +func getArchive(data string, rtOpts lib.RuntimeOptions) (*lib.Archive, error) { + b, err := getSimpleBundle("script.js", data, rtOpts) + if err != nil { + return nil, err + } + return b.makeArchive(), nil +} + func TestNewBundleFromArchive(t *testing.T) { - getArchive := func(data string, rtOpts lib.RuntimeOptions) (*lib.Archive, error) { - b, err := getSimpleBundle("script.js", data, rtOpts) - if err != nil { - return nil, err - } - return b.makeArchive(), nil + t.Parallel() + + es5Code := `module.exports.options = { vus: 12345 }; module.exports.default = function() { return "hi!" };` + es6Code := `export let options = { vus: 12345 }; export default function() { return "hi!"; };` + baseCompatModeRtOpts := lib.RuntimeOptions{CompatibilityMode: null.StringFrom(lib.CompatibilityModeBase.String())} + extCompatModeRtOpts := lib.RuntimeOptions{CompatibilityMode: null.StringFrom(lib.CompatibilityModeExtended.String())} + + checkBundle := func(t *testing.T, b *Bundle) { + assert.Equal(t, lib.Options{VUs: null.IntFrom(12345)}, b.Options) + bi, err := b.Instantiate() + require.NoError(t, err) + val, err := bi.Default(goja.Undefined()) + require.NoError(t, err) + assert.Equal(t, "hi!", val.Export()) } - t.Run("ok", func(t *testing.T) { - testCases := []struct { - compatMode, code string - }{ - // An empty value will assume "extended" - {"", ` - export let options = { vus: 12345 }; - export default function() { return "hi!"; };`}, - {lib.CompatibilityModeExtended.String(), ` - export let options = { vus: 12345 }; - export default function() { return "hi!"; };`}, - {lib.CompatibilityModeBase.String(), ` - module.exports.options = { vus: 12345 }; - module.exports.default = function() { return "hi!" };`}, + checkArchive := func(t *testing.T, arc *lib.Archive, rtOpts lib.RuntimeOptions, expError string) { + b, err := NewBundleFromArchive(arc, rtOpts) + if expError != "" { + require.Error(t, err) + assert.Contains(t, err.Error(), expError) + } else { + require.NoError(t, err) + checkBundle(t, b) } + } - for _, tc := range testCases { - tc := tc - t.Run(tc.compatMode, func(t *testing.T) { - rtOpts := lib.RuntimeOptions{CompatibilityMode: null.StringFrom(tc.compatMode)} - arc, err := getArchive(tc.code, rtOpts) - assert.NoError(t, err) - b, err := NewBundleFromArchive(arc, rtOpts) - if !assert.NoError(t, err) { - return - } - assert.Equal(t, lib.Options{VUs: null.IntFrom(12345)}, b.Options) - expCM := tc.compatMode - if expCM == "" { - expCM = lib.CompatibilityModeExtended.String() - } - assert.Equal(t, expCM, b.CompatibilityMode.String()) + t.Run("es6_script_default", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es6Code, lib.RuntimeOptions{}) // default options + require.NoError(t, err) + require.Equal(t, lib.CompatibilityModeExtended.String(), arc.CompatibilityMode) - bi, err := b.Instantiate() - if !assert.NoError(t, err) { - return - } - val, err := bi.Default(goja.Undefined()) - if !assert.NoError(t, err) { - return - } - assert.Equal(t, "hi!", val.Export()) - }) - } + checkArchive(t, arc, lib.RuntimeOptions{}, "") // default options + checkArchive(t, arc, extCompatModeRtOpts, "") + checkArchive(t, arc, baseCompatModeRtOpts, "Unexpected reserved word") }) - t.Run("err", func(t *testing.T) { - testCases := []struct { - compatMode, code, expErr string - }{ - // Incompatible mode - { - lib.CompatibilityModeBase.String(), ` - export let options = { vus: 12345 }; - export default function() { return "hi!"; };`, - "file://script.js: Line 2:5 Unexpected reserved word (and 2 more errors)", - }, - { - "wrongcompat", ` - export let options = { vus: 12345 }; - export default function() { return "hi!"; };`, - `invalid compatibility mode "wrongcompat". Use: "extended", "base"`, - }, - } - for _, tc := range testCases { - tc := tc - t.Run(tc.compatMode, func(t *testing.T) { - rtOpts := lib.RuntimeOptions{CompatibilityMode: null.StringFrom(tc.compatMode)} - _, err := getArchive(tc.code, rtOpts) - assert.EqualError(t, err, tc.expErr) - }) - } + t.Run("es6_script_explicit", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es6Code, extCompatModeRtOpts) + require.NoError(t, err) + require.Equal(t, lib.CompatibilityModeExtended.String(), arc.CompatibilityMode) + + checkArchive(t, arc, lib.RuntimeOptions{}, "") + checkArchive(t, arc, extCompatModeRtOpts, "") + checkArchive(t, arc, baseCompatModeRtOpts, "Unexpected reserved word") + }) + + t.Run("es5_script_with_extended", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es5Code, lib.RuntimeOptions{}) + require.NoError(t, err) + require.Equal(t, lib.CompatibilityModeExtended.String(), arc.CompatibilityMode) + + checkArchive(t, arc, lib.RuntimeOptions{}, "") + checkArchive(t, arc, extCompatModeRtOpts, "") + checkArchive(t, arc, baseCompatModeRtOpts, "") + }) + + t.Run("es5_script", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es5Code, baseCompatModeRtOpts) + require.NoError(t, err) + require.Equal(t, lib.CompatibilityModeBase.String(), arc.CompatibilityMode) + + checkArchive(t, arc, lib.RuntimeOptions{}, "") + checkArchive(t, arc, extCompatModeRtOpts, "") + checkArchive(t, arc, baseCompatModeRtOpts, "") + }) + + t.Run("es6_archive_with_wrong_compat_mode", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es6Code, baseCompatModeRtOpts) + require.Error(t, err) + require.Nil(t, arc) + }) + + t.Run("messed_up_archive", func(t *testing.T) { + t.Parallel() + arc, err := getArchive(es6Code, extCompatModeRtOpts) + require.NoError(t, err) + arc.CompatibilityMode = "blah" // intentionally break the archive + checkArchive(t, arc, lib.RuntimeOptions{}, "invalid compatibility mode") // fails when it uses the archive one + checkArchive(t, arc, extCompatModeRtOpts, "") // works when I force the compat mode + checkArchive(t, arc, baseCompatModeRtOpts, "Unexpected reserved word") // failes because of ES6 }) } diff --git a/lib/runtime_options.go b/lib/runtime_options.go index 566b94d9c76..4d7e4f04617 100644 --- a/lib/runtime_options.go +++ b/lib/runtime_options.go @@ -42,29 +42,18 @@ const ( // RuntimeOptions are settings passed onto the goja JS runtime type RuntimeOptions struct { // Whether to pass the actual system environment variables to the JS runtime - IncludeSystemEnvVars null.Bool `json:"includeSystemEnvVars" envconfig:"K6_INCLUDE_SYSTEM_ENV_VARS"` + IncludeSystemEnvVars null.Bool `json:"includeSystemEnvVars"` // JS compatibility mode: "extended" (Goja+Babel+core.js) or "base" (plain Goja) - // TODO: maybe use CompatibilityMode directly? this seems strange... + // + // TODO: when we resolve https://github.com/loadimpact/k6/issues/883, we probably + // should use the CompatibilityMode type directly... but by then, we'd need to have + // some way of knowing if the value has been set by the user or if we're using the + // default one, so we can handle `k6 run --compatibility-mode=base es6_extended_archive.tar` CompatibilityMode null.String `json:"compatibilityMode"` // Environment variables passed onto the runner - Env map[string]string `json:"env" envconfig:"K6_ENV"` -} - -// Apply overwrites the receiver RuntimeOptions' fields with any that are set -// on the argument struct and returns the receiver -func (o RuntimeOptions) Apply(opts RuntimeOptions) RuntimeOptions { - if opts.IncludeSystemEnvVars.Valid { - o.IncludeSystemEnvVars = opts.IncludeSystemEnvVars - } - if opts.CompatibilityMode.Valid { - o.CompatibilityMode = opts.CompatibilityMode - } - if opts.Env != nil { - o.Env = opts.Env - } - return o + Env map[string]string `json:"env"` } // ValidateCompatibilityMode checks if the provided val is a valid compatibility mode From c9d75bd1cb74cafd11182125da5ffe2d8e61b37b Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 12 Feb 2020 14:39:47 +0200 Subject: [PATCH 091/350] Fix minor issues in the runtime options test --- cmd/runtime_options_test.go | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/cmd/runtime_options_test.go b/cmd/runtime_options_test.go index ceb7005f0a3..60b8ce983f0 100644 --- a/cmd/runtime_options_test.go +++ b/cmd/runtime_options_test.go @@ -24,7 +24,6 @@ import ( "bytes" "fmt" "net/url" - "strings" "testing" "gopkg.in/guregu/null.v3" @@ -216,26 +215,25 @@ func testRuntimeOptionsCase(t *testing.T, tc runtimeOptionsTestCase) { compatMode, err := lib.ValidateCompatibilityMode(rtOpts.CompatibilityMode.String) require.NoError(t, err) - jsCodeLines := make([]string, 0, len(tc.expEnv)+2) + jsCode := new(bytes.Buffer) if compatMode == lib.CompatibilityModeExtended { - jsCodeLines = append(jsCodeLines, "export default function() {") + fmt.Fprint(jsCode, "export default function() {") } else { - jsCodeLines = append(jsCodeLines, "module.exports.default = function() {") + fmt.Fprint(jsCode, "module.exports.default = function() {") } for key, val := range tc.expEnv { - jsCodeLines = append(jsCodeLines, fmt.Sprintf( + fmt.Fprintf(jsCode, "if (__ENV.%s !== `%s`) { throw new Error('Invalid %s: ' + __ENV.%s); }", key, val, key, key, - )) + ) } - jsCodeLines = append(jsCodeLines, "}") - jsCode := []byte(strings.Join(jsCodeLines, "\n")) + fmt.Fprint(jsCode, "}") fs := afero.NewMemMapFs() - require.NoError(t, afero.WriteFile(fs, "/script.js", jsCode, 0644)) + require.NoError(t, afero.WriteFile(fs, "/script.js", jsCode.Bytes(), 0644)) runner, err := newRunner( - &loader.SourceData{Data: jsCode, URL: &url.URL{Path: "/script.js", Scheme: "file"}}, + &loader.SourceData{Data: jsCode.Bytes(), URL: &url.URL{Path: "/script.js", Scheme: "file"}}, typeJS, map[string]afero.Fs{"file": fs}, rtOpts, @@ -244,7 +242,7 @@ func testRuntimeOptionsCase(t *testing.T, tc runtimeOptionsTestCase) { archive := runner.MakeArchive() archiveBuf := &bytes.Buffer{} - assert.NoError(t, archive.Write(archiveBuf)) + require.NoError(t, archive.Write(archiveBuf)) getRunnerErr := func(rtOpts lib.RuntimeOptions) (lib.Runner, error) { return newRunner( From 9095741dd3b95af915c01fc24c2a10b76f3101cd Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 13 Feb 2020 10:36:45 +0200 Subject: [PATCH 092/350] Move testutils.MiniRunner to its own package to avoid import cycles --- api/server_test.go | 4 +- api/v1/group_routes_test.go | 4 +- api/v1/metric_routes_test.go | 6 +- api/v1/status_routes_test.go | 6 +- cmd/config_consolidation_test.go | 31 ++++----- core/engine_test.go | 8 ++- core/local/local_test.go | 63 ++++++++++--------- lib/executor/common_test.go | 5 +- lib/executor/execution_test.go | 3 +- .../minirunner.go} | 20 +++--- 10 files changed, 80 insertions(+), 70 deletions(-) rename lib/testutils/{mini_runner.go => minirunner/minirunner.go} (89%) diff --git a/api/server_test.go b/api/server_test.go index 25128a1de8d..27299274b88 100644 --- a/api/server_test.go +++ b/api/server_test.go @@ -36,7 +36,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" ) func testHTTPHandler(rw http.ResponseWriter, r *http.Request) { @@ -77,7 +77,7 @@ func TestLogger(t *testing.T) { } func TestWithEngine(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index 7b9241556a1..bb44adefceb 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -29,7 +29,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -44,7 +44,7 @@ func TestGetGroups(t *testing.T) { g2, err := g1.Group("group 2") assert.NoError(t, err) - execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{Group: g0}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{Group: g0}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/metric_routes_test.go b/api/v1/metric_routes_test.go index b276567ba99..9b81eb90ef0 100644 --- a/api/v1/metric_routes_test.go +++ b/api/v1/metric_routes_test.go @@ -33,7 +33,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/stats" "github.com/manyminds/api2go/jsonapi" "github.com/stretchr/testify/assert" @@ -41,7 +41,7 @@ import ( ) func TestGetMetrics(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) @@ -82,7 +82,7 @@ func TestGetMetrics(t *testing.T) { } func TestGetMetric(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index a4dae2dfbc0..720c9389698 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -29,7 +29,7 @@ import ( "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" @@ -37,7 +37,7 @@ import ( ) func TestGetStatus(t *testing.T) { - execScheduler, err := local.NewExecutionScheduler(&testutils.MiniRunner{}, logrus.StandardLogger()) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{}, logrus.StandardLogger()) require.NoError(t, err) engine, err := core.NewEngine(execScheduler, lib.Options{}, logrus.StandardLogger()) require.NoError(t, err) @@ -66,7 +66,7 @@ func TestGetStatus(t *testing.T) { }) } -//TODO: fix after the externally-controlled executor +// TODO: fix after the externally-controlled executor /* func TestPatchStatus(t *testing.T) { testdata := map[string]struct { diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index fd8105159a2..d51a1be5525 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -37,13 +37,14 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" ) // A helper funcion for setting arbitrary environment variables and // restoring the old ones at the end, usually by deferring the returned callback -//TODO: remove these hacks when we improve the configuration... we shouldn't +// TODO: remove these hacks when we improve the configuration... we shouldn't // have to mess with the global environment at all... func setEnv(t *testing.T, newEnv []string) (restoreEnv func()) { actuallSetEnv := func(env []string, abortOnSetErr bool) { @@ -148,7 +149,7 @@ func buildStages(durationsAndVUs ...int64) []executor.Stage { } func mostFlagSets() []flagSetInit { - //TODO: make this unnecessary... currently these are the only commands in which + // TODO: make this unnecessary... currently these are the only commands in which // getConsolidatedConfig() is used, but they also have differences in their CLI flags :/ // sigh... compromises... result := []flagSetInit{} @@ -188,7 +189,7 @@ type opts struct { runner *lib.Options fs afero.Fs - //TODO: remove this when the configuration is more reproducible and sane... + // TODO: remove this when the configuration is more reproducible and sane... // We use a func, because initializing a FlagSet that points to variables // actually will change those variables to their default values :| In our // case, this happens only some of the time, for global variables that @@ -202,7 +203,7 @@ type opts struct { } func resetStickyGlobalVars() { - //TODO: remove after fixing the config, obviously a dirty hack + // TODO: remove after fixing the config, obviously a dirty hack exitOnRunning = false configFilePath = "" runType = "" @@ -344,7 +345,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { verifyVarLoopingVUs(null.NewInt(33, true), buildStages(44, 44, 55, 55)), }, - //TODO: test the future full overwriting of the duration/iterations/stages/execution options + // TODO: test the future full overwriting of the duration/iterations/stages/execution options { opts{ fs: defaultConfig(`{ @@ -358,8 +359,8 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { exp{}, verifySharedIters(I(12), I(25)), }, - //TODO: test the externally controlled executor - //TODO: test execution-segment + // TODO: test the externally controlled executor + // TODO: test execution-segment // Just in case, verify that no options will result in the same 1 vu 1 iter config {opts{}, exp{}, verifyOneIterPerOneVU}, @@ -372,8 +373,10 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { assert.Equal(t, stats.SystemTagSet(0), *c.Options.SystemTags) }}, { - opts{runner: &lib.Options{ - SystemTags: stats.NewSystemTagSet(stats.TagSubproto, stats.TagURL)}, + opts{ + runner: &lib.Options{ + SystemTags: stats.NewSystemTagSet(stats.TagSubproto, stats.TagURL), + }, }, exp{}, func(t *testing.T, c Config) { @@ -405,8 +408,8 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { assert.Equal(t, []string{"avg", "p(90)", "count"}, c.Options.SummaryTrendStats) }, }, - //TODO: test for differences between flagsets - //TODO: more tests in general, especially ones not related to execution parameters... + // TODO: test for differences between flagsets + // TODO: more tests in general, especially ones not related to execution parameters... } } @@ -427,7 +430,7 @@ func runTestCase( flagSet := newFlagSet() defer resetStickyGlobalVars() flagSet.SetOutput(output) - //flagSet.PrintDefaults() + // flagSet.PrintDefaults() cliErr := flagSet.Parse(testCase.options.cli) if testCase.expected.cliParseError { @@ -436,7 +439,7 @@ func runTestCase( } require.NoError(t, cliErr) - //TODO: remove these hacks when we improve the configuration... + // TODO: remove these hacks when we improve the configuration... var cliConf Config if flagSet.Lookup("out") != nil { cliConf, cliErr = getConfig(flagSet) @@ -452,7 +455,7 @@ func runTestCase( var runner lib.Runner if testCase.options.runner != nil { - runner = &testutils.MiniRunner{Options: *testCase.options.runner} + runner = &minirunner.MiniRunner{Options: *testCase.options.runner} } if testCase.options.fs == nil { t.Logf("Creating an empty FS for this test") diff --git a/core/engine_test.go b/core/engine_test.go index f0e2c911252..3d4e9a477ef 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -38,6 +38,7 @@ import ( "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/testutils/httpmultibin" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" @@ -51,7 +52,7 @@ const isWindows = runtime.GOOS == "windows" // Wrapper around NewEngine that applies a logger and manages the options. func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts lib.Options) *Engine { //nolint: golint if runner == nil { - runner = &testutils.MiniRunner{} + runner = &minirunner.MiniRunner{} } if ctx == nil { ctx = context.Background() @@ -109,7 +110,7 @@ func TestEngineRun(t *testing.T) { signalChan := make(chan interface{}) - runner := &testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + runner := &minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { stats.PushIfNotDone(ctx, out, stats.Sample{Metric: testMetric, Time: time.Now(), Value: 1}) close(signalChan) <-ctx.Done() @@ -158,7 +159,7 @@ func TestEngineAtTime(t *testing.T) { func TestEngineCollector(t *testing.T) { testMetric := stats.New("test_metric", stats.Trend) - runner := &testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + runner := &minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { out <- stats.Sample{Metric: testMetric} return nil }} @@ -339,6 +340,7 @@ func getMetricSum(collector *dummy.Collector, name string) (result float64) { } return } + func getMetricCount(collector *dummy.Collector, name string) (result uint) { for _, sc := range collector.SampleContainers { for _, s := range sc.GetSamples() { diff --git a/core/local/local_test.go b/core/local/local_test.go index 78d77b7bba3..f420e546564 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -36,6 +36,7 @@ import ( "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/netext" "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" @@ -50,7 +51,7 @@ func newTestExecutionScheduler( t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, //nolint: golint ) (ctx context.Context, cancel func(), execScheduler *ExecutionScheduler, samples chan stats.SampleContainer) { if runner == nil { - runner = &testutils.MiniRunner{} + runner = &minirunner.MiniRunner{} } ctx, cancel = context.WithCancel(context.Background()) newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ @@ -100,7 +101,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Run("Normal", func(t *testing.T) { setupC := make(chan struct{}) teardownC := make(chan struct{}) - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { close(setupC) return nil, nil @@ -120,7 +121,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.NoError(t, <-err) }) t.Run("Setup Error", func(t *testing.T) { - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, errors.New("setup error") }, @@ -130,7 +131,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.EqualError(t, execScheduler.Run(ctx, samples), "setup error") }) t.Run("Don't Run Setup", func(t *testing.T) { - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, errors.New("setup error") }, @@ -148,7 +149,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { }) t.Run("Teardown Error", func(t *testing.T) { - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, nil }, @@ -165,7 +166,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") }) t.Run("Don't Run Teardown", func(t *testing.T) { - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ SetupFn: func(ctx context.Context, out chan<- stats.SampleContainer) ([]byte, error) { return nil, nil }, @@ -212,7 +213,7 @@ func TestExecutionSchedulerStages(t *testing.T) { data := data t.Run(name, func(t *testing.T) { t.Parallel() - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(100 * time.Millisecond) return nil @@ -231,7 +232,7 @@ func TestExecutionSchedulerStages(t *testing.T) { func TestExecutionSchedulerEndTime(t *testing.T) { t.Parallel() - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(100 * time.Millisecond) return nil @@ -256,7 +257,7 @@ func TestExecutionSchedulerEndTime(t *testing.T) { func TestExecutionSchedulerRuntimeErrors(t *testing.T) { t.Parallel() - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { time.Sleep(10 * time.Millisecond) return errors.New("hi") @@ -294,7 +295,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { exec.Duration = types.NullDurationFrom(1 * time.Second) exec.GracefulStop = types.NullDurationFrom(0 * time.Second) - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { <-ctx.Done() return errors.New("hi") @@ -332,7 +333,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { require.Empty(t, options.Validate()) var i int64 - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { select { case <-ctx.Done(): @@ -361,7 +362,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { assert.Equal(t, uint64(100), execScheduler.GetState().GetFullIterationCount()) assert.Equal(t, uint64(0), execScheduler.GetState().GetPartialIterationCount()) assert.Equal(t, int64(100), i) - require.Equal(t, 100, len(samples)) //TODO: change to 200 https://github.com/loadimpact/k6/issues/1250 + require.Equal(t, 100, len(samples)) // TODO: change to 200 https://github.com/loadimpact/k6/issues/1250 for i := 0; i < 100; i++ { mySample, ok := <-samples require.True(t, ok) @@ -371,7 +372,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { func TestExecutionSchedulerIsRunning(t *testing.T) { t.Parallel() - runner := &testutils.MiniRunner{ + runner := &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { <-ctx.Done() return nil @@ -404,7 +405,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { }) t.Run("Raise", func(t *testing.T) { - e := New(&testutils.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + e := New(&minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { return nil }}) e.ctx = context.Background() @@ -416,7 +417,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { num++ if assert.NotNil(t, handle.vu, "vu %d lacks impl", i) { - assert.Equal(t, int64(0), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(0), handle.vu.(*minirunner.VU).ID) } assert.Nil(t, handle.ctx, "vu %d has ctx", i) assert.Nil(t, handle.cancel, "vu %d has cancel", i) @@ -431,11 +432,11 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { if i < 50 { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) num++ } else { assert.Nil(t, handle.cancel, "vu %d has cancel", i) - assert.Equal(t, int64(0), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(0), handle.vu.(*minirunner.VU).ID) } } assert.Equal(t, 50, num) @@ -447,7 +448,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { num := 0 for i, handle := range e.vus { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) num++ } assert.Equal(t, 100, num) @@ -465,7 +466,7 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { } else { assert.Nil(t, handle.cancel, "vu %d has cancel", i) } - assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) } assert.Equal(t, 50, num) } @@ -477,9 +478,9 @@ func TestExecutionSchedulerSetVUs(t *testing.T) { for i, handle := range e.vus { assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) if i < 50 { - assert.Equal(t, int64(i+1), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) } else { - assert.Equal(t, int64(50+i+1), handle.vu.(*testutils.MiniRunnerVU).ID) + assert.Equal(t, int64(50+i+1), handle.vu.(*minirunner.VU).ID) } } } @@ -658,10 +659,10 @@ func (p pausableExecutor) SetPaused(bool) error { func TestSetPaused(t *testing.T) { t.Run("second pause is an error", func(t *testing.T) { - var runner = &testutils.MiniRunner{} + runner := &minirunner.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - var sched, err = NewExecutionScheduler(runner, logger) + sched, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) sched.executors = []lib.Executor{pausableExecutor{err: nil}} @@ -672,10 +673,10 @@ func TestSetPaused(t *testing.T) { }) t.Run("unpause at the start is an error", func(t *testing.T) { - var runner = &testutils.MiniRunner{} + runner := &minirunner.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - var sched, err = NewExecutionScheduler(runner, logger) + sched, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) sched.executors = []lib.Executor{pausableExecutor{err: nil}} err = sched.SetPaused(false) @@ -684,10 +685,10 @@ func TestSetPaused(t *testing.T) { }) t.Run("second unpause is an error", func(t *testing.T) { - var runner = &testutils.MiniRunner{} + runner := &minirunner.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - var sched, err = NewExecutionScheduler(runner, logger) + sched, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) sched.executors = []lib.Executor{pausableExecutor{err: nil}} require.NoError(t, sched.SetPaused(true)) @@ -698,12 +699,12 @@ func TestSetPaused(t *testing.T) { }) t.Run("an error on pausing is propagated", func(t *testing.T) { - var runner = &testutils.MiniRunner{} + runner := &minirunner.MiniRunner{} logger := logrus.New() logger.SetOutput(testutils.NewTestOutput(t)) - var sched, err = NewExecutionScheduler(runner, logger) + sched, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) - var expectedErr = errors.New("testing pausable executor error") + expectedErr := errors.New("testing pausable executor error") sched.executors = []lib.Executor{pausableExecutor{err: expectedErr}} err = sched.SetPaused(true) require.Error(t, err) @@ -711,7 +712,7 @@ func TestSetPaused(t *testing.T) { }) t.Run("can't pause unpausable executor", func(t *testing.T) { - var runner = &testutils.MiniRunner{} + runner := &minirunner.MiniRunner{} options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ Iterations: null.IntFrom(2), VUs: null.IntFrom(1), diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index eb606cd7f2e..fe08e670df1 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -7,13 +7,14 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" ) func simpleRunner(vuFn func(context.Context) error) lib.Runner { - return &testutils.MiniRunner{ + return &minirunner.MiniRunner{ Fn: func(ctx context.Context, _ chan<- stats.SampleContainer) error { return vuFn(ctx) }, @@ -24,7 +25,7 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, runner lib.Runner) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook, ) { ctx, cancel := context.WithCancel(context.Background()) - engineOut := make(chan stats.SampleContainer, 100) //TODO: return this for more complicated tests? + engineOut := make(chan stats.SampleContainer, 100) // TODO: return this for more complicated tests? logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} testLog := logrus.New() diff --git a/lib/executor/execution_test.go b/lib/executor/execution_test.go index 37770b57be5..05fbc99da5f 100644 --- a/lib/executor/execution_test.go +++ b/lib/executor/execution_test.go @@ -30,6 +30,7 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -83,7 +84,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { es := lib.NewExecutionState(lib.Options{}, 10, 20) es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { - return &testutils.MiniRunnerVU{}, nil + return &minirunner.VU{}, nil }) for i := 0; i < 10; i++ { diff --git a/lib/testutils/mini_runner.go b/lib/testutils/minirunner/minirunner.go similarity index 89% rename from lib/testutils/mini_runner.go rename to lib/testutils/minirunner/minirunner.go index 11f6681f499..ab7c2babdb1 100644 --- a/lib/testutils/mini_runner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -18,7 +18,7 @@ * */ -package testutils +package minirunner import ( "context" @@ -29,8 +29,10 @@ import ( ) // Ensure mock implementations conform to the interfaces. -var _ lib.Runner = &MiniRunner{} -var _ lib.VU = &MiniRunnerVU{} +var ( + _ lib.Runner = &MiniRunner{} + _ lib.VU = &VU{} +) // MiniRunner partially implements the lib.Runner interface, but instead of // using a real JS runtime, it allows us to directly specify the options and @@ -53,10 +55,10 @@ func (r MiniRunner) MakeArchive() *lib.Archive { return nil } -// NewVU returns a new MiniRunnerVU with an incremental ID. +// NewVU returns a new VU with an incremental ID. func (r *MiniRunner) NewVU(out chan<- stats.SampleContainer) (lib.VU, error) { nextVUNum := atomic.AddInt64(&r.NextVUID, 1) - return &MiniRunnerVU{R: r, Out: out, ID: nextVUNum - 1}, nil + return &VU{R: r, Out: out, ID: nextVUNum - 1}, nil } // Setup calls the supplied mock setup() function, if present. @@ -105,8 +107,8 @@ func (r *MiniRunner) SetOptions(opts lib.Options) error { return nil } -// MiniRunnerVU is a mock VU, spawned by a MiniRunner. -type MiniRunnerVU struct { +// VU is a mock VU, spawned by a MiniRunner. +type VU struct { R *MiniRunner Out chan<- stats.SampleContainer ID int64 @@ -114,7 +116,7 @@ type MiniRunnerVU struct { } // RunOnce runs the mock default function once, incrementing its iteration. -func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { +func (vu VU) RunOnce(ctx context.Context) error { if vu.R.Fn == nil { return nil } @@ -131,7 +133,7 @@ func (vu MiniRunnerVU) RunOnce(ctx context.Context) error { } // Reconfigure changes the VU ID. -func (vu *MiniRunnerVU) Reconfigure(id int64) error { +func (vu *VU) Reconfigure(id int64) error { vu.ID = id return nil } From ae338fa1efc50f8c2293c581b483d875238ad8a9 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 13 Feb 2020 10:39:30 +0200 Subject: [PATCH 093/350] Always restore original env vars in tests This is sadly necessary for now, until we properly fix https://github.com/loadimpact/k6/issues/883... --- cmd/config_consolidation_test.go | 35 +------------------------- cmd/config_test.go | 10 +++++--- lib/options_test.go | 20 ++++++++------- lib/testutils/env.go | 42 ++++++++++++++++++++++++++++++++ 4 files changed, 60 insertions(+), 47 deletions(-) create mode 100644 lib/testutils/env.go diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index d51a1be5525..70d3788e021 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -23,7 +23,6 @@ import ( "fmt" "io/ioutil" "os" - "strings" "testing" "time" @@ -42,38 +41,6 @@ import ( "github.com/loadimpact/k6/stats" ) -// A helper funcion for setting arbitrary environment variables and -// restoring the old ones at the end, usually by deferring the returned callback -// TODO: remove these hacks when we improve the configuration... we shouldn't -// have to mess with the global environment at all... -func setEnv(t *testing.T, newEnv []string) (restoreEnv func()) { - actuallSetEnv := func(env []string, abortOnSetErr bool) { - os.Clearenv() - for _, e := range env { - val := "" - pair := strings.SplitN(e, "=", 2) - if len(pair) > 1 { - val = pair[1] - } - err := os.Setenv(pair[0], val) - if abortOnSetErr { - require.NoError(t, err) - } else if err != nil { - t.Logf( - "Received a non-aborting but unexpected error '%s' when setting env.var '%s' to '%s'", - err, pair[0], val, - ) - } - } - } - oldEnv := os.Environ() - actuallSetEnv(newEnv, true) - - return func() { - actuallSetEnv(oldEnv, false) - } -} - func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config exec := c.Execution[lib.DefaultExecutorName] @@ -424,7 +391,7 @@ func runTestCase( logrus.SetOutput(output) logHook.Drain() - restoreEnv := setEnv(t, testCase.options.env) + restoreEnv := testutils.SetEnv(t, testCase.options.env) defer restoreEnv() flagSet := newFlagSet() diff --git a/cmd/config_test.go b/cmd/config_test.go index 37481b99912..75244027ce1 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -21,10 +21,11 @@ package cmd import ( - "os" + "fmt" "testing" "github.com/kelseyhightower/envconfig" + "github.com/loadimpact/k6/lib/testutils" "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" ) @@ -41,7 +42,6 @@ type testCmdTest struct { } func TestConfigCmd(t *testing.T) { - testdata := []testCmdData{ { Name: "Out", @@ -101,11 +101,13 @@ func TestConfigEnv(t *testing.T) { }, } for field, data := range testdata { - os.Clearenv() + field, data := field, data t.Run(field.Name, func(t *testing.T) { for value, fn := range data { + value, fn := value, fn t.Run(`"`+value+`"`, func(t *testing.T) { - assert.NoError(t, os.Setenv(field.Key, value)) + restore := testutils.SetEnv(t, []string{fmt.Sprintf("%s=%s", field.Key, value)}) + defer restore() var config Config assert.NoError(t, envconfig.Process("", &config)) fn(config) diff --git a/lib/options_test.go b/lib/options_test.go index 97645ff203c..76a6419fae4 100644 --- a/lib/options_test.go +++ b/lib/options_test.go @@ -23,13 +23,14 @@ package lib import ( "crypto/tls" "encoding/json" + "fmt" "net" - "os" "reflect" "testing" "time" "github.com/kelseyhightower/envconfig" + "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" @@ -130,7 +131,6 @@ func TestOptions(t *testing.T) { } t.Run("JSON", func(t *testing.T) { - t.Run("String", func(t *testing.T) { var opts Options jsonStr := `{"tlsCipherSuites":["TLS_ECDHE_RSA_WITH_RC4_128_SHA"]}` @@ -385,7 +385,6 @@ func TestOptions(t *testing.T) { assert.True(t, opts.DiscardResponseBodies.Valid) assert.True(t, opts.DiscardResponseBodies.Bool) }) - } func TestOptionsEnv(t *testing.T) { @@ -409,8 +408,10 @@ func TestOptionsEnv(t *testing.T) { }, {"Stages", "K6_STAGES"}: { // "": []Stage{}, - "1s": []Stage{{ - Duration: types.NullDurationFrom(1 * time.Second)}, + "1s": []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + }, }, "1s:100": []Stage{ {Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(100)}, @@ -460,11 +461,13 @@ func TestOptionsEnv(t *testing.T) { // External } for field, data := range testdata { - os.Clearenv() + field, data := field, data t.Run(field.Name, func(t *testing.T) { for str, val := range data { + str, val := str, val t.Run(`"`+str+`"`, func(t *testing.T) { - assert.NoError(t, os.Setenv(field.Key, str)) + restore := testutils.SetEnv(t, []string{fmt.Sprintf("%s=%s", field.Key, str)}) + defer restore() var opts Options assert.NoError(t, envconfig.Process("k6", &opts)) assert.Equal(t, val, reflect.ValueOf(opts).FieldByName(field.Name).Interface()) @@ -475,8 +478,7 @@ func TestOptionsEnv(t *testing.T) { } func TestCIDRUnmarshal(t *testing.T) { - - var testData = []struct { + testData := []struct { input string expectedOutput *IPNet expactFailure bool diff --git a/lib/testutils/env.go b/lib/testutils/env.go new file mode 100644 index 00000000000..d191e880f2d --- /dev/null +++ b/lib/testutils/env.go @@ -0,0 +1,42 @@ +package testutils + +import ( + "os" + "strings" + "testing" + + "github.com/stretchr/testify/require" +) + +// SetEnv is a helper funcion for setting arbitrary environment variables and +// restoring the old ones at the end, usually by deferring the returned callback +// TODO: remove these hacks when we improve the configuration (hopefully +// completely, see https://github.com/loadimpact/k6/issues/883)... we shouldn't +// have to mess with the global environment at all... +func SetEnv(t *testing.T, newEnv []string) (restoreEnv func()) { + actuallSetEnv := func(env []string, abortOnSetErr bool) { + os.Clearenv() + for _, e := range env { + val := "" + pair := strings.SplitN(e, "=", 2) + if len(pair) > 1 { + val = pair[1] + } + err := os.Setenv(pair[0], val) + if abortOnSetErr { + require.NoError(t, err) + } else if err != nil { + t.Logf( + "Received a non-aborting but unexpected error '%s' when setting env.var '%s' to '%s'", + err, pair[0], val, + ) + } + } + } + oldEnv := os.Environ() + actuallSetEnv(newEnv, true) + + return func() { + actuallSetEnv(oldEnv, false) + } +} From 3fb051327ff73a24e4ca0bbb1e3d4423a3ea88db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 31 Jan 2020 10:18:11 +0100 Subject: [PATCH 094/350] Add duration column to arrival-rate progress bars, standardize into 3 columns for all See https://github.com/loadimpact/k6/pull/1310#issuecomment-580284783 --- cmd/ui.go | 2 +- lib/executor/constant_arrival_rate.go | 17 ++++++++++++++--- lib/executor/externally_controlled.go | 25 ++++++++++++++++++------- lib/executor/per_vu_iterations.go | 19 ++++++++++++++----- lib/executor/shared_iterations.go | 16 ++++++++++------ lib/executor/variable_arrival_rate.go | 19 +++++++++++++++---- 6 files changed, 72 insertions(+), 26 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 88f6226e98d..3cf4caab685 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -93,7 +93,7 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) var ( // Maximum length of each right side column except last, // used to calculate the padding between columns. - maxRColumnLen = make([]int, 1) + maxRColumnLen = make([]int, 2) pbsCount = len(pbs) rendered = make([]pb.ProgressBarRender, pbsCount) result = make([]string, pbsCount+2) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 284dce044a4..f98beaaaf0c 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -228,15 +228,26 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } vusFmt := pb.GetFixedLengthIntFormat(maxVUs) + progIters := fmt.Sprintf( + pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0)+" iters/s", arrivalRatePerSec) progresFn := func() (float64, []string) { spent := time.Since(startTime) currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) vusInBuffer := uint64(len(vus)) progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) - progIters := fmt.Sprintf( - pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0)+" iters/s", arrivalRatePerSec) - return math.Min(1, float64(spent)/float64(duration)), []string{progVUs, progIters} + + right := []string{progVUs, duration.String(), progIters} + + if spent > duration { + return 1, right + } + + spentDuration := pb.GetFixedLengthDuration(spent, duration) + progDur := fmt.Sprintf("%s/%s", spentDuration, duration) + right[1] = progDur + + return math.Min(1, float64(spent)/float64(duration)), right } car.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, car, progresFn) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index f28bae31fb8..100173aec44 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -395,18 +395,29 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { } func (rs *externallyControlledRunState) progresFn() (float64, []string) { - spent := rs.executor.executionState.GetCurrentTestRunDuration() - progress := 0.0 - if rs.duration > 0 { - progress = math.Min(1, float64(spent)/float64(rs.duration)) - } //TODO: simulate spinner for the other case or cycle 0-100? currentActiveVUs := atomic.LoadInt64(rs.activeVUsCount) currentMaxVUs := atomic.LoadInt64(rs.maxVUs) vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentActiveVUs, currentMaxVUs) - progDur := fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, rs.duration), rs.duration) - return progress, []string{progVUs, progDur} + + right := []string{progVUs, rs.duration.String(), ""} + + spent := rs.executor.executionState.GetCurrentTestRunDuration() + if spent > rs.duration { + return 1, right + } + + progress := 0.0 + if rs.duration > 0 { + progress = math.Min(1, float64(spent)/float64(rs.duration)) + } + + spentDuration := pb.GetFixedLengthDuration(spent, rs.duration) + progDur := fmt.Sprintf("%s/%s", spentDuration, rs.duration) + right[1] = progDur + + return progress, right } func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg ExternallyControlledConfigParams) error { diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 0ce68c305a9..81b2dc51f0e 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -157,7 +157,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta duration := time.Duration(pvi.config.MaxDuration.Duration) gracefulStop := pvi.config.GetGracefulStop() - _, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) defer cancel() // Make sure the log and the progress bar have accurate information @@ -171,12 +171,21 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) progresFn := func() (float64, []string) { + spent := time.Since(startTime) + progVUs := fmt.Sprintf(vusFmt+" VUs", numVUs) currentDoneIters := atomic.LoadUint64(doneIters) - return float64(currentDoneIters) / float64(totalIters), []string{ - fmt.Sprintf(vusFmt+" VUs", numVUs), - fmt.Sprintf(itersFmt+"/"+itersFmt+" iters, %d per VU", - currentDoneIters, totalIters, iterations), + progIters := fmt.Sprintf(itersFmt+"/"+itersFmt+" iters, %d per VU", + currentDoneIters, totalIters, iterations) + right := []string{progVUs, duration.String(), progIters} + if spent > duration { + return 1, right } + + spentDuration := pb.GetFixedLengthDuration(spent, duration) + progDur := fmt.Sprintf("%s/%s", spentDuration, duration) + right[1] = progDur + + return float64(currentDoneIters) / float64(totalIters), right } pvi.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, pvi, progresFn) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 163757835b9..99a53329353 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -162,7 +162,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta duration := time.Duration(si.config.MaxDuration.Duration) gracefulStop := si.config.GetGracefulStop() - _, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) defer cancel() // Make sure the log and the progress bar have accurate information @@ -175,12 +175,16 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) progresFn := func() (float64, []string) { + spent := time.Since(startTime) + progVUs := fmt.Sprintf(vusFmt+" VUs", numVUs) currentDoneIters := atomic.LoadUint64(doneIters) - return float64(currentDoneIters) / float64(totalIters), []string{ - fmt.Sprintf(vusFmt+" VUs", numVUs), - fmt.Sprintf(itersFmt+"/"+itersFmt+" shared iters", - currentDoneIters, totalIters), - } + progIters := fmt.Sprintf(itersFmt+"/"+itersFmt+" shared iters", + currentDoneIters, totalIters) + spentDuration := pb.GetFixedLengthDuration(spent, duration) + progDur := fmt.Sprintf("%s/%s", spentDuration, duration) + right := []string{progVUs, progDur, progIters} + + return float64(currentDoneIters) / float64(totalIters), right } si.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, si, progresFn) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index e7beac0f787..caca2280f05 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -364,16 +364,27 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) currentTickerPeriod := atomic.LoadInt64(tickerPeriod) vusInBuffer := uint64(len(vus)) + progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", + currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) itersPerSec := 0.0 if currentTickerPeriod > 0 { itersPerSec = float64(time.Second) / float64(currentTickerPeriod) } - return math.Min(1, float64(time.Since(startTime))/float64(duration)), []string{ - fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", - currentInitialisedVUs-vusInBuffer, currentInitialisedVUs), - fmt.Sprintf(itersFmt, itersPerSec), + progIters := fmt.Sprintf(itersFmt, itersPerSec) + + right := []string{progVUs, duration.String(), progIters} + + spent := time.Since(startTime) + if spent > duration { + return 1, right } + + spentDuration := pb.GetFixedLengthDuration(spent, duration) + progDur := fmt.Sprintf("%s/%s", spentDuration, duration) + right[1] = progDur + + return math.Min(1, float64(spent)/float64(duration)), right } varr.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) From 392814898e57f8550a8b7ff6a183147563d2a8cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 31 Jan 2020 11:03:48 +0100 Subject: [PATCH 095/350] Fix progress bar status for executors terminated by maxDuration Resolves https://github.com/loadimpact/k6/pull/1310#pullrequestreview-350685906 --- lib/executor/helpers.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 3f4a4074ba3..34355d8fb72 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -178,7 +178,11 @@ func trackProgress( case <-parentCtx.Done(): progressBar.Modify(pb.WithStatus(pb.Interrupted), constProg) default: - progressBar.Modify(pb.WithStatus(pb.Done), constProg) + status := pb.WithStatus(pb.Done) + if p < 1 { + status = pb.WithStatus(pb.Interrupted) + } + progressBar.Modify(status, constProg) } } From 00d347c6f5ecad1c7de56e4e1a2ce96c4aa67441 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 31 Jan 2020 11:15:32 +0100 Subject: [PATCH 096/350] Make pb.Status a rune instead of a string Resolves https://github.com/loadimpact/k6/pull/1310#discussion_r372843736 --- ui/pb/progressbar.go | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 3bc3409d0b5..a976b5f4565 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -33,15 +33,15 @@ const defaultWidth = 40 const defaultBarColor = color.Faint // Status of the progress bar -type Status string +type Status rune // Progress bar status symbols const ( - Running Status = " " - Waiting Status = "•" - Stopping Status = "↓" - Interrupted Status = "✗" - Done Status = "✓" + Running Status = ' ' + Waiting Status = '•' + Stopping Status = '↓' + Interrupted Status = '✗' + Done Status = '✓' ) //nolint:gochecknoglobals @@ -219,11 +219,16 @@ func (pb *ProgressBar) Render(leftMax int) ProgressBarRender { } out.Left = pb.renderLeft(leftMax) - status := string(pb.status) - if c, ok := statusColors[pb.status]; ok { - status = c.Sprint(pb.status) + + switch c, ok := statusColors[pb.status]; { + case ok: + out.Status = c.Sprint(string(pb.status)) + case pb.status > 0: + out.Status = string(pb.status) + default: + out.Status = " " } - out.Status = fmt.Sprintf("%-1s", status) + out.Progress = fmt.Sprintf("[%s%s%s]", filling, caret, padding) return out From 1318dda1dae27eb13750870836d24b4e42980781 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 31 Jan 2020 11:16:12 +0100 Subject: [PATCH 097/350] Remove confusing string padding, prefer concatenation Resolves https://github.com/loadimpact/k6/pull/1310#discussion_r372853963 --- ui/pb/progressbar.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index a976b5f4565..948d7875d12 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -166,8 +166,7 @@ func (pbr ProgressBarRender) String() string { if len(pbr.Right) > 0 { right = " " + strings.Join(pbr.Right, " ") } - return fmt.Sprintf("%s %-1s %s%s", - pbr.Left, pbr.Status, pbr.Progress, right) + return pbr.Left + " " + pbr.Status + " " + pbr.Progress + right } // Render locks the progressbar struct for reading and calls all of From 39401bf7c992c02568b416232ecac69fc4440128 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 29 Nov 2019 12:45:26 +0100 Subject: [PATCH 098/350] Add SharedIterations.Run unit test --- lib/executor/shared_iterations_test.go | 38 ++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 lib/executor/shared_iterations_test.go diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go new file mode 100644 index 00000000000..a0f6d8a5239 --- /dev/null +++ b/lib/executor/shared_iterations_test.go @@ -0,0 +1,38 @@ +package executor + +import ( + "context" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" +) + +func getTestSharedIterationsConfig() SharedIterationsConfig { + return SharedIterationsConfig{ + VUs: null.IntFrom(10), + Iterations: null.IntFrom(100), + MaxDuration: types.NullDurationFrom(5 * time.Second), + } +} + +func TestSharedIterationsRun(t *testing.T) { + t.Parallel() + var doneIters uint64 + var ctx, cancel, executor, _ = setupExecutor( + t, getTestSharedIterationsConfig(), + simpleRunner(func(ctx context.Context) error { + atomic.AddUint64(&doneIters, 1) + return nil + }), + ) + defer cancel() + err := executor.Run(ctx, nil) + require.NoError(t, err) + assert.Equal(t, uint64(100), doneIters) +} From e88426f6b387175cb7e4d3e7e47b7e9098352d43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 18 Dec 2019 12:57:20 +0100 Subject: [PATCH 099/350] Add VariableLoopingVUs.Run unit test This required access to the ExecutionState from the test, so changes to setupExecutor were needed. And because the number of VUs varies for this executor by definition, we can no longer make the assertions in initializeVUs. --- lib/executor/common_test.go | 23 ++++--- lib/executor/constant_arrival_rate_test.go | 31 +++++---- lib/executor/per_vu_iterations_test.go | 8 ++- lib/executor/shared_iterations_test.go | 4 +- lib/executor/variable_arrival_rate_test.go | 13 ++-- lib/executor/variable_looping_vus_test.go | 76 ++++++++++++++++++++++ 6 files changed, 121 insertions(+), 34 deletions(-) create mode 100644 lib/executor/variable_looping_vus_test.go diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index fe08e670df1..5fb47b7fece 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -5,12 +5,13 @@ import ( "io/ioutil" "testing" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" - "github.com/stretchr/testify/require" ) func simpleRunner(vuFn func(context.Context) error) lib.Runner { @@ -21,7 +22,7 @@ func simpleRunner(vuFn func(context.Context) error) lib.Runner { } } -func setupExecutor(t *testing.T, config lib.ExecutorConfig, runner lib.Runner) ( +func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionState, runner lib.Runner) ( context.Context, context.CancelFunc, lib.Executor, *testutils.SimpleLogrusHook, ) { ctx, cancel := context.WithCancel(context.Background()) @@ -32,32 +33,30 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, runner lib.Runner) ( testLog.AddHook(logHook) testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) - es := lib.NewExecutionState(lib.Options{}, 10, 50) es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.VU, error) { return runner.NewVU(engineOut) }) - initializeVUs(ctx, t, logEntry, es, 10) + segment := es.Options.ExecutionSegment + maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(segment)) + initializeVUs(ctx, t, logEntry, es, maxVUs) executor, err := config.NewExecutor(es, logEntry) require.NoError(t, err) + err = executor.Init(ctx) require.NoError(t, err) return ctx, cancel, executor, logHook } func initializeVUs( - ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number int, + ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number uint64, ) { // This is not how the local ExecutionScheduler initializes VUs, but should do the same job - for i := 0; i < number; i++ { - require.EqualValues(t, i, es.GetInitializedVUsCount()) + for i := uint64(0); i < number; i++ { vu, err := es.InitializeNewVU(ctx, logEntry) require.NoError(t, err) - require.EqualValues(t, i+1, es.GetInitializedVUsCount()) - es.ReturnVU(vu, false) - require.EqualValues(t, 0, es.GetCurrentlyActiveVUsCount()) - require.EqualValues(t, i+1, es.GetInitializedVUsCount()) + es.AddInitializedVU(vu) } } diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index d7f98a00427..163c612a428 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -7,12 +7,13 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" ) func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { @@ -27,8 +28,9 @@ func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( - t, getTestConstantArrivalRateConfig(), + t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) return nil @@ -51,8 +53,9 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestConstantArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( - t, getTestConstantArrivalRateConfig(), + t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) return nil @@ -93,14 +96,16 @@ func TestArrivalRateCancel(t *testing.T) { var ch = make(chan struct{}) var errCh = make(chan error, 1) var weAreDoneCh = make(chan struct{}) - var ctx, cancel, executor, logHook = setupExecutor(t, config, simpleRunner(func(ctx context.Context) error { - select { - case <-ch: - <-ch - default: - } - return nil - })) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, logHook = setupExecutor( + t, config, es, simpleRunner(func(ctx context.Context) error { + select { + case <-ch: + <-ch + default: + } + return nil + })) defer cancel() var wg sync.WaitGroup wg.Add(1) diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 2a4393140d3..eb459e4ef2b 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -6,11 +6,12 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" ) func getTestPerVUIterationsConfig() PerVUIterationsConfig { @@ -24,8 +25,9 @@ func getTestPerVUIterationsConfig() PerVUIterationsConfig { func TestPerVUIterations(t *testing.T) { t.Parallel() var result sync.Map + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, _ = setupExecutor( - t, getTestPerVUIterationsConfig(), + t, getTestPerVUIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { state := lib.GetState(ctx) currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index a0f6d8a5239..5018873b015 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" ) @@ -24,8 +25,9 @@ func getTestSharedIterationsConfig() SharedIterationsConfig { func TestSharedIterationsRun(t *testing.T) { t.Parallel() var doneIters uint64 + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, _ = setupExecutor( - t, getTestSharedIterationsConfig(), + t, getTestSharedIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddUint64(&doneIters, 1) return nil diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index b0e4f4b6a52..eb20d343ce6 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -8,12 +8,13 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" ) func TestGetPlannedRateChanges0DurationStage(t *testing.T) { @@ -222,8 +223,9 @@ func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( - t, getTestVariableArrivalRateConfig(), + t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) return nil @@ -246,8 +248,9 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestVariableArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 + es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( - t, getTestVariableArrivalRateConfig(), + t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) return nil diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go new file mode 100644 index 00000000000..5574c8f1118 --- /dev/null +++ b/lib/executor/variable_looping_vus_test.go @@ -0,0 +1,76 @@ +package executor + +import ( + "context" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" +) + +func getTestVariableLoopingVUsConfig() VariableLoopingVUsConfig { + return VariableLoopingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, + StartVUs: null.IntFrom(5), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(5), + }, + { + Duration: types.NullDurationFrom(0), + Target: null.IntFrom(3), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(3), + }, + }, + GracefulRampDown: types.NullDurationFrom(0), + } +} + +func TestVariableLoopingVUsRun(t *testing.T) { + t.Parallel() + var iterCount int64 + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, getTestVariableLoopingVUsConfig(), es, + simpleRunner(func(ctx context.Context) error { + time.Sleep(200 * time.Millisecond) + atomic.AddInt64(&iterCount, 1) + return nil + }), + ) + defer cancel() + + var ( + wg sync.WaitGroup + result []int64 + ) + + wg.Add(1) + go func() { + defer wg.Done() + time.Sleep(100 * time.Millisecond) + result = append(result, es.GetCurrentlyActiveVUsCount()) + time.Sleep(1 * time.Second) + result = append(result, es.GetCurrentlyActiveVUsCount()) + time.Sleep(1 * time.Second) + result = append(result, es.GetCurrentlyActiveVUsCount()) + }() + + err := executor.Run(ctx, nil) + + wg.Wait() + require.NoError(t, err) + assert.Equal(t, []int64{5, 3, 0}, result) + assert.Equal(t, int64(40), iterCount) +} From 525f83fff2885d8fff52a82c3d1f8718c39dbf2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 12 Dec 2019 12:14:56 +0100 Subject: [PATCH 100/350] Add missing license headers to executor test files --- lib/executor/common_test.go | 20 ++++++++++++++++++++ lib/executor/constant_arrival_rate_test.go | 20 ++++++++++++++++++++ lib/executor/per_vu_iterations_test.go | 20 ++++++++++++++++++++ lib/executor/shared_iterations_test.go | 20 ++++++++++++++++++++ lib/executor/variable_arrival_rate_test.go | 20 ++++++++++++++++++++ lib/executor/variable_looping_vus_test.go | 20 ++++++++++++++++++++ 6 files changed, 120 insertions(+) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 5fb47b7fece..0053606a9dd 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 163c612a428..fbd91caf2cb 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index eb459e4ef2b..a09f199fd49 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index 5018873b015..066a2944c7d 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index eb20d343ce6..f8e624bfebb 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 5574c8f1118..5d2c110d23b 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package executor import ( From 3a00210bd839b6f587f25b4298652405ce1edebe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 12 Dec 2019 13:33:26 +0100 Subject: [PATCH 101/350] Add SharedIterations executor work splitting test Resolves: https://github.com/loadimpact/k6/pull/1266#discussion_r357080519 --- lib/executor/shared_iterations_test.go | 47 ++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index 066a2944c7d..5f8b496fb41 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -22,6 +22,7 @@ package executor import ( "context" + "sync" "sync/atomic" "testing" "time" @@ -42,6 +43,7 @@ func getTestSharedIterationsConfig() SharedIterationsConfig { } } +// Baseline test func TestSharedIterationsRun(t *testing.T) { t.Parallel() var doneIters uint64 @@ -58,3 +60,48 @@ func TestSharedIterationsRun(t *testing.T) { require.NoError(t, err) assert.Equal(t, uint64(100), doneIters) } + +// Test that when one VU "slows down", others will pick up the workload. +// This is the reverse behavior of the PerVUIterations executor. +func TestSharedIterationsRunVariableVU(t *testing.T) { + t.Parallel() + var ( + result sync.Map + slowVUID int64 + ) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, getTestSharedIterationsConfig(), es, + simpleRunner(func(ctx context.Context) error { + time.Sleep(10 * time.Millisecond) // small wait to stabilize the test + state := lib.GetState(ctx) + // Pick one VU randomly and always slow it down. + sid := atomic.LoadInt64(&slowVUID) + if sid == int64(0) { + atomic.StoreInt64(&slowVUID, state.Vu) + } + if sid == state.Vu { + time.Sleep(200 * time.Millisecond) + } + currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) + result.Store(state.Vu, currIter.(uint64)+1) + return nil + }), + ) + defer cancel() + err := executor.Run(ctx, nil) + require.NoError(t, err) + + var totalIters uint64 + result.Range(func(key, value interface{}) bool { + totalIters += value.(uint64) + return true + }) + + // The slow VU should complete 2 iterations given these timings, + // while the rest should randomly complete the other 98 iterations. + val, ok := result.Load(slowVUID) + assert.True(t, ok) + assert.Equal(t, uint64(2), val) + assert.Equal(t, uint64(100), totalIters) +} From 606f1204c226ae2bcae61939e76f27a2d2aa92aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 12 Dec 2019 13:51:10 +0100 Subject: [PATCH 102/350] Add PerVUIterations executor work splitting test Resolves: https://github.com/loadimpact/k6/pull/1266#discussion_r357080519 --- lib/executor/per_vu_iterations_test.go | 55 +++++++++++++++++++++++++- 1 file changed, 53 insertions(+), 2 deletions(-) diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index a09f199fd49..9242ebb94a0 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -23,6 +23,7 @@ package executor import ( "context" "sync" + "sync/atomic" "testing" "time" @@ -38,11 +39,12 @@ func getTestPerVUIterationsConfig() PerVUIterationsConfig { return PerVUIterationsConfig{ VUs: null.IntFrom(10), Iterations: null.IntFrom(100), - MaxDuration: types.NullDurationFrom(5 * time.Second), + MaxDuration: types.NullDurationFrom(3 * time.Second), } } -func TestPerVUIterations(t *testing.T) { +// Baseline test +func TestPerVUIterationsRun(t *testing.T) { t.Parallel() var result sync.Map es := lib.NewExecutionState(lib.Options{}, 10, 50) @@ -68,3 +70,52 @@ func TestPerVUIterations(t *testing.T) { }) assert.Equal(t, uint64(1000), totalIters) } + +// Test that when one VU "slows down", others will *not* pick up the workload. +// This is the reverse behavior of the SharedIterations executor. +func TestPerVUIterationsRunVariableVU(t *testing.T) { + t.Parallel() + var ( + result sync.Map + slowVUID int64 + ) + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, getTestPerVUIterationsConfig(), es, + simpleRunner(func(ctx context.Context) error { + state := lib.GetState(ctx) + // Pick one VU randomly and always slow it down. + sid := atomic.LoadInt64(&slowVUID) + if sid == int64(0) { + atomic.StoreInt64(&slowVUID, state.Vu) + } + if sid == state.Vu { + time.Sleep(200 * time.Millisecond) + } + currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) + result.Store(state.Vu, currIter.(uint64)+1) + return nil + }), + ) + defer cancel() + err := executor.Run(ctx, nil) + require.NoError(t, err) + + val, ok := result.Load(slowVUID) + assert.True(t, ok) + + var totalIters uint64 + result.Range(func(key, value interface{}) bool { + vuIters := value.(uint64) + if key != slowVUID { + assert.Equal(t, uint64(100), vuIters) + } + totalIters += vuIters + return true + }) + + // The slow VU should complete 16 iterations given these timings, + // while the rest should equally complete their assigned 100 iterations. + assert.Equal(t, uint64(16), val) + assert.Equal(t, uint64(916), totalIters) +} From bdae146b1134f8a90c249dbf24143cdcdfc6bbe8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 10 Dec 2019 16:49:07 +0100 Subject: [PATCH 103/350] Add ConstantLoopingVUs.Run unit test --- lib/executor/constant_looping_vus_test.go | 70 +++++++++++++++++++++++ 1 file changed, 70 insertions(+) create mode 100644 lib/executor/constant_looping_vus_test.go diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go new file mode 100644 index 00000000000..8e64c258bc9 --- /dev/null +++ b/lib/executor/constant_looping_vus_test.go @@ -0,0 +1,70 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package executor + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" +) + +func getTestConstantLoopingVUsConfig() ConstantLoopingVUsConfig { + return ConstantLoopingVUsConfig{ + VUs: null.IntFrom(10), + Duration: types.NullDurationFrom(1 * time.Second), + } +} + +func TestConstantLoopingVUsRun(t *testing.T) { + t.Parallel() + var result sync.Map + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, getTestConstantLoopingVUsConfig(), es, + simpleRunner(func(ctx context.Context) error { + time.Sleep(200 * time.Millisecond) + state := lib.GetState(ctx) + currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) + result.Store(state.Vu, currIter.(uint64)+1) + return nil + }), + ) + defer cancel() + err := executor.Run(ctx, nil) + require.NoError(t, err) + + var totalIters uint64 + result.Range(func(key, value interface{}) bool { + vuIters := value.(uint64) + assert.Equal(t, uint64(5), vuIters) + totalIters += vuIters + return true + }) + assert.Equal(t, uint64(50), totalIters) +} From 522b585cfa063054ef2a11d4f776adefd70192f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 11 Dec 2019 17:07:00 +0100 Subject: [PATCH 104/350] Add ExternallyControlled.Run unit test --- lib/executor/externally_controlled_test.go | 110 +++++++++++++++++++++ 1 file changed, 110 insertions(+) create mode 100644 lib/executor/externally_controlled_test.go diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go new file mode 100644 index 00000000000..665261a654b --- /dev/null +++ b/lib/executor/externally_controlled_test.go @@ -0,0 +1,110 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package executor + +import ( + "context" + "sync" + "sync/atomic" + "testing" + "time" + + null "gopkg.in/guregu/null.v3" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" +) + +func getTestExternallyControlledConfig() ExternallyControlledConfig { + return ExternallyControlledConfig{ + ExternallyControlledConfigParams: ExternallyControlledConfigParams{ + VUs: null.IntFrom(2), + MaxVUs: null.IntFrom(10), + Duration: types.NullDurationFrom(2 * time.Second), + }, + } +} + +func TestExternallyControlledRun(t *testing.T) { + t.Parallel() + var doneIters uint64 + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, getTestExternallyControlledConfig(), es, + simpleRunner(func(ctx context.Context) error { + time.Sleep(200 * time.Millisecond) + atomic.AddUint64(&doneIters, 1) + return nil + }), + ) + defer cancel() + + var ( + wg sync.WaitGroup + errCh = make(chan error, 1) + doneCh = make(chan struct{}) + resultVUCount []int64 + ) + wg.Add(1) + go func() { + defer wg.Done() + errCh <- executor.Run(ctx, nil) + close(doneCh) + }() + + updateConfig := func(vus int) { + newConfig := ExternallyControlledConfigParams{ + VUs: null.IntFrom(int64(vus)), + MaxVUs: null.IntFrom(10), + Duration: types.NullDurationFrom(2 * time.Second), + } + err := executor.(*ExternallyControlled).UpdateConfig(ctx, newConfig) + assert.NoError(t, err) + } + + wg.Add(1) + go func() { + defer wg.Done() + es.MarkStarted() + time.Sleep(150 * time.Millisecond) // wait for startup + resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + time.Sleep(500 * time.Millisecond) + updateConfig(4) + time.Sleep(100 * time.Millisecond) + resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + time.Sleep(500 * time.Millisecond) + updateConfig(8) + time.Sleep(100 * time.Millisecond) + resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + time.Sleep(1 * time.Second) + resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + es.MarkEnded() + }() + + <-doneCh + wg.Wait() + require.NoError(t, <-errCh) + assert.Equal(t, uint64(50), doneIters) + assert.Equal(t, []int64{2, 4, 8, 0}, resultVUCount) +} From 8f9e6fdd540d1222e6b358a2bfeb7e635f0e7f18 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 13 Dec 2019 14:26:46 +0100 Subject: [PATCH 105/350] Fix data race reading progress/logger of ExternallyControlled executor Previously TestExternallyControlledRun would trigger a race condition (when testing with `go test -race`) when writing the new config from one goroutine and calling executor.GetProgress() or executor.GetLogger() from another. I'm not sure if using `configLock` for this is the right approach or if there should be a separate mutex for this use case, so let me know. I didn't benchmark if this impacts performance, but I wouldn't expect it to as these should be read infrequently. --- lib/executor/externally_controlled.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 100173aec44..c53e5a1d6f7 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -230,11 +230,15 @@ func (mex *ExternallyControlled) GetConfig() lib.ExecutorConfig { // GetProgress just returns the executor's progress bar instance. func (mex ExternallyControlled) GetProgress() *pb.ProgressBar { + mex.configLock.RLock() + defer mex.configLock.RUnlock() return mex.progress } // GetLogger just returns the executor's logger instance. func (mex ExternallyControlled) GetLogger() *logrus.Entry { + mex.configLock.RLock() + defer mex.configLock.RUnlock() return mex.logger } From 26cc18311831745e83cd3c8c0c546e1f7557d7c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 18 Feb 2020 10:21:24 +0100 Subject: [PATCH 106/350] Fix GetExecutionRequirements documentation reference --- lib/executor/variable_looping_vus.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 17726fe3c24..63ae4dd1ad1 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -422,7 +422,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu // // If gracefulRampDown is specified, it will also be taken into account, and the // number of needed VUs to handle that will also be reserved. See the -// documentation of reserveGracefulVUScalingDown() for more details. +// documentation of reserveVUsForGracefulRampDowns() for more details. // // On the other hand, gracefulStop is handled here. To facilitate it, we'll // ensure that the last execution step will have 0 VUs and will be at time From 2a7d7f9725230845c4dad4d109dba055742eec7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 11 Feb 2020 18:32:28 +0100 Subject: [PATCH 107/350] Fix variable-looping-vus VU wobble during ramp-down bug Part 1 of #1296 --- lib/executor/variable_looping_vus.go | 2 +- lib/executor/variable_looping_vus_test.go | 100 ++++++++++++++++++++-- 2 files changed, 92 insertions(+), 10 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 63ae4dd1ad1..f763cdc0fbd 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -590,7 +590,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } handleNewScheduledVUs(step.PlannedVUs) case step := <-gracefulLimitEvents: - if step.PlannedVUs > currentScheduledVUs { + if step.PlannedVUs > currentMaxAllowedVUs { // Handle the case where a value is read from the // gracefulLimitEvents channel before rawStepEvents handleNewScheduledVUs(step.PlannedVUs) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 5d2c110d23b..45242f0ed4a 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -22,6 +22,7 @@ package executor import ( "context" + "fmt" "sync" "sync/atomic" "testing" @@ -35,10 +36,13 @@ import ( "github.com/loadimpact/k6/lib/types" ) -func getTestVariableLoopingVUsConfig() VariableLoopingVUsConfig { - return VariableLoopingVUsConfig{ - BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, - StartVUs: null.IntFrom(5), +func TestVariableLoopingVUsRun(t *testing.T) { + t.Parallel() + + config := VariableLoopingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, + GracefulRampDown: types.NullDurationFrom(0), + StartVUs: null.IntFrom(5), Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), @@ -53,16 +57,12 @@ func getTestVariableLoopingVUsConfig() VariableLoopingVUsConfig { Target: null.IntFrom(3), }, }, - GracefulRampDown: types.NullDurationFrom(0), } -} -func TestVariableLoopingVUsRun(t *testing.T) { - t.Parallel() var iterCount int64 es := lib.NewExecutionState(lib.Options{}, 10, 50) var ctx, cancel, executor, _ = setupExecutor( - t, getTestVariableLoopingVUsConfig(), es, + t, config, es, simpleRunner(func(ctx context.Context) error { time.Sleep(200 * time.Millisecond) atomic.AddInt64(&iterCount, 1) @@ -94,3 +94,85 @@ func TestVariableLoopingVUsRun(t *testing.T) { assert.Equal(t, []int64{5, 3, 0}, result) assert.Equal(t, int64(40), iterCount) } + +// Ensure there's no wobble of VUs during graceful ramp-down, without segments. +// See https://github.com/loadimpact/k6/issues/1296 +func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { + t.Parallel() + + config := VariableLoopingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, + GracefulRampDown: types.NullDurationFrom(1 * time.Second), + StartVUs: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(3 * time.Second), + Target: null.IntFrom(10), + }, + { + Duration: types.NullDurationFrom(2 * time.Second), + Target: null.IntFrom(0), + }, + }, + } + + es := lib.NewExecutionState(lib.Options{}, 10, 50) + var ctx, cancel, executor, _ = setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + time.Sleep(1 * time.Second) + return nil + }), + ) + defer cancel() + + var ( + wg sync.WaitGroup + result []int64 + m sync.Mutex + ) + + sampleActiveVUs := func(delay time.Duration) { + time.Sleep(delay) + m.Lock() + result = append(result, es.GetCurrentlyActiveVUsCount()) + m.Unlock() + } + + wg.Add(1) + go func() { + defer wg.Done() + sampleActiveVUs(100 * time.Millisecond) + sampleActiveVUs(3 * time.Second) + time.AfterFunc(2*time.Second, func() { + sampleActiveVUs(0) + }) + time.Sleep(1 * time.Second) + // Sample ramp-down at a higher frequency + for i := 0; i < 15; i++ { + sampleActiveVUs(100 * time.Millisecond) + } + }() + + err := executor.Run(ctx, nil) + + wg.Wait() + require.NoError(t, err) + assert.Equal(t, int64(0), result[0]) + assert.Equal(t, int64(10), result[1]) + assert.Equal(t, int64(0), result[len(result)-1]) + + var curr int64 + last := result[2] + // Check all ramp-down samples + for i := 3; i < len(result[2:]); i++ { + curr = result[i] + // Detect ramp-ups, missteps (e.g. 7 -> 4), but ignore pauses + if curr > last || (curr != last && curr != last-1) { + assert.FailNow(t, + fmt.Sprintf("ramping down wobble bug - "+ + "current: %d, previous: %d\nVU samples: %v", curr, last, result)) + } + last = curr + } +} From d1f14a3f29e22c3275f69e229d469c13917d7ad7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 9 Mar 2020 10:55:57 +0100 Subject: [PATCH 108/350] Merge branch 'master' into new-schedulers --- .circleci/config.yml | 10 +- .github/ISSUE_TEMPLATE/bug_report.md | 30 +++++ .github/ISSUE_TEMPLATE/config.yml | 5 + .github/ISSUE_TEMPLATE/feat_req.md | 14 +++ Dockerfile | 5 +- Gopkg.lock | 4 +- README.md | 99 ++++++++-------- SUPPORT.md | 12 ++ appveyor.yml | 2 +- cmd/login_cloud.go | 2 +- github-hr.png | Bin 0 -> 6152 bytes js/bundle.go | 6 +- js/bundle_test.go | 42 +++++++ js/modules/k6/http/request.go | 2 +- js/modules/k6/http/request_test.go | 6 +- js/runner.go | 5 +- js/runner_test.go | 16 +-- lib/consts/consts.go | 2 +- loader/loader.go | 2 +- logo.png | Bin 6443 -> 0 bytes logo.svg | 1 + release notes/v0.26.1.md | 12 ++ samples/http_2.js | 4 +- samples/http_get.js | 4 +- stats/cloud/config.go | 4 +- stats/cloud/util.go | 2 +- vendor/github.com/dop251/goja/builtin_date.go | 2 +- .../github.com/dop251/goja/builtin_global.go | 101 +++++++++++++++- vendor/github.com/dop251/goja/date.go | 1 + vendor/github.com/dop251/goja/runtime.go | 29 +++++ vendor/github.com/dop251/goja/vm.go | 112 +++--------------- 31 files changed, 353 insertions(+), 183 deletions(-) create mode 100644 .github/ISSUE_TEMPLATE/bug_report.md create mode 100644 .github/ISSUE_TEMPLATE/config.yml create mode 100644 .github/ISSUE_TEMPLATE/feat_req.md create mode 100644 SUPPORT.md create mode 100644 github-hr.png delete mode 100644 logo.png create mode 100644 logo.svg create mode 100644 release notes/v0.26.1.md diff --git a/.circleci/config.yml b/.circleci/config.yml index 73d41bb66e0..74d93b100fe 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -2,7 +2,7 @@ version: 2 jobs: deps: docker: - - image: circleci/golang:1.13 + - image: circleci/golang:1.14 environment: GOPATH: /home/circleci/.go_workspace working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 @@ -23,7 +23,7 @@ jobs: test: docker: - - image: circleci/golang:1.13 + - image: circleci/golang:1.14 environment: GOPATH: /home/circleci/.go_workspace working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 @@ -51,7 +51,7 @@ jobs: test-prev-golang: docker: - - image: circleci/golang:1.12 + - image: circleci/golang:1.13 environment: GOPATH: /home/circleci/.go_workspace working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 @@ -67,7 +67,7 @@ jobs: build-docker-images: docker: - - image: circleci/golang:1.13 + - image: circleci/golang:1.14 environment: GOPATH: /home/circleci/.go_workspace working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 @@ -111,7 +111,7 @@ jobs: build-linux-packages: docker: - - image: circleci/golang:1.13 + - image: circleci/golang:1.14 environment: GOPATH: /home/circleci/.go_workspace working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md new file mode 100644 index 00000000000..6484e19e1e3 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -0,0 +1,30 @@ +--- +name: Bug report +about: Use this template for reporting bugs. Please search existing issues first. +labels: bug +--- + + + +## Environment + +- k6 version: +- OS and version: +- Docker version and image, if applicable: + + +## Expected Behavior + + + +## Actual Behavior + + + +## Steps to Reproduce the Problem + + +1. +2. +3. diff --git a/.github/ISSUE_TEMPLATE/config.yml b/.github/ISSUE_TEMPLATE/config.yml new file mode 100644 index 00000000000..dc80b8a6957 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/config.yml @@ -0,0 +1,5 @@ +blank_issues_enabled: true +contact_links: + - name: k6 Community Forum + url: https://community.k6.io/ + about: Please ask and answer questions here. diff --git a/.github/ISSUE_TEMPLATE/feat_req.md b/.github/ISSUE_TEMPLATE/feat_req.md new file mode 100644 index 00000000000..e9839b9b76a --- /dev/null +++ b/.github/ISSUE_TEMPLATE/feat_req.md @@ -0,0 +1,14 @@ +--- +name: Feature request +about: Use this template for suggesting new features. +labels: feature +--- + + + +## Feature Description + + + +## Suggested Solution (optional) + diff --git a/Dockerfile b/Dockerfile index 852b7e59786..89d250b7221 100644 --- a/Dockerfile +++ b/Dockerfile @@ -5,6 +5,9 @@ RUN apk --no-cache add git RUN CGO_ENABLED=0 go install -a -trimpath -ldflags "-s -w -X github.com/loadimpact/k6/lib/consts.VersionDetails=$(date -u +"%FT%T%z")/$(git describe --always --long --dirty)" FROM alpine:3.10 -RUN apk add --no-cache ca-certificates +RUN apk add --no-cache ca-certificates && \ + adduser -D -u 12345 -g 12345 k6 COPY --from=builder /go/bin/k6 /usr/bin/k6 + +USER 12345 ENTRYPOINT ["k6"] diff --git a/Gopkg.lock b/Gopkg.lock index 21b3fdc5f3c..0cb1630216f 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -97,7 +97,7 @@ [[projects]] branch = "master" - digest = "1:a96fa4c60aed526368ef2ee9ee40e1eb89b1901792a7bd29865cf120651ce034" + digest = "1:ae691b322020292729ed19b9a65a645964fed8850762784442287f6581fd2581" name = "github.com/dop251/goja" packages = [ ".", @@ -107,7 +107,7 @@ "token", ] pruneopts = "NUT" - revision = "007eef3bc40fd33b3dbb80ec16da59e8b63b8572" + revision = "2a7d122434f4f4b5efc89b5f85c744faf9d03666" [[projects]] branch = "master" diff --git a/README.md b/README.md index 0c1db1654ac..19e84d17666 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -

k6

+

k6

Like unit testing, for performance

A modern load testing tool for developers and testers in the DevOps era.

@@ -15,13 +15,15 @@

Download · Install · - Documentation · + Documentation · Community

---- +
+--- +
-**k6** is a modern load testing tool, building on [Load Impact](https://loadimpact.com/)'s years of experience in the load and performance testing industry. It provides a clean, approachable scripting API, [local](https://docs.k6.io/docs/running-k6) and [cloud execution](https://docs.k6.io/docs/cloud-execution), flexible configuration, with command & control through CLI or a [REST API](https://docs.k6.io/docs/rest-api). +**k6** is a modern load testing tool, building on [Load Impact](https://loadimpact.com/)'s years of experience in the load and performance testing industry. It provides a clean, approachable scripting API, [local](https://k6.io/docs/getting-started/running-k6) and [cloud execution](https://k6.io/docs/cloud), and flexible configuration. This is how load testing should look in the 21st century. @@ -41,17 +43,17 @@ Menu Features -------- -- **Scripting in ES6 JS**: support for [modules](https://docs.k6.io/docs/modules) to aid code reusability across an organization -- **Everything as code**: test logic and [configuration options](https://docs.k6.io/docs/options) are both in JS for version control friendliness -- **Automation-friendly**: [checks](https://docs.k6.io/docs/checks) (like asserts) and [thresholds](https://docs.k6.io/docs/thresholds) for easy and flexible CI configuration! -- [**HTTP/1.1**](https://docs.k6.io/docs/http-requests), [**HTTP/2**](https://docs.k6.io/docs/http2) and [**WebSocket**](https://docs.k6.io/docs/websockets) protocol support -- **TLS features**: [client certificates](https://docs.k6.io/docs/ssl-tls-client-certificates), [configurable SSL/TLS versions and ciphers](https://docs.k6.io/docs/ssl-tls-version-and-cipher-suites) -- **Batteries included**: [Cookies](https://docs.k6.io/docs/cookies), [Crypto](https://docs.k6.io/docs/k6crypto), [Custom metrics](https://docs.k6.io/docs/result-metrics#section-custom-metrics), [Encodings](https://docs.k6.io/docs/k6encoding), [Environment variables](https://docs.k6.io/docs/environment-variables), JSON, [HTML forms](https://docs.k6.io/docs/working-with-html-forms), [files](https://docs.k6.io/docs/open-filepath-mode), [flexible execution control](https://docs.k6.io/docs/running-k6#section-stages-ramping-updown-vus), and more. -- **Built-in HAR converter**: record browser sessions as [`.har` files](https://en.wikipedia.org/wiki/.har) and [directly convert them to k6 scripts](https://docs.k6.io/docs/session-recording-har-support) -- **Flexible metrics storage and visualization**: [InfluxDB](https://docs.k6.io/docs/influxdb-grafana) (+Grafana), [JSON](https://docs.k6.io/docs/results-output#section-json-output) or [Load Impact Insights](https://docs.k6.io/docs/load-impact-insights) -- [**Cloud execution**](https://docs.k6.io/docs/cloud-execution) and distributed tests _(currently only on infrastructure managed by [Load Impact](https://loadimpact.com/), with native distributed execution in k6 [planned](https://github.com/loadimpact/k6/wiki/Roadmap) for the near future!)_ +- **Scripting in ES6 JS**: support for [modules](https://k6.io/docs/using-k6/modules) to aid code reusability across an organization +- **Everything as code**: test logic and [configuration options](https://k6.io/docs/using-k6/options) are both in JS for version control friendliness +- **Automation-friendly**: [checks](https://k6.io/docs/using-k6/checks) (like asserts) and [thresholds](https://k6.io/docs/using-k6/thresholds) for easy and flexible CI configuration! +- [**HTTP/1.1**](https://k6.io/docs/using-k6/http-requests), [**HTTP/2**](https://k6.io/docs/using-k6/protocols/http-2) and [**WebSocket**](https://k6.io/docs/using-k6/protocols/websockets) protocol support +- **TLS features**: [client certificates](https://k6.io/docs/using-k6/ssl-tls/ssl-tls-client-certificates), [configurable SSL/TLS versions and ciphers](https://k6.io/docs/using-k6/ssl-tls/ssl-tls-version-and-ciphers) +- **Batteries included**: [Cookies](https://k6.io/docs/using-k6/cookies), [Crypto](https://k6.io/docs/javascript-api/k6-crypto), [Custom metrics](https://k6.io/docs/using-k6/metrics#custom-metrics), [Encodings](https://k6.io/docs/javascript-api/k6-encoding), [Environment variables](https://k6.io/docs/using-k6/environment-variables), JSON, [HTML forms](https://k6.io/docs/using-k6/html/working-with-html-forms), [files](https://k6.io/docs/javascript-api/init-context/open-filepath-mode), [flexible execution control](https://k6.io/docs/getting-started/running-k6#section-stages-ramping-updown-vus), and more. +- **Built-in HAR converter**: record browser sessions as [`.har` files](https://en.wikipedia.org/wiki/.har) and [directly convert them to k6 scripts](https://k6.io/docs/using-k6/session-recording-har-support) +- **Flexible metrics storage and visualization**: [InfluxDB](https://k6.io/docs/results-visualization/influxdb-+-grafana) (+Grafana), [JSON](https://k6.io/docs/getting-started/results-output/json) or [k6 Cloud](https://k6.io/docs/cloud/analyzing-results/overview) +- [**Cloud execution**](https://k6.io/docs/using-k6/cloud-execution) and distributed tests _(currently only on infrastructure managed by [Load Impact](https://loadimpact.com/), with native distributed execution in k6 [planned](https://github.com/loadimpact/k6/wiki/Roadmap) for the near future!)_ -There's even more! [See all features available in k6.](https://docs.k6.io/docs/welcome) +There's even more! [See all features available in k6.](https://k6.io/docs/) Install ------- @@ -118,7 +120,7 @@ Scripts must contain, at the very least, an exported `default` function - this d import http from "k6/http"; export default function() { - let response = http.get("https://test.loadimpact.com"); + let response = http.get("https://test-api.k6.io"); }; ``` @@ -126,14 +128,14 @@ The script details and how we can extend and configure it will be explained belo If you decide to use the [k6 docker image](https://hub.docker.com/r/loadimpact/k6/), the command will be slightly different. Instead of passing the script filename to k6, a dash is used to instruct k6 to read the script contents directly via the standard input. This allows us to to avoid messing with docker volumes for such a simple single-file script, greatly simplifying the docker command: `docker run -i loadimpact/k6 run - or write in the `#loadimpact` channel in [Slack](https://k6.io/slack). +Contributing +------------ If you want to contribute or help with the development of k6, start by reading [CONTRIBUTING.md](https://github.com/loadimpact/k6/blob/master/CONTRIBUTING.md). Before you start coding, especially when it comes to big changes and features, it might be a good idea to first discuss your plans and implementation details with the k6 maintainers. You can do this either in the [github issue](https://github.com/loadimpact/k6/issues) for the problem you're solving (create one if it doesn't exist) or in the `#developers` channel on [Slack](https://k6.io/slack). diff --git a/SUPPORT.md b/SUPPORT.md new file mode 100644 index 00000000000..12c068d7c7e --- /dev/null +++ b/SUPPORT.md @@ -0,0 +1,12 @@ +# Support + +Types of questions and where to ask: + +- How do I? -- the Discourse forum at [community.k6.io](https://community.k6.io/) or [Stack Overflow](https://stackoverflow.com/questions/tagged/k6) (use tags: k6, javascript, load-testing) +- I got this error, why? -- [community.k6.io](https://community.k6.io/) or [Stack Overflow](https://stackoverflow.com/questions/tagged/k6) +- I got this error and I'm sure it's a bug -- [open a new issue](https://github.com/loadimpact/k6/issues), if there isn't one for this specific bug already +- I have an idea/request -- search the [GitHub issues](https://github.com/loadimpact/k6/issues) to see if it was already requested and give the issue a :+1: if so. If it wasn't, search [community.k6.io](https://community.k6.io/) or post a forum thread to discuss the idea with the developers before creating a GitHub issue. +- Why do you? -- [community.k6.io](https://community.k6.io/) or [Slack](https://k6.io/slack) +- When will you? -- [community.k6.io](https://community.k6.io/) or [Slack](https://k6.io/slack) + +If your questions are about any of the commercial Load Impact services like managed cloud execution and Load Impact Insights, you can contact or write in the `#loadimpact` channel in [Slack](https://k6.io/slack). diff --git a/appveyor.yml b/appveyor.yml index 55f20a0e2e6..f71ab8d5286 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -10,7 +10,7 @@ environment: # specific to go VERSION: "%APPVEYOR_REPO_TAG_NAME:v=%" GOPATH: c:\gopath - GOVERSION: 1.13 + GOVERSION: 1.14 GOMAXPROCS: 2 CGO_ENABLED: '0' GOARCH: amd64 diff --git a/cmd/login_cloud.go b/cmd/login_cloud.go index 090ae54641d..c43545c7847 100644 --- a/cmd/login_cloud.go +++ b/cmd/login_cloud.go @@ -103,7 +103,7 @@ This will set the default token used when just "k6 run -o cloud" is passed.`, } if res.Token == "" { - return errors.New(`Your account has no API token, please generate one: "https://app.loadimpact.com/account/token".`) + return errors.New(`your account has no API token, please generate one at https://app.k6.io/account/api-token`) } newCloudConf.Token = null.StringFrom(res.Token) diff --git a/github-hr.png b/github-hr.png new file mode 100644 index 0000000000000000000000000000000000000000..34687d65b4f08621333ef4a0e7c3e4b201a81c67 GIT binary patch literal 6152 zcma)Ac|6nqAD^Ly%2AHKX>*iHH1+EnF)T;qh

ms8DVdwK-<1)0pS`nbtP(ceU2FxbXprbad} zn5Y1_#v;~%--j8-<={!+vWBuKHxSVvDTL`;^X_x@}ZT97a2}S|g!D-ug$l8~GkRJ&XfD0ic6tJX! zxnPLQ%~x_4tuh88{kJ8DYFX9$k1#AydPMCozpuSg9hk)ah~sNG2a4q?NT!Ry%Dy@v zrfrpK%hci3en7QCFS*8e zP1IuL1jJwkl(z=VAv;cUBuT@4}kYV8srVv7SkTY zoT5}1Fz0V2;r>x#KC$5`uPtT?h{;mQlL&Y)>1daS7**f?U2;uzn{X5`G-P7&Aw&MFL0$l6lCId&FU+hb?(q9Z*m~2t$DV`a?gz2X~En|E0ebx2;uN^Ul zOhY!WCe^R3F>CPX$doSnAi_Zw$XREa#u*_zj=DPVoL{#u`?%Q>XCA`U%^nUup^C0V z9^CE6!;o8Og4za!R=<$OD*A@LfvJ%NYNz9DUeTW2RD9V^=2=}x^H7|~Bf zEexUHTsV>1=j^j{mC4Zzrn)-bXtq#w zGy9}Z@eWtwm17>n*-n-@k~{2BFzk#jF0_XgigEJ)JdMCx_RmZe#1}-)4n>wZ>F4Nb zi(%##jl&mvJ0>`lsG8ozs>syKO;=Y)CASItg~5vFMaL+kB@+6|>N`MQWtlF=d9J`Q&#qiziJ)yyey zZq$igkWb8*BZREN$iSx13y4>-Q1`+r3vly~o*8Md<-V%Qg+y|yQJ&F+0Bl|Y5YXzM zHRd7N<$mt_oD~y$Qgk&yhw$rQSXO^O;nCUWGoayb^J>01V4;}KZiX)M=uAkk!GIOwE zWOWqt`86PSayAikgS2=cx3MsGvM8Hh~l zic6aeR8Yke&tTLnoMwjh=Yzooo!zAl&MpWHEr`JO z@g2&?8V^&{2MM*8jk1$;XF2VaVu1|Q&))$(7zVvQKaGpEkiEBVwFR~B%xxBIalUoz zNQbD7Zx|=LO~SbNc_{(-JdkmS#xUR(}RGZBQkGrf3aUk-iQzW!`mZqS3}X;^3BTDv}X-(8w)(T<#lAm6CH!2W)RoKJh` zHSF}9U#fdhY0bllm)lF>sqFJxE9^2b!#S-dD%~@aPRP(N$6+&O|LKJMdCse_+*W5} zxfFRoXhmSAEx8oMq#sJF+eDKUjd->f#4uF>F&tbYQoQv}YB@PW(}QdaN-S$yzQiEy z-JDwe^k70|%%$y@E<6q13h!2m6bURu#mV`D72wp{-}kkjQD`zXLqoWiS`b|%4i77C zy6N>j1ebX)a{=Om0c>_kMlAK`L2O$0oEmrfIkl2>02dTpJ(>5MwP_RDdW@)X30PH*QGWSg zZa21C@kQd2Ngy2F!bkN;IlkUaEM2sl+3s=4^CS8>*f#T5*PHU*}yX@UIZL6>^9#v1Wq%g))z>5QX@;Ph&>F zXKIr$gVYzn{BhwofbR})lNHkSF*`;4cTq<>sC2sSllb5MvngCiDpNy0>mWn5M>4bI ze2UF8oQKYZb6d%xQ`eOtrrd905BXEn?s#S z+hs?5kZkX%GumEJ*craV%dtc~ilS|Cx6=DT_lO&Yn-S?{s15S|;l~m(R8+Q1_o6aT zHGKmQMN;S=h0G8bwZWN%s^WoWivEMe(|;Li@Bhru_;D$TGu-uy5z zjT2|idwUI%sT&P3m_vHqe%woWnNPDM+<$v_(8NcT(tPbJNYMbz|w(KF};|Hfox`+X~?RQy2 z;T_MZ{%j20E=3oy1DKDdu){Lc+|^HSqG^EvQgbB7aO-^|x*z)XfYh8`L_&Z$7{57c|7h>D~aq2v#CBH39BU`K$F&EWcr_=P}i%kYqI8Qg>O zyy2>eAFPb}zKpi8c=fJ`k37B{s>;%&;UVT;pJZ6Asa9Ks>^x*?QUfAzymSsw znsZ)8EcdrMN2U_9lh4r=fNz9&+MFk=!g_T;5BXUch=JrrE$lHUHPZab2!aoa=oN~T_K{~Ql_aTxo$qQ=o(AiNaaCW7;Td4eO zSsL+v2?iHaKHE00ic_(5>>)~U7D|Fr-K(4D{W+=udMirHljFGcTEXW|OL^+iabEf_ z;-6ixgFFRIG9R?7WEbygjVVwd0Vr@Iu2=t7%mjQd{VPMAgV9rJa_PhHawaCvQ|!^4 zklqcqg|)wnYsVE>@UMJO9xuhVt&pTFbl|g<3ErsaXgMF>7aUDBNe^z6Sx$m^0uCni z&XFR}vbC^j+}LQrLRsgpVt*Q27+x9F(o<@7sl?peG7}SPL{F0LxO4#yttiJ?>V}Ne zank_vI-`|?=?>)o)<>&>rtxZ$cYgp=M8L3=*&C}bLMmG552eiaRVzSHRYouE;)IfHAHSZPu*ll7o7b*J{v+_=dD8!96< zNoI!XYjmDfFHfpeG;B9sDdR^%s`}wKuF*U#D){)`;E&Ww3y0Q~+cdj@`mX$_pakHB zBw^JuYGFI)T-n&$9{WW{I__eq8@()s&~8I9`I+(t7e9LG^F_Gy$oh)5bMrjs799TOU0_UD58^g zdni@5vaA%*Nhw2{y*#PRKmY+}_whFW6DtV$dZomR#N-d_4|%wp(ZfB%srG#F-7tWL zjs)Lq{#ScK3(D4z5~ih)5|dYKs~1f$)91>dZ*ah-$DN5;j#mLccwFY{wGw0Fz{ z$$<7+C?tTSr$4j~=c7ks;9`2e_-d^}G6#@w1dcj*n~X?$`2LrfN$Y?1M0^zJZ*w^? zG`;(DxF@%Taj-!5+?U{1nUAf1iKLf;7_nxa_Kb1+yOF6YZKXr_=4Y$oBeJQKjGUyu z2!Y#9&6xj+J^@vTRZ}py;Q}h;D}!42EJ^Sx@2#5xZz0J{Q!l33r3k}Ae+jbNjYD$+ z66d(WI-E@$jjaz|fA2Pw9;vcA!L3{$ibJ->X{B^Y5Z)S`U8 z6d-0g!v}B{?Tu9e>j`HLzzB0Yzs6BiCX-YsV^4zHq*EHIe0TF|R1(-g{SMpH#l3qe z#p3}I|C7cKI4swa&GH<4atzsV^Zts%CWRL^V(x_Rak(I?4Ff;NjE@`94P3AN7jgpo A?f?J) literal 0 HcmV?d00001 diff --git a/js/bundle.go b/js/bundle.go index 526c6a96e67..4d1b33ec30c 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -262,7 +262,11 @@ func (b *Bundle) instantiate(rt *goja.Runtime, init *InitContext) error { _ = module.Set("exports", exports) rt.Set("module", module) - rt.Set("__ENV", b.Env) + env := make(map[string]string, len(b.Env)) + for key, value := range b.Env { + env[key] = value + } + rt.Set("__ENV", env) rt.Set("console", common.Bind(rt, newConsole(), init.ctxPtr)) *init.ctxPtr = common.WithRuntime(context.Background(), rt) diff --git a/js/bundle_test.go b/js/bundle_test.go index dc3c50a8188..84f76c5f116 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -755,6 +755,48 @@ func TestBundleEnv(t *testing.T) { } } +func TestBundleNotSharable(t *testing.T) { + data := ` + export default function() { + if (__ITER == 0) { + if (typeof __ENV.something !== "undefined") { + throw new Error("invalid something: " + __ENV.something + " should be undefined"); + } + __ENV.something = __VU; + } else if (__ENV.something != __VU) { + throw new Error("invalid something: " + __ENV.something+ " should be "+ __VU); + } + } + ` + b1, err := getSimpleBundle("/script.js", data) + if !assert.NoError(t, err) { + return + } + + b2, err := NewBundleFromArchive(b1.makeArchive(), lib.RuntimeOptions{}) + if !assert.NoError(t, err) { + return + } + + bundles := map[string]*Bundle{"Source": b1, "Archive": b2} + vus, iters := 10, 1000 + for name, b := range bundles { + b := b + t.Run(name, func(t *testing.T) { + for i := 0; i < vus; i++ { + bi, err := b.Instantiate() + bi.Runtime.Set("__VU", i) + require.NoError(t, err) + for j := 0; j < iters; j++ { + bi.Runtime.Set("__ITER", j) + _, err := bi.Default(goja.Undefined()) + assert.NoError(t, err) + } + } + }) + } +} + func TestBundleMakeArchive(t *testing.T) { testCases := []struct { cm lib.CompatibilityMode diff --git a/js/modules/k6/http/request.go b/js/modules/k6/http/request.go index 287c768c0ca..45cbaa6a63e 100644 --- a/js/modules/k6/http/request.go +++ b/js/modules/k6/http/request.go @@ -481,7 +481,7 @@ func (h *HTTP) parseBatchRequest( } case map[string]interface{}: - // Handling of {method: "GET", url: "http://test.loadimpact.com"} + // Handling of {method: "GET", url: "https://test.k6.io"} if murl, ok := data["url"]; !ok { return nil, fmt.Errorf("batch request %q doesn't have an url key", key) } else if reqURL, err = ToURL(murl); err != nil { diff --git a/js/modules/k6/http/request_test.go b/js/modules/k6/http/request_test.go index a77fc022b26..d5688685c58 100644 --- a/js/modules/k6/http/request_test.go +++ b/js/modules/k6/http/request_test.go @@ -1627,7 +1627,7 @@ func checkErrorCode(t testing.TB, tags *stats.SampleTags, code int, msg string) if msg == "" { assert.False(t, ok) } else { - assert.Equal(t, msg, errorMsg) + assert.Contains(t, errorMsg, msg) } errorCodeStr, ok := tags.Get("error_code") if code == 0 { @@ -1693,7 +1693,7 @@ func TestErrorCodes(t *testing.T) { { name: "Bad location redirect", expectedErrorCode: 1000, - expectedErrorMsg: "failed to parse Location header \"h\\t:/\": parse h\t:/: net/url: invalid control character in URL", //nolint: lll + expectedErrorMsg: "failed to parse Location header \"h\\t:/\": ", script: `let res = http.request("GET", "HTTPBIN_URL/bad-location-redirect");`, }, { @@ -1730,7 +1730,7 @@ func TestErrorCodes(t *testing.T) { _, err := common.RunString(rt, sr(testCase.script+"\n"+fmt.Sprintf(` if (res.status != %d) { throw new Error("wrong status: "+ res.status);} - if (res.error != %q) { throw new Error("wrong error: '" + res.error + "'");} + if (res.error.indexOf(%q, 0) === -1) { throw new Error("wrong error: '" + res.error + "'");} if (res.error_code != %d) { throw new Error("wrong error_code: "+ res.error_code);} `, testCase.status, testCase.expectedErrorMsg, testCase.expectedErrorCode))) if testCase.expectedScriptError == "" { diff --git a/js/runner.go b/js/runner.go index 6bdd01a6061..aba2ca7d0df 100644 --- a/js/runner.go +++ b/js/runner.go @@ -199,7 +199,10 @@ func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) common.BindToGlobal(vu.Runtime, map[string]interface{}{ "open": func() { - common.Throw(vu.Runtime, errors.New("\"open\" function is only available to the init code (aka global scope), see https://docs.k6.io/docs/test-life-cycle for more information")) + common.Throw(vu.Runtime, errors.New( + `The "open()" function is only available to init code (aka the global scope), see `+ + ` https://k6.io/docs/using-k6/test-life-cycle for more information`, + )) }, }) diff --git a/js/runner_test.go b/js/runner_test.go index 267e2f656aa..8a87eabcfdf 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -336,6 +336,7 @@ func testSetupDataHelper(t *testing.T, data string) { }) } } + func TestSetupDataReturnValue(t *testing.T) { testSetupDataHelper(t, ` export let options = { setupTimeout: "1s", teardownTimeout: "1s" }; @@ -410,6 +411,7 @@ func TestSetupDataNoReturn(t *testing.T) { } };`) } + func TestRunnerIntegrationImports(t *testing.T) { t.Run("Modules", func(t *testing.T) { modules := []string{ @@ -513,7 +515,7 @@ func TestVURunContext(t *testing.T) { } func TestVURunInterrupt(t *testing.T) { - //TODO: figure out why interrupt sometimes fails... data race in goja? + // TODO: figure out why interrupt sometimes fails... data race in goja? if isWindows { t.Skip() } @@ -550,7 +552,7 @@ func TestVURunInterrupt(t *testing.T) { } func TestVURunInterruptDoesntPanic(t *testing.T) { - //TODO: figure out why interrupt sometimes fails... data race in goja? + // TODO: figure out why interrupt sometimes fails... data race in goja? if isWindows { t.Skip() } @@ -582,7 +584,7 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { for i := 0; i < 1000; i++ { wg.Add(1) newCtx, newCancel := context.WithCancel(ctx) - var ch = make(chan struct{}) + ch := make(chan struct{}) go func() { defer wg.Done() close(ch) @@ -878,7 +880,7 @@ func TestVUIntegrationHosts(t *testing.T) { } func TestVUIntegrationTLSConfig(t *testing.T) { - var unsupportedVersionErrorMsg = "remote error: tls: handshake failure" + unsupportedVersionErrorMsg := "remote error: tls: handshake failure" for _, tag := range build.Default.ReleaseTags { if tag == "go1.12" { unsupportedVersionErrorMsg = "tls: no supported versions satisfy MinVersion and MaxVersion" @@ -1009,11 +1011,11 @@ func TestVUIntegrationOpenFunctionError(t *testing.T) { vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) assert.NoError(t, err) 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") + assert.Error(t, err) + assert.Contains(t, err.Error(), "only available to init code") } func TestVUIntegrationCookiesReset(t *testing.T) { - tb := httpmultibin.NewHTTPMultiBin(t) defer tb.Cleanup() @@ -1312,7 +1314,7 @@ func TestHTTPRequestInInitContext(t *testing.T) { } func TestInitContextForbidden(t *testing.T) { - var table = [...][3]string{ + table := [...][3]string{ { "http.request", `import http from "k6/http"; diff --git a/lib/consts/consts.go b/lib/consts/consts.go index 313c393631a..04efe1543ad 100644 --- a/lib/consts/consts.go +++ b/lib/consts/consts.go @@ -8,7 +8,7 @@ import ( ) // Version contains the current semantic version of k6. -var Version = "0.26.0" //nolint:gochecknoglobals +var Version = "0.26.1" //nolint:gochecknoglobals // VersionDetails can be set externally as part of the build process var VersionDetails = "" // nolint:gochecknoglobals diff --git a/loader/loader.go b/loader/loader.go index 0983ed4016a..6a91edb28c1 100644 --- a/loader/loader.go +++ b/loader/loader.go @@ -62,7 +62,7 @@ var ( `local directory (-v /local/path/:/inside/docker/path) containing ` + `your script and modules so that they're accessible by k6 from ` + `inside of the container, see ` + - `https://docs.k6.io/v1.0/docs/modules#section-using-local-modules-with-docker.` + `https://k6.io/docs/using-k6/modules#using-local-modules-with-docker` errNoLoaderMatched = errors.New("no loader matched") ) diff --git a/logo.png b/logo.png deleted file mode 100644 index 9e1464280cc038899fe8c80e38b13e51d07912ff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6443 zcmW+*c|4Tg7alXlzKw0{zD-GFiR_^g!bG+-#x~gsHH~GEC8kuC5VDl5#Zn9gVT|ym zkZdgsF&I>0FeD`6_xkm!ZZ+BLZPlgWyfe2b#nL9!t9BSYr z!@~t?Dm?=`ArPrw*5;?3BMX)$ZoKlmaHD&b-EW+6%HImAb>M&qiuXS!PG}d;0TEGA zGbpeAOgt8=HZ+!;_@a~7{l6yyHm6FSd4yr;HdU2X$qg6Oo`iqJHeeb8;S2GY%96On z=;kxa+ilUs5yfkZu>;}KsH8_iF_1i*N8%!yq@+D$!+hOVtfa-KmhrT4<6lf$fP^**p7TE50Z41} zV!u)|w=4rhr3re+8rUPpNey=!-t&lKP%sVYjqTh=&Ii<0DCPwLz}-uV;|WEQhb>4- zGw!FRUcWot#o;Cpm>~&b2heUY?zwk(9dJp&0U%i2G(z=QQwO$05T3*1&X`_%JoA7D z6c?9Qwp?;k9JSy%Iuo@(hmcPWM{lpDg^>f!`io$3^zf2_N(L1Dn9M!8Y`F1En;&tK zPc5Ksoyz;%)Ssg~8 zn~SADPA7@jh4S({cf~QE&H3ZS_}ezZaN+N;hvLCGUVL#ZuzTg)owzTL0}HPnp0#-f zWR|(|%8h;3%7NWG`@#aMLMaLGp3oV&bp-|*@>ss`UgRf7`F%O_IN`*J?qkRi#(C8e zAoFli>nqF9I@L0eAy44y{g^&=YN~1p3PdKW19|Ve3liPDj!5j=^pLn!*4x1cQu7Ky z?KI{`gMwS@N}wR}R$cCAcg(NV(sav=(w!(XbfWdrOuOP@I9L)6W_ z^nbZ!>}MV25D3aY`1dBph!5lKeg;gA61<~pBDvH^30#|rlDeby&`M4al=UV4E8XJv zBjEyH5A%s5loC5fH<{=8>jyf8Yt>YHhnU;YkbOXuKt;#+&q>rUV2!0`)R3>2e^9GA%)`)gC?v&jx>J=i=1-g zryA}t5z!1LUs{Og!e`jkCJod@KD8c)1$3}cPwUO&pxiHtD4ot*DNT2w%s+7b< zC_{&P6-+tz0@%4)4OCqKz{Ns^fbP2wt^cx?mxCUm9OCANu&Ppy^l;%SK-)kbp@r8E z{lu%6?(c6Pp@k^J(Hi{ulj!F_=8!rxtx@BduQC&R1kKH-X6@nRpP>JJ@T&wor=%K; zUGi@%@&OENEJi*Keff&Vpo^9`1lr?AZ2xNN{$=;|8*xl##V2a%QCr8kd07W=`U|pz zmOh=HU1|~g^DQXaad@crdD(7rE7)b>6tH|E)kx!4nk7j|g%xbijQNP+Qr(q}Vd@ z?7{mXWZ;olle?gD-UKMQr08b5D=Tv);Dh}MGQU+S^ytZy_>otC7#sI?-a2_PNR2W~OLRU$`L*hTyE`{}Jl+XSR?;&O3_73N8w+yf z!CoI&(N$D^{y?;^e9aI4rB%e>JuOij96>1uf+Qgq&8wdaU~kP_&LwiF*yBpC3wA1A zYv<3Cta<3TzhX%fba=@rQg&NQOXM|S!dbGJ4TZ0X)56H@1K>#Ra>SJ`@zJJ!1Rlkw zdM4|1A2~OxXm^D2Syv*$8u|9#L3w38^-kW<3?c_z95YXY?Q0#G5*cX{z0%ZD<(cbT zlmQwKnvbkgg}Uj*-h_&w6zgSaa%yFta_@lJ>|kDfA!>8NtfUsN!3Zt!+%@Q7Eb@{C zDZ29vy$tC9Hb9cDrRnG|M0yi^QEKWv8jm+1PgC z%L_f9uhDG}7E7v9J`)s%uPa0uP7FR1UXNiE9GjoiW?HdO6=i_U+d|8^DjGaha;0mS z8sW;P7Ja=Nzjnuo&~o--=_(qH3Sx{nk^F)bR<^rzog2Okqy*>tS{c{JBVCaWO7NU2 zJhuCtwrr#md@f*-@ibF+q0XM(Hoqi^M#M0FNVD1B{EAK{JNf)YS37esg;U@TbJGIoPWWN`P ztPlyzFl~ee5jLTAe$rAae?94`Jru&^VWB>h0TlzK)HDIZQ7$?8X@jVU3sqxrtsp)R z!nopL{zPixpiEQPJIqr1Bp*4DVRLisLC&X%u!Jt-W}EkN?xF2tj<{?#;aBT{sNxpt0cN*uuQE_w0$+`f!)##_6aRbBo`BOQf2w=djWL z>Gha@E;e$?gH{@wP$%+P|NZvxuF^zn+mH7K!(DDn3NBvuFhA4{aWm;2fZA!+O3q(Z z*IhCx15~@;0BkH0@l~E0ky<#k`zh*h;MV37K=rJ2N@A5QZ1THS^+p!*xI=HR8Qdl2 zvi(%^v5OG?v6$UsGSn>ev$R7ym#lw@V^)`HHJZ@94JU5F%5B0@$aRzATitBd*iQ|NQTvK8|XI zjauyFw<;3$A6V&TFoFfwJj^>M`vCW1UO@6Kwxpp7EMF*Z2VAE^cYuKDU;RraVolk( zC6`4B)kH+?5eh9rn)05;b5J(EbZF7kT5#C^<&YE$s;w_geM95%DYn>KEyc?pgGF}G z9^mH{u@RH^%;NNPJ(^sOBe6)4+t5m?#1Nq5OO-b=!W%PIo#o&;?Sj6$;aJXy6F$$02n!X;&cyJIL@UXs$1F9B zAFww4T zC*Xl-CyrU+U_s4AGj<#c%ed)8fMdVQ`@jm6g&GS8!O3&Jo}}?)ygOu~T-ie+HW4{c zykmFse>^BD0eCk9j7V_*fjH2^px@FgqmHoy9ETvgK2&LW=W#3Gx64 zw!Gwv5H#pm&$$o~DS{D;&j(K?cKqS7n6E zSWfNosLiYN!S!=jztS)(@z2CmICTSmAAp|ghKW)DS|SZjN5|`eaT*%ZU_looF<>DS zd0FcT%Ho*okRMp&0Y{&E&`LG_7igR<_hg_)f)y-pGNuNNi~R^@&E3&UH+~s$(s=<( zFnluJMuf8A9+n6uuus`hz5Xt=nVVio>IC_9`YdsEok^iWWNRRGzx`1PSNZ?;=5!kY zKq)+D zTxnYHe!*{)#g=K2mvf-R_bmmy*TCE0RQ1@7K&mn+eI}}<7_u|}WN>M@5fCwi)6JL> z|#ay1o?<8484?-$m{y6zZnecR-?HqQ}I5&Se!WB#bQaQ;dSbwzg*Kww)D$RFb zTAr>8sDaVfCWsE-9mYjh8dxuobonE*z)w!;QMj?chxu1WVX_Ih1AUoJI^Ui;f7qnh zb@gwHWNbw7x8Vf{I{1aRP=;(?H|aXm>s|lKd7BR;Vmg>5(o{wLhX39fGyLDW5oCqa zzeNKuU|l@_DmK|!lUK9bR$L-u!))Z&5~Pi^tq>&P!Zcf|YpczbJgVL%4h!Pb7`7;B zy*^Dh`%4oUnfLVug1Tm+20f%#9)B*8!Zc<1d!Sx!t;N^!|HSXzj>rJ27hb~OIsoDO z-mEsq*XmmsO#MToeaarf(<-%!6WePd)AIQXWT#{5NllQr+bl)B%vGRCB>)-(R;QGp zST$7PKk>cthVpZ`~xp7#;K*HU8(F#ZP)=J6DGbdj8R z$@zrw@TA1za%h)^dQ~#r)1~XQ6A!r)sNCP4eaDMtqv8$yBtLg>;Q|+Lml1@~ZvAMd ze;-2`^+385Z7ccuiChOD;lVT1{2P+NhDFT+*b3u|Tk1_*a+Fb9u?dl;qc`eKnfezvs7ur3rvuS&^ ze1dbJJXn9P9lj&1;auFc9$1`^KVgbAfHo;;4ZWdm4|yF~8H>em(q(XkZ*EwqEpfKV z9NVoSDIo@0a-UxFRp`9a{gzs4G6mdunC?RmGiwslYhsCN3#r-qJSb~UqQq@BacHK6 zEn8DJdW2zdN#mK7FYayMbiW<_VatDPb7-MSu<7hfy+?uUi(LhJHnj_fvRxIq)N;@( zX&h^sbD1FP%&>~Mq|4wAWX1D``! zznLRV2}$i)<6Ny9FjqD;0+k3?^tm-pP`aLX#e9}Kj|k8DC&^S&8W#1Iq(CunxzX>n zH}6N^8nwCpf7oOSzk0@D1BuOVeceH;(KMGFA{DnKQyo3;1LTs}%Yfw6d7fpKK7a7rkDYm7(e8FKCmj*? zRoA}c@UGCNP!QD(mR6hZ9a_i z!8{2~pBSi#zaE*k-m0@4!6}bjN#LX(VKxF#^96t8GOw4ivS{ONyPpj5?Ui=@bpo?? ze2cGegl9u`KBPITkzHp7WktCk)Xpd~N~JVvM{wScm5X!({zwCFihNEg5Eyo*^M?47 z$xEk4H%xP`fgyd$kdftMym)8XFNT!CO*dfFMgQMf(K=b;(Fi71ahvDmHEK?1T?{nHWO_R@ zX{g_mIS;MLPj0bhvyKDBk!dE%OscSGxZw$gKItO-{?y0LX1&?NqeOyfYS<>uEF^^GJypX^_sk14uA}8SVVQP>KVt&Au!>5?b zbJ}nt888Z;Ib7K|_UHzMJv%91%~>R1&|ON>P4#`(cF6?7QjGL18SF0!6}yqah?Qw? zg`b@*`qs~yaZe##hu@ErG8pq==rGIS3$7EJK3dD$vEtRj+q1EAHy?P{uN0STn~1!z zo%mC;I~C)-_MuT9!A>TaA*2b)`2K<#^*?Ziy7(xGt0`yoDYPbNPIV?hf@*l3Eww{( z$xW!7UoLx;xX9A!C@PB0zcfWrcQ+s1aPxa~hGFW}axd=51)M~)mNKgFPNR67$;z4Z zV8!mZ(8M}psjbL+(?3E@E|&l_fxT-n=7;*UJAPoN4R31WkJk3Qk?0K8|4u%xpsgGf ziL>RJbn39S@CDRKvwI4WfDc2IiGJ5|?;Fbn8qv;6_{5wN5GIc&PA#I%?>^>ut+Aqu zoYfZ&lwRT4UAjiol0=pq5VJXB%S~V3^C#e{AR20`?@-#zYd}f->dEf7n_u1l7kNih z%i^5~q0M^Pk!d#v*ZEn(WDCGYq5yM9H^sPn{o{ljdAbWyj3WDdD5Sd1nOaWMcmF{m zm4fl_(>vAA?FuYwH(2T4MKylZmoJG^pp&TZfq#X(n=`wBQ*fL^w29DPwPPF+ihWJ7 zta_@R_=2$J!5e(~!Y|)f#3?=})kLSoN>aHZ;@)pgMVv69RzICx9sk@qWyxh!9`?6Q zE`{|f1iCDLu_?6Lf41!Wj$Fj{Po#&!;E3 \ No newline at end of file diff --git a/release notes/v0.26.1.md b/release notes/v0.26.1.md new file mode 100644 index 00000000000..ef36f4a4427 --- /dev/null +++ b/release notes/v0.26.1.md @@ -0,0 +1,12 @@ +k6 v0.26.1 is here! This is a minor release that supports the rebranding of LoadImpact to k6, the new [k6.io website](https://k6.io/), and the new [k6 cloud service](https://k6.io/cloud)! :tada: + +In practical terms, all that it means for k6 is that the URLs for cloud tests will point to https://app.k6.io, instead of https://app.loadimpact.com. The old URLs (and old k6 versions) will still continue to work - for the next 3 months the old app and the new one would work in parallel, and after that period the old app will redirect to the new one. Nothing changes in regards to the k6 open source project and our commitment to it! + +You can find more information about the rebranding in our blog post about it: https://k6.io/blog/load-impact-rebranding-to-k6 + +# Changes in this release compared to v0.26.0: + +* Fix how HTTP request timeouts are specified internally. This is not a bug in current k6 releases, it only affects k6 if it is compiled with Go 1.14, which at this time is still not officially released. (#1261) +* Improve the official docker image to use an unprivileged user. Thanks, @funkypenguin! (#1314) +* Fix the unintentional sharing of `__ENV` between VUs, which could result in data races and crashes of k6. (#1329) +* Update cloud URLs to point to https://app.k6.io instead of https://app.loadimpact.com. (#1335) \ No newline at end of file diff --git a/samples/http_2.js b/samples/http_2.js index 868e2891d18..a1d1a356c7f 100644 --- a/samples/http_2.js +++ b/samples/http_2.js @@ -1,8 +1,8 @@ import http from "k6/http"; import { check } from "k6"; -export default function() { - check(http.get("https://www.bbc.co.uk/"), { +export default function () { + check(http.get("https://test-api.k6.io/"), { "status is 200": (r) => r.status == 200, "protocol is HTTP/2": (r) => r.proto == "HTTP/2.0", }); diff --git a/samples/http_get.js b/samples/http_get.js index 7ce7c0ac555..6af937d0c6e 100644 --- a/samples/http_get.js +++ b/samples/http_get.js @@ -1,5 +1,5 @@ import http from 'k6/http'; -export default function() { - const response = http.get("http://test.loadimpact.com"); +export default function () { + const response = http.get("https://test-api.k6.io/"); }; diff --git a/stats/cloud/config.go b/stats/cloud/config.go index 61259805f6a..21ecc85e90b 100644 --- a/stats/cloud/config.go +++ b/stats/cloud/config.go @@ -150,8 +150,8 @@ type Config struct { // NewConfig creates a new Config instance with default values for some fields. func NewConfig() Config { return Config{ - Host: null.NewString("https://ingest.loadimpact.com", false), - WebAppURL: null.NewString("https://app.loadimpact.com", false), + Host: null.NewString("https://ingest.k6.io", false), + WebAppURL: null.NewString("https://app.k6.io", false), MetricPushInterval: types.NewNullDuration(1*time.Second, false), MaxMetricSamplesPerPackage: null.NewInt(100000, false), // Aggregation is disabled by default, since AggregationPeriod has no default value diff --git a/stats/cloud/util.go b/stats/cloud/util.go index f56407434dd..c955ce3e5ed 100644 --- a/stats/cloud/util.go +++ b/stats/cloud/util.go @@ -27,5 +27,5 @@ func URLForResults(refID string, config Config) string { if config.Token.String == "" { path = "anonymous" } - return fmt.Sprintf("%s/k6/%s/%s", config.WebAppURL.String, path, refID) + return fmt.Sprintf("%s/%s/%s", config.WebAppURL.String, path, refID) } diff --git a/vendor/github.com/dop251/goja/builtin_date.go b/vendor/github.com/dop251/goja/builtin_date.go index 5344b5fda51..f4bbdbc6689 100644 --- a/vendor/github.com/dop251/goja/builtin_date.go +++ b/vendor/github.com/dop251/goja/builtin_date.go @@ -152,7 +152,7 @@ func (r *Runtime) dateproto_toUTCString(call FunctionCall) Value { obj := r.toObject(call.This) if d, ok := obj.self.(*dateObject); ok { if d.isSet { - return asciiString(d.time.In(time.UTC).Format(dateTimeLayout)) + return asciiString(d.time.In(time.UTC).Format(utcDateTimeLayout)) } else { return stringInvalidDate } diff --git a/vendor/github.com/dop251/goja/builtin_global.go b/vendor/github.com/dop251/goja/builtin_global.go index 88cac2b3a2c..da41f7e2764 100644 --- a/vendor/github.com/dop251/goja/builtin_global.go +++ b/vendor/github.com/dop251/goja/builtin_global.go @@ -6,10 +6,13 @@ import ( "math" "regexp" "strconv" + "strings" "unicode/utf16" "unicode/utf8" ) +const hexUpper = "0123456789ABCDEF" + var ( parseFloatRegexp = regexp.MustCompile(`^([+-]?(?:Infinity|[0-9]*\.?[0-9]*(?:[eE][+-]?[0-9]+)?))`) ) @@ -100,14 +103,14 @@ func (r *Runtime) _encode(uriString valueString, unescaped *[256]bool) valueStri n := utf8.EncodeRune(utf8Buf, rn) for _, b := range utf8Buf[:n] { buf[i] = '%' - buf[i+1] = "0123456789ABCDEF"[b>>4] - buf[i+2] = "0123456789ABCDEF"[b&15] + buf[i+1] = hexUpper[b>>4] + buf[i+2] = hexUpper[b&15] i += 3 } } else if !unescaped[rn] { buf[i] = '%' - buf[i+1] = "0123456789ABCDEF"[rn>>4] - buf[i+2] = "0123456789ABCDEF"[rn&15] + buf[i+1] = hexUpper[rn>>4] + buf[i+2] = hexUpper[rn&15] i += 3 } else { buf[i] = byte(rn) @@ -233,6 +236,94 @@ func (r *Runtime) builtin_encodeURIComponent(call FunctionCall) Value { return r._encode(uriString, &uriUnescaped) } +func (r *Runtime) builtin_escape(call FunctionCall) Value { + s := call.Argument(0).ToString() + var sb strings.Builder + l := s.length() + for i := int64(0); i < l; i++ { + r := uint16(s.charAt(i)) + if r >= 'A' && r <= 'Z' || r >= 'a' && r <= 'z' || r >= '0' && r <= '9' || + r == '@' || r == '*' || r == '_' || r == '+' || r == '-' || r == '.' || r == '/' { + sb.WriteByte(byte(r)) + } else if r <= 0xff { + sb.WriteByte('%') + sb.WriteByte(hexUpper[r>>4]) + sb.WriteByte(hexUpper[r&0xf]) + } else { + sb.WriteString("%u") + sb.WriteByte(hexUpper[r>>12]) + sb.WriteByte(hexUpper[(r>>8)&0xf]) + sb.WriteByte(hexUpper[(r>>4)&0xf]) + sb.WriteByte(hexUpper[r&0xf]) + } + } + return asciiString(sb.String()) +} + +func (r *Runtime) builtin_unescape(call FunctionCall) Value { + s := call.Argument(0).ToString() + l := s.length() + _, unicode := s.(unicodeString) + var asciiBuf []byte + var unicodeBuf []uint16 + if unicode { + unicodeBuf = make([]uint16, 0, l) + } else { + asciiBuf = make([]byte, 0, l) + } + for i := int64(0); i < l; { + r := s.charAt(i) + if r == '%' { + if i <= l-6 && s.charAt(i+1) == 'u' { + c0 := s.charAt(i + 2) + c1 := s.charAt(i + 3) + c2 := s.charAt(i + 4) + c3 := s.charAt(i + 5) + if c0 <= 0xff && ishex(byte(c0)) && + c1 <= 0xff && ishex(byte(c1)) && + c2 <= 0xff && ishex(byte(c2)) && + c3 <= 0xff && ishex(byte(c3)) { + r = rune(unhex(byte(c0)))<<12 | + rune(unhex(byte(c1)))<<8 | + rune(unhex(byte(c2)))<<4 | + rune(unhex(byte(c3))) + i += 5 + goto out + } + } + if i <= l-3 { + c0 := s.charAt(i + 1) + c1 := s.charAt(i + 2) + if c0 <= 0xff && ishex(byte(c0)) && + c1 <= 0xff && ishex(byte(c1)) { + r = rune(unhex(byte(c0))<<4 | unhex(byte(c1))) + i += 2 + } + } + } + out: + if r >= utf8.RuneSelf && !unicode { + unicodeBuf = make([]uint16, 0, l) + for _, b := range asciiBuf { + unicodeBuf = append(unicodeBuf, uint16(b)) + } + asciiBuf = nil + unicode = true + } + if unicode { + unicodeBuf = append(unicodeBuf, uint16(r)) + } else { + asciiBuf = append(asciiBuf, byte(r)) + } + i++ + } + if unicode { + return unicodeString(unicodeBuf) + } + + return asciiString(asciiBuf) +} + func (r *Runtime) initGlobalObject() { o := r.globalObject.self o._putProp("NaN", _NaN, false, false, false) @@ -247,6 +338,8 @@ func (r *Runtime) initGlobalObject() { o._putProp("decodeURIComponent", r.newNativeFunc(r.builtin_decodeURIComponent, nil, "decodeURIComponent", nil, 1), true, false, true) o._putProp("encodeURI", r.newNativeFunc(r.builtin_encodeURI, nil, "encodeURI", nil, 1), true, false, true) o._putProp("encodeURIComponent", r.newNativeFunc(r.builtin_encodeURIComponent, nil, "encodeURIComponent", nil, 1), true, false, true) + o._putProp("escape", r.newNativeFunc(r.builtin_escape, nil, "escape", nil, 1), true, false, true) + o._putProp("unescape", r.newNativeFunc(r.builtin_unescape, nil, "unescape", nil, 1), true, false, true) o._putProp("toString", r.newNativeFunc(func(FunctionCall) Value { return stringGlobalObject diff --git a/vendor/github.com/dop251/goja/date.go b/vendor/github.com/dop251/goja/date.go index c5f33115bc5..67aaf0297eb 100644 --- a/vendor/github.com/dop251/goja/date.go +++ b/vendor/github.com/dop251/goja/date.go @@ -6,6 +6,7 @@ import ( const ( dateTimeLayout = "Mon Jan 02 2006 15:04:05 GMT-0700 (MST)" + utcDateTimeLayout = "Mon, 02 Jan 2006 15:04:05 GMT" isoDateTimeLayout = "2006-01-02T15:04:05.000Z" dateLayout = "Mon Jan 02 2006" timeLayout = "15:04:05 GMT-0700 (MST)" diff --git a/vendor/github.com/dop251/goja/runtime.go b/vendor/github.com/dop251/goja/runtime.go index 26c62e703e9..82e56d5d13e 100644 --- a/vendor/github.com/dop251/goja/runtime.go +++ b/vendor/github.com/dop251/goja/runtime.go @@ -915,10 +915,22 @@ func (r *Runtime) RunProgram(p *Program) (result Value, err error) { // Interrupt a running JavaScript. The corresponding Go call will return an *InterruptedError containing v. // Note, it only works while in JavaScript code, it does not interrupt native Go functions (which includes all built-ins). +// If the runtime is currently not running, it will be immediately interrupted on the next Run*() call. +// To avoid that use ClearInterrupt() func (r *Runtime) Interrupt(v interface{}) { r.vm.Interrupt(v) } +// ClearInterrupt resets the interrupt flag. Typically this needs to be called before the runtime +// is made available for re-use if there is a chance it could have been interrupted with Interrupt(). +// Otherwise if Interrupt() was called when runtime was not running (e.g. if it had already finished) +// so that Interrupt() didn't actually trigger, an attempt to use the runtime will immediately cause +// an interruption. It is up to the user to ensure proper synchronisation so that ClearInterrupt() is +// only called when the runtime has finished and there is no chance of a concurrent Interrupt() call. +func (r *Runtime) ClearInterrupt() { + r.vm.ClearInterrupt() +} + /* ToValue converts a Go value into JavaScript value. @@ -1440,6 +1452,23 @@ func (r *Runtime) SetTimeSource(now Now) { r.now = now } +// New is an equivalent of the 'new' operator allowing to call it directly from Go. +func (r *Runtime) New(construct Value, args ...Value) (o *Object, err error) { + defer func() { + if x := recover(); x != nil { + switch x := x.(type) { + case *Exception: + err = x + case *InterruptedError: + err = x + default: + panic(x) + } + } + }() + return r.builtin_new(r.toObject(construct), args), nil +} + // Callable represents a JavaScript function that can be called from Go. type Callable func(this Value, args ...Value) (Value, error) diff --git a/vendor/github.com/dop251/goja/vm.go b/vendor/github.com/dop251/goja/vm.go index 96005aa42a4..b7fb6aef17c 100644 --- a/vendor/github.com/dop251/goja/vm.go +++ b/vendor/github.com/dop251/goja/vm.go @@ -88,7 +88,7 @@ func (r *unresolvedRef) get() Value { panic("Unreachable") } -func (r *unresolvedRef) set(v Value) { +func (r *unresolvedRef) set(Value) { r.get() } @@ -230,7 +230,7 @@ func (s *stash) getByIdx(idx uint32) Value { return _undefined } -func (s *stash) getByName(name string, vm *vm) (v Value, exists bool) { +func (s *stash) getByName(name string, _ *vm) (v Value, exists bool) { if s.obj != nil { v = s.obj.getStr(name) if v == nil { @@ -313,6 +313,10 @@ func (vm *vm) Interrupt(v interface{}) { vm.interruptLock.Unlock() } +func (vm *vm) ClearInterrupt() { + atomic.StoreUint32(&vm.interrupted, 0) +} + func (vm *vm) captureStack(stack []stackFrame, ctxOffset int) []stackFrame { // Unroll the context stack stack = append(stack, stackFrame{prg: vm.prg, pc: vm.pc, funcName: vm.funcName}) @@ -342,12 +346,12 @@ func (vm *vm) try(f func()) (ex *Exception) { // Restore other stacks iterTail := vm.iterStack[iterLen:] - for i, _ := range iterTail { + for i := range iterTail { iterTail[i] = iterStackItem{} } vm.iterStack = vm.iterStack[:iterLen] refTail := vm.refStack[refLen:] - for i, _ := range refTail { + for i := range refTail { refTail[i] = nil } vm.refStack = vm.refStack[:refLen] @@ -450,11 +454,10 @@ func (r *Runtime) toObject(v Value, args ...interface{}) *Object { return obj } if len(args) > 0 { - r.typeErrorResult(true, args) + panic(r.NewTypeError(args...)) } else { - r.typeErrorResult(true, "Value is not an object: %s", v.ToString()) + panic(r.NewTypeError("Value is not an object: %s", v.String())) } - panic("Unreachable") } func (r *Runtime) toCallee(v Value) *Object { @@ -482,14 +485,6 @@ func (_newStash) exec(vm *vm) { vm.pc++ } -type _noop struct{} - -var noop _noop - -func (_noop) exec(vm *vm) { - vm.pc++ -} - type loadVal uint32 func (l loadVal) exec(vm *vm) { @@ -497,13 +492,6 @@ func (l loadVal) exec(vm *vm) { vm.pc++ } -type loadVal1 uint32 - -func (l *loadVal1) exec(vm *vm) { - vm.push(vm.prg.values[*l]) - vm.pc++ -} - type _loadUndef struct{} var loadUndef _loadUndef @@ -971,7 +959,7 @@ type _setElem struct{} var setElem _setElem func (_setElem) exec(vm *vm) { - obj := vm.r.toObject(vm.stack[vm.sp-3]) + obj := vm.stack[vm.sp-3].ToObject(vm.r) propName := vm.stack[vm.sp-2] val := vm.stack[vm.sp-1] @@ -1052,8 +1040,7 @@ type setProp string func (p setProp) exec(vm *vm) { val := vm.stack[vm.sp-1] - - vm.r.toObject(vm.stack[vm.sp-2]).self.putStr(string(p), val, false) + vm.stack[vm.sp-2].ToObject(vm.r).self.putStr(string(p), val, false) vm.stack[vm.sp-2] = val vm.sp-- vm.pc++ @@ -1134,7 +1121,7 @@ func (g getProp) exec(vm *vm) { v := vm.stack[vm.sp-1] obj := v.baseObject(vm.r) if obj == nil { - vm.r.typeErrorResult(true, "Cannot read property '%s' of undefined", g) + panic(vm.r.NewTypeError("Cannot read property '%s' of undefined", g)) } prop := obj.self.getPropStr(string(g)) if prop1, ok := prop.(*valueProperty); ok { @@ -1155,7 +1142,7 @@ func (g getPropCallee) exec(vm *vm) { v := vm.stack[vm.sp-1] obj := v.baseObject(vm.r) if obj == nil { - vm.r.typeErrorResult(true, "Cannot read property '%s' of undefined", g) + panic(vm.r.NewTypeError("Cannot read property '%s' of undefined", g)) } prop := obj.self.getPropStr(string(g)) if prop1, ok := prop.(*valueProperty); ok { @@ -1179,7 +1166,7 @@ func (_getElem) exec(vm *vm) { obj := v.baseObject(vm.r) propName := vm.stack[vm.sp-1] if obj == nil { - vm.r.typeErrorResult(true, "Cannot read property '%s' of undefined", propName.String()) + panic(vm.r.NewTypeError("Cannot read property '%s' of undefined", propName.String())) } prop := obj.self.getProp(propName) @@ -1475,62 +1462,6 @@ func (s setGlobal) exec(vm *vm) { vm.pc++ } -type setVarStrict struct { - name string - idx uint32 -} - -func (s setVarStrict) exec(vm *vm) { - v := vm.peek() - - level := int(s.idx >> 24) - idx := uint32(s.idx & 0x00FFFFFF) - stash := vm.stash - name := s.name - for i := 0; i < level; i++ { - if stash.put(name, v) { - goto end - } - stash = stash.outer - } - - if stash != nil { - stash.putByIdx(idx, v) - } else { - o := vm.r.globalObject.self - if o.hasOwnPropertyStr(name) { - o.putStr(name, v, true) - } else { - vm.r.throwReferenceError(name) - } - } - -end: - vm.pc++ -} - -type setVar1Strict string - -func (s setVar1Strict) exec(vm *vm) { - v := vm.peek() - var o objectImpl - - name := string(s) - for stash := vm.stash; stash != nil; stash = stash.outer { - if stash.put(name, v) { - goto end - } - } - o = vm.r.globalObject.self - if o.hasOwnPropertyStr(name) { - o.putStr(name, v, true) - } else { - vm.r.throwReferenceError(name) - } -end: - vm.pc++ -} - type setGlobalStrict string func (s setGlobalStrict) exec(vm *vm) { @@ -1731,15 +1662,6 @@ func (_pop) exec(vm *vm) { vm.pc++ } -type _swap struct{} - -var swap _swap - -func (_swap) exec(vm *vm) { - vm.stack[vm.sp-1], vm.stack[vm.sp-2] = vm.stack[vm.sp-2], vm.stack[vm.sp-1] - vm.pc++ -} - func (vm *vm) callEval(n int, strict bool) { if vm.r.toObject(vm.stack[vm.sp-n-1]) == vm.r.global.Eval { if n > 0 { @@ -1883,7 +1805,7 @@ func (e enterFunc) exec(vm *vm) { } else { copy(vm.stash.values, vm.stack[vm.sp-vm.args:]) vv := vm.stash.values[vm.args:] - for i, _ := range vv { + for i := range vv { vv[i] = _undefined } } @@ -1929,7 +1851,7 @@ func (e enterFuncStashless) exec(vm *vm) { vm.sp += int(ss) vm.stack.expand(vm.sp) s := vm.stack[sp:vm.sp] - for i, _ := range s { + for i := range s { s[i] = _undefined } } From 0ca8f1325a1dc6c9a88246863f7cf6746ee7fcda Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 10 Mar 2020 23:01:16 +0200 Subject: [PATCH 109/350] Fix running result in the test status REST API call --- api/v1/status.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/v1/status.go b/api/v1/status.go index 93ecf945602..cbac01d2c6a 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -38,7 +38,7 @@ type Status struct { func NewStatus(engine *core.Engine) Status { executionState := engine.ExecutionScheduler.GetState() return Status{ - Running: executionState.HasStarted(), + Running: executionState.HasStarted() && !executionState.HasEnded(), Paused: null.BoolFrom(executionState.IsPaused()), VUs: null.IntFrom(executionState.GetCurrentlyActiveVUsCount()), VUsMax: null.IntFrom(executionState.GetInitializedVUsCount()), From 560bb73bb9778786d41d6bc1861ea4ca786bb0c8 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 10 Mar 2020 23:02:48 +0200 Subject: [PATCH 110/350] Fix a comment typo --- lib/consts/consts.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/consts/consts.go b/lib/consts/consts.go index 04efe1543ad..37ba155ac6a 100644 --- a/lib/consts/consts.go +++ b/lib/consts/consts.go @@ -13,7 +13,7 @@ var Version = "0.26.1" //nolint:gochecknoglobals // VersionDetails can be set externally as part of the build process var VersionDetails = "" // nolint:gochecknoglobals -// FullVersion returns the maximully full version and build information for +// FullVersion returns the maximally full version and build information for // the currently running k6 executable. func FullVersion() string { goVersionArch := fmt.Sprintf("%s, %s/%s", runtime.Version(), runtime.GOOS, runtime.GOARCH) @@ -29,7 +29,7 @@ func FullVersion() string { } // Banner contains the ASCII-art banner with the k6 logo and stylized website URL -//TODO: make these into methods, only the version needs to be a variable +// TODO: make these into methods, only the version needs to be a variable //nolint:gochecknoglobals var Banner = strings.Join([]string{ ` /\ |‾‾| /‾‾/ /‾/ `, From 7ce11f76f1a04f078c770e39e00fe5477fb625f6 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 00:10:01 +0200 Subject: [PATCH 111/350] Add a machine-readable test execution status field --- core/local/local.go | 11 +++++++++-- lib/execution.go | 43 +++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 506ac9a6c61..a06c28d9990 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -148,7 +148,7 @@ func (e *ExecutionScheduler) GetExecutionPlan() []lib.ExecutionStep { // initVU is just a helper method that's used to both initialize the planned VUs // in the Init() method, and also passed to executors so they can initialize // any unplanned VUs themselves. -//TODO: actually use the context... +// TODO: actually use the context... func (e *ExecutionScheduler) initVU( _ context.Context, logger *logrus.Entry, engineOut chan<- stats.SampleContainer, ) (lib.VU, error) { @@ -200,12 +200,13 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa // Initialize VUs concurrently doneInits := make(chan error, vusToInitialize) // poor man's early-return waitgroup - //TODO: make this an option? + // TODO: make this an option? initConcurrency := runtime.NumCPU() limiter := make(chan struct{}) subctx, cancel := context.WithCancel(ctx) defer cancel() + e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) for i := 0; i < initConcurrency; i++ { go func() { for range limiter { @@ -258,6 +259,7 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa return e.initVU(ctx, logger, engineOut) }) + e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) logger.Debugf("Finished initializing needed VUs, start initializing executors...") for _, exec := range e.executors { executorConfig := exec.GetConfig() @@ -268,6 +270,7 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa logger.Debugf("Initialized executor %s", executorConfig.GetName()) } + e.state.SetExecutionStatus(lib.ExecutionStatusInitDone) logger.Debugf("Initialization completed") return nil } @@ -332,6 +335,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam if e.state.IsPaused() { logger.Debug("Execution is paused, waiting for resume or interrupt...") + e.state.SetExecutionStatus(lib.ExecutionStatusPausedBeforeRun) e.initProgress.Modify(pb.WithConstProgress(1, "paused")) select { case <-e.state.ResumeNotify(): @@ -355,6 +359,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam // Run setup() before any executors, if it's not disabled if !e.options.NoSetup.Bool { logger.Debug("Running setup()") + e.state.SetExecutionStatus(lib.ExecutionStatusSetup) e.initProgress.Modify(pb.WithConstProgress(1, "setup()")) if err := e.runner.Setup(runCtx, engineOut); err != nil { logger.WithField("error", err).Debug("setup() aborted by error") @@ -365,6 +370,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam // Start all executors at their particular startTime in a separate goroutine... logger.Debug("Start all executors...") + e.state.SetExecutionStatus(lib.ExecutionStatusRunning) for _, exec := range e.executors { go e.runExecutor(runCtx, runResults, engineOut, exec) } @@ -383,6 +389,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam // Run teardown() after all executors are done, if it's not disabled if !e.options.NoTeardown.Bool { logger.Debug("Running teardown()") + e.state.SetExecutionStatus(lib.ExecutionStatusTeardown) if err := e.runner.Teardown(ctx, engineOut); err != nil { logger.WithField("error", err).Debug("teardown() aborted by error") return err diff --git a/lib/execution.go b/lib/execution.go index 28d879f52cb..45514cdfcc9 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -103,6 +103,24 @@ const MaxTimeToWaitForPlannedVU = 400 * time.Millisecond // MaxTimeToWaitForPlannedVU before we actually return an error. const MaxRetriesGetPlannedVU = 5 +// ExecutionStatus is similar to RunStatus, but more fine grained and concerns +// only local execution. +type ExecutionStatus uint32 + +// Possible execution status values +const ( + ExecutionStatusCreated ExecutionStatus = iota + ExecutionStatusInitVUs + ExecutionStatusInitExecutors + ExecutionStatusInitDone + ExecutionStatusPausedBeforeRun + ExecutionStatusStarted + ExecutionStatusSetup + ExecutionStatusRunning + ExecutionStatusTeardown + ExecutionStatusEnded +) + // ExecutionState contains a few different things: // - Some convenience items, that are needed by all executors, like the // execution segment and the unique VU ID generator. By keeping those here, @@ -161,7 +179,7 @@ type ExecutionState struct { // (for backwards compatibility...) currentVUIdentifier *uint64 - //TODO: add something similar, but for iterations? Currently, there isn't + // TODO: add something similar, but for iterations? Currently, there isn't // a straightforward way to get a unique sequential identifier per iteration // in the context of a single k6 instance. Combining __VU and __ITER gives us // a unique identifier, but it's unwieldy and somewhat cumbersome. @@ -202,6 +220,11 @@ type ExecutionState struct { // API, etc. interruptedIterationsCount *uint64 + // A machine-readable indicator in which the current state of the test + // execution is currently stored. Useful for the REST API and external + // observability of the k6 test run progress. + executionStatus *uint32 + // A nanosecond UNIX timestamp that is set when the test is actually // started. The default 0 value is used to denote that the test hasn't // started yet... @@ -256,6 +279,7 @@ func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *E Options: options, vus: make(chan VU, maxPossibleVUs), + executionStatus: new(uint32), currentVUIdentifier: new(uint64), initializedVUs: new(int64), uninitializedUnplannedVUs: &maxUnplannedUninitializedVUs, @@ -343,6 +367,19 @@ func (es *ExecutionState) AddInterruptedIterations(count uint64) uint64 { return atomic.AddUint64(es.interruptedIterationsCount, count) } +// SetExecutionStatus changes the current execution status to the supplied value +// and returns the current value. +func (es *ExecutionState) SetExecutionStatus(newStatus ExecutionStatus) (oldStatus ExecutionStatus) { + return ExecutionStatus(atomic.SwapUint32(es.executionStatus, uint32(newStatus))) +} + +// GetCurrentExecutionStatus returns the current execution status. Don't use +// this for synchronization unless you've made the k6 behavior somewhat +// predictable with options like --paused or --linger. +func (es *ExecutionState) GetCurrentExecutionStatus() ExecutionStatus { + return ExecutionStatus(atomic.LoadUint32(es.executionStatus)) +} + // MarkStarted saves the current timestamp as the test start time. // // CAUTION: Calling MarkStarted() a second time for the same execution state will @@ -351,6 +388,7 @@ func (es *ExecutionState) MarkStarted() { if !atomic.CompareAndSwapInt64(es.startTime, 0, time.Now().UnixNano()) { panic("the execution scheduler was started a second time") } + es.SetExecutionStatus(ExecutionStatusStarted) } // MarkEnded saves the current timestamp as the test end time. @@ -361,6 +399,7 @@ func (es *ExecutionState) MarkEnded() { if !atomic.CompareAndSwapInt64(es.endTime, 0, time.Now().UnixNano()) { panic("the execution scheduler was stopped a second time") } + es.SetExecutionStatus(ExecutionStatusEnded) } // HasStarted returns true if the test has actually started executing. @@ -493,7 +532,7 @@ func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount if modifyActiveVUCount { es.ModCurrentlyActiveVUsCount(+1) } - //TODO: set environment and exec + // TODO: set environment and exec return vu, nil case <-time.After(MaxTimeToWaitForPlannedVU): logger.Warnf("Could not get a VU from the buffer for %s", time.Duration(i)*MaxTimeToWaitForPlannedVU) From 6806126f11aeb909bdef703d673b868c03d7b46b Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 00:15:45 +0200 Subject: [PATCH 112/350] Refactor VU initialization to reduce the Init() method length --- core/local/local.go | 49 ++++++++++++++++++++++++++------------------- 1 file changed, 28 insertions(+), 21 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index a06c28d9990..c2cb4cbc517 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -187,27 +187,13 @@ func (e *ExecutionScheduler) getRunStats() string { ) } -// Init concurrently initializes all of the planned VUs and then sequentially -// initializes all of the configured executors. -func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { - logger := e.logger.WithField("phase", "local-execution-scheduler-init") - - vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) - logger.WithFields(logrus.Fields{ - "neededVUs": vusToInitialize, - "executorsCount": len(e.executors), - }).Debugf("Start of initialization") - - // Initialize VUs concurrently - doneInits := make(chan error, vusToInitialize) // poor man's early-return waitgroup - // TODO: make this an option? - initConcurrency := runtime.NumCPU() +func (e *ExecutionScheduler) initVUsConcurrently( + ctx context.Context, engineOut chan<- stats.SampleContainer, count uint64, concurrency int, logger *logrus.Entry, +) chan error { + doneInits := make(chan error, count) // poor man's early-return waitgroup limiter := make(chan struct{}) - subctx, cancel := context.WithCancel(ctx) - defer cancel() - e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) - for i := 0; i < initConcurrency; i++ { + for i := 0; i < concurrency; i++ { go func() { for range limiter { newVU, err := e.initVU(ctx, logger, engineOut) @@ -221,15 +207,36 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa go func() { defer close(limiter) - for vuNum := uint64(0); vuNum < vusToInitialize; vuNum++ { + for vuNum := uint64(0); vuNum < count; vuNum++ { select { case limiter <- struct{}{}: - case <-subctx.Done(): + case <-ctx.Done(): return } } }() + return doneInits +} + +// Init concurrently initializes all of the planned VUs and then sequentially +// initializes all of the configured executors. +func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { + logger := e.logger.WithField("phase", "local-execution-scheduler-init") + + vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) + logger.WithFields(logrus.Fields{ + "neededVUs": vusToInitialize, + "executorsCount": len(e.executors), + }).Debugf("Start of initialization") + + subctx, cancel := context.WithCancel(ctx) + defer cancel() + + // Initialize VUs concurrently + e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) + doneInits := e.initVUsConcurrently(subctx, engineOut, vusToInitialize, runtime.NumCPU(), logger) + initializedVUs := new(uint64) vusFmt := pb.GetFixedLengthIntFormat(int64(vusToInitialize)) e.initProgress.Modify( From b49afd9607460f94563166815d0cd8b6ec94bfc8 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 00:22:08 +0200 Subject: [PATCH 113/350] Expose the execution status via the REST API --- api/v1/status.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/api/v1/status.go b/api/v1/status.go index cbac01d2c6a..b30edd44ca5 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -22,22 +22,24 @@ package v1 import ( "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/lib" "gopkg.in/guregu/null.v3" ) type Status struct { - Paused null.Bool `json:"paused" yaml:"paused"` - VUs null.Int `json:"vus" yaml:"vus"` - VUsMax null.Int `json:"vus-max" yaml:"vus-max"` + Status lib.ExecutionStatus `json:"status" yaml:"status"` - // Readonly. - Running bool `json:"running" yaml:"running"` - Tainted bool `json:"tainted" yaml:"tainted"` + Paused null.Bool `json:"paused" yaml:"paused"` + VUs null.Int `json:"vus" yaml:"vus"` + VUsMax null.Int `json:"vus-max" yaml:"vus-max"` + Running bool `json:"running" yaml:"running"` + Tainted bool `json:"tainted" yaml:"tainted"` } func NewStatus(engine *core.Engine) Status { executionState := engine.ExecutionScheduler.GetState() return Status{ + Status: executionState.GetCurrentExecutionStatus(), Running: executionState.HasStarted() && !executionState.HasEnded(), Paused: null.BoolFrom(executionState.IsPaused()), VUs: null.IntFrom(executionState.GetCurrentlyActiveVUsCount()), From 18b17df1636215f44765673e83a093429c4a320c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 13:05:25 +0200 Subject: [PATCH 114/350] Remove a redundant comment --- core/local/local.go | 1 - 1 file changed, 1 deletion(-) diff --git a/core/local/local.go b/core/local/local.go index c2cb4cbc517..7fecc4651eb 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -233,7 +233,6 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa subctx, cancel := context.WithCancel(ctx) defer cancel() - // Initialize VUs concurrently e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) doneInits := e.initVUsConcurrently(subctx, engineOut, vusToInitialize, runtime.NumCPU(), logger) From 89c0e4c39634d2ebaedadd9b3697639a97f1f87a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 11 Mar 2020 12:24:34 +0100 Subject: [PATCH 115/350] Re-enable the usage report (#1343) * Re-enable the usage report Closes #1301 * Sync usage report goroutine with WaitGroup Resolves https://github.com/loadimpact/k6/pull/1343#discussion_r388145629 * Add initialized VUs count to usage report Partly resolves https://github.com/loadimpact/k6/pull/1343#discussion_r388141537 * Use full version in usage report Resolves https://github.com/loadimpact/k6/pull/1343#discussion_r387787531 * Latest changes from code review (will squash) * Fix nil pointer panic if timeout is reached * Use channel to sync reportUsage goroutine Resolves https://github.com/loadimpact/k6/pull/1343#discussion_r390826793 * Fix: only wait for report ch if it wasn't disabled --- cmd/run.go | 84 +++++++++++++++++++++++++++++++++--------------------- 1 file changed, 51 insertions(+), 33 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index e874113cd76..29a9a345051 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -24,8 +24,11 @@ import ( "archive/tar" "bytes" "context" + "encoding/json" + "net/http" "os" "os/signal" + "runtime" "sync" "syscall" "time" @@ -260,39 +263,6 @@ a commandline interface for interacting with it.`, signal.Notify(sigC, os.Interrupt, syscall.SIGINT, syscall.SIGTERM) defer signal.Stop(sigC) - // If the user hasn't opted out: report usage. - //TODO: fix - //TODO: move to a separate function - /* - if !conf.NoUsageReport.Bool { - go func() { - u := "http://k6reports.loadimpact.com/" - mime := "application/json" - var endTSeconds float64 - if endT := engine.Executor.GetEndTime(); endT.Valid { - endTSeconds = time.Duration(endT.Duration).Seconds() - } - var stagesEndTSeconds float64 - if stagesEndT := lib.SumStages(engine.Executor.GetStages()); stagesEndT.Valid { - stagesEndTSeconds = time.Duration(stagesEndT.Duration).Seconds() - } - body, err := json.Marshal(map[string]interface{}{ - "k6_version": Version, - "vus_max": engine.Executor.GetVUsMax(), - "iterations": engine.Executor.GetEndIterations(), - "duration": endTSeconds, - "st_duration": stagesEndTSeconds, - "goos": runtime.GOOS, - "goarch": runtime.GOARCH, - }) - if err != nil { - panic(err) // This should never happen!! - } - _, _ = http.Post(u, mime, bytes.NewBuffer(body)) - }() - } - */ - // Ticker for progress bar updates. Less frequent updates for non-TTYs, none if quiet. updateFreq := 50 * time.Millisecond if !stdoutTTY { @@ -350,6 +320,16 @@ a commandline interface for interacting with it.`, // but with uninterruptible iterations it will be even more problematic. } } + + var reportCh chan struct{} + if !conf.NoUsageReport.Bool { + reportCh = make(chan struct{}) + go func() { + _ = reportUsage(execScheduler) + close(reportCh) + }() + } + if quiet || !stdoutTTY { e := logger.WithFields(logrus.Fields{ "t": executionState.GetCurrentTestRunDuration(), @@ -407,6 +387,13 @@ a commandline interface for interacting with it.`, <-sigC } + if reportCh != nil { + select { + case <-reportCh: + case <-time.After(3 * time.Second): + } + } + if engine.IsTainted() { return ExitCode{error: errors.New("some thresholds have failed"), Code: thresholdHaveFailedErrorCode} } @@ -414,6 +401,37 @@ a commandline interface for interacting with it.`, }, } +func reportUsage(execScheduler *local.ExecutionScheduler) error { + execState := execScheduler.GetState() + executorConfigs := execScheduler.GetExecutorConfigs() + + executors := make(map[string]int) + for _, ec := range executorConfigs { + executors[ec.GetType()]++ + } + + body, err := json.Marshal(map[string]interface{}{ + "k6_version": consts.Version, + "executors": executors, + "vus_max": execState.GetInitializedVUsCount(), + "iterations": execState.GetFullIterationCount(), + "duration": execState.GetCurrentTestRunDuration().String(), + "goos": runtime.GOOS, + "goarch": runtime.GOARCH, + }) + if err != nil { + return err + } + res, err := http.Post("https://reports.k6.io/", "application/json", bytes.NewBuffer(body)) + defer func() { + if err == nil { + _ = res.Body.Close() + } + }() + + return err +} + func runCmdFlagSet() *pflag.FlagSet { flags := pflag.NewFlagSet("", pflag.ContinueOnError) flags.SortFlags = false From 50515800f5f4c0ade20ecc47a1976ed37903d768 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 14:05:29 +0200 Subject: [PATCH 116/350] Increase setupTimeout and teardownTimeout from 10 to 60s --- cmd/options.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/options.go b/cmd/options.go index 3e4099e52df..d82c525fed0 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -46,7 +46,7 @@ func optionFlagSet() *pflag.FlagSet { flags.SortFlags = false flags.Int64P("vus", "u", 1, "number of virtual users") - //TODO: delete in a few versions + // TODO: delete in a few versions flags.Int64P("max", "m", 0, "max available virtual users") _ = flags.MarkDeprecated("max", "the global MaxVUs option is obsolete and doesn't affect the k6 script execution") @@ -114,8 +114,8 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { DiscardResponseBodies: getNullBool(flags, "discard-response-bodies"), // Default values for options without CLI flags: // 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}, + SetupTimeout: types.NullDuration{Duration: types.Duration(60 * time.Second), Valid: false}, + TeardownTimeout: types.NullDuration{Duration: types.Duration(60 * time.Second), Valid: false}, MetricSamplesBufferSize: null.NewInt(1000, false), } From 567e5a0e225e3bb8f2d5dec00c36fd4fdd9a5d7a Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 11 Mar 2020 14:09:01 +0200 Subject: [PATCH 117/350] Move the default gracefulStop value to a variable --- lib/executor/base_config.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index 32547b3bb8a..45823625d06 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -30,6 +30,11 @@ import ( null "gopkg.in/guregu/null.v3" ) +// DefaultGracefulStopValue is the graceful top value for all executors, unless +// it's manually changed by the gracefulStop in each one. +// TODO?: Discard? Or make this actually user-configurable somehow? hello #883... +var DefaultGracefulStopValue = 30 * time.Second //nolint:gochecknoglobals + var executorNameWhitelist = regexp.MustCompile(`^[0-9a-zA-Z_-]+$`) //nolint:gochecknoglobals const executorNameErr = "the executor name should contain only numbers, latin letters, underscores, and dashes" @@ -42,7 +47,7 @@ type BaseConfig struct { Env map[string]string `json:"env"` Exec null.String `json:"exec"` // function name, externally validated - //TODO: future extensions like tags, distribution, others? + // TODO: future extensions like tags, distribution, others? } // NewBaseConfig returns a default base config with the default values @@ -50,7 +55,7 @@ func NewBaseConfig(name, configType string) BaseConfig { return BaseConfig{ Name: name, Type: configType, - GracefulStop: types.NewNullDuration(30*time.Second, false), + GracefulStop: types.NewNullDuration(DefaultGracefulStopValue, false), } } From 4017d71724af45ab036f38290773b36d4cdbc243 Mon Sep 17 00:00:00 2001 From: Kieren Hynd Date: Tue, 10 Mar 2020 21:56:39 +0000 Subject: [PATCH 118/350] Add a method to stop a running Engine via the HTTP API Closes #1352. --- api/v1/status.go | 2 ++ api/v1/status_routes.go | 53 ++++++++++++++++++++---------------- api/v1/status_routes_test.go | 1 + core/engine.go | 31 +++++++++++++++++---- core/engine_test.go | 14 ++++++++++ 5 files changed, 72 insertions(+), 29 deletions(-) diff --git a/api/v1/status.go b/api/v1/status.go index b30edd44ca5..187d3e42193 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -32,6 +32,7 @@ type Status struct { Paused null.Bool `json:"paused" yaml:"paused"` VUs null.Int `json:"vus" yaml:"vus"` VUsMax null.Int `json:"vus-max" yaml:"vus-max"` + Stopped bool `json:"stopped" yaml:"stopped"` Running bool `json:"running" yaml:"running"` Tainted bool `json:"tainted" yaml:"tainted"` } @@ -42,6 +43,7 @@ func NewStatus(engine *core.Engine) Status { Status: executionState.GetCurrentExecutionStatus(), Running: executionState.HasStarted() && !executionState.HasEnded(), Paused: null.BoolFrom(executionState.IsPaused()), + Stopped: engine.IsStopped(), VUs: null.IntFrom(executionState.GetCurrentlyActiveVUsCount()), VUsMax: null.IntFrom(executionState.GetInitializedVUsCount()), Tainted: engine.IsTainted(), diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index e1a9097d39e..01368a27bfb 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -71,34 +71,39 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par return } - if status.Paused.Valid { - if err = engine.ExecutionScheduler.SetPaused(status.Paused.Bool); err != nil { - apiError(rw, "Pause error", err.Error(), http.StatusInternalServerError) - return + if status.Stopped { + engine.Stop() + } else { + if status.Paused.Valid { + if err = engine.ExecutionScheduler.SetPaused(status.Paused.Bool); err != nil { + apiError(rw, "Pause error", err.Error(), http.StatusInternalServerError) + return + } } - } - if status.VUsMax.Valid || status.VUs.Valid { - //TODO: add ability to specify the actual executor id? Though this should - //likely be in the v2 REST API, where we could implement it in a way that - //may allow us to eventually support other executor types. - executor, updateErr := getFirstExternallyControlledExecutor(engine.ExecutionScheduler) - if updateErr != nil { - apiError(rw, "Execution config error", updateErr.Error(), http.StatusInternalServerError) - return - } - newConfig := executor.GetCurrentConfig().ExternallyControlledConfigParams - if status.VUsMax.Valid { - newConfig.MaxVUs = status.VUsMax - } - if status.VUs.Valid { - newConfig.VUs = status.VUs - } - if updateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { - apiError(rw, "Config update error", updateErr.Error(), http.StatusInternalServerError) - return + if status.VUsMax.Valid || status.VUs.Valid { + //TODO: add ability to specify the actual executor id? Though this should + //likely be in the v2 REST API, where we could implement it in a way that + //may allow us to eventually support other executor types. + executor, updateErr := getFirstExternallyControlledExecutor(engine.ExecutionScheduler) + if updateErr != nil { + apiError(rw, "Execution config error", updateErr.Error(), http.StatusInternalServerError) + return + } + newConfig := executor.GetCurrentConfig().ExternallyControlledConfigParams + if status.VUsMax.Valid { + newConfig.MaxVUs = status.VUsMax + } + if status.VUs.Valid { + newConfig.VUs = status.VUs + } + if updateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { + apiError(rw, "Config update error", updateErr.Error(), http.StatusInternalServerError) + return + } } } + data, err := jsonapi.Marshal(NewStatus(engine)) if err != nil { apiError(rw, "Encoding error", err.Error(), http.StatusInternalServerError) diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index 720c9389698..f7bc36a5d05 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -62,6 +62,7 @@ func TestGetStatus(t *testing.T) { assert.True(t, status.Paused.Valid) assert.True(t, status.VUs.Valid) assert.True(t, status.VUsMax.Valid) + assert.False(t, status.Stopped) assert.False(t, status.Tainted) }) } diff --git a/core/engine.go b/core/engine.go index c7bea4dec3d..9717cddb473 100644 --- a/core/engine.go +++ b/core/engine.go @@ -59,7 +59,8 @@ type Engine struct { NoSummary bool SummaryExport bool - logger *logrus.Logger + logger *logrus.Logger + stopChan chan struct{} Metrics map[string]*stats.Metric MetricsLock sync.Mutex @@ -84,10 +85,11 @@ func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) ExecutionScheduler: ex, executionState: ex.GetState(), - Options: o, - Metrics: make(map[string]*stats.Metric), - Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), - logger: logger, + Options: o, + Metrics: make(map[string]*stats.Metric), + Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), + stopChan: make(chan struct{}), + logger: logger, } e.thresholds = o.Thresholds @@ -218,6 +220,10 @@ func (e *Engine) Run(ctx context.Context) error { e.logger.Debug("run: context expired; exiting...") e.setRunStatus(lib.RunStatusAbortedUser) return nil + case <-e.stopChan: + e.logger.Debug("run: stopped by user; exiting...") + e.setRunStatus(lib.RunStatusAbortedUser) + return nil } } } @@ -226,6 +232,21 @@ func (e *Engine) IsTainted() bool { return e.thresholdsTainted } +// Stop closes a signal channel, forcing a running Engine to return +func (e *Engine) Stop() { + close(e.stopChan) +} + +// IsStopped returns a bool indicating whether the Engine has been stopped +func (e *Engine) IsStopped() bool { + select { + case <-e.stopChan: + return true + default: + return false + } +} + func (e *Engine) runMetricsEmission(ctx context.Context) { ticker := time.NewTicker(MetricsRate) for { diff --git a/core/engine_test.go b/core/engine_test.go index 3d4e9a477ef..7b711629d77 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -156,6 +156,20 @@ func TestEngineAtTime(t *testing.T) { assert.NoError(t, e.Run(ctx)) } +func TestEngineStopped(t *testing.T) { + e := newTestEngine(t, nil, nil, lib.Options{ + VUs: null.IntFrom(1), + Duration: types.NullDurationFrom(20 * time.Second), + }) + + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) + defer cancel() + assert.NoError(t, e.Run(ctx)) + assert.Equal(t, false, e.IsStopped(), "engine should be running") + e.Stop() + assert.Equal(t, true, e.IsStopped(), "engine should be stopped") +} + func TestEngineCollector(t *testing.T) { testMetric := stats.New("test_metric", stats.Trend) From 55a49d5a3db3d1af330644a87d0f406ec5f11a3d Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 17 Mar 2020 14:27:18 +0200 Subject: [PATCH 119/350] Always get the biggest plannedVUs for a givem timeoffset (#1361) fixes #1358 --- lib/executor/executors_test.go | 39 ++++++++++++++++++++++++++++++++++ lib/executors.go | 3 +++ 2 files changed, 42 insertions(+) diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index dcf2945b861..66d552b2ec0 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -151,6 +151,45 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) }}, }, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "10s", + "stages": [{"duration": "10s", "target": 10}]}}`, + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + assert.Empty(t, cm["varloops"].Validate()) + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "Up to 10 looping VUs for 10s over 1 stages (gracefulRampDown: 10s)", cm["varloops"].GetDescription(nil)) + + schedReqs := cm["varloops"].GetExecutionRequirements(nil) + assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) + }}, + }, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", + "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + assert.Empty(t, cm["varloops"].Validate()) + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "Up to 10 looping VUs for 20s over 3 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(nil)) + + schedReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) + }}, + }, + {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", + "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 11},{"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, + exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + assert.Empty(t, cm["varloops"].Validate()) + assert.Empty(t, cm.Validate()) + + assert.Equal(t, "Up to 11 looping VUs for 20s over 4 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(nil)) + + schedReqs := cm.GetFullExecutionRequirements(nil) + assert.Equal(t, uint64(11), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(11), lib.GetMaxPossibleVUs(schedReqs)) + }}, + }, {`{"varloops": {"type": "variable-looping-vus", "startVUs": 0, "stages": [{"duration": "60s", "target": 0}]}}`, exp{}}, {`{"varloops": {"type": "variable-looping-vus", "startVUs": -1, "stages": [{"duration": "60s", "target": 30}]}}`, exp{validationError: true}}, {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "-60s", "target": 30}]}}`, exp{validationError: true}}, diff --git a/lib/executors.go b/lib/executors.go index ea914688131..aec2de575c3 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -270,6 +270,9 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec case trackedSteps[a].TimeOffset < trackedSteps[b].TimeOffset: return true case trackedSteps[a].TimeOffset == trackedSteps[b].TimeOffset: + if trackedSteps[a].configID == trackedSteps[b].configID { + return trackedSteps[a].PlannedVUs < trackedSteps[b].PlannedVUs + } return trackedSteps[a].configID < trackedSteps[b].configID default: return false From d50c5d1beded8699c2fade0191de6c24f429cc24 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 17 Mar 2020 15:55:36 +0200 Subject: [PATCH 120/350] Don't skip steps in GetFullExecutionRequirements --- lib/executors.go | 21 +++++++-------------- 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/lib/executors.go b/lib/executors.go index aec2de575c3..bd21ad823bc 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -266,17 +266,11 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec // sorting algorithm, since there could be steps with the same time from // the same executor and their order is important. sort.SliceStable(trackedSteps, func(a, b int) bool { - switch { - case trackedSteps[a].TimeOffset < trackedSteps[b].TimeOffset: - return true - case trackedSteps[a].TimeOffset == trackedSteps[b].TimeOffset: - if trackedSteps[a].configID == trackedSteps[b].configID { - return trackedSteps[a].PlannedVUs < trackedSteps[b].PlannedVUs - } + if trackedSteps[a].TimeOffset == trackedSteps[b].TimeOffset { return trackedSteps[a].configID < trackedSteps[b].configID - default: - return false } + + return trackedSteps[a].TimeOffset < trackedSteps[b].TimeOffset }) // Go through all of the sorted steps from all of the executors, and @@ -309,15 +303,14 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec } } for _, step := range trackedSteps { + // TODO: optimize by skipping some steps // If the time offset is different, create a new step with the current values - if step.TimeOffset != currentTimeOffset { - addCurrentStepIfDifferent() - currentTimeOffset = step.TimeOffset - } + + currentTimeOffset = step.TimeOffset currentPlannedVUs[step.configID] = step.PlannedVUs currentMaxUnplannedVUs[step.configID] = step.MaxUnplannedVUs + addCurrentStepIfDifferent() } - addCurrentStepIfDifferent() // Add the last step return consolidatedSteps } From 886a3f03549e9c6c55bc03753154d07a8e9ad079 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 9 Mar 2020 10:54:08 +0200 Subject: [PATCH 121/350] Add an option to make k6 directly start pushing metrics to the cloud --- stats/cloud/collector.go | 8 +++++++- stats/cloud/config.go | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index 3af156406bb..db8c12515f1 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -150,6 +150,12 @@ func New( // Init is called between the collector's creation and the call to Run(). // You should do any lengthy setup here rather than in New. func (c *Collector) Init() error { + if c.config.PushRefID.Valid { + c.referenceID = c.config.PushRefID.String + logrus.WithField("referenceId", c.referenceID).Debug("Cloud: directly pushing metrics without init") + return nil + } + thresholds := make(map[string][]string) for name, t := range c.thresholds { @@ -527,7 +533,7 @@ func (c *Collector) pushMetrics() { } func (c *Collector) testFinished() { - if c.referenceID == "" { + if c.referenceID == "" || c.config.PushRefID.Valid { return } diff --git a/stats/cloud/config.go b/stats/cloud/config.go index 21ecc85e90b..238c9b54b33 100644 --- a/stats/cloud/config.go +++ b/stats/cloud/config.go @@ -37,6 +37,7 @@ type Config struct { Name null.String `json:"name" envconfig:"K6_CLOUD_NAME"` Host null.String `json:"host" envconfig:"K6_CLOUD_HOST"` + PushRefID null.String `json:"pushRefID" envconfig:"K6_CLOUD_PUSH_REF_ID"` WebAppURL null.String `json:"webAppURL" envconfig:"K6_CLOUD_WEB_APP_URL"` NoCompress null.Bool `json:"noCompress" envconfig:"K6_CLOUD_NO_COMPRESS"` From e731f6b9b7031c829eeb17ca9bcdc528806af4b6 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Sun, 15 Mar 2020 16:00:28 +0200 Subject: [PATCH 122/350] Fix style issues --- stats/cloud/collector.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index db8c12515f1..e492af49671 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -299,7 +299,7 @@ func (c *Collector) Collect(sampleContainers []stats.SampleContainer) { newSamples = append(newSamples, NewSampleFromTrail(sc)) } case *netext.NetTrail: - //TODO: aggregate? + // TODO: aggregate? values := map[string]float64{ metrics.DataSent.Name: float64(sc.BytesWritten), metrics.DataReceived.Name: float64(sc.BytesRead), @@ -317,7 +317,8 @@ func (c *Collector) Collect(sampleContainers []stats.SampleContainer) { Time: Timestamp(sc.GetTime()), Tags: sc.GetTags(), Values: values, - }}) + }, + }) default: for _, sample := range sampleContainer.GetSamples() { newSamples = append(newSamples, &Sample{ @@ -515,7 +516,7 @@ func (c *Collector) pushMetrics() { }).Debug("Pushing metrics to cloud") for len(buffer) > 0 { - var size = len(buffer) + size := len(buffer) if size > int(c.config.MaxMetricSamplesPerPackage.Int64) { size = int(c.config.MaxMetricSamplesPerPackage.Int64) } From f98206564edc8ddcec2ebced16c522b0a8114318 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 18 Mar 2020 21:13:37 +0200 Subject: [PATCH 123/350] Clean up and refactor the simple k6 REST API client --- api/v1/client/client.go | 67 ++++++++++++++++++++++++++++++++++------ api/v1/client/metrics.go | 7 ++--- api/v1/client/status.go | 11 ++++--- 3 files changed, 66 insertions(+), 19 deletions(-) diff --git a/api/v1/client/client.go b/api/v1/client/client.go index 0ef1b3bbe6f..e6c395e22f2 100644 --- a/api/v1/client/client.go +++ b/api/v1/client/client.go @@ -29,29 +29,73 @@ import ( "net/http" "net/url" + "github.com/sirupsen/logrus" + "github.com/manyminds/api2go/jsonapi" - "github.com/loadimpact/k6/api/v1" + v1 "github.com/loadimpact/k6/api/v1" ) +// Client is a simple HTTP client for the REST API. type Client struct { - BaseURL *url.URL + BaseURL *url.URL + httpClient *http.Client + logger *logrus.Entry } -func New(base string) (*Client, error) { +// Option function are helpers that enable the flexible configuration of the +// REST API client. +type Option func(*Client) + +// New returns a newly configured REST API Client. +func New(base string, options ...Option) (*Client, error) { baseURL, err := url.Parse("http://" + base) if err != nil { return nil, err } - return &Client{BaseURL: baseURL}, nil + c := &Client{ + BaseURL: baseURL, + httpClient: http.DefaultClient, + } + + for _, option := range options { + option(c) + } + + return c, nil +} + +// WithHTTPClient configures the supplied HTTP client to be used when making +// REST API requests. +func WithHTTPClient(httpClient *http.Client) Option { + return Option(func(c *Client) { + c.httpClient = httpClient + }) +} + +// WithLogger sets the specifield logger to the client. +func WithLogger(logger *logrus.Entry) Option { + return Option(func(c *Client) { + c.logger = logger + }) } -func (c *Client) call(ctx context.Context, method string, rel *url.URL, body, out interface{}) error { +// Call executes the desired REST API request. +func (c *Client) Call(ctx context.Context, method string, rel *url.URL, body, out interface{}) (err error) { + if c.logger != nil { + c.logger.Debugf("[REST API] Making a %s request to '%s'", method, rel.String()) + defer func() { + if err != nil { + c.logger.WithError(err).Error("[REST API] Error") + } + }() + } + var bodyReader io.ReadCloser if body != nil { - bodyData, err := jsonapi.Marshal(body) - if err != nil { - return err + bodyData, errm := jsonapi.Marshal(body) + if errm != nil { + return errm } bodyReader = ioutil.NopCloser(bytes.NewBuffer(bodyData)) } @@ -63,7 +107,7 @@ func (c *Client) call(ctx context.Context, method string, rel *url.URL, body, ou } req = req.WithContext(ctx) - res, err := http.DefaultClient.Do(req) + res, err := c.httpClient.Do(req) if err != nil { return err } @@ -82,5 +126,8 @@ func (c *Client) call(ctx context.Context, method string, rel *url.URL, body, ou return errs.Errors[0] } - return jsonapi.Unmarshal(data, out) + if out != nil { + return jsonapi.Unmarshal(data, out) + } + return nil } diff --git a/api/v1/client/metrics.go b/api/v1/client/metrics.go index 8d3753ef520..8666d43ffb5 100644 --- a/api/v1/client/metrics.go +++ b/api/v1/client/metrics.go @@ -24,11 +24,10 @@ import ( "context" "net/url" - "github.com/loadimpact/k6/api/v1" + v1 "github.com/loadimpact/k6/api/v1" ) -var MetricsURL = &url.URL{Path: "/v1/metrics"} - +// Metrics returns the current metrics summary. func (c *Client) Metrics(ctx context.Context) (ret []v1.Metric, err error) { - return ret, c.call(ctx, "GET", MetricsURL, nil, &ret) + return ret, c.Call(ctx, "GET", &url.URL{Path: "/v1/metrics"}, nil, &ret) } diff --git a/api/v1/client/status.go b/api/v1/client/status.go index 70cd23f35f6..a49aabd0c61 100644 --- a/api/v1/client/status.go +++ b/api/v1/client/status.go @@ -24,15 +24,16 @@ import ( "context" "net/url" - "github.com/loadimpact/k6/api/v1" + v1 "github.com/loadimpact/k6/api/v1" ) -var StatusURL = &url.URL{Path: "/v1/status"} - +// Status returns the current k6 status. func (c *Client) Status(ctx context.Context) (ret v1.Status, err error) { - return ret, c.call(ctx, "GET", StatusURL, nil, &ret) + return ret, c.Call(ctx, "GET", &url.URL{Path: "/v1/status"}, nil, &ret) } +// SetStatus tries to change the current status and returns the new one if it +// was successful. func (c *Client) SetStatus(ctx context.Context, patch v1.Status) (ret v1.Status, err error) { - return ret, c.call(ctx, "PATCH", StatusURL, patch, &ret) + return ret, c.Call(ctx, "PATCH", &url.URL{Path: "/v1/status"}, patch, &ret) } From 4fa069c5cd3217643f665195748acc4fe45e168f Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 20 Mar 2020 15:10:56 +0200 Subject: [PATCH 124/350] Improve the REST API client flexibility --- api/v1/client/client.go | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/api/v1/client/client.go b/api/v1/client/client.go index e6c395e22f2..51c1998af7b 100644 --- a/api/v1/client/client.go +++ b/api/v1/client/client.go @@ -93,9 +93,17 @@ func (c *Client) Call(ctx context.Context, method string, rel *url.URL, body, ou var bodyReader io.ReadCloser if body != nil { - bodyData, errm := jsonapi.Marshal(body) - if errm != nil { - return errm + var bodyData []byte + switch val := body.(type) { + case []byte: + bodyData = val + case string: + bodyData = []byte(val) + default: + bodyData, err = jsonapi.Marshal(body) + if err != nil { + return err + } } bodyReader = ioutil.NopCloser(bytes.NewBuffer(bodyData)) } From 0b1482807cc9691e7d6f82cb5860108187f8de88 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Sun, 22 Mar 2020 21:04:14 +0200 Subject: [PATCH 125/350] Add enumer generation for ExecutionStatus --- lib/execution.go | 1 + lib/execution_status_gen.go | 58 +++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+) create mode 100644 lib/execution_status_gen.go diff --git a/lib/execution.go b/lib/execution.go index 45514cdfcc9..c8278188325 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -105,6 +105,7 @@ const MaxRetriesGetPlannedVU = 5 // ExecutionStatus is similar to RunStatus, but more fine grained and concerns // only local execution. +//go:generate enumer -type=ExecutionStatus -trimprefix ExecutionStatus -output execution_status_gen.go type ExecutionStatus uint32 // Possible execution status values diff --git a/lib/execution_status_gen.go b/lib/execution_status_gen.go new file mode 100644 index 00000000000..aafbcc6b303 --- /dev/null +++ b/lib/execution_status_gen.go @@ -0,0 +1,58 @@ +// Code generated by "enumer -type=ExecutionStatus -trimprefix ExecutionStatus -output execution_status_gen.go"; DO NOT EDIT. + +// +package lib + +import ( + "fmt" +) + +const _ExecutionStatusName = "CreatedInitVUsInitExecutorsInitDonePausedBeforeRunStartedSetupRunningTeardownEnded" + +var _ExecutionStatusIndex = [...]uint8{0, 7, 14, 27, 35, 50, 57, 62, 69, 77, 82} + +func (i ExecutionStatus) String() string { + if i >= ExecutionStatus(len(_ExecutionStatusIndex)-1) { + return fmt.Sprintf("ExecutionStatus(%d)", i) + } + return _ExecutionStatusName[_ExecutionStatusIndex[i]:_ExecutionStatusIndex[i+1]] +} + +var _ExecutionStatusValues = []ExecutionStatus{0, 1, 2, 3, 4, 5, 6, 7, 8, 9} + +var _ExecutionStatusNameToValueMap = map[string]ExecutionStatus{ + _ExecutionStatusName[0:7]: 0, + _ExecutionStatusName[7:14]: 1, + _ExecutionStatusName[14:27]: 2, + _ExecutionStatusName[27:35]: 3, + _ExecutionStatusName[35:50]: 4, + _ExecutionStatusName[50:57]: 5, + _ExecutionStatusName[57:62]: 6, + _ExecutionStatusName[62:69]: 7, + _ExecutionStatusName[69:77]: 8, + _ExecutionStatusName[77:82]: 9, +} + +// ExecutionStatusString retrieves an enum value from the enum constants string name. +// Throws an error if the param is not part of the enum. +func ExecutionStatusString(s string) (ExecutionStatus, error) { + if val, ok := _ExecutionStatusNameToValueMap[s]; ok { + return val, nil + } + return 0, fmt.Errorf("%s does not belong to ExecutionStatus values", s) +} + +// ExecutionStatusValues returns all values of the enum +func ExecutionStatusValues() []ExecutionStatus { + return _ExecutionStatusValues +} + +// IsAExecutionStatus returns "true" if the value is listed in the enum definition. "false" otherwise +func (i ExecutionStatus) IsAExecutionStatus() bool { + for _, v := range _ExecutionStatusValues { + if i == v { + return true + } + } + return false +} From 298cf6bfab04d64392db34872240aca0f96c3e78 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Sun, 22 Mar 2020 23:24:27 +0200 Subject: [PATCH 126/350] Improve setup() and teardown() errors --- js/runner.go | 6 +++--- lib/timeout_error.go | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/js/runner.go b/js/runner.go index aba2ca7d0df..d3374266f56 100644 --- a/js/runner.go +++ b/js/runner.go @@ -223,7 +223,7 @@ func (r *Runner) Setup(ctx context.Context, out chan<- stats.SampleContainer) er v, err := r.runPart(setupCtx, out, stageSetup, nil) if err != nil { - return errors.Wrap(err, stageSetup) + return err } // r.setupData = nil is special it means undefined from this moment forward if goja.IsUndefined(v) { @@ -284,7 +284,7 @@ func (r *Runner) SetOptions(opts lib.Options) error { r.RPSLimit = rate.NewLimiter(rate.Limit(rps.Int64), 1) } - //TODO: validate that all exec values are either nil or valid exported methods (or HTTP requests in the future) + // TODO: validate that all exec values are either nil or valid exported methods (or HTTP requests in the future) if opts.ConsoleOutput.Valid { c, err := newFileConsole(opts.ConsoleOutput.String) @@ -332,7 +332,7 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, if deadline, ok := ctx.Deadline(); ok && time.Now().After(deadline) { // we could have an error that is not errInterrupt in which case we should return it instead if err, ok := err.(*goja.InterruptedError); ok && v != nil && err.Value() != errInterrupt { - //TODO: silence this error? + // TODO: silence this error? return v, err } // otherwise we have timeouted diff --git a/lib/timeout_error.go b/lib/timeout_error.go index 682fe000bf9..13da407c4eb 100644 --- a/lib/timeout_error.go +++ b/lib/timeout_error.go @@ -27,7 +27,7 @@ func NewTimeoutError(place string, d time.Duration) TimeoutError { // String returns timeout error in human readable format. func (t TimeoutError) String() string { - return fmt.Sprintf("%s execution timed out after %.f seconds", t.place, t.d.Seconds()) + return fmt.Sprintf("%s() execution timed out after %.f seconds", t.place, t.d.Seconds()) } // Error implements error interface. From 94c73a95f4e536e244f3412d3207c78d4dcbf04e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 23 Mar 2020 17:43:27 +0100 Subject: [PATCH 127/350] Make TestVariableLoopingVUsRun more deterministic This doesn't get rid of the flakiness entirely, but merely shifts things around and uses safer timings to reduce the possibility of failures. In my tests the behavior is much more stable now, though there are still issues on Windows during quick CPU usage bursts, which I'm hopeful we won't run into on GH Actions. --- lib/executor/variable_looping_vus_test.go | 39 +++++++++++------------ 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 45242f0ed4a..3a347cc1684 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -23,7 +23,6 @@ package executor import ( "context" "fmt" - "sync" "sync/atomic" "testing" "time" @@ -64,35 +63,35 @@ func TestVariableLoopingVUsRun(t *testing.T) { var ctx, cancel, executor, _ = setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { - time.Sleep(200 * time.Millisecond) + // Sleeping for a weird duration somewhat offset from the + // executor ticks to hopefully keep race conditions out of + // our control from failing the test. + time.Sleep(300 * time.Millisecond) atomic.AddInt64(&iterCount, 1) return nil }), ) defer cancel() - var ( - wg sync.WaitGroup - result []int64 - ) + sampleTimes := []time.Duration{ + 500 * time.Millisecond, + 1000 * time.Millisecond, + 700 * time.Millisecond, + } - wg.Add(1) - go func() { - defer wg.Done() - time.Sleep(100 * time.Millisecond) - result = append(result, es.GetCurrentlyActiveVUsCount()) - time.Sleep(1 * time.Second) - result = append(result, es.GetCurrentlyActiveVUsCount()) - time.Sleep(1 * time.Second) - result = append(result, es.GetCurrentlyActiveVUsCount()) - }() + errCh := make(chan error) + go func() { errCh <- executor.Run(ctx, nil) }() - err := executor.Run(ctx, nil) + var result = make([]int64, len(sampleTimes)) + for i, d := range sampleTimes { + time.Sleep(d) + result[i] = es.GetCurrentlyActiveVUsCount() + } + + require.NoError(t, <-errCh) - wg.Wait() - require.NoError(t, err) assert.Equal(t, []int64{5, 3, 0}, result) - assert.Equal(t, int64(40), iterCount) + assert.Equal(t, int64(29), iterCount) } // Ensure there's no wobble of VUs during graceful ramp-down, without segments. From 277e31b850b0ae6f9f090a638a344ba7694ba0d2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 24 Mar 2020 16:56:20 +0100 Subject: [PATCH 128/350] Make TestVariableLoopingVUsRampDownNoWobble more deterministic This doesn't get rid of the flakiness entirely, but merely shifts things around and uses safer timings to reduce the possibility of failures. In my tests the behavior is much more stable now, though there are still issues on Windows during quick CPU usage bursts, which I'm hopeful we won't run into on GH Actions. --- lib/executor/variable_looping_vus_test.go | 50 ++++++++++------------- 1 file changed, 21 insertions(+), 29 deletions(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 3a347cc1684..ba6f35e0354 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -125,48 +125,40 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { ) defer cancel() - var ( - wg sync.WaitGroup - result []int64 - m sync.Mutex - ) + sampleTimes := []time.Duration{ + 100 * time.Millisecond, + 3400 * time.Millisecond, + } + const rampDownSamples = 50 - sampleActiveVUs := func(delay time.Duration) { - time.Sleep(delay) - m.Lock() - result = append(result, es.GetCurrentlyActiveVUsCount()) - m.Unlock() + errCh := make(chan error) + go func() { errCh <- executor.Run(ctx, nil) }() + + var result = make([]int64, len(sampleTimes)+rampDownSamples) + for i, d := range sampleTimes { + time.Sleep(d) + result[i] = es.GetCurrentlyActiveVUsCount() } - wg.Add(1) - go func() { - defer wg.Done() - sampleActiveVUs(100 * time.Millisecond) - sampleActiveVUs(3 * time.Second) - time.AfterFunc(2*time.Second, func() { - sampleActiveVUs(0) - }) - time.Sleep(1 * time.Second) - // Sample ramp-down at a higher frequency - for i := 0; i < 15; i++ { - sampleActiveVUs(100 * time.Millisecond) - } - }() + // Sample ramp-down at a higher rate + for i := len(sampleTimes); i < rampDownSamples; i++ { + time.Sleep(50 * time.Millisecond) + result[i] = es.GetCurrentlyActiveVUsCount() + } - err := executor.Run(ctx, nil) + require.NoError(t, <-errCh) - wg.Wait() - require.NoError(t, err) + // Some baseline checks assert.Equal(t, int64(0), result[0]) assert.Equal(t, int64(10), result[1]) assert.Equal(t, int64(0), result[len(result)-1]) var curr int64 last := result[2] - // Check all ramp-down samples + // Check all ramp-down samples for wobble for i := 3; i < len(result[2:]); i++ { curr = result[i] - // Detect ramp-ups, missteps (e.g. 7 -> 4), but ignore pauses + // Detect ramp-ups, missteps (e.g. 7 -> 4), but ignore pauses (repeats) if curr > last || (curr != last && curr != last-1) { assert.FailNow(t, fmt.Sprintf("ramping down wobble bug - "+ From 8841d3a141b4b5113bfb23150fc5924222a96cf0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 25 Mar 2020 13:32:17 +0100 Subject: [PATCH 129/350] Simplify TestVariableLoopingVUsRampDownNoWobble This does the same functional check as before in a simpler way, while also ensuring all values were covered, whereas in the previous version a sequence of repeating values that doesn't decrement would erroneously pass the test. --- lib/executor/variable_looping_vus_test.go | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index ba6f35e0354..fa565fc8c9e 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -22,7 +22,6 @@ package executor import ( "context" - "fmt" "sync/atomic" "testing" "time" @@ -153,17 +152,12 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { assert.Equal(t, int64(10), result[1]) assert.Equal(t, int64(0), result[len(result)-1]) - var curr int64 - last := result[2] - // Check all ramp-down samples for wobble + vuChanges := []int64{result[2]} + // Check ramp-down consistency for i := 3; i < len(result[2:]); i++ { - curr = result[i] - // Detect ramp-ups, missteps (e.g. 7 -> 4), but ignore pauses (repeats) - if curr > last || (curr != last && curr != last-1) { - assert.FailNow(t, - fmt.Sprintf("ramping down wobble bug - "+ - "current: %d, previous: %d\nVU samples: %v", curr, last, result)) + if result[i] != result[i-1] { + vuChanges = append(vuChanges, result[i]) } - last = curr } + assert.Equal(t, []int64{10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}, vuChanges) } From 355ac254b9484c3f3166efb442505114e0b6574d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 26 Mar 2020 12:46:28 +0100 Subject: [PATCH 130/350] Add test for ExecutionSegment.Scale consistency --- lib/execution_segment_test.go | 69 ++++++++++++++++++++++++++++++++++- 1 file changed, 68 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 386a3f63062..994f398552c 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -21,8 +21,13 @@ package lib import ( + "fmt" "math/big" + "math/rand" + "os" + "sort" "testing" + "time" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -379,4 +384,66 @@ func TestExecutionSegmentStringSequences(t *testing.T) { } } -// TODO: test with randomized things +// Return a randomly distributed sequence of numSegments amount of +// execution segments whose length totals 1. +func genRandomExecutionSegmentSequence(numSegments int) (ExecutionSegmentSequence, error) { + const denom int = 1000 + + bounds := make(map[int]struct{}, numSegments-1) + for i := 0; i < numSegments-1; i++ { + b := rand.Intn(denom-1) + 1 + // Avoid duplicates + if _, ok := bounds[b]; ok { + numSegments++ + continue + } + bounds[b] = struct{}{} + } + + nums := make([]int, 0, len(bounds)+2) + for k := range bounds { + nums = append(nums, k) + } + nums = append(nums, []int{0, denom}...) + + sort.Ints(nums) + + segments := make([]*ExecutionSegment, 0, len(bounds)+1) + denom64 := int64(denom) + for i := 0; i < len(nums)-1; i++ { + from, to := big.NewRat(int64(nums[i]), denom64), big.NewRat(int64(nums[i+1]), denom64) + segment, err := NewExecutionSegment(from, to) + if err != nil { + return nil, err + } + segments = append(segments, segment) + } + + return NewExecutionSegmentSequence(segments...) +} + +// Ensure that the sum of scaling all execution segments in +// the same sequence with scaling factor M results in M itself. +func TestExecutionSegmentScaleConsistency(t *testing.T) { + t.Parallel() + + const numTests = 10 + for i := 0; i < numTests; i++ { + scale := rand.Int31n(99) + 2 + seq, err := genRandomExecutionSegmentSequence(rand.Intn(9) + 2) + require.NoError(t, err) + + t.Run(fmt.Sprintf("%d_%s", scale, seq), func(t *testing.T) { + var total int64 + for _, segment := range seq { + total += segment.Scale(int64(scale)) + } + assert.Equal(t, int64(scale), total) + }) + } +} + +func TestMain(m *testing.M) { + rand.Seed(time.Now().UnixNano()) + os.Exit(m.Run()) +} From 0a2bc9c2dd65c754e5f6b2fc0ebb3f78c2e37f2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 26 Mar 2020 17:14:43 +0100 Subject: [PATCH 131/350] Replace random ExecutionSegmentSequence generation, log random seed This ports @MStoykov's simpler implementation from https://github.com/loadimpact/k6/blob/9fa61d917c9f3bdfbb744ff6a40db2f851522098/lib/execution_segment_test.go#L506 --- lib/execution_segment_test.go | 56 +++++++++++++---------------------- 1 file changed, 21 insertions(+), 35 deletions(-) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 994f398552c..5b73648867a 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -24,8 +24,6 @@ import ( "fmt" "math/big" "math/rand" - "os" - "sort" "testing" "time" @@ -384,42 +382,31 @@ func TestExecutionSegmentStringSequences(t *testing.T) { } } -// Return a randomly distributed sequence of numSegments amount of +// Return a randomly distributed sequence of n amount of // execution segments whose length totals 1. -func genRandomExecutionSegmentSequence(numSegments int) (ExecutionSegmentSequence, error) { - const denom int = 1000 - - bounds := make(map[int]struct{}, numSegments-1) - for i := 0; i < numSegments-1; i++ { - b := rand.Intn(denom-1) + 1 - // Avoid duplicates - if _, ok := bounds[b]; ok { - numSegments++ - continue +func generateRandomSequence(n int64, r *rand.Rand) (ExecutionSegmentSequence, error) { + var err error + var ess = ExecutionSegmentSequence(make([]*ExecutionSegment, n)) + var numerators = make([]int64, n) + var denominator int64 + for i := int64(0); i < n; i++ { + for numerators[i] == 0 { + numerators[i] = r.Int63n(n) + denominator += numerators[i] } - bounds[b] = struct{}{} } - - nums := make([]int, 0, len(bounds)+2) - for k := range bounds { - nums = append(nums, k) + ess[0], err = NewExecutionSegment(big.NewRat(0, 1), big.NewRat(numerators[0], denominator)) + if err != nil { + return nil, err } - nums = append(nums, []int{0, denom}...) - - sort.Ints(nums) - - segments := make([]*ExecutionSegment, 0, len(bounds)+1) - denom64 := int64(denom) - for i := 0; i < len(nums)-1; i++ { - from, to := big.NewRat(int64(nums[i]), denom64), big.NewRat(int64(nums[i+1]), denom64) - segment, err := NewExecutionSegment(from, to) + for i := int64(1); i < n; i++ { + ess[i], err = NewExecutionSegment(ess[i-1].to, new(big.Rat).Add(big.NewRat(numerators[i], denominator), ess[i-1].to)) if err != nil { return nil, err } - segments = append(segments, segment) } - return NewExecutionSegmentSequence(segments...) + return ess, nil } // Ensure that the sum of scaling all execution segments in @@ -427,10 +414,14 @@ func genRandomExecutionSegmentSequence(numSegments int) (ExecutionSegmentSequenc func TestExecutionSegmentScaleConsistency(t *testing.T) { t.Parallel() + seed := time.Now().UnixNano() + r := rand.New(rand.NewSource(seed)) + t.Logf("Random source seeded with %d\n", seed) + const numTests = 10 for i := 0; i < numTests; i++ { scale := rand.Int31n(99) + 2 - seq, err := genRandomExecutionSegmentSequence(rand.Intn(9) + 2) + seq, err := generateRandomSequence(r.Int63n(9)+2, r) require.NoError(t, err) t.Run(fmt.Sprintf("%d_%s", scale, seq), func(t *testing.T) { @@ -442,8 +433,3 @@ func TestExecutionSegmentScaleConsistency(t *testing.T) { }) } } - -func TestMain(m *testing.M) { - rand.Seed(time.Now().UnixNano()) - os.Exit(m.Run()) -} From d2af11cff80b59dbcca3f7cad566e3a74be5e8e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 17 Jan 2020 16:37:46 +0100 Subject: [PATCH 132/350] Add responsive progress bars Progress bars will now dynamically resize depending on terminal window width, being replaced with percentages if "squished" too far. It works best on *nix platforms because of SIGWINCH, while on Windows retrieving the size is required before each render iteration, and isn't as responsive. Terminal multiplexers are a bigger issue: tmux seems to buffer SIGWINCH/terminfo and is plain broken on versions including the current latest stable (3.0a) (see issue #2005[1]). This was fixed on `master`, but is yet to be released, and even with the fix the experience is far from ideal (lags quite a bit, no continuous resize). GNU screen also has some rendering issues I haven't looked too much into yet. Untested on macOS terminals, but I expect it to work as well as on Linux. Terminals tested: - Linux: st, urxvt, xterm - Windows: cmd.exe and PowerShell standalone, and both inside the new Windows Terminal[2] app. Resizing works fine in all cases, though isn't as responsive as on Linux. Part of #1279 [1]: https://github.com/tmux/tmux/issues/2005 [2]: https://github.com/microsoft/terminal --- cmd/run.go | 2 +- cmd/ui.go | 106 ++++++++++++++++++++----------- cmd/ui_test.go | 27 +++++--- cmd/ui_unix.go | 43 +++++++++++++ cmd/ui_windows.go | 41 ++++++++++++ ui/pb/progressbar.go | 128 +++++++++++++++++++++++++------------- ui/pb/progressbar_test.go | 39 +++++------- 7 files changed, 274 insertions(+), 112 deletions(-) create mode 100644 cmd/ui_unix.go create mode 100644 cmd/ui_windows.go diff --git a/cmd/run.go b/cmd/run.go index 29a9a345051..6b2c1718643 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -161,7 +161,7 @@ a commandline interface for interacting with it.`, progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) go func() { - showProgress(ctx, conf, execScheduler) + showProgress(ctx, conf, execScheduler, logger) progressBarWG.Done() }() diff --git a/cmd/ui.go b/cmd/ui.go index 3cf4caab685..6c5820cd474 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -25,9 +25,15 @@ import ( "context" "fmt" "io" + "os" "strings" "sync" "time" + "unicode/utf8" + + "golang.org/x/crypto/ssh/terminal" + + "github.com/sirupsen/logrus" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" @@ -78,12 +84,14 @@ func printBar(bar *pb.ProgressBar, rightText string) { // TODO: check for cross platform support end = "\x1b[0K\r" } - rendered := bar.Render(0) + rendered := bar.Render(0, 0) // Only output the left and middle part of the progress bar - fprintf(stdout, "%s %s %s%s", rendered.Left, rendered.Progress, rightText, end) + fprintf(stdout, "%s %s %s%s", rendered.Left, rendered.Progress(), rightText, end) } -func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) string { +func renderMultipleBars( + isTTY, goBack bool, maxLeft, widthDelta int, pbs []*pb.ProgressBar, +) (string, int) { lineEnd := "\n" if isTTY { //TODO: check for cross platform support @@ -91,6 +99,7 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) } var ( + longestLine int // Maximum length of each right side column except last, // used to calculate the padding between columns. maxRColumnLen = make([]int, 2) @@ -104,10 +113,9 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) // First pass to render all progressbars and get the maximum // lengths of right-side columns. for i, pb := range pbs { - rend := pb.Render(leftMax) + rend := pb.Render(maxLeft, widthDelta) for i := range rend.Right { - // Don't calculate for last column, since there's nothing to align - // after it (yet?). + // Skip last column, since there's nothing to align after it (yet?). if i == len(rend.Right)-1 { break } @@ -126,8 +134,8 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) continue } var leftText, rightText string - leftPadFmt := fmt.Sprintf("%%-%ds %%s ", leftMax) - leftText = fmt.Sprintf(leftPadFmt, rend.Left, rend.Status) + leftPadFmt := fmt.Sprintf("%%-%ds", maxLeft) + leftText = fmt.Sprintf(leftPadFmt, rend.Left) for i := range rend.Right { rpad := 0 if len(maxRColumnLen) > i { @@ -136,7 +144,20 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) rightPadFmt := fmt.Sprintf(" %%-%ds", rpad+1) rightText += fmt.Sprintf(rightPadFmt, rend.Right[i]) } - result[i+1] = leftText + rend.Progress + rightText + lineEnd + // Get visible line length, without ANSI escape sequences (color) + status := fmt.Sprintf(" %s ", rend.Status()) + line := leftText + status + rend.Progress() + rightText + lineRuneLen := utf8.RuneCountInString(line) + if lineRuneLen > longestLine { + longestLine = lineRuneLen + } + if !noColor { + rend.Color = true + status = fmt.Sprintf(" %s ", rend.Status()) + line = fmt.Sprintf(leftPadFmt+"%s%s%s", + rend.Left, status, rend.Progress(), rightText) + } + result[i+1] = line + lineEnd } if isTTY && goBack { @@ -146,13 +167,18 @@ func renderMultipleBars(isTTY, goBack bool, leftMax int, pbs []*pb.ProgressBar) } else { result[pbsCount+1] = "\n" } - return strings.Join(result, "") + + return strings.Join(result, ""), longestLine } //TODO: show other information here? //TODO: add a no-progress option that will disable these //TODO: don't use global variables... -func showProgress(ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler) { +// nolint:funlen +func showProgress( + ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler, + logger *logrus.Logger, +) { if quiet || conf.HTTPDebug.Valid && conf.HTTPDebug.String != "" { return } @@ -162,6 +188,12 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi pbs = append(pbs, s.GetProgress()) } + termWidth, _, err := terminal.GetSize(int(os.Stdout.Fd())) + if err != nil { + logger.WithError(err).Warn("error getting terminal size") + termWidth = 80 // TODO: something safer, return error? + } + // Get the longest left side string length, to align progress bars // horizontally and trim excess text. var leftLen int64 @@ -169,13 +201,20 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi l := pb.Left() leftLen = lib.Max(int64(len(l)), leftLen) } - // Limit to maximum left text length - leftMax := int(lib.Min(leftLen, maxLeftLength)) + maxLeft := int(lib.Min(leftLen, maxLeftLength)) + + var widthDelta int + var progressBarsLastRender []byte + // default responsive render + renderProgressBars := func(goBack bool) { + barText, longestLine := renderMultipleBars(stdoutTTY, goBack, maxLeft, widthDelta, pbs) + // -1 to allow some "breathing room" near the edge + widthDelta = termWidth - longestLine - 1 + progressBarsLastRender = []byte(barText) + } - // For flicker-free progressbars! - progressBarsLastRender := []byte(renderMultipleBars(stdoutTTY, true, leftMax, pbs)) - progressBarsPrint := func() { + printProgressBars := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } @@ -185,35 +224,28 @@ func showProgress(ctx context.Context, conf Config, execScheduler *local.Executi //description in the TODO message in cmd/root.go) if stdoutTTY && !noColor { updateFreq = 100 * time.Millisecond - outMutex.Lock() - stdout.PersistentText = progressBarsPrint - stderr.PersistentText = progressBarsPrint - outMutex.Unlock() - defer func() { - outMutex.Lock() - stdout.PersistentText = nil - stderr.PersistentText = nil - if ctx.Err() != nil { - // Render a last plain-text progressbar in an error - progressBarsLastRender = []byte(renderMultipleBars(stdoutTTY, false, leftMax, pbs)) - progressBarsPrint() - } - outMutex.Unlock() - }() } ctxDone := ctx.Done() ticker := time.NewTicker(updateFreq) + sigwinch := NotifyWindowResize() + fd := int(os.Stdout.Fd()) for { select { - case <-ticker.C: - barText := renderMultipleBars(stdoutTTY, true, leftMax, pbs) - outMutex.Lock() - progressBarsLastRender = []byte(barText) - progressBarsPrint() - outMutex.Unlock() case <-ctxDone: + renderProgressBars(false) + printProgressBars() return + case <-ticker.C: + // Optional "polling" method, platform dependent. + termWidth, _, _ = GetTermSize(fd, termWidth) + case <-sigwinch: + // More efficient SIGWINCH method on *nix. + termWidth, _, _ = terminal.GetSize(fd) } + renderProgressBars(true) + outMutex.Lock() + printProgressBars() + outMutex.Unlock() } } diff --git a/cmd/ui_test.go b/cmd/ui_test.go index 95c7df02ade..284eee36367 100644 --- a/cmd/ui_test.go +++ b/cmd/ui_test.go @@ -41,30 +41,39 @@ func TestRenderMultipleBars(t *testing.T) { t.Parallel() testCases := []struct { - name string - padding int - expected string + name string + padding int + widthDelta int + expOut string + expLongLine int }{ - {"nopad", 0, ` + {"pad0", 0, 0, ` left 0 [--------------------------------------] right 0 000 left 1 ✓ [======================================] right 1 000 left 2 [--------------------------------------] right 2 000 -`}, - {"pad2", 2, ` +`, 62}, + {"pad2", 2, 0, ` left 0 [--------------------------------------] right 0 000 left 1++ ✓ [======================================] right 1++ 000 left 2 [--------------------------------------] right 2 000 -`}, +`, 66}, + {"pad0compact", 0, -50, ` +left 0 [ 0% ] right 0 000 +left 1 ✓ [ 100% ] right 1 000 +left 2 [ 0% ] right 2 000 + +`, 30}, } for _, tc := range testCases { tc := tc t.Run(tc.name, func(t *testing.T) { pbs := createTestProgressBars(3, tc.padding, 1) - out := renderMultipleBars(false, false, 6+tc.padding, pbs) - assert.Equal(t, tc.expected, out) + out, longestLine := renderMultipleBars(false, false, 6+tc.padding, tc.widthDelta, pbs) + assert.Equal(t, tc.expOut, out) + assert.Equal(t, tc.expLongLine, longestLine) }) } } diff --git a/cmd/ui_unix.go b/cmd/ui_unix.go new file mode 100644 index 00000000000..6c158e7ed27 --- /dev/null +++ b/cmd/ui_unix.go @@ -0,0 +1,43 @@ +// +build darwin dragonfly freebsd linux netbsd openbsd + +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2020 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package cmd + +import ( + "os" + "os/signal" + "syscall" +) + +// GetTermSize returns the current terminal window size on Windows, +// but is a no-op on all other platforms. +func GetTermSize(fd, termWidth int) (width, height int, err error) { + return termWidth, 0, nil +} + +// NotifyWindowResize listens for SIGWINCH (terminal window size changes) +// on *nix platforms, and is a no-op on Windows. +func NotifyWindowResize() <-chan os.Signal { + sch := make(chan os.Signal, 1) + signal.Notify(sch, os.Signal(syscall.SIGWINCH)) + return sch +} diff --git a/cmd/ui_windows.go b/cmd/ui_windows.go new file mode 100644 index 00000000000..b5d982c4342 --- /dev/null +++ b/cmd/ui_windows.go @@ -0,0 +1,41 @@ +// +build windows + +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2020 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package cmd + +import ( + "os" + + "golang.org/x/crypto/ssh/terminal" +) + +// GetTermSize returns the current terminal window size on Windows, +// but is a no-op on all other platforms. +func GetTermSize(fd, termWidth int) (width, height int, err error) { + return terminal.GetSize(fd) +} + +// NotifyWindowResize listens for SIGWINCH (terminal window size changes) +// on *nix platforms, and is a no-op on Windows. +func NotifyWindowResize() <-chan os.Signal { + return make(<-chan os.Signal, 1) +} diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 948d7875d12..68a4aa85eee 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -29,8 +29,22 @@ import ( "github.com/sirupsen/logrus" ) -const defaultWidth = 40 -const defaultBarColor = color.Faint +//nolint:gochecknoglobals +var ( + colorFaint = color.New(color.Faint) + statusColors = map[Status]*color.Color{ + Interrupted: color.New(color.FgRed), + Done: color.New(color.FgGreen), + Waiting: colorFaint, + } +) + +const ( + defaultWidth = 40 + // threshold below which progress should be rendered as + // percentages instead of filling bars + minWidth = 8 +) // Status of the progress bar type Status rune @@ -44,19 +58,11 @@ const ( Done Status = '✓' ) -//nolint:gochecknoglobals -var statusColors = map[Status]*color.Color{ - Interrupted: color.New(color.FgRed), - Done: color.New(color.FgGreen), - Waiting: color.New(defaultBarColor), -} - // ProgressBar is a simple thread-safe progressbar implementation with // callbacks. type ProgressBar struct { mutex sync.RWMutex width int - color *color.Color logger *logrus.Entry status Status @@ -114,7 +120,6 @@ func New(options ...ProgressBarOption) *ProgressBar { pb := &ProgressBar{ mutex: sync.RWMutex{}, width: defaultWidth, - color: color.New(defaultBarColor), } pb.Modify(options...) return pb @@ -152,10 +157,44 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { } // ProgressBarRender stores the different rendered parts of the -// progress bar UI. +// progress bar UI to allow dynamic positioning and padding of +// elements in the terminal output (e.g. for responsive progress +// bars). type ProgressBarRender struct { - Left, Status, Progress, Hijack string - Right []string + Color bool + progress, progressFill, progressPadding string + Left, Hijack string + status Status + Right []string +} + +// Status returns an optionally colorized status string +func (pbr *ProgressBarRender) Status() string { + status := " " + + if pbr.status > 0 { + status = string(pbr.status) + if c, ok := statusColors[pbr.status]; pbr.Color && ok { + status = c.Sprint(status) + } + } + + return status +} + +// Progress returns an assembled and optionally colorized progress string +func (pbr *ProgressBarRender) Progress() string { + var body string + if pbr.progress != "" { + body = fmt.Sprintf(" %s ", pbr.progress) + } else { + padding := pbr.progressPadding + if pbr.Color { + padding = colorFaint.Sprint(pbr.progressPadding) + } + body = pbr.progressFill + padding + } + return fmt.Sprintf("[%s]", body) } func (pbr ProgressBarRender) String() string { @@ -166,17 +205,21 @@ func (pbr ProgressBarRender) String() string { if len(pbr.Right) > 0 { right = " " + strings.Join(pbr.Right, " ") } - return pbr.Left + " " + pbr.Status + " " + pbr.Progress + right + return pbr.Left + " " + pbr.Status() + " " + pbr.Progress() + right } // Render locks the progressbar struct for reading and calls all of // its methods to return the final output. A struct is returned over a // plain string to allow dynamic padding and positioning of elements // depending on other elements on the screen. -// - leftMax defines the maximum character length of the left-side +// - maxLeft defines the maximum character length of the left-side // text. Characters exceeding this length will be replaced with a // single ellipsis. Passing <=0 disables this. -func (pb *ProgressBar) Render(leftMax int) ProgressBarRender { +// - widthDelta changes the progress bar width the specified amount of +// characters. E.g. passing -2 would shorten the width by 2 chars. +// If the resulting width is lower than minWidth, progress will be +// rendered as a percentage instead of a filling bar. +func (pb *ProgressBar) Render(maxLeft, widthDelta int) ProgressBarRender { pb.mutex.RLock() defer pb.mutex.RUnlock() @@ -198,37 +241,36 @@ func (pb *ProgressBar) Render(leftMax int) ProgressBarRender { } } - space := pb.width - 2 - filled := int(float64(space) * progress) - - filling := "" - caret := "" - if filled > 0 { - if filled < space { - filling = strings.Repeat("=", filled-1) - caret = ">" - } else { - filling = strings.Repeat("=", filled) + width := Clampf(float64(pb.width+widthDelta), minWidth, defaultWidth) + pb.width = int(width) + + if pb.width > minWidth { + space := pb.width - 2 + filled := int(float64(space) * progress) + + filling := "" + caret := "" + if filled > 0 { + if filled < space { + filling = strings.Repeat("=", filled-1) + caret = ">" + } else { + filling = strings.Repeat("=", filled) + } } - } - padding := "" - if space > filled { - padding = pb.color.Sprint(strings.Repeat("-", space-filled)) - } - - out.Left = pb.renderLeft(leftMax) + out.progressPadding = "" + if space > filled { + out.progressPadding = strings.Repeat("-", space-filled) + } - switch c, ok := statusColors[pb.status]; { - case ok: - out.Status = c.Sprint(string(pb.status)) - case pb.status > 0: - out.Status = string(pb.status) - default: - out.Status = " " + out.progressFill = filling + caret + } else { + out.progress = fmt.Sprintf("%3.f%%", progress*100) } - out.Progress = fmt.Sprintf("[%s%s%s]", filling, caret, padding) + out.Left = pb.renderLeft(maxLeft) + out.status = pb.status return out } diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index a27a230c37b..c48830b3cfc 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -40,52 +40,47 @@ func TestProgressBarRender(t *testing.T) { t.Parallel() testCases := []struct { - options []ProgressBarOption - expected string + options []ProgressBarOption + pbWidthDelta int + expected string }{ {[]ProgressBarOption{WithLeft(func() string { return "left" })}, - "left [--------------------------------------]"}, + 0, "left [--------------------------------------]"}, {[]ProgressBarOption{WithConstLeft("constLeft")}, - "constLeft [--------------------------------------]"}, + 0, "constLeft [--------------------------------------]"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithStatus(Done), - }, - "left ✓ [--------------------------------------]"}, + }, 0, "left ✓ [--------------------------------------]"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, []string) { return 0, []string{"right"} }), - }, - "left [--------------------------------------] right"}, + }, 0, "left [--------------------------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, []string) { return 0.5, []string{"right"} }), - }, - "left [==================>-------------------] right"}, + }, 0, "left [==================>-------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, []string) { return 1.0, []string{"right"} }), - }, - "left [======================================] right"}, + }, 0, "left [======================================] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, []string) { return -1, []string{"right"} }), - }, - "left [--------------------------------------] right"}, + }, 0, "left [--------------------------------------] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithProgress(func() (float64, []string) { return 2, []string{"right"} }), - }, - "left [======================================] right"}, + }, 0, "left [======================================] right"}, {[]ProgressBarOption{ WithLeft(func() string { return "left" }), WithConstProgress(0.2, "constProgress"), - }, - "left [======>-------------------------------] constProgress"}, + }, 0, "left [======>-------------------------------] constProgress"}, {[]ProgressBarOption{ WithHijack(func() string { return "progressbar hijack!" }), - }, - "progressbar hijack!"}, + }, 0, "progressbar hijack!"}, + {[]ProgressBarOption{WithConstProgress(0.25, "")}, + -defaultWidth, " [ 25% ] "}, } for _, tc := range testCases { @@ -93,7 +88,7 @@ func TestProgressBarRender(t *testing.T) { t.Run(tc.expected, func(t *testing.T) { pbar := New(tc.options...) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(0).String()) + assert.Equal(t, tc.expected, pbar.Render(0, tc.pbWidthDelta).String()) }) } } @@ -117,7 +112,7 @@ func TestProgressBarRenderPaddingMaxLeft(t *testing.T) { t.Run(tc.left, func(t *testing.T) { pbar := New(WithLeft(func() string { return tc.left })) assert.NotNil(t, pbar) - assert.Equal(t, tc.expected, pbar.Render(tc.maxLen).String()) + assert.Equal(t, tc.expected, pbar.Render(tc.maxLen, 0).String()) }) } } From b1dfe9c4ac28601b0d786fa5a1eefec7cee19e29 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 23 Jan 2020 13:34:27 +0100 Subject: [PATCH 133/350] Make UI mode configurable via CLI/env --- cmd/config.go | 5 +++ cmd/config_consolidation_test.go | 4 +++ cmd/options.go | 14 +++++++-- cmd/runtime_options.go | 3 +- cmd/ui.go | 36 +++++++++++++++++++--- cmd/ui_mode_gen.go | 52 ++++++++++++++++++++++++++++++++ lib/options.go | 13 ++++++-- ui/pb/progressbar.go | 7 +++-- ui/pb/progressbar_test.go | 2 +- 9 files changed, 121 insertions(+), 15 deletions(-) create mode 100644 cmd/ui_mode_gen.go diff --git a/cmd/config.go b/cmd/config.go index 9c06bf2824a..3abe3da5873 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -87,6 +87,11 @@ func (c Config) Validate() []error { errors := c.Options.Validate() //TODO: validate all of the other options... that we should have already been validating... //TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation + + if _, err := UIModeString(c.Options.UIMode.String); err != nil { + errors = append(errors, err) + } + return errors } diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 70d3788e021..f0e441f8344 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -377,6 +377,10 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, // TODO: test for differences between flagsets // TODO: more tests in general, especially ones not related to execution parameters... + {opts{cli: []string{"--ui-mode", "compact"}}, exp{}, func(t *testing.T, c Config) { + assert.Equal(t, UIModeCompact.String(), c.UIMode.String) + }}, + {opts{cli: []string{"--ui-mode", "blah"}}, exp{validationErrors: true}, nil}, } } diff --git a/cmd/options.go b/cmd/options.go index d82c525fed0..e52a1d5963b 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -25,14 +25,15 @@ import ( "strings" "time" + "github.com/pkg/errors" + "github.com/spf13/pflag" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui" - "github.com/pkg/errors" - "github.com/spf13/pflag" - null "gopkg.in/guregu/null.v3" ) var ( @@ -89,6 +90,12 @@ func optionFlagSet() *pflag.FlagSet { flags.StringSlice("tag", nil, "add a `tag` to be applied to all samples, as `[name]=[value]`") flags.String("console-output", "", "redirects the console logging to the provided output file") flags.Bool("discard-response-bodies", false, "Read but don't process or save HTTP response bodies") + flags.String("ui-mode", UIModeResponsive.String(), + `Change the way the UI is rendered. Options: + compact: show compact fixed-length progress bars with percentages + responsive: resize progress bars to fit terminal window + full: show full fixed-length progress bars + `) return flags } @@ -118,6 +125,7 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { TeardownTimeout: types.NullDuration{Duration: types.Duration(60 * time.Second), Valid: false}, MetricSamplesBufferSize: null.NewInt(1000, false), + UIMode: getNullString(flags, "ui-mode"), } // Using Changed() because GetStringSlice() doesn't differentiate between empty and no value diff --git a/cmd/runtime_options.go b/cmd/runtime_options.go index ec36b37d366..edc1680a1a2 100644 --- a/cmd/runtime_options.go +++ b/cmd/runtime_options.go @@ -25,10 +25,11 @@ import ( "strconv" "strings" - "github.com/loadimpact/k6/lib" "github.com/pkg/errors" "github.com/spf13/pflag" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" ) // TODO: move this whole file out of the cmd package? maybe when fixing diff --git a/cmd/ui.go b/cmd/ui.go index 6c5820cd474..b9d5fbcac4c 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -18,6 +18,8 @@ * */ +// go:generate enumer -type=UIMode -transform=snake -trimprefix=UIMode -output ui_mode_gen.go + package cmd import ( @@ -40,8 +42,23 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -// TODO: Make configurable -const maxLeftLength = 30 +// UIMode defines various rendering methods +type UIMode uint8 + +//nolint: golint +const ( + // max length of left-side progress bar text before trimming is forced + maxLeftLength = 30 + UIModeResponsive UIMode = iota + 1 + UIModeCompact + UIModeFull +) + +// Decode implements envconfig.Decoder +func (i *UIMode) Decode(value string) (err error) { + *i, err = UIModeString(value) + return +} // A writer that syncs writes with a mutex and, if the output is a TTY, clears before newlines. type consoleWriter struct { @@ -176,8 +193,8 @@ func renderMultipleBars( //TODO: don't use global variables... // nolint:funlen func showProgress( - ctx context.Context, conf Config, execScheduler *local.ExecutionScheduler, - logger *logrus.Logger, + ctx context.Context, conf Config, + execScheduler *local.ExecutionScheduler, logger *logrus.Logger, ) { if quiet || conf.HTTPDebug.Valid && conf.HTTPDebug.String != "" { return @@ -214,6 +231,17 @@ func showProgress( progressBarsLastRender = []byte(barText) } + if conf.UIMode.String == UIModeCompact.String() { + widthDelta = -pb.DefaultWidth + } + + if conf.UIMode.String != UIModeResponsive.String() { + renderProgressBars = func(goBack bool) { + barText, _ := renderMultipleBars(stdoutTTY, goBack, maxLeft, widthDelta, pbs) + progressBarsLastRender = []byte(barText) + } + } + printProgressBars := func() { _, _ = stdout.Writer.Write(progressBarsLastRender) } diff --git a/cmd/ui_mode_gen.go b/cmd/ui_mode_gen.go new file mode 100644 index 00000000000..856b01e493a --- /dev/null +++ b/cmd/ui_mode_gen.go @@ -0,0 +1,52 @@ +// Code generated by "enumer -type=UIMode -transform=snake -trimprefix=UIMode -output ui_mode_gen.go"; DO NOT EDIT. + +// +package cmd + +import ( + "fmt" +) + +const _UIModeName = "responsivecompactfull" + +var _UIModeIndex = [...]uint8{0, 10, 17, 21} + +func (i UIMode) String() string { + i -= 2 + if i >= UIMode(len(_UIModeIndex)-1) { + return fmt.Sprintf("UIMode(%d)", i+2) + } + return _UIModeName[_UIModeIndex[i]:_UIModeIndex[i+1]] +} + +var _UIModeValues = []UIMode{2, 3, 4} + +var _UIModeNameToValueMap = map[string]UIMode{ + _UIModeName[0:10]: 2, + _UIModeName[10:17]: 3, + _UIModeName[17:21]: 4, +} + +// UIModeString retrieves an enum value from the enum constants string name. +// Throws an error if the param is not part of the enum. +func UIModeString(s string) (UIMode, error) { + if val, ok := _UIModeNameToValueMap[s]; ok { + return val, nil + } + return 0, fmt.Errorf("%s does not belong to UIMode values", s) +} + +// UIModeValues returns all values of the enum +func UIModeValues() []UIMode { + return _UIModeValues +} + +// IsAUIMode returns "true" if the value is listed in the enum definition. "false" otherwise +func (i UIMode) IsAUIMode() bool { + for _, v := range _UIModeValues { + if i == v { + return true + } + } + return false +} diff --git a/lib/options.go b/lib/options.go index ba590922dc8..1690819d4a5 100644 --- a/lib/options.go +++ b/lib/options.go @@ -27,10 +27,11 @@ import ( "net" "reflect" - "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats" "github.com/pkg/errors" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" ) // DefaultExecutorName is used as the default key/ID of the executor config entries @@ -192,7 +193,7 @@ type Options struct { // Initial values for VUs, max VUs, duration cap, iteration cap, and stages. // See the Runner or Executor interfaces for more information. - VUs null.Int `json:"vus" envconfig:"K6_VUS"` + VUs null.Int `json:"vus" envconfig:"K6_VUS"` Duration types.NullDuration `json:"duration" envconfig:"K6_DURATION"` Iterations null.Int `json:"iterations" envconfig:"K6_ITERATIONS"` Stages []Stage `json:"stages" envconfig:"K6_STAGES"` @@ -290,6 +291,9 @@ type Options struct { // Redirect console logging to a file ConsoleOutput null.String `json:"-" envconfig:"K6_CONSOLE_OUTPUT"` + + // UI render mode: "compact", "full" or "responsive" + UIMode null.String `json:"uiMode" envconfig:"K6_UI_MODE"` } // Returns the result of overwriting any fields with any that are set on the argument. @@ -437,6 +441,9 @@ func (o Options) Apply(opts Options) Options { if opts.ConsoleOutput.Valid { o.ConsoleOutput = opts.ConsoleOutput } + if opts.UIMode.Valid { + o.UIMode = opts.UIMode + } return o } diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 68a4aa85eee..3878ca4ddc6 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -40,7 +40,8 @@ var ( ) const ( - defaultWidth = 40 + // DefaultWidth of the progress bar + DefaultWidth = 40 // threshold below which progress should be rendered as // percentages instead of filling bars minWidth = 8 @@ -119,7 +120,7 @@ func WithHijack(hijack func() string) ProgressBarOption { func New(options ...ProgressBarOption) *ProgressBar { pb := &ProgressBar{ mutex: sync.RWMutex{}, - width: defaultWidth, + width: DefaultWidth, } pb.Modify(options...) return pb @@ -241,7 +242,7 @@ func (pb *ProgressBar) Render(maxLeft, widthDelta int) ProgressBarRender { } } - width := Clampf(float64(pb.width+widthDelta), minWidth, defaultWidth) + width := Clampf(float64(pb.width+widthDelta), minWidth, DefaultWidth) pb.width = int(width) if pb.width > minWidth { diff --git a/ui/pb/progressbar_test.go b/ui/pb/progressbar_test.go index c48830b3cfc..b9b9900da80 100644 --- a/ui/pb/progressbar_test.go +++ b/ui/pb/progressbar_test.go @@ -80,7 +80,7 @@ func TestProgressBarRender(t *testing.T) { WithHijack(func() string { return "progressbar hijack!" }), }, 0, "progressbar hijack!"}, {[]ProgressBarOption{WithConstProgress(0.25, "")}, - -defaultWidth, " [ 25% ] "}, + -DefaultWidth, " [ 25% ] "}, } for _, tc := range testCases { From 242d9553d612aecf37bc901a0fb2b1dbbdf27f00 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 5 Mar 2020 11:41:11 +0100 Subject: [PATCH 134/350] Refactor terminal resize handling, simplify? See [1]. This also accidentally fixes [2], and the time.Sleep hack is not needed! [1]: https://github.com/loadimpact/k6/pull/1332#discussion_r383805414 [2]: https://github.com/loadimpact/k6/pull/1332#discussion_r380129011 --- cmd/ui.go | 37 ++++++++++++++++++++++++++++--------- cmd/ui_unix.go | 15 ++------------- cmd/ui_windows.go | 14 ++------------ 3 files changed, 32 insertions(+), 34 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index b9d5fbcac4c..bc23510339c 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -28,6 +28,7 @@ import ( "fmt" "io" "os" + "os/signal" "strings" "sync" "time" @@ -254,23 +255,41 @@ func showProgress( updateFreq = 100 * time.Millisecond } + var ( + fd = int(os.Stdout.Fd()) + ticker = time.NewTicker(updateFreq) + updateTermWidth func() + winchSignal = getWinchSignal() + ) + // On platforms that have SIGWINCH (*nix), progress bar resizing will be + // more responsive. On others (Windows), we fallback to resize on every tick. + if winchSignal != nil { + winch := make(chan os.Signal, 1) + signal.Notify(winch, winchSignal) + updateTermWidth = func() { + select { + case <-ticker.C: + case <-winch: + termWidth, _, _ = terminal.GetSize(fd) + } + } + } else { + updateTermWidth = func() { + <-ticker.C + termWidth, _, _ = terminal.GetSize(fd) + } + } + ctxDone := ctx.Done() - ticker := time.NewTicker(updateFreq) - sigwinch := NotifyWindowResize() - fd := int(os.Stdout.Fd()) for { select { case <-ctxDone: renderProgressBars(false) printProgressBars() return - case <-ticker.C: - // Optional "polling" method, platform dependent. - termWidth, _, _ = GetTermSize(fd, termWidth) - case <-sigwinch: - // More efficient SIGWINCH method on *nix. - termWidth, _, _ = terminal.GetSize(fd) + default: } + updateTermWidth() renderProgressBars(true) outMutex.Lock() printProgressBars() diff --git a/cmd/ui_unix.go b/cmd/ui_unix.go index 6c158e7ed27..c12d4ed1ad3 100644 --- a/cmd/ui_unix.go +++ b/cmd/ui_unix.go @@ -24,20 +24,9 @@ package cmd import ( "os" - "os/signal" "syscall" ) -// GetTermSize returns the current terminal window size on Windows, -// but is a no-op on all other platforms. -func GetTermSize(fd, termWidth int) (width, height int, err error) { - return termWidth, 0, nil -} - -// NotifyWindowResize listens for SIGWINCH (terminal window size changes) -// on *nix platforms, and is a no-op on Windows. -func NotifyWindowResize() <-chan os.Signal { - sch := make(chan os.Signal, 1) - signal.Notify(sch, os.Signal(syscall.SIGWINCH)) - return sch +func getWinchSignal() os.Signal { + return syscall.SIGWINCH } diff --git a/cmd/ui_windows.go b/cmd/ui_windows.go index b5d982c4342..62cc92a6ea6 100644 --- a/cmd/ui_windows.go +++ b/cmd/ui_windows.go @@ -24,18 +24,8 @@ package cmd import ( "os" - - "golang.org/x/crypto/ssh/terminal" ) -// GetTermSize returns the current terminal window size on Windows, -// but is a no-op on all other platforms. -func GetTermSize(fd, termWidth int) (width, height int, err error) { - return terminal.GetSize(fd) -} - -// NotifyWindowResize listens for SIGWINCH (terminal window size changes) -// on *nix platforms, and is a no-op on Windows. -func NotifyWindowResize() <-chan os.Signal { - return make(<-chan os.Signal, 1) +func getWinchSignal() os.Signal { + return nil } From 4ea5cd289065f487b6229f949ee8bc2da6281214 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 5 Mar 2020 17:13:42 +0100 Subject: [PATCH 135/350] Fix scrollback when rendering progress bars with limited window width Resolves https://github.com/loadimpact/k6/pull/1332#discussion_r383918768 --- cmd/ui.go | 34 ++++++++++++++++++++++------------ cmd/ui_test.go | 2 +- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index bc23510339c..84ae8b4edaa 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -48,11 +48,14 @@ type UIMode uint8 //nolint: golint const ( - // max length of left-side progress bar text before trimming is forced + // Max length of left-side progress bar text before trimming is forced maxLeftLength = 30 UIModeResponsive UIMode = iota + 1 UIModeCompact UIModeFull + // Amount of padding in chars between rendered progress + // bar text and right-side terminal window edge. + termPadding = 1 ) // Decode implements envconfig.Decoder @@ -107,8 +110,9 @@ func printBar(bar *pb.ProgressBar, rightText string) { fprintf(stdout, "%s %s %s%s", rendered.Left, rendered.Progress(), rightText, end) } +//nolint: funlen func renderMultipleBars( - isTTY, goBack bool, maxLeft, widthDelta int, pbs []*pb.ProgressBar, + isTTY, goBack bool, maxLeft, termWidth, widthDelta int, pbs []*pb.ProgressBar, ) (string, int) { lineEnd := "\n" if isTTY { @@ -117,6 +121,10 @@ func renderMultipleBars( } var ( + // Amount of times line lengths exceed termWidth. + // Needed to factor into the amount of lines to jump + // back with [A and avoid scrollback issues. + lineBreaks int longestLine int // Maximum length of each right side column except last, // used to calculate the padding between columns. @@ -149,6 +157,8 @@ func renderMultipleBars( rend := rendered[i] if rend.Hijack != "" { result[i+1] = rend.Hijack + lineEnd + runeCount := utf8.RuneCountInString(rend.Hijack) + lineBreaks += (runeCount - termPadding) / termWidth continue } var leftText, rightText string @@ -165,10 +175,11 @@ func renderMultipleBars( // Get visible line length, without ANSI escape sequences (color) status := fmt.Sprintf(" %s ", rend.Status()) line := leftText + status + rend.Progress() + rightText - lineRuneLen := utf8.RuneCountInString(line) - if lineRuneLen > longestLine { - longestLine = lineRuneLen + lineRuneCount := utf8.RuneCountInString(line) + if lineRuneCount > longestLine { + longestLine = lineRuneCount } + lineBreaks += (lineRuneCount - termPadding) / termWidth if !noColor { rend.Color = true status = fmt.Sprintf(" %s ", rend.Status()) @@ -179,11 +190,11 @@ func renderMultipleBars( } if isTTY && goBack { - // Go back to the beginning + // Clear screen and go back to the beginning //TODO: check for cross platform support - result[pbsCount+1] = fmt.Sprintf("\r\x1b[%dA", pbsCount+1) + result[pbsCount+1] = fmt.Sprintf("\r\x1b[J\x1b[%dA", pbsCount+lineBreaks+1) } else { - result[pbsCount+1] = "\n" + result[pbsCount+1] = lineEnd } return strings.Join(result, ""), longestLine @@ -226,9 +237,8 @@ func showProgress( var progressBarsLastRender []byte // default responsive render renderProgressBars := func(goBack bool) { - barText, longestLine := renderMultipleBars(stdoutTTY, goBack, maxLeft, widthDelta, pbs) - // -1 to allow some "breathing room" near the edge - widthDelta = termWidth - longestLine - 1 + barText, longestLine := renderMultipleBars(stdoutTTY, goBack, maxLeft, termWidth, widthDelta, pbs) + widthDelta = termWidth - longestLine - termPadding progressBarsLastRender = []byte(barText) } @@ -238,7 +248,7 @@ func showProgress( if conf.UIMode.String != UIModeResponsive.String() { renderProgressBars = func(goBack bool) { - barText, _ := renderMultipleBars(stdoutTTY, goBack, maxLeft, widthDelta, pbs) + barText, _ := renderMultipleBars(stdoutTTY, goBack, maxLeft, termWidth, widthDelta, pbs) progressBarsLastRender = []byte(barText) } } diff --git a/cmd/ui_test.go b/cmd/ui_test.go index 284eee36367..9f03df39bae 100644 --- a/cmd/ui_test.go +++ b/cmd/ui_test.go @@ -71,7 +71,7 @@ left 2 [ 0% ] right 2 000 tc := tc t.Run(tc.name, func(t *testing.T) { pbs := createTestProgressBars(3, tc.padding, 1) - out, longestLine := renderMultipleBars(false, false, 6+tc.padding, tc.widthDelta, pbs) + out, longestLine := renderMultipleBars(false, false, 6+tc.padding, 80, tc.widthDelta, pbs) assert.Equal(t, tc.expOut, out) assert.Equal(t, tc.expLongLine, longestLine) }) From ce07a70a2853838f6400e41a0a499f3ffd8bd6c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 9 Mar 2020 11:38:52 +0100 Subject: [PATCH 136/350] Bring back PersistentText for avoiding stdout flicker --- cmd/ui.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/cmd/ui.go b/cmd/ui.go index 84ae8b4edaa..74714215d43 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -263,6 +263,16 @@ func showProgress( //description in the TODO message in cmd/root.go) if stdoutTTY && !noColor { updateFreq = 100 * time.Millisecond + outMutex.Lock() + stdout.PersistentText = printProgressBars + stderr.PersistentText = printProgressBars + outMutex.Unlock() + defer func() { + outMutex.Lock() + stdout.PersistentText = nil + stderr.PersistentText = nil + outMutex.Unlock() + }() } var ( From ff5eab2cbf453d3b8305c5b24ed3598db6b1acab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 27 Mar 2020 13:00:08 +0100 Subject: [PATCH 137/350] Remove ui-mode option Progress bars will be responsive by default, and fixed compact (showing percentages) only in non-interactive terminals (e.g. CI). Resolves https://github.com/loadimpact/k6/pull/1332#pullrequestreview-381830699 --- cmd/config.go | 4 --- cmd/config_consolidation_test.go | 4 --- cmd/options.go | 7 ----- cmd/ui.go | 36 ++++++---------------- cmd/ui_mode_gen.go | 52 -------------------------------- lib/options.go | 6 ---- 6 files changed, 10 insertions(+), 99 deletions(-) delete mode 100644 cmd/ui_mode_gen.go diff --git a/cmd/config.go b/cmd/config.go index 3abe3da5873..f2ad066185b 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -88,10 +88,6 @@ func (c Config) Validate() []error { //TODO: validate all of the other options... that we should have already been validating... //TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation - if _, err := UIModeString(c.Options.UIMode.String); err != nil { - errors = append(errors, err) - } - return errors } diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index f0e441f8344..70d3788e021 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -377,10 +377,6 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, // TODO: test for differences between flagsets // TODO: more tests in general, especially ones not related to execution parameters... - {opts{cli: []string{"--ui-mode", "compact"}}, exp{}, func(t *testing.T, c Config) { - assert.Equal(t, UIModeCompact.String(), c.UIMode.String) - }}, - {opts{cli: []string{"--ui-mode", "blah"}}, exp{validationErrors: true}, nil}, } } diff --git a/cmd/options.go b/cmd/options.go index e52a1d5963b..9c684dbf5a2 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -90,12 +90,6 @@ func optionFlagSet() *pflag.FlagSet { flags.StringSlice("tag", nil, "add a `tag` to be applied to all samples, as `[name]=[value]`") flags.String("console-output", "", "redirects the console logging to the provided output file") flags.Bool("discard-response-bodies", false, "Read but don't process or save HTTP response bodies") - flags.String("ui-mode", UIModeResponsive.String(), - `Change the way the UI is rendered. Options: - compact: show compact fixed-length progress bars with percentages - responsive: resize progress bars to fit terminal window - full: show full fixed-length progress bars - `) return flags } @@ -125,7 +119,6 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { TeardownTimeout: types.NullDuration{Duration: types.Duration(60 * time.Second), Valid: false}, MetricSamplesBufferSize: null.NewInt(1000, false), - UIMode: getNullString(flags, "ui-mode"), } // Using Changed() because GetStringSlice() doesn't differentiate between empty and no value diff --git a/cmd/ui.go b/cmd/ui.go index 74714215d43..e61ead5121b 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -18,8 +18,6 @@ * */ -// go:generate enumer -type=UIMode -transform=snake -trimprefix=UIMode -output ui_mode_gen.go - package cmd import ( @@ -43,27 +41,14 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -// UIMode defines various rendering methods -type UIMode uint8 - -//nolint: golint const ( // Max length of left-side progress bar text before trimming is forced - maxLeftLength = 30 - UIModeResponsive UIMode = iota + 1 - UIModeCompact - UIModeFull + maxLeftLength = 30 // Amount of padding in chars between rendered progress // bar text and right-side terminal window edge. termPadding = 1 ) -// Decode implements envconfig.Decoder -func (i *UIMode) Decode(value string) (err error) { - *i, err = UIModeString(value) - return -} - // A writer that syncs writes with a mutex and, if the output is a TTY, clears before newlines. type consoleWriter struct { Writer io.Writer @@ -233,30 +218,29 @@ func showProgress( // Limit to maximum left text length maxLeft := int(lib.Min(leftLen, maxLeftLength)) - var widthDelta int var progressBarsLastRender []byte - // default responsive render + + printProgressBars := func() { + _, _ = stdout.Writer.Write(progressBarsLastRender) + } + + var widthDelta int + // Default to responsive progress bars when in an interactive terminal renderProgressBars := func(goBack bool) { barText, longestLine := renderMultipleBars(stdoutTTY, goBack, maxLeft, termWidth, widthDelta, pbs) widthDelta = termWidth - longestLine - termPadding progressBarsLastRender = []byte(barText) } - if conf.UIMode.String == UIModeCompact.String() { + // Otherwise fallback to fixed compact progress bars + if !stdoutTTY { widthDelta = -pb.DefaultWidth - } - - if conf.UIMode.String != UIModeResponsive.String() { renderProgressBars = func(goBack bool) { barText, _ := renderMultipleBars(stdoutTTY, goBack, maxLeft, termWidth, widthDelta, pbs) progressBarsLastRender = []byte(barText) } } - printProgressBars := func() { - _, _ = stdout.Writer.Write(progressBarsLastRender) - } - //TODO: make configurable? updateFreq := 1 * time.Second //TODO: remove !noColor after we fix how we handle colors (see the related diff --git a/cmd/ui_mode_gen.go b/cmd/ui_mode_gen.go deleted file mode 100644 index 856b01e493a..00000000000 --- a/cmd/ui_mode_gen.go +++ /dev/null @@ -1,52 +0,0 @@ -// Code generated by "enumer -type=UIMode -transform=snake -trimprefix=UIMode -output ui_mode_gen.go"; DO NOT EDIT. - -// -package cmd - -import ( - "fmt" -) - -const _UIModeName = "responsivecompactfull" - -var _UIModeIndex = [...]uint8{0, 10, 17, 21} - -func (i UIMode) String() string { - i -= 2 - if i >= UIMode(len(_UIModeIndex)-1) { - return fmt.Sprintf("UIMode(%d)", i+2) - } - return _UIModeName[_UIModeIndex[i]:_UIModeIndex[i+1]] -} - -var _UIModeValues = []UIMode{2, 3, 4} - -var _UIModeNameToValueMap = map[string]UIMode{ - _UIModeName[0:10]: 2, - _UIModeName[10:17]: 3, - _UIModeName[17:21]: 4, -} - -// UIModeString retrieves an enum value from the enum constants string name. -// Throws an error if the param is not part of the enum. -func UIModeString(s string) (UIMode, error) { - if val, ok := _UIModeNameToValueMap[s]; ok { - return val, nil - } - return 0, fmt.Errorf("%s does not belong to UIMode values", s) -} - -// UIModeValues returns all values of the enum -func UIModeValues() []UIMode { - return _UIModeValues -} - -// IsAUIMode returns "true" if the value is listed in the enum definition. "false" otherwise -func (i UIMode) IsAUIMode() bool { - for _, v := range _UIModeValues { - if i == v { - return true - } - } - return false -} diff --git a/lib/options.go b/lib/options.go index 1690819d4a5..6b89733f3e4 100644 --- a/lib/options.go +++ b/lib/options.go @@ -291,9 +291,6 @@ type Options struct { // Redirect console logging to a file ConsoleOutput null.String `json:"-" envconfig:"K6_CONSOLE_OUTPUT"` - - // UI render mode: "compact", "full" or "responsive" - UIMode null.String `json:"uiMode" envconfig:"K6_UI_MODE"` } // Returns the result of overwriting any fields with any that are set on the argument. @@ -441,9 +438,6 @@ func (o Options) Apply(opts Options) Options { if opts.ConsoleOutput.Valid { o.ConsoleOutput = opts.ConsoleOutput } - if opts.UIMode.Valid { - o.UIMode = opts.UIMode - } return o } From 17c92555aea12972ea999612b30c3b91ab455762 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 27 Mar 2020 13:50:53 +0100 Subject: [PATCH 138/350] More compact updateTermWidth init Resolves https://github.com/loadimpact/k6/pull/1332#discussion_r397880860 --- cmd/ui.go | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index e61ead5121b..523c0825b40 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -260,13 +260,17 @@ func showProgress( } var ( - fd = int(os.Stdout.Fd()) - ticker = time.NewTicker(updateFreq) - updateTermWidth func() - winchSignal = getWinchSignal() + fd = int(os.Stdout.Fd()) + ticker = time.NewTicker(updateFreq) + winchSignal = getWinchSignal() ) - // On platforms that have SIGWINCH (*nix), progress bar resizing will be - // more responsive. On others (Windows), we fallback to resize on every tick. + + // Default ticker-based progress bar resizing + updateTermWidth := func() { + <-ticker.C + termWidth, _, _ = terminal.GetSize(fd) + } + // More responsive progress bar resizing on platforms with SIGWINCH (*nix) if winchSignal != nil { winch := make(chan os.Signal, 1) signal.Notify(winch, winchSignal) @@ -277,11 +281,6 @@ func showProgress( termWidth, _, _ = terminal.GetSize(fd) } } - } else { - updateTermWidth = func() { - <-ticker.C - termWidth, _, _ = terminal.GetSize(fd) - } } ctxDone := ctx.Done() From 1affab2943e0b5b4b86b27c34a21294a841a0ac7 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Jan 2020 12:18:19 +0200 Subject: [PATCH 139/350] WIP Implement GetStripedOffsets --- lib/execution_segment.go | 80 ++++++++++++++++++++++++++++++++--- lib/execution_segment_test.go | 41 ++++++++++++++++++ 2 files changed, 116 insertions(+), 5 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 94ff0dd8b9e..adb4e95fa98 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -24,6 +24,7 @@ import ( "encoding" "fmt" "math/big" + "sort" "strings" ) @@ -394,13 +395,82 @@ func (ess ExecutionSegmentSequence) String() string { return strings.Join(result, ",") } +func (ess ExecutionSegmentSequence) Len() int { + return len(ess) +} + +func (ess ExecutionSegmentSequence) Less(i, j int) bool { + // Yes this Less is actually More, but we want it sorted in descending order and the alternative is to reverse it after sort + return ess[i].length.Cmp(ess[j].length) > 0 +} + +func (ess ExecutionSegmentSequence) Swap(i, j int) { + ess[i], ess[j] = ess[j], ess[i] +} + +var _ sort.Interface = ExecutionSegmentSequence{} + +// lowest common denominator based on https://rosettacode.org/wiki/Least_common_multiple#Go +func (ess ExecutionSegmentSequence) lcd() int64 { + var m, n, z big.Int + z = *ess[0].length.Denom() + for _, seg := range ess[1:] { + m = z + n = *seg.length.Denom() + z.Mul(z.Div(&m, z.GCD(nil, nil, &m, &n)), &n) + } + + return z.Int64() +} + // GetStripedOffsets returns everything that you need in order to execute only // the iterations that belong to the supplied segment... // // TODO: add a more detailed algorithm description -func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int, []int, error) { - start := 0 - offsets := []int{} - // TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit - return start, offsets, fmt.Errorf("not implemented") +// TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit +func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, error) { + // TODO check if ExecutionSegmentSequence actually starts from 0 and goes to 1 ? + var matchingSegment *ExecutionSegment + for _, seg := range ess { + if seg.Equal(segment) { + matchingSegment = seg + break + } + } + if matchingSegment == nil { + return -1, nil, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) // TODO: make a seperate error ? + } + sort.Stable(ess) // TODO: Use Stable ? copy the sequence ? + + var numerators = make([]int64, len(ess)) + var soonest = make([]*big.Rat, len(ess)) + var lcd = ess.lcd() + + for i := range ess { + numerators[i] = ess[i].length.Num().Int64() * (lcd / ess[i].length.Denom().Int64()) + soonest[i] = big.NewRat(0, 1) + } + + start := int64(-1) + var offsets []int64 // TODO we can preallocate this + + for i := int64(0); i < lcd; i++ { + var iRat = big.NewRat(i, 1) + for index, value := range soonest { + if iRat.Cmp(value) >= 0 { + value.Add(value, big.NewRat(lcd, numerators[index])) + if ess[index] == matchingSegment { // TODO: this can be done for all segments and then we only get what we care about + if start < 0 { + start = i + } else { + offsets = append(offsets, i) + } + // we can stop iterating the moment len(offsets) + 1 = numerator and we have set start ... + } + break + } + } + } + + return start, offsets, nil } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 5b73648867a..fa79fca7430 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -433,3 +433,44 @@ func TestExecutionSegmentScaleConsistency(t *testing.T) { }) } } + +func TestGetStripedOffsets(t *testing.T) { + t.Parallel() + testCases := []struct { + seq string + seg string + start int64 + offsets []int64 + expError string + }{ + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", expError: "missing segment"}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 7}}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{5}}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9}, + } + + for _, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("seq:%s;segment:%s", tc.seq, tc.seg), func(t *testing.T) { + result, err := NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(t, err) + segment, err := NewExecutionSegmentFromString(tc.seg) + require.NoError(t, err) + start, offsets, err := result.GetStripedOffsets(segment) + if len(tc.expError) != 0 { + require.Error(t, err, tc.expError) + require.Contains(t, err.Error(), tc.expError) + return + } + require.NoError(t, err) + + assert.Equal(t, tc.start, start) + assert.Equal(t, tc.offsets, offsets) + }) + } +} + +// TODO: test with randomized things From cac2e09f92fbb41fb63fe89adbb5cc872fee23b6 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Jan 2020 12:54:50 +0200 Subject: [PATCH 140/350] two more test cases --- lib/execution_segment_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index fa79fca7430..908e91758f8 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -450,6 +450,8 @@ func TestGetStripedOffsets(t *testing.T) { {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3}, {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8}, {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{5}}, } for _, tc := range testCases { From c7133d0a6d6fcb87802b40746921b06469c3508b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 3 Feb 2020 15:03:56 +0200 Subject: [PATCH 141/350] Use sort.SliceStable and copy ExecutionSegmentSequence --- lib/execution_segment.go | 23 +++++++---------------- lib/execution_segment_test.go | 8 ++++++-- 2 files changed, 13 insertions(+), 18 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index adb4e95fa98..7fc1ab05a53 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -395,21 +395,6 @@ func (ess ExecutionSegmentSequence) String() string { return strings.Join(result, ",") } -func (ess ExecutionSegmentSequence) Len() int { - return len(ess) -} - -func (ess ExecutionSegmentSequence) Less(i, j int) bool { - // Yes this Less is actually More, but we want it sorted in descending order and the alternative is to reverse it after sort - return ess[i].length.Cmp(ess[j].length) > 0 -} - -func (ess ExecutionSegmentSequence) Swap(i, j int) { - ess[i], ess[j] = ess[j], ess[i] -} - -var _ sort.Interface = ExecutionSegmentSequence{} - // lowest common denominator based on https://rosettacode.org/wiki/Least_common_multiple#Go func (ess ExecutionSegmentSequence) lcd() int64 { var m, n, z big.Int @@ -440,7 +425,13 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) if matchingSegment == nil { return -1, nil, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) // TODO: make a seperate error ? } - sort.Stable(ess) // TODO: Use Stable ? copy the sequence ? + + ess = append([]*ExecutionSegment{}, ess...) + sort.SliceStable(ess, + func(i, j int) bool { + // Yes this Less is actually More, but we want it sorted in descending order and the alternative is to reverse it after sort + return ess[i].length.Cmp(ess[j].length) > 0 + }) var numerators = make([]int64, len(ess)) var soonest = make([]*big.Rat, len(ess)) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 908e91758f8..890751ae52a 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -457,11 +457,11 @@ func TestGetStripedOffsets(t *testing.T) { for _, tc := range testCases { tc := tc t.Run(fmt.Sprintf("seq:%s;segment:%s", tc.seq, tc.seg), func(t *testing.T) { - result, err := NewExecutionSegmentSequenceFromString(tc.seq) + ess, err := NewExecutionSegmentSequenceFromString(tc.seq) require.NoError(t, err) segment, err := NewExecutionSegmentFromString(tc.seg) require.NoError(t, err) - start, offsets, err := result.GetStripedOffsets(segment) + start, offsets, err := ess.GetStripedOffsets(segment) if len(tc.expError) != 0 { require.Error(t, err, tc.expError) require.Contains(t, err.Error(), tc.expError) @@ -471,6 +471,10 @@ func TestGetStripedOffsets(t *testing.T) { assert.Equal(t, tc.start, start) assert.Equal(t, tc.offsets, offsets) + + ess2, err := NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(t, err) + assert.Equal(t, ess, ess2) }) } } From cee06022bfce87682dd240995dd3e2022dbbe200 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 3 Feb 2020 16:31:04 +0200 Subject: [PATCH 142/350] Add benchmarks MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name time/op GetStripedOffsets/length10,seed777-4 58.9µs ±41% GetStripedOffsets/length100,seed777-4 19.3ms ±19% GetStripedOffsetsEven/length10-4 10.4µs ± 2% GetStripedOffsetsEven/length100-4 395µs ± 2% GetStripedOffsetsEven/length1000-4 32.5ms ± 2% name alloc/op GetStripedOffsets/length10,seed777-4 21.7kB ±54% GetStripedOffsets/length100,seed777-4 9.82MB ±21% GetStripedOffsetsEven/length10-4 3.93kB ± 0% GetStripedOffsetsEven/length100-4 116kB ± 0% GetStripedOffsetsEven/length1000-4 8.35MB ± 0% name allocs/op GetStripedOffsets/length10,seed777-4 1.07k ±42% GetStripedOffsets/length100,seed777-4 389k ±17% GetStripedOffsetsEven/length10-4 233 ± 0% GetStripedOffsetsEven/length100-4 11.4k ± 0% GetStripedOffsetsEven/length1000-4 1.01M ± 0% --- lib/execution_segment.go | 3 +- lib/execution_segment_test.go | 53 +++++++++++++++++++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 7fc1ab05a53..79c32f3edb0 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -450,7 +450,8 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) for index, value := range soonest { if iRat.Cmp(value) >= 0 { value.Add(value, big.NewRat(lcd, numerators[index])) - if ess[index] == matchingSegment { // TODO: this can be done for all segments and then we only get what we care about + if ess[index] == matchingSegment { + // TODO: this can be done for all segments and then we only get what we care about if start < 0 { start = i } else { diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 890751ae52a..0f7505878dd 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -479,4 +479,57 @@ func TestGetStripedOffsets(t *testing.T) { } } +func BenchmarkGetStripedOffsets(b *testing.B) { + var lengths = [...]int64{10, 100} + const seed = 777 + r := rand.New(rand.NewSource(seed)) + + for _, length := range lengths { + length := length + b.Run(fmt.Sprintf("length%d,seed%d", length, seed), func(b *testing.B) { + sequence := generateRandomSequence(length, r) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, _, err := sequence.GetStripedOffsets(sequence[int(r.Int63())%len(sequence)]) + require.NoError(b, err) + } + }) + } +} + +func BenchmarkGetStripedOffsetsEven(b *testing.B) { + var lengths = [...]int64{10, 100, 1000} + generateSequence := func(n int64) ExecutionSegmentSequence { + // try to randomly generate an executionsegmentsequence + var err error + var ess = ExecutionSegmentSequence(make([]*ExecutionSegment, n)) + var numerators = make([]int64, n) + var denominator int64 + for i := int64(0); i < n; i++ { + numerators[i] = 1 // nice and simple :) + denominator += numerators[i] + } + ess[0], err = NewExecutionSegment(big.NewRat(0, 1), big.NewRat(numerators[0], denominator)) + require.NoError(b, err) + for i := int64(1); i < n; i++ { + ess[i], err = NewExecutionSegment(ess[i-1].to, new(big.Rat).Add(big.NewRat(numerators[i], denominator), ess[i-1].to)) + require.NoError(b, err, "%d", i) + } + + return ess + } + + for _, length := range lengths { + length := length + b.Run(fmt.Sprintf("length%d", length), func(b *testing.B) { + sequence := generateSequence(length) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, _, err := sequence.GetStripedOffsets(sequence[111233%len(sequence)]) + require.NoError(b, err) + } + }) + } +} + // TODO: test with randomized things From ce94b5251ec89f531ac2ccdf6221eaae9c8fc00e Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 3 Feb 2020 15:54:37 +0200 Subject: [PATCH 143/350] Precalculate the numerator change MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 58.9µs ±41% 47.3µs ±35% -19.67% (p=0.000 n=50+49) GetStripedOffsets/length100,seed777-4 19.3ms ±19% 18.6ms ±23% ~ (p=0.058 n=49+50) GetStripedOffsetsEven/length10-4 10.4µs ± 2% 10.9µs ± 2% +5.37% (p=0.000 n=43+44) GetStripedOffsetsEven/length100-4 395µs ± 2% 400µs ± 2% +1.32% (p=0.000 n=43+42) GetStripedOffsetsEven/length1000-4 32.5ms ± 2% 32.9ms ± 2% +1.37% (p=0.000 n=43+44) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 21.7kB ±54% 20.4kB ±47% ~ (p=0.179 n=50+50) GetStripedOffsets/length100,seed777-4 9.82MB ±21% 10.09MB ±21% ~ (p=0.143 n=48+50) GetStripedOffsetsEven/length10-4 3.93kB ± 0% 4.65kB ± 0% +18.33% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 116kB ± 0% 123kB ± 0% +6.30% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 8.35MB ± 0% 8.42MB ± 0% +0.86% (p=0.000 n=46+49) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 1.07k ±42% 0.88k ±38% -17.98% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 389k ±17% 381k ±21% ~ (p=0.246 n=48+50) GetStripedOffsetsEven/length10-4 233 ± 0% 244 ± 0% +4.72% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 11.4k ± 0% 11.5k ± 0% +0.89% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 1.01M ± 0% 1.01M ± 0% +0.10% (p=0.000 n=50+50) --- lib/execution_segment.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 79c32f3edb0..252b0ce8ac3 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -434,12 +434,14 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) }) var numerators = make([]int64, len(ess)) + var numeratorChanges = make([]*big.Rat, len(ess)) var soonest = make([]*big.Rat, len(ess)) var lcd = ess.lcd() for i := range ess { numerators[i] = ess[i].length.Num().Int64() * (lcd / ess[i].length.Denom().Int64()) soonest[i] = big.NewRat(0, 1) + numeratorChanges[i] = big.NewRat(lcd, numerators[i]) } start := int64(-1) @@ -449,7 +451,7 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) var iRat = big.NewRat(i, 1) for index, value := range soonest { if iRat.Cmp(value) >= 0 { - value.Add(value, big.NewRat(lcd, numerators[index])) + value.Add(value, numeratorChanges[index]) if ess[index] == matchingSegment { // TODO: this can be done for all segments and then we only get what we care about if start < 0 { From b45e60b5e38dd41574026ece5e9e59bb86f7f2ff Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 4 Feb 2020 08:41:40 +0200 Subject: [PATCH 144/350] Return correct offsets and return the LCD as well MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 47.3µs ±35% 48.6µs ±37% ~ (p=0.694 n=49+50) GetStripedOffsets/length100,seed777-4 18.6ms ±23% 18.9ms ±20% ~ (p=0.557 n=50+50) GetStripedOffsetsEven/length10-4 10.9µs ± 2% 11.0µs ± 3% +1.01% (p=0.000 n=44+41) GetStripedOffsetsEven/length100-4 400µs ± 2% 411µs ± 5% +2.71% (p=0.000 n=42+42) GetStripedOffsetsEven/length1000-4 32.9ms ± 2% 32.9ms ± 3% ~ (p=0.718 n=44+44) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 20.4kB ±47% 20.2kB ±45% ~ (p=0.683 n=50+50) GetStripedOffsets/length100,seed777-4 10.1MB ±21% 10.0MB ±14% ~ (p=0.515 n=50+47) GetStripedOffsetsEven/length10-4 4.65kB ± 0% 4.65kB ± 0% ~ (all equal) GetStripedOffsetsEven/length100-4 123kB ± 0% 123kB ± 0% ~ (all equal) GetStripedOffsetsEven/length1000-4 8.42MB ± 0% 8.42MB ± 0% ~ (p=0.556 n=49+47) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 881 ±38% 887 ±35% ~ (p=0.896 n=50+50) GetStripedOffsets/length100,seed777-4 381k ±21% 377k ±14% ~ (p=0.617 n=50+47) GetStripedOffsetsEven/length10-4 244 ± 0% 244 ± 0% ~ (all equal) GetStripedOffsetsEven/length100-4 11.5k ± 0% 11.5k ± 0% ~ (all equal) GetStripedOffsetsEven/length1000-4 1.01M ± 0% 1.01M ± 0% ~ (all equal) --- lib/execution_segment.go | 12 ++++++++---- lib/execution_segment_test.go | 25 +++++++++++++------------ 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 252b0ce8ac3..a1447621b98 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -413,7 +413,7 @@ func (ess ExecutionSegmentSequence) lcd() int64 { // // TODO: add a more detailed algorithm description // TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit -func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, error) { +func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64, error) { // TODO check if ExecutionSegmentSequence actually starts from 0 and goes to 1 ? var matchingSegment *ExecutionSegment for _, seg := range ess { @@ -423,7 +423,7 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) } } if matchingSegment == nil { - return -1, nil, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) // TODO: make a seperate error ? + return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) } ess = append([]*ExecutionSegment{}, ess...) @@ -457,7 +457,11 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) if start < 0 { start = i } else { - offsets = append(offsets, i) + prev := start + if len(offsets) > 0 { + prev = offsets[len(offsets)-1] + } + offsets = append(offsets, i-prev) } // we can stop iterating the moment len(offsets) + 1 = numerator and we have set start ... } @@ -466,5 +470,5 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) } } - return start, offsets, nil + return start, offsets, lcd, nil } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 0f7505878dd..42304576f15 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -441,17 +441,18 @@ func TestGetStripedOffsets(t *testing.T) { seg string start int64 offsets []int64 + lcd int64 expError string }{ {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", expError: "missing segment"}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 7}}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{5}}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9}, - {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9}, - {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{5}}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 3}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{4}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{4}, lcd: 10}, } for _, tc := range testCases { @@ -461,7 +462,7 @@ func TestGetStripedOffsets(t *testing.T) { require.NoError(t, err) segment, err := NewExecutionSegmentFromString(tc.seg) require.NoError(t, err) - start, offsets, err := ess.GetStripedOffsets(segment) + start, offsets, lcd, err := ess.GetStripedOffsets(segment) if len(tc.expError) != 0 { require.Error(t, err, tc.expError) require.Contains(t, err.Error(), tc.expError) @@ -471,6 +472,7 @@ func TestGetStripedOffsets(t *testing.T) { assert.Equal(t, tc.start, start) assert.Equal(t, tc.offsets, offsets) + assert.Equal(t, tc.lcd, lcd) ess2, err := NewExecutionSegmentSequenceFromString(tc.seq) require.NoError(t, err) @@ -490,7 +492,7 @@ func BenchmarkGetStripedOffsets(b *testing.B) { sequence := generateRandomSequence(length, r) b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := sequence.GetStripedOffsets(sequence[int(r.Int63())%len(sequence)]) + _, _, _, err := sequence.GetStripedOffsets(sequence[int(r.Int63())%len(sequence)]) require.NoError(b, err) } }) @@ -500,7 +502,6 @@ func BenchmarkGetStripedOffsets(b *testing.B) { func BenchmarkGetStripedOffsetsEven(b *testing.B) { var lengths = [...]int64{10, 100, 1000} generateSequence := func(n int64) ExecutionSegmentSequence { - // try to randomly generate an executionsegmentsequence var err error var ess = ExecutionSegmentSequence(make([]*ExecutionSegment, n)) var numerators = make([]int64, n) @@ -525,7 +526,7 @@ func BenchmarkGetStripedOffsetsEven(b *testing.B) { sequence := generateSequence(length) b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := sequence.GetStripedOffsets(sequence[111233%len(sequence)]) + _, _, _, err := sequence.GetStripedOffsets(sequence[111233%len(sequence)]) require.NoError(b, err) } }) From 72c7023de855b16b3735943198459774e8a08a7b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 4 Feb 2020 09:18:29 +0200 Subject: [PATCH 145/350] Preemptivly break the loop if we know we have enough offsets MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 48.6µs ±37% 44.5µs ±38% -8.48% (p=0.021 n=50+49) GetStripedOffsets/length100,seed777-4 18.9ms ±20% 17.9ms ±18% -5.49% (p=0.004 n=50+50) GetStripedOffsetsEven/length10-4 11.0µs ± 3% 7.2µs ± 5% -34.87% (p=0.000 n=41+43) GetStripedOffsetsEven/length100-4 411µs ± 5% 104µs ± 3% -74.70% (p=0.000 n=42+45) GetStripedOffsetsEven/length1000-4 32.9ms ± 3% 2.5ms ± 7% -92.53% (p=0.000 n=44+42) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 20.2kB ±45% 19.5kB ±45% ~ (p=0.334 n=50+50) GetStripedOffsets/length100,seed777-4 10.0MB ±14% 9.7MB ±20% -3.14% (p=0.027 n=47+50) GetStripedOffsetsEven/length10-4 4.65kB ± 0% 3.78kB ± 0% -18.59% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 123kB ± 0% 50kB ± 0% -59.16% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 8.42MB ± 0% 0.83MB ± 0% -90.09% (p=0.000 n=47+33) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 887 ±35% 806 ±37% -9.13% (p=0.007 n=50+50) GetStripedOffsets/length100,seed777-4 377k ±14% 361k ±18% -4.13% (p=0.005 n=47+50) GetStripedOffsetsEven/length10-4 244 ± 0% 136 ± 0% -44.26% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 11.5k ± 0% 2.4k ± 0% -79.13% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 1.01M ± 0% 0.07M ± 0% -93.44% (p=0.000 n=50+50) --- lib/execution_segment.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index a1447621b98..ebcd57e8be7 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -447,6 +447,7 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) start := int64(-1) var offsets []int64 // TODO we can preallocate this +OUTER: for i := int64(0); i < lcd; i++ { var iRat = big.NewRat(i, 1) for index, value := range soonest { @@ -463,7 +464,9 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) } offsets = append(offsets, i-prev) } - // we can stop iterating the moment len(offsets) + 1 = numerator and we have set start ... + if int64(len(offsets)) == numerators[index]-1 { + break OUTER + } } break } From 6bdc4c95271f974328fb006e2360a90478aa4ba2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 4 Feb 2020 10:04:30 +0200 Subject: [PATCH 146/350] Optimize the calculation of LCD as na-- proposed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 44.5µs ±38% 41.7µs ±31% ~ (p=0.218 n=49+50) GetStripedOffsets/length100,seed777-4 17.9ms ±18% 18.3ms ±19% ~ (p=0.238 n=50+50) GetStripedOffsetsEven/length10-4 7.18µs ± 5% 5.47µs ± 3% -23.82% (p=0.000 n=43+43) GetStripedOffsetsEven/length100-4 104µs ± 3% 87µs ± 3% -16.25% (p=0.000 n=45+44) GetStripedOffsetsEven/length1000-4 2.46ms ± 7% 2.27ms ± 3% -7.85% (p=0.000 n=42+43) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 19.5kB ±45% 17.9kB ±32% ~ (p=0.074 n=50+49) GetStripedOffsets/length100,seed777-4 9.70MB ±20% 9.95MB ±16% ~ (p=0.168 n=50+49) GetStripedOffsetsEven/length10-4 3.78kB ± 0% 3.21kB ± 0% -15.22% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 50.3kB ± 0% 44.0kB ± 0% -12.60% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 835kB ± 0% 771kB ± 0% -7.66% (p=0.000 n=33+37) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 806 ±37% 757 ±37% ~ (p=0.272 n=50+49) GetStripedOffsets/length100,seed777-4 361k ±18% 371k ±22% ~ (p=0.122 n=50+50) GetStripedOffsetsEven/length10-4 136 ± 0% 109 ± 0% -19.85% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 2.40k ± 0% 2.10k ± 0% -12.36% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 66.6k ± 0% 63.6k ± 0% -4.50% (p=0.000 n=50+50) --- lib/execution_segment.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index ebcd57e8be7..8496f16ed5c 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -402,6 +402,9 @@ func (ess ExecutionSegmentSequence) lcd() int64 { for _, seg := range ess[1:] { m = z n = *seg.length.Denom() + if m.Cmp(&n) == 0 { + continue + } z.Mul(z.Div(&m, z.GCD(nil, nil, &m, &n)), &n) } From 32fdd3dd448cb105b2366435660ced6b42b30a65 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 4 Feb 2020 11:21:56 +0200 Subject: [PATCH 147/350] Don't use rat.Cmp in order to save a lot memory and CPU MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 42.3µs ±43% 28.8µs ±31% -32.06% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 18.0ms ±24% 4.8ms ±14% -73.40% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 5.47µs ± 3% 4.95µs ± 3% -9.55% (p=0.000 n=42+44) GetStripedOffsetsEven/length100-4 86.9µs ± 6% 57.7µs ± 4% -33.55% (p=0.000 n=44+43) GetStripedOffsetsEven/length1000-4 2.28ms ± 9% 0.91ms ± 6% -60.03% (p=0.000 n=44+43) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 18.3kB ±51% 10.8kB ±22% -40.93% (p=0.000 n=50+49) GetStripedOffsets/length100,seed777-4 9.71MB ±17% 0.66MB ± 9% -93.21% (p=0.000 n=47+50) GetStripedOffsetsEven/length10-4 3.21kB ± 0% 3.19kB ± 0% -0.50% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 44.0kB ± 0% 35.5kB ± 0% -19.14% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 771kB ± 0% 338kB ± 0% -56.11% (p=0.000 n=40+45) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 759 ±35% 343 ±25% -54.80% (p=0.000 n=48+50) GetStripedOffsets/length100,seed777-4 366k ±23% 22k ±10% -93.91% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 109 ± 0% 87 ± 0% -20.18% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 2.10k ± 0% 0.88k ± 0% -58.05% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 63.6k ± 0% 8.4k ± 0% -86.80% (p=0.000 n=50+50) --- lib/execution_segment.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 8496f16ed5c..71b1852b861 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -452,9 +452,9 @@ func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) OUTER: for i := int64(0); i < lcd; i++ { - var iRat = big.NewRat(i, 1) for index, value := range soonest { - if iRat.Cmp(value) >= 0 { + num, denom := value.Num().Int64(), value.Denom().Int64() + if i > num/denom || (i == num/denom && num%denom == 0) { value.Add(value, numeratorChanges[index]) if ess[index] == matchingSegment { // TODO: this can be done for all segments and then we only get what we care about From 91bb4ad9bd448dfb19e584c7d72ee2eb0497e513 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 4 Feb 2020 13:26:15 +0200 Subject: [PATCH 148/350] Sort by the numerator array MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 28.8µs ±31% 25.6µs ±32% -10.91% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 4.78ms ±14% 4.86ms ±11% ~ (p=0.121 n=50+45) GetStripedOffsetsEven/length10-4 4.95µs ± 3% 3.94µs ± 3% -20.34% (p=0.000 n=44+43) GetStripedOffsetsEven/length100-4 57.7µs ± 4% 44.6µs ± 2% -22.76% (p=0.000 n=43+41) GetStripedOffsetsEven/length1000-4 912µs ± 6% 776µs ± 4% -14.89% (p=0.000 n=43+43) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 10.8kB ±22% 7.9kB ±21% -26.81% (p=0.000 n=49+47) GetStripedOffsets/length100,seed777-4 660kB ± 9% 579kB ±11% -12.25% (p=0.000 n=50+49) GetStripedOffsetsEven/length10-4 3.19kB ± 0% 2.34kB ± 0% -26.57% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 35.5kB ± 0% 23.8kB ± 0% -32.91% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 338kB ± 0% 220kB ± 0% -34.88% (p=0.000 n=45+46) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 343 ±25% 279 ±22% -18.59% (p=0.000 n=50+47) GetStripedOffsets/length100,seed777-4 22.3k ±10% 20.6k ±10% -7.65% (p=0.000 n=50+49) GetStripedOffsetsEven/length10-4 87.0 ± 0% 68.0 ± 0% -21.84% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 883 ± 0% 638 ± 0% -27.75% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 8.40k ± 0% 5.94k ± 0% -29.28% (p=0.000 n=50+50) --- lib/execution_segment.go | 67 +++++++++++++++++++++++++++------------- 1 file changed, 46 insertions(+), 21 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 71b1852b861..57d38a36f87 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -418,35 +418,34 @@ func (ess ExecutionSegmentSequence) lcd() int64 { // TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64, error) { // TODO check if ExecutionSegmentSequence actually starts from 0 and goes to 1 ? - var matchingSegment *ExecutionSegment - for _, seg := range ess { - if seg.Equal(segment) { - matchingSegment = seg - break - } - } - if matchingSegment == nil { - return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) - } - - ess = append([]*ExecutionSegment{}, ess...) - sort.SliceStable(ess, - func(i, j int) bool { - // Yes this Less is actually More, but we want it sorted in descending order and the alternative is to reverse it after sort - return ess[i].length.Cmp(ess[j].length) > 0 - }) + ess = append([]*ExecutionSegment{}, ess...) // copy the original sequence var numerators = make([]int64, len(ess)) - var numeratorChanges = make([]*big.Rat, len(ess)) + var steps = make([]*big.Rat, len(ess)) var soonest = make([]*big.Rat, len(ess)) var lcd = ess.lcd() for i := range ess { numerators[i] = ess[i].length.Num().Int64() * (lcd / ess[i].length.Denom().Int64()) soonest[i] = big.NewRat(0, 1) - numeratorChanges[i] = big.NewRat(lcd, numerators[i]) + steps[i] = big.NewRat(lcd, numerators[i]) } + sort.Stable(sortInterfaceWrapper{ + numerators: numerators, + steps: steps, + ess: ess, + }) + var segmentIndex = -1 + for i, seg := range ess { + if seg.Equal(segment) { + segmentIndex = i + break + } + } + if segmentIndex == -1 { + return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) + } start := int64(-1) var offsets []int64 // TODO we can preallocate this @@ -455,8 +454,8 @@ OUTER: for index, value := range soonest { num, denom := value.Num().Int64(), value.Denom().Int64() if i > num/denom || (i == num/denom && num%denom == 0) { - value.Add(value, numeratorChanges[index]) - if ess[index] == matchingSegment { + value.Add(value, steps[index]) + if index == segmentIndex { // TODO: this can be done for all segments and then we only get what we care about if start < 0 { start = i @@ -478,3 +477,29 @@ OUTER: return start, offsets, lcd, nil } + +// This is only needed in order to sort all three at the same time +type sortInterfaceWrapper struct { // TODO: rename ? + numerators []int64 + steps []*big.Rat + ess ExecutionSegmentSequence +} + +// Len is the number of elements in the collection. +func (e sortInterfaceWrapper) Len() int { + return len(e.numerators) +} + +// Less reports whether the element with +// index i should sort before the element with index j. +func (e sortInterfaceWrapper) Less(i, j int) bool { + // Yes this Less is actually More, but we want it sorted in descending order + return e.numerators[i] > e.numerators[j] +} + +// Swap swaps the elements with indexes i and j. +func (e sortInterfaceWrapper) Swap(i, j int) { + e.numerators[i], e.numerators[j] = e.numerators[j], e.numerators[i] + e.steps[i], e.steps[j] = e.steps[j], e.steps[i] + e.ess[i], e.ess[j] = e.ess[j], e.ess[i] +} From 037db770169b5d9a705f30b2c80a99d3678cd1ed Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 20 Feb 2020 17:33:25 +0200 Subject: [PATCH 149/350] Refactor the GetStrippedOffsets This one handles ... more elegantly all cases where the execution segmetn sequence isn't full or missing at all. This is mostly done through the fact that the algorithm works even if we don't fill the sequence as long as we don't care about whether 0:1/3 and 1/3:2/3 get the same offsets when there is no sequence provided. If we care about that we need to fill the sequence which will be more involved but not by much :D. The other thing is that the construction of the numerator, steps and so on arrays is separate from the actual algorithm of calculating the offsets which at least makes silence golangci-lint. I would argue also makes it easier to read. --- lib/execution_segment.go | 117 +++++++++++++++++++++------------- lib/execution_segment_test.go | 32 +++++++--- 2 files changed, 96 insertions(+), 53 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 57d38a36f87..75dabdacd23 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -416,58 +416,91 @@ func (ess ExecutionSegmentSequence) lcd() int64 { // // TODO: add a more detailed algorithm description // TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit -func (ess ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64, error) { - // TODO check if ExecutionSegmentSequence actually starts from 0 and goes to 1 ? - ess = append([]*ExecutionSegment{}, ess...) // copy the original sequence +func (ess *ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64, error) { + if segment == nil || segment.length.Cmp(oneRat) == 0 { + return 0, []int64{1}, 1, nil + } + + // we will copy the sequnce to this in order to sort it :) + var copyESS ExecutionSegmentSequence + // Here we fix the problem with having no sequence + // No filling up is required as the algorithm will accommodate for it + // through just going through the iterations that need to be in the values will fill up + // this has the consequence that if this is ran without sequence, + // but with segments: 0:1/3 and 1/3:2/3 it will get the same results instead + // of 1/3:2/3 to get start=1 and offset={3} it will get as 0:1/3 will start=0 and offsets={3} + // if the above behaviour is desired this will definitely need to be outside of this function. + if ess == nil || len(*ess) == 0 { + copyESS = []*ExecutionSegment{segment} + } else { + copyESS = append([]*ExecutionSegment{}, *ess...) // copy the original sequence + } + var wrapper = newWrapper(copyESS) + + var segmentIndex = wrapper.indexOf(segment) + if segmentIndex == -1 { + return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) + } + start, offsets := wrapper.strippedOffsetsFor(segmentIndex) + return start, offsets, wrapper.lcd, nil +} - var numerators = make([]int64, len(ess)) - var steps = make([]*big.Rat, len(ess)) - var soonest = make([]*big.Rat, len(ess)) - var lcd = ess.lcd() +// This is only needed in order to sort all three at the same time +type sortInterfaceWrapper struct { // TODO: rename ? + numerators []int64 + steps []*big.Rat + ess ExecutionSegmentSequence + lcd int64 +} + +func newWrapper(ess ExecutionSegmentSequence) sortInterfaceWrapper { + var result = sortInterfaceWrapper{ + numerators: make([]int64, len(ess)), + steps: make([]*big.Rat, len(ess)), + lcd: ess.lcd(), + ess: ess, + } for i := range ess { - numerators[i] = ess[i].length.Num().Int64() * (lcd / ess[i].length.Denom().Int64()) - soonest[i] = big.NewRat(0, 1) - steps[i] = big.NewRat(lcd, numerators[i]) + result.numerators[i] = ess[i].length.Num().Int64() * (result.lcd / ess[i].length.Denom().Int64()) + result.steps[i] = big.NewRat(result.lcd, result.numerators[i]) } - sort.Stable(sortInterfaceWrapper{ - numerators: numerators, - steps: steps, - ess: ess, - }) - var segmentIndex = -1 - for i, seg := range ess { + sort.Stable(result) + return result +} + +func (e sortInterfaceWrapper) indexOf(segment *ExecutionSegment) int { + for i, seg := range e.ess { if seg.Equal(segment) { - segmentIndex = i - break + return i } } - if segmentIndex == -1 { - return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) + + return -1 +} + +func (e sortInterfaceWrapper) strippedOffsetsFor(segmentIndex int) (int64, []int64) { + var offsets = make([]int64, 0, e.numerators[segmentIndex]+1) + var soonest = make([]*big.Rat, len(e.ess)) + for i := range e.ess { + soonest[i] = big.NewRat(0, 1) } - start := int64(-1) - var offsets []int64 // TODO we can preallocate this -OUTER: - for i := int64(0); i < lcd; i++ { + for i := int64(0); i < e.lcd; i++ { for index, value := range soonest { num, denom := value.Num().Int64(), value.Denom().Int64() if i > num/denom || (i == num/denom && num%denom == 0) { - value.Add(value, steps[index]) + value.Add(value, e.steps[index]) if index == segmentIndex { - // TODO: this can be done for all segments and then we only get what we care about - if start < 0 { - start = i - } else { - prev := start - if len(offsets) > 0 { - prev = offsets[len(offsets)-1] - } - offsets = append(offsets, i-prev) + prev := int64(0) + if len(offsets) > 0 { + prev = offsets[len(offsets)-1] } - if int64(len(offsets)) == numerators[index]-1 { - break OUTER + offsets = append(offsets, i-prev) + if int64(len(offsets)) == e.numerators[index] { + offsets = append(offsets, offsets[0]+e.lcd-i) + return offsets[0], offsets[1:] } } break @@ -475,14 +508,8 @@ OUTER: } } - return start, offsets, lcd, nil -} - -// This is only needed in order to sort all three at the same time -type sortInterfaceWrapper struct { // TODO: rename ? - numerators []int64 - steps []*big.Rat - ess ExecutionSegmentSequence + // TODO return some error if we get to here + return offsets[0], offsets[1:] } // Len is the number of elements in the collection. diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 42304576f15..27e01a3a7d2 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -444,15 +444,31 @@ func TestGetStripedOffsets(t *testing.T) { lcd int64 expError string }{ + // full sequences {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", expError: "missing segment"}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 3}, lcd: 10}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{4}, lcd: 10}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2, lcd: 10}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3, lcd: 10}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8, lcd: 10}, - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, lcd: 10}, - {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, lcd: 10}, - {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{4}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 3, 3}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{4, 6}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2, offsets: []int64{10}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3, offsets: []int64{10}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.8:0.9", start: 8, offsets: []int64{10}, lcd: 10}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, offsets: []int64{10}, lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.9:1", start: 9, offsets: []int64{10}, lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{4, 6}, lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3, offsets: []int64{10}, lcd: 10}, + // not full sequences + {seq: "0,0.2,0.5", seg: "0:0.2", start: 1, offsets: []int64{4, 6}, lcd: 10}, + {seq: "0,0.2,0.5", seg: "0.2:0.5", start: 0, offsets: []int64{4, 3, 3}, lcd: 10}, + {seq: "0,2/5,4/5", seg: "0:2/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, + {seq: "0,2/5,4/5", seg: "2/5:4/5", start: 1, offsets: []int64{3, 2}, lcd: 5}, + // no sequence + {seg: "0:0.2", start: 0, offsets: []int64{5}, lcd: 5}, + {seg: "0:1/5", start: 0, offsets: []int64{5}, lcd: 5}, + {seg: "0:2/10", start: 0, offsets: []int64{5}, lcd: 5}, + {seg: "0:0.4", start: 0, offsets: []int64{3, 2}, lcd: 5}, + {seg: "0:2/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, + {seg: "2/5:4/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, // this is the same as the previous one as there is no sequence + {seg: "0:4/10", start: 0, offsets: []int64{3, 2}, lcd: 5}, + {seg: "1/10:5/10", start: 0, offsets: []int64{3, 2}, lcd: 5}, } for _, tc := range testCases { From 56248a788096a6a167c3041c2745d2f67ca1bdc6 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 17 Feb 2020 17:16:53 +0200 Subject: [PATCH 150/350] Add ExecutionSegmentSequence cli flag --- cmd/options.go | 14 ++++++++++++++ lib/options.go | 7 ++++--- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/cmd/options.go b/cmd/options.go index d82c525fed0..461c0451e78 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -54,6 +54,7 @@ func optionFlagSet() *pflag.FlagSet { flags.Int64P("iterations", "i", 0, "script total iteration limit (among all VUs)") flags.StringSliceP("stage", "s", nil, "add a `stage`, as `[duration]:[target]`") flags.String("execution-segment", "", "limit execution to the specified segment, e.g. 10%, 1/3, 0.2:2/3") + flags.String("execution-segment-sequence", "", "the execution segment sequence") // TODO better description flags.BoolP("paused", "p", false, "start the test in a paused state") flags.Bool("no-setup", false, "don't run setup()") flags.Bool("no-teardown", false, "don't run teardown()") @@ -151,6 +152,19 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { } opts.ExecutionSegment = segment } + + if flags.Changed("execution-segment-sequence") { + executionSegmentSequenceStr, err := flags.GetString("execution-segment-sequence") + if err != nil { + return opts, err + } + segmentSequence := new(lib.ExecutionSegmentSequence) + err = segmentSequence.UnmarshalText([]byte(executionSegmentSequenceStr)) + if err != nil { + return opts, err + } + opts.ESS = segmentSequence + } if flags.Changed("system-tags") { systemTagList, err := flags.GetStringSlice("system-tags") if err != nil { diff --git a/lib/options.go b/lib/options.go index ba590922dc8..102a59004c2 100644 --- a/lib/options.go +++ b/lib/options.go @@ -192,7 +192,7 @@ type Options struct { // Initial values for VUs, max VUs, duration cap, iteration cap, and stages. // See the Runner or Executor interfaces for more information. - VUs null.Int `json:"vus" envconfig:"K6_VUS"` + VUs null.Int `json:"vus" envconfig:"K6_VUS"` Duration types.NullDuration `json:"duration" envconfig:"K6_DURATION"` Iterations null.Int `json:"iterations" envconfig:"K6_ITERATIONS"` Stages []Stage `json:"stages" envconfig:"K6_STAGES"` @@ -203,8 +203,9 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` - ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` + Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` + ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` + ESS *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` // Timeouts for the setup() and teardown() functions NoSetup null.Bool `json:"noSetup" envconfig:"NO_SETUP"` From 08aacadb3725b08c0de1df9e4583c8afacf5218b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 20 Feb 2020 17:54:51 +0200 Subject: [PATCH 151/350] Use GetStrippedOffsets in constant-arrival-rate --- lib/executor/constant_arrival_rate.go | 22 +++- lib/executor/constant_arrival_rate_test.go | 117 +++++++++++++++++++++ 2 files changed, 136 insertions(+), 3 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index f98beaaaf0c..87be88e5eb1 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -24,6 +24,7 @@ import ( "context" "fmt" "math" + "math/big" "sync/atomic" "time" @@ -196,7 +197,6 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) defer cancel() - ticker := time.NewTicker(tickerPeriod) // the rate can't be 0 because of the validation // Make sure the log and the progress bar have accurate information car.logger.WithFields(logrus.Fields{ @@ -260,9 +260,25 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs - for { + start, offsets, _, err := car.executionState.Options.ESS.GetStripedOffsets(segment) + if err != nil { + return err + } + startTime = time.Now() + timer := time.NewTimer(time.Hour * 24) + // here the we need the not scaled one + notScaledTickerPeriod := time.Duration( + getTickerPeriod( + big.NewRat( + car.config.Rate.Int64, + int64(time.Duration(car.config.TimeUnit.Duration)), + )).Duration) + + for li, gi := 0, start; ; li, gi = li+1, gi+offsets[li%len(offsets)] { + var t = notScaledTickerPeriod*time.Duration(gi) - time.Since(startTime) + timer.Reset(t) select { - case <-ticker.C: + case <-timer.C: select { case vu := <-vus: // ideally, we get the VU from the buffer without any issues diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index fbd91caf2cb..b014d9d255e 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -22,12 +22,14 @@ package executor import ( "context" + "fmt" "sync" "sync/atomic" "testing" "time" "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" @@ -102,6 +104,121 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { require.Empty(t, logHook.Drain()) } +func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { + newExecutionSegmentFromString := func(str string) *lib.ExecutionSegment { + r, err := lib.NewExecutionSegmentFromString(str) + require.NoError(t, err) + return r + } + + newExecutionSegmentSequenceFromString := func(str string) *lib.ExecutionSegmentSequence { + r, err := lib.NewExecutionSegmentSequenceFromString(str) + require.NoError(t, err) + return &r + } + + var tests = []struct { + segment *lib.ExecutionSegment + sequence *lib.ExecutionSegmentSequence + start time.Duration + step time.Duration + }{ + { + segment: newExecutionSegmentFromString("0:1/3"), + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("1/3:2/3"), + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("2/3:1"), + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("1/6:3/6"), + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("1/6:3/6"), + sequence: &lib.ExecutionSegmentSequence{}, + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + // sequences + { + segment: newExecutionSegmentFromString("0:1/3"), + sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), + start: time.Millisecond * 00, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("1/3:2/3"), + sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), + start: time.Millisecond * 20, + step: time.Millisecond * 60, + }, + { + segment: newExecutionSegmentFromString("2/3:1"), + sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), + start: time.Millisecond * 40, + step: time.Millisecond * 60, + }, + } + for _, test := range tests { + test := test + + t.Run(fmt.Sprintf("segment %s sequence %s", test.segment, test.sequence), func(t *testing.T) { + t.Parallel() + es := lib.NewExecutionState(lib.Options{ + ExecutionSegment: test.segment, + ESS: test.sequence, + }, 10, 50) + var count int64 + var startTime = time.Now() + var ctx, cancel, executor, logHook = setupExecutor( + t, getTestConstantArrivalRateConfig(), es, + simpleRunner(func(ctx context.Context) error { + current := atomic.AddInt64(&count, 1) + expectedTime := test.start + time.Duration(current-1)*test.step + assert.WithinDuration(t, + startTime.Add(expectedTime), + time.Now(), + time.Millisecond*10, + "%d expectedTime %s", current, expectedTime, + ) + + return nil + }), + ) + defer cancel() + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + // check that we got around the amount of VU iterations as we would expect + var currentCount int64 + + for i := 0; i < 5; i++ { + time.Sleep(time.Second) + currentCount = atomic.LoadInt64(&count) + assert.InDelta(t, (i+1)*17, currentCount, 2) + } + }() + startTime = time.Now() + var engineOut = make(chan stats.SampleContainer, 1000) + err := executor.Run(ctx, engineOut) + wg.Wait() + require.NoError(t, err) + require.Empty(t, logHook.Drain()) + }) + } +} + func TestArrivalRateCancel(t *testing.T) { t.Parallel() From f219a1b974adc5f395a2bc92320d00fab78df938 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 21 Feb 2020 15:39:15 +0200 Subject: [PATCH 152/350] Considerable speedup of strippedOffsetFor by using less big.Rat MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 23.8µs ±34% 4.5µs ±48% -81.12% (p=0.000 n=50+48) GetStripedOffsets/length100,seed777-4 4.54ms ±15% 1.94ms ±21% -57.14% (p=0.000 n=49+50) GetStripedOffsetsEven/length10-4 3.92µs ± 2% 1.15µs ± 2% -70.68% (p=0.000 n=43+43) GetStripedOffsetsEven/length100-4 42.4µs ± 2% 15.0µs ± 1% -64.54% (p=0.000 n=43+43) GetStripedOffsetsEven/length1000-4 726µs ± 1% 377µs ± 1% -48.11% (p=0.000 n=41+44) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 7.94kB ±30% 0.98kB ±37% -87.63% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 584kB ±11% 10kB ±25% -98.22% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 2.42kB ± 0% 0.58kB ± 0% -76.16% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 23.9kB ± 0% 5.9kB ± 0% -75.32% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 220kB ± 0% 47kB ± 0% -78.67% (p=0.000 n=48+50) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 278 ±32% 25 ±67% -90.99% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 20.8k ±11% 0.2k ±54% -98.93% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 71.0 ± 0% 12.0 ± 0% -83.10% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 641 ± 0% 72 ± 0% -88.77% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 5.94k ± 0% 0.47k ± 0% -92.06% (p=0.000 n=50+50) --- lib/execution_segment.go | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 75dabdacd23..131c0bcd8ee 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -447,23 +447,20 @@ func (ess *ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment // This is only needed in order to sort all three at the same time type sortInterfaceWrapper struct { // TODO: rename ? - numerators []int64 - steps []*big.Rat ess ExecutionSegmentSequence + numerators []int64 lcd int64 } func newWrapper(ess ExecutionSegmentSequence) sortInterfaceWrapper { var result = sortInterfaceWrapper{ + ess: ess, numerators: make([]int64, len(ess)), - steps: make([]*big.Rat, len(ess)), lcd: ess.lcd(), - ess: ess, } for i := range ess { result.numerators[i] = ess[i].length.Num().Int64() * (result.lcd / ess[i].length.Denom().Int64()) - result.steps[i] = big.NewRat(result.lcd, result.numerators[i]) } sort.Stable(result) @@ -482,16 +479,20 @@ func (e sortInterfaceWrapper) indexOf(segment *ExecutionSegment) int { func (e sortInterfaceWrapper) strippedOffsetsFor(segmentIndex int) (int64, []int64) { var offsets = make([]int64, 0, e.numerators[segmentIndex]+1) - var soonest = make([]*big.Rat, len(e.ess)) - for i := range e.ess { - soonest[i] = big.NewRat(0, 1) - } + var chosenCounts = make([]int64, len(e.ess)) + // Here instead of calculating steps which need to be big.Rat, we use the fact that + // the steps are always the length of the segment inverted which also is lcd/numerator + // So instead of creating and adding up big.Rat we just multiply the step by the amount + // of times given segment has been chosen which is count * lcd / numerator and use that + // this both saves on a lot of big.Rat allocations and also on a lot of unneeded calculations + // with them. for i := int64(0); i < e.lcd; i++ { - for index, value := range soonest { - num, denom := value.Num().Int64(), value.Denom().Int64() + for index, chosenCount := range chosenCounts { + num := chosenCount * e.lcd + denom := e.numerators[index] if i > num/denom || (i == num/denom && num%denom == 0) { - value.Add(value, e.steps[index]) + chosenCounts[index]++ if index == segmentIndex { prev := int64(0) if len(offsets) > 0 { @@ -527,6 +528,5 @@ func (e sortInterfaceWrapper) Less(i, j int) bool { // Swap swaps the elements with indexes i and j. func (e sortInterfaceWrapper) Swap(i, j int) { e.numerators[i], e.numerators[j] = e.numerators[j], e.numerators[i] - e.steps[i], e.steps[j] = e.steps[j], e.steps[i] e.ess[i], e.ess[j] = e.ess[j], e.ess[i] } From cdf8ead56d9aec92b1301d04efc2aa037822e332 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 21 Feb 2020 16:47:03 +0200 Subject: [PATCH 153/350] Rewrite lcd calculation to not use big.Rat as well MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Very small speed up but it does also drop the memory usage in not "even" sequences. There are better algorithms for calculating the gcd but unless there are some quite big and divers denominators this will have very small perfomance impact if at all. name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 4.50µs ±48% 3.48µs ±35% -22.59% (p=0.000 n=48+48) GetStripedOffsets/length100,seed777-4 1.94ms ±21% 1.81ms ±16% -6.87% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 1.15µs ± 2% 1.08µs ± 1% -6.42% (p=0.000 n=43+43) GetStripedOffsetsEven/length100-4 15.0µs ± 1% 14.0µs ± 4% -6.80% (p=0.000 n=43+43) GetStripedOffsetsEven/length1000-4 377µs ± 1% 351µs ± 1% -6.80% (p=0.000 n=44+44) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 983B ±37% 764B ± 2% -22.20% (p=0.000 n=50+48) GetStripedOffsets/length100,seed777-4 10.4kB ±25% 7.9kB ± 3% -23.90% (p=0.000 n=50+49) GetStripedOffsetsEven/length10-4 576B ± 0% 576B ± 0% ~ (all equal) GetStripedOffsetsEven/length100-4 5.90kB ± 0% 5.90kB ± 0% ~ (all equal) GetStripedOffsetsEven/length1000-4 47.0kB ± 0% 47.0kB ± 0% ~ (all equal) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 25.1 ±67% 15.0 ± 0% -40.19% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 222 ±54% 105 ± 5% -52.88% (p=0.000 n=50+48) GetStripedOffsetsEven/length10-4 12.0 ± 0% 12.0 ± 0% ~ (all equal) GetStripedOffsetsEven/length100-4 72.0 ± 0% 72.0 ± 0% ~ (all equal) GetStripedOffsetsEven/length1000-4 472 ± 0% 472 ± 0% ~ (all equal) --- lib/execution_segment.go | 29 +++++++++++++++++++++-------- lib/execution_segment_test.go | 22 ++++++++++++++++++++++ 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 131c0bcd8ee..b2a9efd4e73 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -395,20 +395,33 @@ func (ess ExecutionSegmentSequence) String() string { return strings.Join(result, ",") } -// lowest common denominator based on https://rosettacode.org/wiki/Least_common_multiple#Go +// lowest common denominator +// https://en.wikipedia.org/wiki/Least_common_multiple#Using_the_greatest_common_divisor func (ess ExecutionSegmentSequence) lcd() int64 { - var m, n, z big.Int - z = *ess[0].length.Denom() + var acc = ess[0].length.Denom().Int64() + var n int64 for _, seg := range ess[1:] { - m = z - n = *seg.length.Denom() - if m.Cmp(&n) == 0 { + n = seg.length.Denom().Int64() + if acc == n || acc%n == 0 { // short circuit continue } - z.Mul(z.Div(&m, z.GCD(nil, nil, &m, &n)), &n) + acc = acc * n / gcd(acc, n) } - return z.Int64() + return acc +} + +// Greatest common divisor +// https://en.wikipedia.org/wiki/Euclidean_algorithm +func gcd(a, b int64) int64 { + for a != b { + if a > b { + a -= b + } else { + b -= a + } + } + return a } // GetStripedOffsets returns everything that you need in order to execute only diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 27e01a3a7d2..568421b0b99 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -497,6 +497,28 @@ func TestGetStripedOffsets(t *testing.T) { } } +func TestSequenceLCD(t *testing.T) { + testCases := []struct { + seq string + lcd int64 + }{ + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", lcd: 10}, + {seq: "0,0.1,0.5,0.6,0.7,0.8,0.9,1", lcd: 10}, + {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", lcd: 10}, + {seq: "0,1/3,5/6", lcd: 6}, + {seq: "0,1/3,4/7", lcd: 21}, + } + + for _, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("seq:%s", tc.seq), func(t *testing.T) { + ess, err := NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(t, err) + require.Equal(t, tc.lcd, ess.lcd()) + }) + } +} + func BenchmarkGetStripedOffsets(b *testing.B) { var lengths = [...]int64{10, 100} const seed = 777 From 005b85530eed1eee8031d37245dac10c83fcc879 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 4 Mar 2020 11:09:52 +0200 Subject: [PATCH 154/350] Refactor and use the striping algorithm MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This required some interface changes as well as a lot of test retouching. There is somewhat significant perfomance degradation which comes from the fact that now we calculate the offset for everything once, this should be negligible in the cases where it is not needed and when it is it is going to significantly speed up the execution. The constant arrival rate executor has been used to showcase how the striping algorithm should be used. name old time/op new time/op delta GetStripedOffsets/length10,seed777-4 3.64µs ±40% 5.56µs ±33% +52.95% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 1.97ms ±21% 1.89ms ±18% -3.98% (p=0.046 n=50+50) GetStripedOffsetsEven/length10-4 1.06µs ± 2% 2.67µs ± 1% +152.75% (p=0.000 n=44+43) GetStripedOffsetsEven/length100-4 14.3µs ± 1% 62.4µs ± 3% +337.62% (p=0.000 n=42+43) GetStripedOffsetsEven/length1000-4 370µs ± 1% 4807µs ± 1% +1198.18% (p=0.000 n=44+45) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-4 766B ± 2% 2144B ±17% +180.06% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 7.90kB ± 3% 103.93kB ±11% +1216.44% (p=0.000 n=50+50) GetStripedOffsetsEven/length10-4 576B ± 0% 1392B ± 0% +141.67% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 5.90kB ± 0% 9.55kB ± 0% +61.79% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 47.0kB ± 0% 82.5kB ± 0% +75.59% (p=0.000 n=50+50) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-4 15.0 ± 0% 36.6 ±15% +144.00% (p=0.000 n=50+50) GetStripedOffsets/length100,seed777-4 104 ± 5% 223 ± 7% +113.13% (p=0.000 n=50+49) GetStripedOffsetsEven/length10-4 12.0 ± 0% 39.0 ± 0% +225.00% (p=0.000 n=50+50) GetStripedOffsetsEven/length100-4 72.0 ± 0% 225.0 ± 0% +212.50% (p=0.000 n=50+50) GetStripedOffsetsEven/length1000-4 472 ± 0% 2031 ± 0% +330.30% (p=0.000 n=50+50) --- cmd/options.go | 3 +- cmd/run.go | 2 +- core/local/local.go | 7 +- lib/execution.go | 6 + lib/execution_segment.go | 303 +++++++++++++++------ lib/execution_segment_test.go | 291 +++++++++++++++++--- lib/executor/common_test.go | 4 +- lib/executor/constant_arrival_rate.go | 45 +-- lib/executor/constant_arrival_rate_test.go | 50 ++-- lib/executor/constant_looping_vus.go | 19 +- lib/executor/executors_test.go | 68 +++-- lib/executor/externally_controlled.go | 10 +- lib/executor/per_vu_iterations.go | 19 +- lib/executor/shared_iterations.go | 25 +- lib/executor/variable_arrival_rate.go | 38 +-- lib/executor/variable_arrival_rate_test.go | 12 +- lib/executor/variable_looping_vus.go | 20 +- lib/executors.go | 10 +- lib/options.go | 27 +- 19 files changed, 674 insertions(+), 285 deletions(-) diff --git a/cmd/options.go b/cmd/options.go index 461c0451e78..f85cb9dadf0 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -163,8 +163,9 @@ func getOptions(flags *pflag.FlagSet) (lib.Options, error) { if err != nil { return opts, err } - opts.ESS = segmentSequence + opts.ExecutionSegmentSequence = segmentSequence } + if flags.Changed("system-tags") { systemTagList, err := flags.GetStringSlice("system-tags") if err != nil { diff --git a/cmd/run.go b/cmd/run.go index 29a9a345051..a307a1b14ce 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -241,7 +241,7 @@ a commandline interface for interacting with it.`, ) for _, ec := range executorConfigs { fprintf(stdout, " * %s: %s\n", - ec.GetName(), ec.GetDescription(conf.ExecutionSegment)) + ec.GetName(), ec.GetDescription(execScheduler.GetState().ExecutionTuple)) } fprintf(stdout, "\n") } diff --git a/core/local/local.go b/core/local/local.go index 7fecc4651eb..e990ae35c3a 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -59,8 +59,9 @@ var _ lib.ExecutionScheduler = &ExecutionScheduler{} // doesn't initialize the executors and it doesn't initialize or run VUs. func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*ExecutionScheduler, error) { options := runner.GetOptions() - - executionPlan := options.Execution.GetFullExecutionRequirements(options.ExecutionSegment) + // TODO figure out a way to give it to executionStage that is not terrible + et := lib.NewExecutionTuple(options.ExecutionSegment, options.ExecutionSegmentSequence) + executionPlan := options.Execution.GetFullExecutionRequirements(et) maxPlannedVUs := lib.GetMaxPlannedVUs(executionPlan) maxPossibleVUs := lib.GetMaxPossibleVUs(executionPlan) @@ -71,7 +72,7 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution executors := make([]lib.Executor, 0, len(executorConfigs)) // Only take executors which have work. for _, sc := range executorConfigs { - if !sc.HasWork(options.ExecutionSegment) { + if !sc.HasWork(et) { logger.Warnf( "Executor '%s' is disabled for segment %s due to lack of work!", sc.GetName(), options.ExecutionSegment, diff --git a/lib/execution.go b/lib/execution.go index c8278188325..1c11bbc5df3 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -150,6 +150,8 @@ type ExecutionState struct { // via the Go type system... Options Options + ExecutionTuple *ExecutionTuple // TODO Rename, possibly move + // vus is the shared channel buffer that contains all of the VUs that have // been initialized and aren't currently being used by a executor. // @@ -293,6 +295,10 @@ func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *E pauseStateLock: sync.RWMutex{}, totalPausedDuration: 0, // Accessed only behind the pauseStateLock resumeNotify: resumeNotify, + ExecutionTuple: NewExecutionTuple( + options.ExecutionSegment, + options.ExecutionSegmentSequence, + ), } } diff --git a/lib/execution_segment.go b/lib/execution_segment.go index b2a9efd4e73..11c5c50d7d1 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -26,6 +26,7 @@ import ( "math/big" "sort" "strings" + "sync" ) // ExecutionSegment represents a (start, end] partition of the total execution @@ -405,7 +406,7 @@ func (ess ExecutionSegmentSequence) lcd() int64 { if acc == n || acc%n == 0 { // short circuit continue } - acc = acc * n / gcd(acc, n) + acc *= (n / gcd(acc, n)) } return acc @@ -424,122 +425,246 @@ func gcd(a, b int64) int64 { return a } -// GetStripedOffsets returns everything that you need in order to execute only -// the iterations that belong to the supplied segment... -// -// TODO: add a more detailed algorithm description -// TODO: basically https://docs.google.com/spreadsheets/d/1V_ivN2xuaMJIgOf1HkpOw1ex8QOhxp960itGGiRrNzo/edit -func (ess *ExecutionSegmentSequence) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64, error) { - if segment == nil || segment.length.Cmp(oneRat) == 0 { - return 0, []int64{1}, 1, nil - } - - // we will copy the sequnce to this in order to sort it :) - var copyESS ExecutionSegmentSequence - // Here we fix the problem with having no sequence - // No filling up is required as the algorithm will accommodate for it - // through just going through the iterations that need to be in the values will fill up - // this has the consequence that if this is ran without sequence, - // but with segments: 0:1/3 and 1/3:2/3 it will get the same results instead - // of 1/3:2/3 to get start=1 and offset={3} it will get as 0:1/3 will start=0 and offsets={3} - // if the above behaviour is desired this will definitely need to be outside of this function. - if ess == nil || len(*ess) == 0 { - copyESS = []*ExecutionSegment{segment} - } else { - copyESS = append([]*ExecutionSegment{}, *ess...) // copy the original sequence - } - var wrapper = newWrapper(copyESS) - - var segmentIndex = wrapper.indexOf(segment) - if segmentIndex == -1 { - return -1, nil, -1, fmt.Errorf("missing segment %s inside segment sequence %s", segment, ess) +type sortInterfaceWrapper struct { // TODO: rename ? delete ? and replace ? + slice []struct { // TODO better name ? maybe a type of it's own ? + numerator int64 + originalIndex int } - start, offsets := wrapper.strippedOffsetsFor(segmentIndex) - return start, offsets, wrapper.lcd, nil -} - -// This is only needed in order to sort all three at the same time -type sortInterfaceWrapper struct { // TODO: rename ? - ess ExecutionSegmentSequence - numerators []int64 - lcd int64 + lcd int64 } func newWrapper(ess ExecutionSegmentSequence) sortInterfaceWrapper { var result = sortInterfaceWrapper{ - ess: ess, - numerators: make([]int64, len(ess)), - lcd: ess.lcd(), + slice: make([]struct { + numerator int64 + originalIndex int + }, len(ess)), + lcd: ess.lcd(), } for i := range ess { - result.numerators[i] = ess[i].length.Num().Int64() * (result.lcd / ess[i].length.Denom().Int64()) + result.slice[i].numerator = ess[i].length.Num().Int64() * (result.lcd / ess[i].length.Denom().Int64()) + result.slice[i].originalIndex = i } - sort.Stable(result) + sort.SliceStable(result.slice, func(i, j int) bool { + return result.slice[i].numerator > result.slice[j].numerator + }) return result } -func (e sortInterfaceWrapper) indexOf(segment *ExecutionSegment) int { - for i, seg := range e.ess { - if seg.Equal(segment) { - return i - } - } - - return -1 -} - -func (e sortInterfaceWrapper) strippedOffsetsFor(segmentIndex int) (int64, []int64) { - var offsets = make([]int64, 0, e.numerators[segmentIndex]+1) - var chosenCounts = make([]int64, len(e.ess)) - // Here instead of calculating steps which need to be big.Rat, we use the fact that - // the steps are always the length of the segment inverted which also is lcd/numerator - // So instead of creating and adding up big.Rat we just multiply the step by the amount - // of times given segment has been chosen which is count * lcd / numerator and use that - // this both saves on a lot of big.Rat allocations and also on a lot of unneeded calculations - // with them. - +// Imagine you have a number of rational numbers which all add up to 1 (or less) and call them +// segments. +// If you want each to get proportional amount of anything you need to give them their numerator +// count of elements for each denominator amount from the original elements. So for 1/3 you give 1 +// element for each 3 elements. For 3/5 - 3 elements for each 5. +// If you have for example a sequence of with element with length 3/5 and 1/3 in order to know how +// to distribute it accurately you need to get the LCD(lowest common denominitor) in this case +// between 3 and 5 this is 15 and then to transform the numbers to have the same, LCD equal, +// denominator. So 3/5 becomes 9/15 and 1/3 becomes 5/15. So now for each 15 elements 9 need to go +// to the 3/5, and 5 need to go to 1/3. +// +// We use the below algorithm to split elements between ExecutionSegments by using their length as +// the rational number. As we would like to get non sequential elements we try to get the maximum +// distance between them. That is the number of elements divided by the number of elements for any +// given segment, which concidently is the length of the segment reversed. +// The algorithm below does the following: +// 1. Goes through the elements from 0 to the lcd-1 +// 2. For each of element goes through the segments and looks if the amount of already taken +// elements by the given segment multiplied by that segment length inverted is equal to or less to +// the current element index. if it is give that element to that segment if not continue with the +// next element. +// +// The code below specifically avoids using big.Rat which complicates the code somewhat. +// As additional note the sorting of the segments from biggest to smallest helps with the fact that +// the biggest elements will need to take the most elements and for them it will be the hardest to +// not get sequential elements. +func (e sortInterfaceWrapper) stripingAlgorithm(saveIndex func(iteration int64, index int, numerator int64) bool) { + var chosenCounts = make([]int64, len(e.slice)) + +outer: for i := int64(0); i < e.lcd; i++ { for index, chosenCount := range chosenCounts { num := chosenCount * e.lcd - denom := e.numerators[index] + denom := e.slice[index].numerator if i > num/denom || (i == num/denom && num%denom == 0) { chosenCounts[index]++ - if index == segmentIndex { - prev := int64(0) - if len(offsets) > 0 { - prev = offsets[len(offsets)-1] - } - offsets = append(offsets, i-prev) - if int64(len(offsets)) == e.numerators[index] { - offsets = append(offsets, offsets[0]+e.lcd-i) - return offsets[0], offsets[1:] - } + if saveIndex(i, e.slice[index].originalIndex, denom) { + break outer } break } } } +} + +// ExecutionTuple is here to represent the combination of ExecutionSegmentSequence and +// ExecutionSegment and to give easy access to a couple of algorithms based on them in a way that is +// somewhat perfomant for which it generally needs to cache the results +type ExecutionTuple struct { // TODO rename + ES *ExecutionSegment // TODO unexport this as well? + + // TODO: have the index of the segment, cached? + sequence ExecutionSegmentSequence + offsetsCache [][]int64 + lcd int64 + // TODO discuss if we just don't want to fillCache in the constructor and not need to use pointer receivers everywhere + once *sync.Once +} + +func fillSequence(sequence ExecutionSegmentSequence) ExecutionSegmentSequence { + // TODO: discuss if we want to get the lcd of the sequence and fill with it elements of length 1/lcd ? + if sequence[0].from.Cmp(zeroRat) != 0 { + es, err := NewExecutionSegment(zeroRat, sequence[0].from) + if err != nil { + panic(err) // this really can't happen + } + + sequence = append(ExecutionSegmentSequence{es}, sequence...) + } + + if sequence[len(sequence)-1].to.Cmp(oneRat) != 0 { + es, err := NewExecutionSegment(sequence[len(sequence)-1].to, oneRat) + if err != nil { + panic(err) // this really can't happen + } + + sequence = append(sequence, es) + } + return sequence +} + +// NewExecutionTuple returns a new ExecutionTuple for the provided segment and sequence +func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequence) *ExecutionTuple { + if segment == nil { // TODO: try to do something better, maybe have bool flag in the ExecutionTuple or something + // this is needed in order to know that a segment == nil means that after + // GetNewExecutionTupleBasedOnValues the original segment scaled to 0 length one and as such + // should it be used it should always get 0 as values + segment, _ = NewExecutionSegmentFromString("0:1") + } + et := ExecutionTuple{ + once: new(sync.Once), + ES: segment, + } + if sequence == nil || len(*sequence) == 0 { + if segment == nil || segment.length.Cmp(oneRat) == 0 { + et.sequence = ExecutionSegmentSequence{segment} + } else { + et.sequence = fillSequence(ExecutionSegmentSequence{segment}) + } + } else { + et.sequence = fillSequence(*sequence) + } + return &et +} + +func (et *ExecutionTuple) find(segment *ExecutionSegment) int { + index := sort.Search(len(et.sequence), func(i int) bool { + return et.sequence[i].from.Cmp(segment.from) >= 0 + }) - // TODO return some error if we get to here - return offsets[0], offsets[1:] + if index < 0 || index >= len(et.sequence) || !et.sequence[index].Equal(segment) { + return -1 + } + return index } -// Len is the number of elements in the collection. -func (e sortInterfaceWrapper) Len() int { - return len(e.numerators) +// ScaleInt64 scales the provided value based on the ExecutionTuple +func (et *ExecutionTuple) ScaleInt64(value int64) int64 { + return et.scaleInt64With(value, et.ES) } -// Less reports whether the element with -// index i should sort before the element with index j. -func (e sortInterfaceWrapper) Less(i, j int) bool { - // Yes this Less is actually More, but we want it sorted in descending order - return e.numerators[i] > e.numerators[j] +// scaleInt64With scales the provided value based on the ExecutionTuples' +// sequence and the segment provided +func (et *ExecutionTuple) scaleInt64With(value int64, es *ExecutionSegment) int64 { + if es == nil { + return 0 + } + start, offsets, lcd := et.GetStripedOffsets(es) + return scaleInt64(value, start, offsets, lcd) +} + +func scaleInt64(value, start int64, offsets []int64, lcd int64) int64 { + endValue := (value / lcd) * int64(len(offsets)) + for gi, i := 0, start; i < value%lcd; gi, i = gi+1, i+offsets[gi] { + endValue++ + } + return endValue } -// Swap swaps the elements with indexes i and j. -func (e sortInterfaceWrapper) Swap(i, j int) { - e.numerators[i], e.numerators[j] = e.numerators[j], e.numerators[i] - e.ess[i], e.ess[j] = e.ess[j], e.ess[i] +func (et *ExecutionTuple) fillCache() { + var wrapper = newWrapper(et.sequence) + + et.offsetsCache = make([][]int64, len(et.sequence)) + for i := range et.offsetsCache { + et.offsetsCache[i] = make([]int64, 0, wrapper.slice[i].numerator) + } + + var prev = make([]int64, len(et.sequence)) + var saveIndex = func(iteration int64, index int, numerator int64) bool { + et.offsetsCache[index] = append(et.offsetsCache[index], iteration-prev[index]) + prev[index] = iteration + if int64(len(et.offsetsCache[index])) == numerator { + et.offsetsCache[index] = append(et.offsetsCache[index], et.offsetsCache[index][0]+wrapper.lcd-iteration) + } + return false + } + + wrapper.stripingAlgorithm(saveIndex) + et.lcd = wrapper.lcd +} + +// GetStripedOffsets returns the stripped offsets for the given segment +// the returned values are as follows in order: +// - start: the first value that is for the segment +// - offsets: a list of offsets from the previous value for the segment. This are only the offsets +// to from the start to the next start if we chunk the elements we are going to strip +// into lcd sized chunks +// - lcd: the LCD of the lengths of all segments in the sequence. This is also the number of +// elements after which the algorithm starts to loop and give the same values +func (et *ExecutionTuple) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64) { + et.once.Do(et.fillCache) + index := et.find(segment) + if index == -1 { + return -1, nil, et.lcd + } + offsets := et.offsetsCache[index] + return offsets[0], offsets[1:], et.lcd +} + +// GetNewExecutionTupleBasedOnValue uses the value provided, splits it using the striping offsets +// between all the segments in the sequence and returns a new ExecutionTuple with a new sequence and +// segments, such that each new segment in the new sequence has length `Scale(value)/value` while +// keeping the order. The main segment in the new ExecutionTuple is the correspoding one from the +// original, if that segmetn would've been with length 0 then it is nil, and obviously isn't part of +// the sequence. +func (et *ExecutionTuple) GetNewExecutionTupleBasedOnValue(value int64) *ExecutionTuple { + var ( + newESS = make(ExecutionSegmentSequence, 0, len(et.sequence)) // this can be smaller + newES *ExecutionSegment + ) + et.once.Do(et.fillCache) + var prev int64 + for i, es := range et.sequence { + offsets := et.offsetsCache[i] + newValue := scaleInt64(value, offsets[0], offsets[1:], et.lcd) + // TODO optimize this, somewhat + if newValue == 0 { + continue + } + var currentES, err = NewExecutionSegmentFromString(fmt.Sprintf("%d/%d:%d/%d", prev, value, prev+newValue, value)) + if err != nil { + panic(err) // TODO this really can't happen but during the optimization it will probably disappear + } + prev += newValue + if es.Equal(et.ES) { + newES = currentES + } + newESS = append(newESS, currentES) + } + return &ExecutionTuple{ + ES: newES, // in case newES is nil we want to keep it that way + sequence: newESS, + once: new(sync.Once), + } } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 568421b0b99..760212f1a69 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -188,18 +188,60 @@ func TestExecutionSegmentSplit(t *testing.T) { assert.Equal(t, "7/16:1/2", segments[3].String()) } -func TestExecutionSegmentScale(t *testing.T) { +func TestExecutionSegmentFailures(t *testing.T) { t.Parallel() es := new(ExecutionSegment) - require.NoError(t, es.UnmarshalText([]byte("0.5"))) + require.NoError(t, es.UnmarshalText([]byte("0:0.25"))) require.Equal(t, int64(1), es.Scale(2)) - require.Equal(t, int64(2), es.Scale(3)) + require.Equal(t, int64(1), es.Scale(3)) - require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) + require.NoError(t, es.UnmarshalText([]byte("0.25:0.5"))) + require.Equal(t, int64(0), es.Scale(2)) + require.Equal(t, int64(1), es.Scale(3)) + + require.NoError(t, es.UnmarshalText([]byte("0.5:0.75"))) require.Equal(t, int64(1), es.Scale(2)) + require.Equal(t, int64(0), es.Scale(3)) + + require.NoError(t, es.UnmarshalText([]byte("0.75:1"))) + require.Equal(t, int64(0), es.Scale(2)) require.Equal(t, int64(1), es.Scale(3)) } +func TestExecutionTupleScale(t *testing.T) { + t.Parallel() + es := new(ExecutionSegment) + require.NoError(t, es.UnmarshalText([]byte("0.5"))) + et := NewExecutionTuple(es, nil) + require.Equal(t, int64(1), et.ScaleInt64(2)) + require.Equal(t, int64(2), et.ScaleInt64(3)) + + require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) + et = NewExecutionTuple(es, nil) + require.Equal(t, int64(1), et.ScaleInt64(2)) + require.Equal(t, int64(1), et.ScaleInt64(3)) + + ess, err := NewExecutionSegmentSequenceFromString("0,0.5,1") + require.NoError(t, err) + require.NoError(t, es.UnmarshalText([]byte("0.5"))) + et = NewExecutionTuple(es, &ess) + require.Equal(t, int64(1), et.ScaleInt64(2)) + require.Equal(t, int64(2), et.ScaleInt64(3)) + + require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) + et = NewExecutionTuple(es, &ess) + require.Equal(t, int64(1), et.ScaleInt64(2)) + require.Equal(t, int64(1), et.ScaleInt64(3)) +} +func TestBigScale(t *testing.T) { + es := new(ExecutionSegment) + ess, err := NewExecutionSegmentSequenceFromString("0,7/20,7/10,1") + require.NoError(t, err) + require.NoError(t, es.UnmarshalText([]byte("0:7/20"))) + et := NewExecutionTuple(es, &ess) + require.Equal(t, int64(18), et.ScaleInt64(50)) +} + func TestExecutionSegmentCopyScaleRat(t *testing.T) { t.Parallel() es := new(ExecutionSegment) @@ -437,15 +479,13 @@ func TestExecutionSegmentScaleConsistency(t *testing.T) { func TestGetStripedOffsets(t *testing.T) { t.Parallel() testCases := []struct { - seq string - seg string - start int64 - offsets []int64 - lcd int64 - expError string + seq string + seg string + start int64 + offsets []int64 + lcd int64 }{ // full sequences - {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", expError: "missing segment"}, {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3", start: 0, offsets: []int64{4, 3, 3}, lcd: 10}, {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.3:0.5", start: 1, offsets: []int64{4, 6}, lcd: 10}, {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0.5:0.6", start: 2, offsets: []int64{10}, lcd: 10}, @@ -456,19 +496,19 @@ func TestGetStripedOffsets(t *testing.T) { {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.2", start: 1, offsets: []int64{4, 6}, lcd: 10}, {seq: "0,0.2,0.5,0.6,0.7,0.8,0.9,1", seg: "0.6:0.7", start: 3, offsets: []int64{10}, lcd: 10}, // not full sequences - {seq: "0,0.2,0.5", seg: "0:0.2", start: 1, offsets: []int64{4, 6}, lcd: 10}, - {seq: "0,0.2,0.5", seg: "0.2:0.5", start: 0, offsets: []int64{4, 3, 3}, lcd: 10}, + {seq: "0,0.2,0.5", seg: "0:0.2", start: 3, offsets: []int64{6, 4}, lcd: 10}, + {seq: "0,0.2,0.5", seg: "0.2:0.5", start: 1, offsets: []int64{4, 2, 4}, lcd: 10}, {seq: "0,2/5,4/5", seg: "0:2/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, {seq: "0,2/5,4/5", seg: "2/5:4/5", start: 1, offsets: []int64{3, 2}, lcd: 5}, // no sequence - {seg: "0:0.2", start: 0, offsets: []int64{5}, lcd: 5}, - {seg: "0:1/5", start: 0, offsets: []int64{5}, lcd: 5}, - {seg: "0:2/10", start: 0, offsets: []int64{5}, lcd: 5}, - {seg: "0:0.4", start: 0, offsets: []int64{3, 2}, lcd: 5}, - {seg: "0:2/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, - {seg: "2/5:4/5", start: 0, offsets: []int64{3, 2}, lcd: 5}, // this is the same as the previous one as there is no sequence - {seg: "0:4/10", start: 0, offsets: []int64{3, 2}, lcd: 5}, - {seg: "1/10:5/10", start: 0, offsets: []int64{3, 2}, lcd: 5}, + {seg: "0:0.2", start: 1, offsets: []int64{5}, lcd: 5}, + {seg: "0:1/5", start: 1, offsets: []int64{5}, lcd: 5}, + {seg: "0:2/10", start: 1, offsets: []int64{5}, lcd: 5}, + {seg: "0:0.4", start: 1, offsets: []int64{2, 3}, lcd: 5}, + {seg: "0:2/5", start: 1, offsets: []int64{2, 3}, lcd: 5}, + {seg: "2/5:4/5", start: 1, offsets: []int64{3, 2}, lcd: 5}, + {seg: "0:4/10", start: 1, offsets: []int64{2, 3}, lcd: 5}, + {seg: "1/10:5/10", start: 1, offsets: []int64{2, 2, 4, 2}, lcd: 10}, } for _, tc := range testCases { @@ -478,13 +518,8 @@ func TestGetStripedOffsets(t *testing.T) { require.NoError(t, err) segment, err := NewExecutionSegmentFromString(tc.seg) require.NoError(t, err) - start, offsets, lcd, err := ess.GetStripedOffsets(segment) - if len(tc.expError) != 0 { - require.Error(t, err, tc.expError) - require.Contains(t, err.Error(), tc.expError) - return - } - require.NoError(t, err) + et := NewExecutionTuple(segment, &ess) + start, offsets, lcd := et.GetStripedOffsets(segment) assert.Equal(t, tc.start, start) assert.Equal(t, tc.offsets, offsets) @@ -492,7 +527,7 @@ func TestGetStripedOffsets(t *testing.T) { ess2, err := NewExecutionSegmentSequenceFromString(tc.seq) require.NoError(t, err) - assert.Equal(t, ess, ess2) + assert.Equal(t, ess.String(), ess2.String()) }) } } @@ -527,11 +562,13 @@ func BenchmarkGetStripedOffsets(b *testing.B) { for _, length := range lengths { length := length b.Run(fmt.Sprintf("length%d,seed%d", length, seed), func(b *testing.B) { - sequence := generateRandomSequence(length, r) + sequence, err := generateRandomSequence(length, r) + require.NoError(b, err) b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, _, err := sequence.GetStripedOffsets(sequence[int(r.Int63())%len(sequence)]) - require.NoError(b, err) + segment := sequence[int(r.Int63())%len(sequence)] + et := NewExecutionTuple(segment, &sequence) + _, _, _ = et.GetStripedOffsets(segment) } }) } @@ -564,8 +601,194 @@ func BenchmarkGetStripedOffsetsEven(b *testing.B) { sequence := generateSequence(length) b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, _, err := sequence.GetStripedOffsets(sequence[111233%len(sequence)]) - require.NoError(b, err) + segment := sequence[111233%len(sequence)] + et := NewExecutionTuple(segment, &sequence) + _, _, _ = et.GetStripedOffsets(segment) + } + }) + } +} + +func TestGetNewExecutionTupleBesedOnValue(t *testing.T) { + t.Parallel() + + testCases := []struct { + seq string + seg string + value int64 + expected string + }{ + // full sequences + {seq: "0,1/3,2/3,1", seg: "0:1/3", value: 20, expected: "0,7/20,7/10,1"}, + } + + for _, tc := range testCases { + tc := tc + t.Run(fmt.Sprintf("seq:%s;segment:%s", tc.seq, tc.seg), func(t *testing.T) { + ess, err := NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(t, err) + + segment, err := NewExecutionSegmentFromString(tc.seg) + require.NoError(t, err) + + et := NewExecutionTuple(segment, &ess) + newET := et.GetNewExecutionTupleBasedOnValue(tc.value) + require.Equal(t, tc.expected, newET.sequence.String()) + }) + } +} + +func mustNewExecutionSegment(str string) *ExecutionSegment { + res, err := NewExecutionSegmentFromString(str) + if err != nil { + panic(err) + } + return res +} + +func mustNewExecutionSegmentSequence(str string) *ExecutionSegmentSequence { + res, err := NewExecutionSegmentSequenceFromString(str) + if err != nil { + panic(err) + } + return &res +} + +func TestNewExecutionTuple(t *testing.T) { + var testCases = []struct { + seg *ExecutionSegment + seq *ExecutionSegmentSequence + scaleTests map[int64]int64 + newScaleTests map[int64]map[int64]int64 // this is for after calling GetNewExecutionTupleBasedOnValue + }{ + { + // both segment and sequence are nil + scaleTests: map[int64]int64{ + 50: 50, + 1: 1, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 50, 1: 1, 0: 0}, + 1: {50: 50, 1: 1, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("0:1"), + // nil sequence + scaleTests: map[int64]int64{ + 50: 50, + 1: 1, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 50, 1: 1, 0: 0}, + 1: {50: 50, 1: 1, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("0:1"), + seq: mustNewExecutionSegmentSequence("0,1"), + scaleTests: map[int64]int64{ + 50: 50, + 1: 1, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 50, 1: 1, 0: 0}, + 1: {50: 50, 1: 1, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("0:1"), + seq: mustNewExecutionSegmentSequence(""), + scaleTests: map[int64]int64{ + 50: 50, + 1: 1, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 50, 1: 1, 0: 0}, + 1: {50: 50, 1: 1, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("0:1/3"), + seq: mustNewExecutionSegmentSequence("0,1/3,2/3,1"), + scaleTests: map[int64]int64{ + 50: 17, + 3: 1, + 2: 1, + 1: 1, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 17, 1: 1, 0: 0}, + 20: {50: 18, 1: 1, 0: 0}, + 3: {50: 17, 1: 1, 0: 0}, + 2: {50: 25, 1: 1, 0: 0}, + 1: {50: 50, 1: 1, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("1/3:2/3"), + seq: mustNewExecutionSegmentSequence("0,1/3,2/3,1"), + scaleTests: map[int64]int64{ + 50: 17, + 3: 1, + 2: 1, + 1: 0, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 17, 1: 0, 0: 0}, + 20: {50: 17, 1: 0, 0: 0}, + 3: {50: 17, 1: 0, 0: 0}, + 2: {50: 25, 1: 0, 0: 0}, + 1: {50: 0, 1: 0, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + { + seg: mustNewExecutionSegment("2/3:1"), + seq: mustNewExecutionSegmentSequence("0,1/3,2/3,1"), + scaleTests: map[int64]int64{ + 50: 16, + 3: 1, + 2: 0, + 1: 0, + 0: 0, + }, + newScaleTests: map[int64]map[int64]int64{ + 50: {50: 16, 1: 0, 0: 0}, + 20: {50: 15, 1: 0, 0: 0}, + 3: {50: 16, 1: 0, 0: 0}, + 2: {50: 0, 1: 0, 0: 0}, + 1: {50: 0, 1: 0, 0: 0}, + 0: {50: 0, 1: 0, 0: 0}, + }, + }, + } + + for _, testCase := range testCases { + testCase := testCase + t.Run(fmt.Sprintf("seg:'%s',seq:'%s'", testCase.seg, testCase.seq), func(t *testing.T) { + et := NewExecutionTuple(testCase.seg, testCase.seq) + for scaleValue, result := range testCase.scaleTests { + require.Equal(t, result, et.ScaleInt64(scaleValue), "%d->%d", scaleValue, result) + } + + for value, newResult := range testCase.newScaleTests { + newET := et.GetNewExecutionTupleBasedOnValue(value) + for scaleValue, result := range newResult { + require.Equal(t, result, newET.ScaleInt64(scaleValue), + "getNewExecutionTupleBasedOnValue(%d)%d->%d", value, scaleValue, result) + } } }) } diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 0053606a9dd..95e48583589 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -58,8 +58,8 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta return runner.NewVU(engineOut) }) - segment := es.Options.ExecutionSegment - maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(segment)) + et := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) + maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(et)) initializeVUs(ctx, t, logEntry, es, maxVUs) executor, err := config.NewExecutor(es, logEntry) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 87be88e5eb1..58e4024d799 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -76,26 +76,31 @@ func NewConstantArrivalRateConfig(name string) ConstantArrivalRateConfig { var _ lib.ExecutorConfig = &ConstantArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. -func (carc ConstantArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(carc.PreAllocatedVUs.Int64) +func (carc ConstantArrivalRateConfig) GetPreAllocatedVUs(et *lib.ExecutionTuple) int64 { + return et.ScaleInt64(carc.PreAllocatedVUs.Int64) } // GetMaxVUs is just a helper method that returns the scaled max VUs. -func (carc ConstantArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(carc.MaxVUs.Int64) +func (carc ConstantArrivalRateConfig) GetMaxVUs(et *lib.ExecutionTuple) int64 { + return et.ScaleInt64(carc.MaxVUs.Int64) } // GetDescription returns a human-readable description of the executor options -func (carc ConstantArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { - preAllocatedVUs, maxVUs := carc.GetPreAllocatedVUs(es), carc.GetMaxVUs(es) +func (carc ConstantArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) string { + preAllocatedVUs, maxVUs := carc.GetPreAllocatedVUs(et), carc.GetMaxVUs(et) maxVUsRange := fmt.Sprintf("maxVUs: %d", preAllocatedVUs) if maxVUs > preAllocatedVUs { maxVUsRange += fmt.Sprintf("-%d", maxVUs) } timeUnit := time.Duration(carc.TimeUnit.Duration) - arrRate := getScaledArrivalRate(es, carc.Rate.Int64, timeUnit) - arrRatePerSec, _ := getArrivalRatePerSec(arrRate).Float64() + var arrRatePerSec float64 + if maxVUs != 0 { // TODO: do something better? + ratio := big.NewRat(maxVUs, carc.MaxVUs.Int64) + arrRate := big.NewRat(carc.Rate.Int64, int64(timeUnit)) + arrRate.Mul(arrRate, ratio) + arrRatePerSec, _ = getArrivalRatePerSec(arrRate).Float64() + } return fmt.Sprintf("%.2f iterations/s for %s%s", arrRatePerSec, carc.Duration.Duration, carc.getBaseInfo(maxVUsRange)) @@ -142,12 +147,12 @@ func (carc ConstantArrivalRateConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (carc ConstantArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (carc ConstantArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(es.Scale(carc.PreAllocatedVUs.Int64)), - MaxUnplannedVUs: uint64(es.Scale(carc.MaxVUs.Int64 - carc.PreAllocatedVUs.Int64)), + PlannedVUs: uint64(et.ScaleInt64(carc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(et.ScaleInt64(carc.MaxVUs.Int64 - carc.PreAllocatedVUs.Int64)), }, { TimeOffset: time.Duration(carc.Duration.Duration + carc.GracefulStop.Duration), PlannedVUs: 0, @@ -167,8 +172,8 @@ func (carc ConstantArrivalRateConfig) NewExecutor( } // HasWork reports whether there is any work to be done for the given execution segment. -func (carc ConstantArrivalRateConfig) HasWork(es *lib.ExecutionSegment) bool { - return carc.GetMaxVUs(es) > 0 +func (carc ConstantArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { + return carc.GetMaxVUs(et) > 0 } // ConstantArrivalRate tries to execute a specific number of iterations for a @@ -185,13 +190,12 @@ var _ lib.Executor = &ConstantArrivalRate{} // // TODO: Reuse the variable arrival rate method? func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen - segment := car.executionState.Options.ExecutionSegment + newET := car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(car.config.MaxVUs.Int64) gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) - preAllocatedVUs := car.config.GetPreAllocatedVUs(segment) - maxVUs := car.config.GetMaxVUs(segment) - - arrivalRate := getScaledArrivalRate(segment, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) + preAllocatedVUs := car.config.GetPreAllocatedVUs(car.executionState.ExecutionTuple) + maxVUs := car.config.GetMaxVUs(car.executionState.ExecutionTuple) + arrivalRate := getScaledArrivalRate(newET.ES, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() @@ -260,10 +264,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs - start, offsets, _, err := car.executionState.Options.ESS.GetStripedOffsets(segment) - if err != nil { - return err - } + start, offsets, _ := newET.GetStripedOffsets(newET.ES) startTime = time.Now() timer := time.NewTimer(time.Hour * 24) // here the we need the not scaled one diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index b014d9d255e..47df943dc70 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -121,52 +121,52 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { segment *lib.ExecutionSegment sequence *lib.ExecutionSegmentSequence start time.Duration - step time.Duration + steps []int64 }{ { segment: newExecutionSegmentFromString("0:1/3"), - start: time.Millisecond * 00, - step: time.Millisecond * 60, + start: time.Millisecond * 20, + steps: []int64{40, 60, 60, 60, 60, 60, 60}, }, { segment: newExecutionSegmentFromString("1/3:2/3"), - start: time.Millisecond * 00, - step: time.Millisecond * 60, + start: time.Millisecond * 20, + steps: []int64{60, 60, 60, 60, 60, 60, 40}, }, { segment: newExecutionSegmentFromString("2/3:1"), - start: time.Millisecond * 00, - step: time.Millisecond * 60, + start: time.Millisecond * 20, + steps: []int64{40, 60, 60, 60, 60, 60, 60}, }, { segment: newExecutionSegmentFromString("1/6:3/6"), - start: time.Millisecond * 00, - step: time.Millisecond * 60, + start: time.Millisecond * 20, + steps: []int64{40, 80, 40, 80, 40, 80, 40}, }, { segment: newExecutionSegmentFromString("1/6:3/6"), - sequence: &lib.ExecutionSegmentSequence{}, - start: time.Millisecond * 00, - step: time.Millisecond * 60, + sequence: newExecutionSegmentSequenceFromString("1/6,3/6"), + start: time.Millisecond * 20, + steps: []int64{40, 80, 40, 80, 40, 80, 40}, }, // sequences { segment: newExecutionSegmentFromString("0:1/3"), sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), start: time.Millisecond * 00, - step: time.Millisecond * 60, + steps: []int64{60, 60, 60, 60, 60, 60, 40}, }, { segment: newExecutionSegmentFromString("1/3:2/3"), sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), start: time.Millisecond * 20, - step: time.Millisecond * 60, + steps: []int64{60, 60, 60, 60, 60, 60, 40}, }, { segment: newExecutionSegmentFromString("2/3:1"), sequence: newExecutionSegmentSequenceFromString("0,1/3,2/3,1"), start: time.Millisecond * 40, - step: time.Millisecond * 60, + steps: []int64{60, 60, 60, 60, 60, 100}, }, } for _, test := range tests { @@ -175,16 +175,25 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { t.Run(fmt.Sprintf("segment %s sequence %s", test.segment, test.sequence), func(t *testing.T) { t.Parallel() es := lib.NewExecutionState(lib.Options{ - ExecutionSegment: test.segment, - ESS: test.sequence, + ExecutionSegment: test.segment, + ExecutionSegmentSequence: test.sequence, }, 10, 50) var count int64 + var config = getTestConstantArrivalRateConfig() + newET := es.ExecutionTuple.GetNewExecutionTupleBasedOnValue(config.MaxVUs.Int64) + rateScaled := newET.ScaleInt64(config.Rate.Int64) var startTime = time.Now() + var expectedTimeInt64 = int64(test.start) var ctx, cancel, executor, logHook = setupExecutor( - t, getTestConstantArrivalRateConfig(), es, + t, config, es, simpleRunner(func(ctx context.Context) error { current := atomic.AddInt64(&count, 1) - expectedTime := test.start + time.Duration(current-1)*test.step + + var expectedTime = test.start + if current != 1 { + expectedTime = time.Duration(atomic.AddInt64(&expectedTimeInt64, + int64(time.Millisecond)*test.steps[(current-2)%int64(len(test.steps))])) + } assert.WithinDuration(t, startTime.Add(expectedTime), time.Now(), @@ -195,6 +204,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { return nil }), ) + defer cancel() var wg sync.WaitGroup wg.Add(1) @@ -206,7 +216,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { for i := 0; i < 5; i++ { time.Sleep(time.Second) currentCount = atomic.LoadInt64(&count) - assert.InDelta(t, (i+1)*17, currentCount, 2) + assert.InDelta(t, int64(i+1)*rateScaled, currentCount, 3) } }() startTime = time.Now() diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index a96e971aa0f..bb3f86b8ef5 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -71,14 +71,14 @@ func NewConstantLoopingVUsConfig(name string) ConstantLoopingVUsConfig { var _ lib.ExecutorConfig = &ConstantLoopingVUsConfig{} // GetVUs returns the scaled VUs for the executor. -func (clvc ConstantLoopingVUsConfig) GetVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(clvc.VUs.Int64) +func (clvc ConstantLoopingVUsConfig) GetVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(clvc.VUs.Int64) } // GetDescription returns a human-readable description of the executor options -func (clvc ConstantLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { +func (clvc ConstantLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { return fmt.Sprintf("%d looping VUs for %s%s", - clvc.GetVUs(es), clvc.Duration.Duration, clvc.getBaseInfo()) + clvc.GetVUs(et), clvc.Duration.Duration, clvc.getBaseInfo()) } // Validate makes sure all options are configured and valid @@ -104,11 +104,11 @@ func (clvc ConstantLoopingVUsConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(clvc.GetVUs(es)), + PlannedVUs: uint64(clvc.GetVUs(et)), }, { TimeOffset: time.Duration(clvc.Duration.Duration + clvc.GracefulStop.Duration), @@ -118,8 +118,8 @@ func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionS } // HasWork reports whether there is any work to be done for the given execution segment. -func (clvc ConstantLoopingVUsConfig) HasWork(es *lib.ExecutionSegment) bool { - return clvc.GetVUs(es) > 0 +func (clvc ConstantLoopingVUsConfig) HasWork(et *lib.ExecutionTuple) bool { + return clvc.GetVUs(et) > 0 } // NewExecutor creates a new ConstantLoopingVUs executor @@ -143,8 +143,7 @@ var _ lib.Executor = &ConstantLoopingVUs{} // Run constantly loops through as many iterations as possible on a fixed number // of VUs for the specified duration. func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := clv.executionState.Options.ExecutionSegment - numVUs := clv.config.GetVUs(segment) + numVUs := clv.config.GetVUs(clv.executionState.ExecutionTuple) duration := time.Duration(clv.config.Duration.Duration) gracefulStop := clv.config.GetGracefulStop() diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 66d552b2ec0..f5117221feb 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -87,16 +87,17 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["someKey"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "10 looping VUs for 1m0s (exec: someFunc, startTime: 1m10s, gracefulStop: 10s)", cm["someKey"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "10 looping VUs for 1m0s (exec: someFunc, startTime: 1m10s, gracefulStop: 10s)", cm["someKey"].GetDescription(et)) - schedReqs := cm["someKey"].GetExecutionRequirements(nil) + schedReqs := cm["someKey"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 70*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) endOffset, isFinal = lib.GetEndOffset(totalReqs) assert.Equal(t, 140*time.Second, endOffset) assert.Equal(t, true, isFinal) @@ -134,16 +135,17 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "Up to 30 looping VUs for 3m10s over 2 stages (gracefulRampDown: 10s, startTime: 23s, gracefulStop: 15s)", cm["varloops"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "Up to 30 looping VUs for 3m10s over 2 stages (gracefulRampDown: 10s, startTime: 23s, gracefulStop: 15s)", cm["varloops"].GetDescription(et)) - schedReqs := cm["varloops"].GetExecutionRequirements(nil) + schedReqs := cm["varloops"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 205*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(30), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) endOffset, isFinal = lib.GetEndOffset(totalReqs) assert.Equal(t, 228*time.Second, endOffset) assert.Equal(t, true, isFinal) @@ -157,9 +159,10 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "Up to 10 looping VUs for 10s over 1 stages (gracefulRampDown: 10s)", cm["varloops"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "Up to 10 looping VUs for 10s over 1 stages (gracefulRampDown: 10s)", cm["varloops"].GetDescription(et)) - schedReqs := cm["varloops"].GetExecutionRequirements(nil) + schedReqs := cm["varloops"].GetExecutionRequirements(et) assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) }}, @@ -170,9 +173,10 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "Up to 10 looping VUs for 20s over 3 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "Up to 10 looping VUs for 20s over 3 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(et)) - schedReqs := cm.GetFullExecutionRequirements(nil) + schedReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, uint64(10), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) }}, @@ -183,9 +187,10 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "Up to 11 looping VUs for 20s over 4 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "Up to 11 looping VUs for 20s over 4 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(et)) - schedReqs := cm.GetFullExecutionRequirements(nil) + schedReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, uint64(11), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(11), lib.GetMaxPossibleVUs(schedReqs)) }}, @@ -209,16 +214,17 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["ishared"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "22 iterations shared among 12 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "22 iterations shared among 12 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(et)) - schedReqs := cm["ishared"].GetExecutionRequirements(nil) + schedReqs := cm["ishared"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 130*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(12), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(12), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, schedReqs, totalReqs) }}, }, @@ -242,16 +248,17 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["ipervu"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "23 iterations for each of 13 VUs (maxDuration: 10m0s)", cm["ipervu"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "23 iterations for each of 13 VUs (maxDuration: 10m0s)", cm["ipervu"].GetDescription(et)) - schedReqs := cm["ipervu"].GetExecutionRequirements(nil) + schedReqs := cm["ipervu"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 600*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(13), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(13), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, schedReqs, totalReqs) }}, }, @@ -267,6 +274,7 @@ var configMapTestCases = []configMapTestCase{ // constant-arrival-rate {`{"carrival": {"type": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + et := lib.NewExecutionTuple(nil, nil) sched := NewConstantArrivalRateConfig("carrival") sched.Rate = null.IntFrom(30) sched.Duration = types.NullDurationFrom(10 * time.Minute) @@ -277,16 +285,16 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["carrival"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "0.50 iterations/s for 10m0s (maxVUs: 20-30, gracefulStop: 30s)", cm["carrival"].GetDescription(nil)) + assert.Equal(t, "0.50 iterations/s for 10m0s (maxVUs: 20-30, gracefulStop: 30s)", cm["carrival"].GetDescription(et)) - schedReqs := cm["carrival"].GetExecutionRequirements(nil) + schedReqs := cm["carrival"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 630*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(20), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, schedReqs, totalReqs) }}, }, @@ -319,16 +327,17 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varrival"].Validate()) assert.Empty(t, cm.Validate()) - assert.Equal(t, "Up to 1.00 iterations/s for 8m0s over 2 stages (maxVUs: 20-50, gracefulStop: 30s)", cm["varrival"].GetDescription(nil)) + et := lib.NewExecutionTuple(nil, nil) + assert.Equal(t, "Up to 1.00 iterations/s for 8m0s over 2 stages (maxVUs: 20-50, gracefulStop: 30s)", cm["varrival"].GetDescription(et)) - schedReqs := cm["varrival"].GetExecutionRequirements(nil) + schedReqs := cm["varrival"].GetExecutionRequirements(et) endOffset, isFinal := lib.GetEndOffset(schedReqs) assert.Equal(t, 510*time.Second, endOffset) assert.Equal(t, true, isFinal) assert.Equal(t, uint64(20), lib.GetMaxPlannedVUs(schedReqs)) assert.Equal(t, uint64(50), lib.GetMaxPossibleVUs(schedReqs)) - totalReqs := cm.GetFullExecutionRequirements(nil) + totalReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, schedReqs, totalReqs) }}, }, @@ -373,6 +382,7 @@ func TestConfigMapParsingAndValidation(t *testing.T) { func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { t.Parallel() + et := lib.NewExecutionTuple(nil, nil) conf := NewVariableLoopingVUsConfig("test") conf.StartVUs = null.IntFrom(4) conf.Stages = []Stage{ @@ -432,7 +442,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 42 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 53 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(nil)) + }, conf.GetExecutionRequirements(et)) // Try a longer GracefulStop than the GracefulRampDown conf.GracefulStop = types.NullDurationFrom(80 * time.Second) @@ -444,7 +454,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 42 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 103 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(nil)) + }, conf.GetExecutionRequirements(et)) // Try a much shorter GracefulStop than the GracefulRampDown conf.GracefulStop = types.NullDurationFrom(3 * time.Second) @@ -453,7 +463,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(nil)) + }, conf.GetExecutionRequirements(et)) // Try a zero GracefulStop conf.GracefulStop = types.NullDurationFrom(0 * time.Second) @@ -462,9 +472,9 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, {TimeOffset: 23 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(nil)) + }, conf.GetExecutionRequirements(et)) // Try a zero GracefulStop and GracefulRampDown, i.e. raw steps with 0 end cap conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) - assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(nil)) + assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) } diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index c53e5a1d6f7..a91ea45e475 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -101,7 +101,7 @@ type ExternallyControlledConfig struct { var _ lib.ExecutorConfig = &ExternallyControlledConfig{} // GetDescription returns a human-readable description of the executor options -func (mec ExternallyControlledConfig) GetDescription(_ *lib.ExecutionSegment) string { +func (mec ExternallyControlledConfig) GetDescription(_ *lib.ExecutionTuple) string { duration := "infinite" if mec.Duration.Duration != 0 { duration = mec.Duration.String() @@ -140,11 +140,11 @@ func (mec ExternallyControlledConfig) Validate() []error { // point during a test run. That's used for sizing purposes and for user qouta // checking in the cloud execution, where the externally controlled executor // isn't supported. -func (mec ExternallyControlledConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (mec ExternallyControlledConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { startVUs := lib.ExecutionStep{ TimeOffset: 0, - PlannedVUs: uint64(es.Scale(mec.MaxVUs.Int64)), // user-configured, VUs to be pre-initialized - MaxUnplannedVUs: 0, // intentional, see function comment + PlannedVUs: uint64(et.ES.Scale(mec.MaxVUs.Int64)), // user-configured, VUs to be pre-initialized + MaxUnplannedVUs: 0, // intentional, see function comment } maxDuration := time.Duration(mec.Duration.Duration) @@ -179,7 +179,7 @@ func (mec ExternallyControlledConfig) NewExecutor(es *lib.ExecutionState, logger } // HasWork reports whether there is any work to be done for the given execution segment. -func (mec ExternallyControlledConfig) HasWork(es *lib.ExecutionSegment) bool { +func (mec ExternallyControlledConfig) HasWork(_ *lib.ExecutionTuple) bool { // We can always initialize new VUs via the REST API, so return true. return true } diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 81b2dc51f0e..30eb6f34f8a 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -68,8 +68,8 @@ func NewPerVUIterationsConfig(name string) PerVUIterationsConfig { var _ lib.ExecutorConfig = &PerVUIterationsConfig{} // GetVUs returns the scaled VUs for the executor. -func (pvic PerVUIterationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(pvic.VUs.Int64) +func (pvic PerVUIterationsConfig) GetVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(pvic.VUs.Int64) } // GetIterations returns the UNSCALED iteration count for the executor. It's @@ -81,9 +81,9 @@ func (pvic PerVUIterationsConfig) GetIterations() int64 { } // GetDescription returns a human-readable description of the executor options -func (pvic PerVUIterationsConfig) GetDescription(es *lib.ExecutionSegment) string { +func (pvic PerVUIterationsConfig) GetDescription(et *lib.ExecutionTuple) string { return fmt.Sprintf("%d iterations for each of %d VUs%s", - pvic.GetIterations(), pvic.GetVUs(es), + pvic.GetIterations(), pvic.GetVUs(et), pvic.getBaseInfo(fmt.Sprintf("maxDuration: %s", pvic.MaxDuration.Duration))) } @@ -112,11 +112,11 @@ func (pvic PerVUIterationsConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (pvic PerVUIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (pvic PerVUIterationsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(pvic.GetVUs(es)), + PlannedVUs: uint64(pvic.GetVUs(et)), }, { TimeOffset: time.Duration(pvic.MaxDuration.Duration + pvic.GracefulStop.Duration), @@ -136,8 +136,8 @@ func (pvic PerVUIterationsConfig) NewExecutor( } // HasWork reports whether there is any work to be done for the given execution segment. -func (pvic PerVUIterationsConfig) HasWork(es *lib.ExecutionSegment) bool { - return pvic.GetVUs(es) > 0 && pvic.GetIterations() > 0 +func (pvic PerVUIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { + return pvic.GetVUs(et) > 0 && pvic.GetIterations() > 0 } // PerVUIterations executes a specific number of iterations with each VU. @@ -151,8 +151,7 @@ var _ lib.Executor = &PerVUIterations{} // Run executes a specific number of iterations with each configured VU. func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := pvi.executionState.Options.ExecutionSegment - numVUs := pvi.config.GetVUs(segment) + numVUs := pvi.config.GetVUs(pvi.executionState.ExecutionTuple) iterations := pvi.config.GetIterations() duration := time.Duration(pvi.config.MaxDuration.Duration) gracefulStop := pvi.config.GetGracefulStop() diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 99a53329353..04af864484a 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -71,19 +71,19 @@ func NewSharedIterationsConfig(name string) SharedIterationsConfig { var _ lib.ExecutorConfig = &SharedIterationsConfig{} // GetVUs returns the scaled VUs for the executor. -func (sic SharedIterationsConfig) GetVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(sic.VUs.Int64) +func (sic SharedIterationsConfig) GetVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(sic.VUs.Int64) } // GetIterations returns the scaled iteration count for the executor. -func (sic SharedIterationsConfig) GetIterations(es *lib.ExecutionSegment) int64 { - return es.Scale(sic.Iterations.Int64) +func (sic SharedIterationsConfig) GetIterations(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(sic.Iterations.Int64) } // GetDescription returns a human-readable description of the executor options -func (sic SharedIterationsConfig) GetDescription(es *lib.ExecutionSegment) string { +func (sic SharedIterationsConfig) GetDescription(et *lib.ExecutionTuple) string { return fmt.Sprintf("%d iterations shared among %d VUs%s", - sic.GetIterations(es), sic.GetVUs(es), + sic.GetIterations(et), sic.GetVUs(et), sic.getBaseInfo(fmt.Sprintf("maxDuration: %s", sic.MaxDuration.Duration))) } @@ -115,11 +115,11 @@ func (sic SharedIterationsConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (sic SharedIterationsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (sic SharedIterationsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(sic.GetVUs(es)), + PlannedVUs: uint64(sic.GetVUs(et)), }, { TimeOffset: time.Duration(sic.MaxDuration.Duration + sic.GracefulStop.Duration), @@ -149,16 +149,15 @@ type SharedIterations struct { var _ lib.Executor = &SharedIterations{} // HasWork reports whether there is any work to be done for the given execution segment. -func (sic SharedIterationsConfig) HasWork(es *lib.ExecutionSegment) bool { - return sic.GetVUs(es) > 0 && sic.GetIterations(es) > 0 +func (sic SharedIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { + return sic.GetVUs(et) > 0 && sic.GetIterations(et) > 0 } // Run executes a specific total number of iterations, which are all shared by // the configured VUs. func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := si.executionState.Options.ExecutionSegment - numVUs := si.config.GetVUs(segment) - iterations := si.config.GetIterations(segment) + numVUs := si.config.GetVUs(si.executionState.ExecutionTuple) + iterations := si.config.GetIterations(si.executionState.ExecutionTuple) duration := time.Duration(si.config.MaxDuration.Duration) gracefulStop := si.config.GetGracefulStop() diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index caca2280f05..7fdd1969014 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -80,25 +80,25 @@ func NewVariableArrivalRateConfig(name string) VariableArrivalRateConfig { var _ lib.ExecutorConfig = &VariableArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. -func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(varc.PreAllocatedVUs.Int64) +func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(varc.PreAllocatedVUs.Int64) } // GetMaxVUs is just a helper method that returns the scaled max VUs. -func (varc VariableArrivalRateConfig) GetMaxVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(varc.MaxVUs.Int64) +func (varc VariableArrivalRateConfig) GetMaxVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(varc.MaxVUs.Int64) } // GetDescription returns a human-readable description of the executor options -func (varc VariableArrivalRateConfig) GetDescription(es *lib.ExecutionSegment) string { +func (varc VariableArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) string { //TODO: something better? always show iterations per second? - maxVUsRange := fmt.Sprintf("maxVUs: %d", es.Scale(varc.PreAllocatedVUs.Int64)) + maxVUsRange := fmt.Sprintf("maxVUs: %d", et.ES.Scale(varc.PreAllocatedVUs.Int64)) if varc.MaxVUs.Int64 > varc.PreAllocatedVUs.Int64 { - maxVUsRange += fmt.Sprintf("-%d", es.Scale(varc.MaxVUs.Int64)) + maxVUsRange += fmt.Sprintf("-%d", et.ES.Scale(varc.MaxVUs.Int64)) } maxUnscaledRate := getStagesUnscaledMaxTarget(varc.StartRate.Int64, varc.Stages) maxArrRatePerSec, _ := getArrivalRatePerSec( - getScaledArrivalRate(es, maxUnscaledRate, time.Duration(varc.TimeUnit.Duration)), + getScaledArrivalRate(et.ES, maxUnscaledRate, time.Duration(varc.TimeUnit.Duration)), ).Float64() return fmt.Sprintf("Up to %.2f iterations/s for %s over %d stages%s", @@ -140,12 +140,12 @@ func (varc VariableArrivalRateConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (varc VariableArrivalRateConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { +func (varc VariableArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(es.Scale(varc.PreAllocatedVUs.Int64)), - MaxUnplannedVUs: uint64(es.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), + PlannedVUs: uint64(et.ES.Scale(varc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(et.ES.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), }, { TimeOffset: sumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), @@ -167,7 +167,7 @@ type rateChange struct { } // A helper method to generate the plan how the rate changes would happen. -func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.ExecutionSegment) []rateChange { +func (varc VariableArrivalRateConfig) getPlannedRateChanges(et *lib.ExecutionTuple) []rateChange { timeUnit := time.Duration(varc.TimeUnit.Duration) // Important note for accuracy: we must work with and scale only the // rational numbers, never the raw target values directly. It matters most @@ -179,7 +179,7 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi // numbers for scaling, then the instance executing the first segment won't // ever do even a single request, since scale(20%, 1) would be 0, whereas // the rational value for scale(20%, 1/sec) is 0.2/sec, or rather 1/5sec... - currentRate := getScaledArrivalRate(segment, varc.StartRate.Int64, timeUnit) + currentRate := getScaledArrivalRate(et.ES, varc.StartRate.Int64, timeUnit) rateChanges := []rateChange{} timeFromStart := time.Duration(0) @@ -188,7 +188,7 @@ func (varc VariableArrivalRateConfig) getPlannedRateChanges(segment *lib.Executi var tArrivalRateStep = new(big.Rat) var stepCoef = new(big.Rat) for _, stage := range varc.Stages { - stageTargetRate := getScaledArrivalRate(segment, stage.Target.Int64, timeUnit) + stageTargetRate := getScaledArrivalRate(et.ES, stage.Target.Int64, timeUnit) stageDuration := time.Duration(stage.Duration.Duration) if currentRate.Cmp(stageTargetRate) == 0 { @@ -257,13 +257,13 @@ func (varc VariableArrivalRateConfig) NewExecutor( return VariableArrivalRate{ BaseExecutor: NewBaseExecutor(varc, es, logger), config: varc, - plannedRateChanges: varc.getPlannedRateChanges(es.Options.ExecutionSegment), + plannedRateChanges: varc.getPlannedRateChanges(es.ExecutionTuple), }, nil } // HasWork reports whether there is any work to be done for the given execution segment. -func (varc VariableArrivalRateConfig) HasWork(es *lib.ExecutionSegment) bool { - return varc.GetMaxVUs(es) > 0 +func (varc VariableArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { + return varc.GetMaxVUs(et) > 0 } // VariableArrivalRate tries to execute a specific number of iterations for a @@ -308,8 +308,8 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample segment := varr.executionState.Options.ExecutionSegment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) - preAllocatedVUs := varr.config.GetPreAllocatedVUs(segment) - maxVUs := varr.config.GetMaxVUs(segment) + preAllocatedVUs := varr.config.GetPreAllocatedVUs(varr.executionState.ExecutionTuple) + maxVUs := varr.config.GetMaxVUs(varr.executionState.ExecutionTuple) timeUnit := time.Duration(varr.config.TimeUnit.Duration) startArrivalRate := getScaledArrivalRate(segment, varr.config.StartRate.Int64, timeUnit) diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index f8e624bfebb..8344d820410 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -61,8 +61,7 @@ func TestGetPlannedRateChanges0DurationStage(t *testing.T) { }, }, } - var es *lib.ExecutionSegment - changes := config.getPlannedRateChanges(es) + changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) require.Equal(t, 2, len(changes)) require.Equal(t, time.Duration(0), changes[0].timeOffset) require.Equal(t, types.NullDurationFrom(time.Millisecond*20), changes[0].tickerPeriod) @@ -114,8 +113,7 @@ func TestGetPlannedRateChangesZeroDurationStart(t *testing.T) { }, } - var es *lib.ExecutionSegment - changes := config.getPlannedRateChanges(es) + changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) var expectedTickerPeriod types.Duration for i, change := range changes { switch { @@ -167,8 +165,7 @@ func TestGetPlannedRateChanges(t *testing.T) { }, } - var es *lib.ExecutionSegment - changes := config.getPlannedRateChanges(es) + changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) var expectedTickerPeriod types.Duration for i, change := range changes { switch { @@ -207,10 +204,9 @@ func BenchmarkGetPlannedRateChanges(b *testing.B) { }, } - var es *lib.ExecutionSegment b.RunParallel(func(pb *testing.PB) { for pb.Next() { - changes := config.getPlannedRateChanges(es) + changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) require.Equal(b, time.Duration(0), changes[0].timeOffset%minIntervalBetweenRateAdjustments, "%+v", changes[0]) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index f763cdc0fbd..b2965303a6d 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -82,8 +82,8 @@ func NewVariableLoopingVUsConfig(name string) VariableLoopingVUsConfig { var _ lib.ExecutorConfig = &VariableLoopingVUsConfig{} // GetStartVUs is just a helper method that returns the scaled starting VUs. -func (vlvc VariableLoopingVUsConfig) GetStartVUs(es *lib.ExecutionSegment) int64 { - return es.Scale(vlvc.StartVUs.Int64) +func (vlvc VariableLoopingVUsConfig) GetStartVUs(et *lib.ExecutionTuple) int64 { + return et.ES.Scale(vlvc.StartVUs.Int64) } // GetGracefulRampDown is just a helper method that returns the graceful @@ -93,8 +93,8 @@ func (vlvc VariableLoopingVUsConfig) GetGracefulRampDown() time.Duration { } // GetDescription returns a human-readable description of the executor options -func (vlvc VariableLoopingVUsConfig) GetDescription(es *lib.ExecutionSegment) string { - maxVUs := es.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) +func (vlvc VariableLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { + maxVUs := et.ES.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) @@ -435,8 +435,8 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu // last stage's target is 0), then this will have no effect. // - If the last stage's target is more than 0, the VUs at the end of the // executor's life will have more time to finish their last iterations. -func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(es *lib.ExecutionSegment) []lib.ExecutionStep { - steps := vlvc.getRawExecutionSteps(es, false) +func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { + steps := vlvc.getRawExecutionSteps(et.ES, false) executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) // Handle graceful ramp-downs, if we have them @@ -461,8 +461,8 @@ func (vlvc VariableLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger } // HasWork reports whether there is any work to be done for the given execution segment. -func (vlvc VariableLoopingVUsConfig) HasWork(es *lib.ExecutionSegment) bool { - return lib.GetMaxPlannedVUs(vlvc.GetExecutionRequirements(es)) > 0 +func (vlvc VariableLoopingVUsConfig) HasWork(et *lib.ExecutionTuple) bool { + return lib.GetMaxPlannedVUs(vlvc.GetExecutionRequirements(et)) > 0 } // VariableLoopingVUs handles the old "stages" execution configuration - it @@ -491,7 +491,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo return fmt.Errorf("%s expected raw end offset at %s to be final", vlv.config.GetName(), regularDuration) } - gracefulExecutionSteps := vlv.config.GetExecutionRequirements(segment) + gracefulExecutionSteps := vlv.config.GetExecutionRequirements(vlv.executionState.ExecutionTuple) maxDuration, isFinal := lib.GetEndOffset(gracefulExecutionSteps) if !isFinal { return fmt.Errorf("%s expected graceful end offset at %s to be final", vlv.config.GetName(), maxDuration) @@ -504,7 +504,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo // Make sure the log and the progress bar have accurate information vlv.logger.WithFields(logrus.Fields{ - "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(segment), "maxVUs": maxVUs, + "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(vlv.executionState.ExecutionTuple), "maxVUs": maxVUs, "duration": regularDuration, "numStages": len(vlv.config.Stages)}, ).Debug("Starting executor run...") diff --git a/lib/executors.go b/lib/executors.go index bd21ad823bc..b9a110a2aed 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -97,15 +97,15 @@ type ExecutorConfig interface { // Calculates the VU requirements in different stages of the executor's // execution, including any extensions caused by waiting for iterations to // finish with graceful stops or ramp-downs. - GetExecutionRequirements(*ExecutionSegment) []ExecutionStep + GetExecutionRequirements(*ExecutionTuple) []ExecutionStep // Return a human-readable description of the executor - GetDescription(es *ExecutionSegment) string + GetDescription(*ExecutionTuple) string NewExecutor(*ExecutionState, *logrus.Entry) (Executor, error) // HasWork reports whether there is any work for the executor to do with a given segment. - HasWork(*ExecutionSegment) bool + HasWork(*ExecutionTuple) bool } // InitVUFunc is just a shorthand so we don't have to type the function @@ -243,7 +243,7 @@ func (scs ExecutorConfigMap) GetSortedConfigs() []ExecutorConfig { // the configured executors. It takes into account their start times and their // individual VU requirements and calculates the total VU requirements for each // moment in the test execution. -func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *ExecutionSegment) []ExecutionStep { +func (scs ExecutorConfigMap) GetFullExecutionRequirements(et *ExecutionTuple) []ExecutionStep { sortedConfigs := scs.GetSortedConfigs() // Combine the steps and requirements from all different executors, and @@ -256,7 +256,7 @@ func (scs ExecutorConfigMap) GetFullExecutionRequirements(executionSegment *Exec trackedSteps := []trackedStep{} for configID, config := range sortedConfigs { // orderly iteration over a slice configStartTime := config.GetStartTime() - configSteps := config.GetExecutionRequirements(executionSegment) + configSteps := config.GetExecutionRequirements(et) for _, cs := range configSteps { cs.TimeOffset += configStartTime // add the executor start time to the step time offset trackedSteps = append(trackedSteps, trackedStep{cs, configID}) diff --git a/lib/options.go b/lib/options.go index 102a59004c2..ff942a08ec6 100644 --- a/lib/options.go +++ b/lib/options.go @@ -203,9 +203,9 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` - ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` - ESS *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` + Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` + ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` + ExecutionSegmentSequence *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` // Timeouts for the setup() and teardown() functions NoSetup null.Bool `json:"noSetup" envconfig:"NO_SETUP"` @@ -345,6 +345,10 @@ func (o Options) Apply(opts Options) Options { if opts.ExecutionSegment != nil { o.ExecutionSegment = opts.ExecutionSegment } + + if opts.ExecutionSegmentSequence != nil { + o.ExecutionSegmentSequence = opts.ExecutionSegmentSequence + } if opts.NoSetup.Valid { o.NoSetup = opts.NoSetup } @@ -446,7 +450,22 @@ func (o Options) Apply(opts Options) Options { func (o Options) Validate() []error { //TODO: validate all of the other options... that we should have already been validating... //TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation - return o.Execution.Validate() + var errors []error + if o.ExecutionSegmentSequence != nil { + var segmentFound bool + for _, segment := range *o.ExecutionSegmentSequence { + if o.ExecutionSegment.Equal(segment) { + segmentFound = true + break + } + } + if !segmentFound { + errors = append(errors, + fmt.Errorf("provided segment %s can't be found in sequence %s", + o.ExecutionSegment, o.ExecutionSegmentSequence)) + } + } + return append(errors, o.Execution.Validate()...) } // ForEachSpecified enumerates all struct fields and calls the supplied function with each From 886859a6c92de36ebcefdaab49745b33506a75e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 30 Mar 2020 14:48:44 +0200 Subject: [PATCH 155/350] Use single select for rendering progress bars Resolves https://github.com/loadimpact/k6/pull/1332#discussion_r400104834 --- cmd/ui.go | 37 ++++++++++++++++--------------------- 1 file changed, 16 insertions(+), 21 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 523c0825b40..9a310944cda 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -260,27 +260,14 @@ func showProgress( } var ( - fd = int(os.Stdout.Fd()) - ticker = time.NewTicker(updateFreq) - winchSignal = getWinchSignal() + fd = int(os.Stdout.Fd()) + ticker = time.NewTicker(updateFreq) ) - // Default ticker-based progress bar resizing - updateTermWidth := func() { - <-ticker.C - termWidth, _, _ = terminal.GetSize(fd) - } - // More responsive progress bar resizing on platforms with SIGWINCH (*nix) - if winchSignal != nil { - winch := make(chan os.Signal, 1) - signal.Notify(winch, winchSignal) - updateTermWidth = func() { - select { - case <-ticker.C: - case <-winch: - termWidth, _, _ = terminal.GetSize(fd) - } - } + var winch chan os.Signal + if sig := getWinchSignal(); sig != nil { + winch = make(chan os.Signal, 1) + signal.Notify(winch, sig) } ctxDone := ctx.Done() @@ -288,11 +275,19 @@ func showProgress( select { case <-ctxDone: renderProgressBars(false) + outMutex.Lock() printProgressBars() + outMutex.Unlock() return - default: + case <-winch: + // More responsive progress bar resizing on platforms with SIGWINCH (*nix) + termWidth, _, _ = terminal.GetSize(fd) + case <-ticker.C: + // Default ticker-based progress bar resizing + if winch == nil { + termWidth, _, _ = terminal.GetSize(fd) + } } - updateTermWidth() renderProgressBars(true) outMutex.Lock() printProgressBars() From 363ed9c2cb49e6bbf3a645c8a33c6f26bcd2e63d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 30 Mar 2020 15:35:10 +0200 Subject: [PATCH 156/350] Remove redundant logging in non-interactive mode With the new progress bars this logging is not needed, though we might bring it back once something like `--ui-mode=log` is implemented. Resolves https://github.com/loadimpact/k6/pull/1332#pullrequestreview-383745205 --- cmd/run.go | 26 -------------------------- 1 file changed, 26 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 6b2c1718643..d1b78cde4fe 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -262,35 +262,9 @@ a commandline interface for interacting with it.`, sigC := make(chan os.Signal, 1) signal.Notify(sigC, os.Interrupt, syscall.SIGINT, syscall.SIGTERM) defer signal.Stop(sigC) - - // Ticker for progress bar updates. Less frequent updates for non-TTYs, none if quiet. - updateFreq := 50 * time.Millisecond - if !stdoutTTY { - updateFreq = 1 * time.Second - } - ticker := time.NewTicker(updateFreq) - if quiet || conf.HTTPDebug.Valid && conf.HTTPDebug.String != "" { - ticker.Stop() - } mainLoop: for { select { - case <-ticker.C: - if quiet || !stdoutTTY { - l := logrus.WithFields(logrus.Fields{ - "t": executionState.GetCurrentTestRunDuration(), - "i": executionState.GetFullIterationCount(), - }) - fn := l.Info - if quiet { - fn = l.Debug - } - if executionState.IsPaused() { - fn("Paused") - } else { - fn("Running") - } - } case err := <-errC: cancel() if err == nil { From 1d34063ddc578788dc431e90040daeb324eb250a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 30 Mar 2020 15:50:33 +0200 Subject: [PATCH 157/350] Silence warning retrieving terminal size in non-interactive mode --- cmd/ui.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ui.go b/cmd/ui.go index 9a310944cda..e52636680b3 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -203,7 +203,7 @@ func showProgress( } termWidth, _, err := terminal.GetSize(int(os.Stdout.Fd())) - if err != nil { + if err != nil && stdoutTTY { logger.WithError(err).Warn("error getting terminal size") termWidth = 80 // TODO: something safer, return error? } From 73e6f5b403369f1e72be4129b902158bfa292f1f Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 31 Mar 2020 13:26:06 +0300 Subject: [PATCH 158/350] refactor some panics away --- lib/execution_segment.go | 27 ++++++++++----------------- 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 11c5c50d7d1..68cb2174c6a 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -77,11 +77,16 @@ func NewExecutionSegment(from, to *big.Rat) (*ExecutionSegment, error) { if to.Cmp(oneRat) > 0 { return nil, fmt.Errorf("segment end value shouldn't be more than 1 but was %s", to.FloatString(2)) } + return newExecutionSegment(from, to), nil +} + +// newExecutionSegment just creates an ExecutionSegment without validating the arguments +func newExecutionSegment(from, to *big.Rat) *ExecutionSegment { return &ExecutionSegment{ from: from, to: to, length: new(big.Rat).Sub(to, from), - }, nil + } } // stringToRat is a helper function that tries to convert a string to a rational @@ -513,22 +518,13 @@ type ExecutionTuple struct { // TODO rename } func fillSequence(sequence ExecutionSegmentSequence) ExecutionSegmentSequence { - // TODO: discuss if we want to get the lcd of the sequence and fill with it elements of length 1/lcd ? if sequence[0].from.Cmp(zeroRat) != 0 { - es, err := NewExecutionSegment(zeroRat, sequence[0].from) - if err != nil { - panic(err) // this really can't happen - } - + es := newExecutionSegment(zeroRat, sequence[0].from) sequence = append(ExecutionSegmentSequence{es}, sequence...) } if sequence[len(sequence)-1].to.Cmp(oneRat) != 0 { - es, err := NewExecutionSegment(sequence[len(sequence)-1].to, oneRat) - if err != nil { - panic(err) // this really can't happen - } - + es := newExecutionSegment(sequence[len(sequence)-1].to, oneRat) sequence = append(sequence, es) } return sequence @@ -540,7 +536,7 @@ func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequ // this is needed in order to know that a segment == nil means that after // GetNewExecutionTupleBasedOnValues the original segment scaled to 0 length one and as such // should it be used it should always get 0 as values - segment, _ = NewExecutionSegmentFromString("0:1") + segment = newExecutionSegment(zeroRat, oneRat) } et := ExecutionTuple{ once: new(sync.Once), @@ -652,10 +648,7 @@ func (et *ExecutionTuple) GetNewExecutionTupleBasedOnValue(value int64) *Executi if newValue == 0 { continue } - var currentES, err = NewExecutionSegmentFromString(fmt.Sprintf("%d/%d:%d/%d", prev, value, prev+newValue, value)) - if err != nil { - panic(err) // TODO this really can't happen but during the optimization it will probably disappear - } + var currentES = newExecutionSegment(big.NewRat(prev, value), big.NewRat(prev+newValue, value)) prev += newValue if es.Equal(et.ES) { newES = currentES From 67e3738c3515503e5f716bb5bb2c3593d9ab7686 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 31 Mar 2020 16:03:27 +0300 Subject: [PATCH 159/350] more errors --- core/local/local.go | 7 ++- lib/execution.go | 7 +-- lib/execution_segment.go | 50 +++++++++++++--------- lib/execution_segment_test.go | 31 +++++++++----- lib/executor/common_test.go | 4 +- lib/executor/constant_arrival_rate_test.go | 22 +++++++--- lib/executor/constant_looping_vus_test.go | 6 ++- lib/executor/execution_test.go | 20 ++++++--- lib/executor/executors_test.go | 30 ++++++++----- lib/executor/externally_controlled_test.go | 4 +- lib/executor/per_vu_iterations_test.go | 12 ++++-- lib/executor/shared_iterations_test.go | 12 ++++-- lib/executor/variable_arrival_rate_test.go | 28 ++++++++---- lib/executor/variable_looping_vus_test.go | 8 +++- 14 files changed, 160 insertions(+), 81 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index e990ae35c3a..e0f74bf5921 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -60,12 +60,15 @@ var _ lib.ExecutionScheduler = &ExecutionScheduler{} func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*ExecutionScheduler, error) { options := runner.GetOptions() // TODO figure out a way to give it to executionStage that is not terrible - et := lib.NewExecutionTuple(options.ExecutionSegment, options.ExecutionSegmentSequence) + et, err := lib.NewExecutionTuple(options.ExecutionSegment, options.ExecutionSegmentSequence) + if err != nil { + return nil, err + } executionPlan := options.Execution.GetFullExecutionRequirements(et) maxPlannedVUs := lib.GetMaxPlannedVUs(executionPlan) maxPossibleVUs := lib.GetMaxPossibleVUs(executionPlan) - executionState := lib.NewExecutionState(options, maxPlannedVUs, maxPossibleVUs) + executionState := lib.NewExecutionState(options, et, maxPlannedVUs, maxPossibleVUs) maxDuration, _ := lib.GetEndOffset(executionPlan) // we don't care if the end offset is final executorConfigs := options.Execution.GetSortedConfigs() diff --git a/lib/execution.go b/lib/execution.go index 1c11bbc5df3..b6dd2deff34 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -272,7 +272,7 @@ type ExecutionState struct { // NewExecutionState initializes all of the pointers in the ExecutionState // with zeros. It also makes sure that the initial state is unpaused, by // setting resumeNotify to an already closed channel. -func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *ExecutionState { +func NewExecutionState(options Options, et *ExecutionTuple, maxPlannedVUs, maxPossibleVUs uint64) *ExecutionState { resumeNotify := make(chan struct{}) close(resumeNotify) // By default the ExecutionState starts unpaused @@ -295,10 +295,7 @@ func NewExecutionState(options Options, maxPlannedVUs, maxPossibleVUs uint64) *E pauseStateLock: sync.RWMutex{}, totalPausedDuration: 0, // Accessed only behind the pauseStateLock resumeNotify: resumeNotify, - ExecutionTuple: NewExecutionTuple( - options.ExecutionSegment, - options.ExecutionSegmentSequence, - ), + ExecutionTuple: et, } } diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 68cb2174c6a..db34b76a360 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -509,7 +509,7 @@ outer: type ExecutionTuple struct { // TODO rename ES *ExecutionSegment // TODO unexport this as well? - // TODO: have the index of the segment, cached? + esIndex int sequence ExecutionSegmentSequence offsetsCache [][]int64 lcd int64 @@ -531,30 +531,36 @@ func fillSequence(sequence ExecutionSegmentSequence) ExecutionSegmentSequence { } // NewExecutionTuple returns a new ExecutionTuple for the provided segment and sequence -func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequence) *ExecutionTuple { - if segment == nil { // TODO: try to do something better, maybe have bool flag in the ExecutionTuple or something - // this is needed in order to know that a segment == nil means that after - // GetNewExecutionTupleBasedOnValues the original segment scaled to 0 length one and as such - // should it be used it should always get 0 as values - segment = newExecutionSegment(zeroRat, oneRat) - } +func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequence) (*ExecutionTuple, error) { et := ExecutionTuple{ once: new(sync.Once), ES: segment, } if sequence == nil || len(*sequence) == 0 { if segment == nil || segment.length.Cmp(oneRat) == 0 { - et.sequence = ExecutionSegmentSequence{segment} + // here we replace it with a not nil as we otherwise will need to check it everywhere + et.sequence = ExecutionSegmentSequence{newExecutionSegment(zeroRat, oneRat)} } else { et.sequence = fillSequence(ExecutionSegmentSequence{segment}) } } else { et.sequence = fillSequence(*sequence) } - return &et + + et.esIndex = et.find(segment) + if et.esIndex == -1 { + return nil, fmt.Errorf("coulnd't find segment %s in sequence %s", segment, sequence) + } + return &et, nil } func (et *ExecutionTuple) find(segment *ExecutionSegment) int { + if segment == nil { + if len(et.sequence) == 1 { + return 0 + } + return -1 + } index := sort.Search(len(et.sequence), func(i int) bool { return et.sequence[i].from.Cmp(segment.from) >= 0 }) @@ -567,15 +573,17 @@ func (et *ExecutionTuple) find(segment *ExecutionSegment) int { // ScaleInt64 scales the provided value based on the ExecutionTuple func (et *ExecutionTuple) ScaleInt64(value int64) int64 { - return et.scaleInt64With(value, et.ES) + if et.esIndex == -1 { + return 0 + } + et.once.Do(et.fillCache) + offsets := et.offsetsCache[et.esIndex] + return scaleInt64(value, offsets[0], offsets[1:], et.lcd) } // scaleInt64With scales the provided value based on the ExecutionTuples' // sequence and the segment provided func (et *ExecutionTuple) scaleInt64With(value int64, es *ExecutionSegment) int64 { - if es == nil { - return 0 - } start, offsets, lcd := et.GetStripedOffsets(es) return scaleInt64(value, start, offsets, lcd) } @@ -636,28 +644,30 @@ func (et *ExecutionTuple) GetStripedOffsets(segment *ExecutionSegment) (int64, [ // the sequence. func (et *ExecutionTuple) GetNewExecutionTupleBasedOnValue(value int64) *ExecutionTuple { var ( - newESS = make(ExecutionSegmentSequence, 0, len(et.sequence)) // this can be smaller - newES *ExecutionSegment + newESS = make(ExecutionSegmentSequence, 0, len(et.sequence)) // this can be smaller + newES *ExecutionSegment + esIndex = -1 ) et.once.Do(et.fillCache) var prev int64 - for i, es := range et.sequence { + for i := range et.sequence { offsets := et.offsetsCache[i] newValue := scaleInt64(value, offsets[0], offsets[1:], et.lcd) - // TODO optimize this, somewhat if newValue == 0 { continue } var currentES = newExecutionSegment(big.NewRat(prev, value), big.NewRat(prev+newValue, value)) prev += newValue - if es.Equal(et.ES) { + if i == et.esIndex { newES = currentES + esIndex = len(newESS) } newESS = append(newESS, currentES) } return &ExecutionTuple{ - ES: newES, // in case newES is nil we want to keep it that way + ES: newES, sequence: newESS, + esIndex: esIndex, once: new(sync.Once), } } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 760212f1a69..12e019f1047 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -212,24 +212,28 @@ func TestExecutionTupleScale(t *testing.T) { t.Parallel() es := new(ExecutionSegment) require.NoError(t, es.UnmarshalText([]byte("0.5"))) - et := NewExecutionTuple(es, nil) + et, err := NewExecutionTuple(es, nil) + require.NoError(t, err) require.Equal(t, int64(1), et.ScaleInt64(2)) require.Equal(t, int64(2), et.ScaleInt64(3)) require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) - et = NewExecutionTuple(es, nil) + et, err = NewExecutionTuple(es, nil) + require.NoError(t, err) require.Equal(t, int64(1), et.ScaleInt64(2)) require.Equal(t, int64(1), et.ScaleInt64(3)) ess, err := NewExecutionSegmentSequenceFromString("0,0.5,1") require.NoError(t, err) require.NoError(t, es.UnmarshalText([]byte("0.5"))) - et = NewExecutionTuple(es, &ess) + et, err = NewExecutionTuple(es, &ess) + require.NoError(t, err) require.Equal(t, int64(1), et.ScaleInt64(2)) require.Equal(t, int64(2), et.ScaleInt64(3)) require.NoError(t, es.UnmarshalText([]byte("0.5:1.0"))) - et = NewExecutionTuple(es, &ess) + et, err = NewExecutionTuple(es, &ess) + require.NoError(t, err) require.Equal(t, int64(1), et.ScaleInt64(2)) require.Equal(t, int64(1), et.ScaleInt64(3)) } @@ -238,7 +242,8 @@ func TestBigScale(t *testing.T) { ess, err := NewExecutionSegmentSequenceFromString("0,7/20,7/10,1") require.NoError(t, err) require.NoError(t, es.UnmarshalText([]byte("0:7/20"))) - et := NewExecutionTuple(es, &ess) + et, err := NewExecutionTuple(es, &ess) + require.NoError(t, err) require.Equal(t, int64(18), et.ScaleInt64(50)) } @@ -518,7 +523,9 @@ func TestGetStripedOffsets(t *testing.T) { require.NoError(t, err) segment, err := NewExecutionSegmentFromString(tc.seg) require.NoError(t, err) - et := NewExecutionTuple(segment, &ess) + et, err := NewExecutionTuple(segment, &ess) + require.NoError(t, err) + start, offsets, lcd := et.GetStripedOffsets(segment) assert.Equal(t, tc.start, start) @@ -567,7 +574,8 @@ func BenchmarkGetStripedOffsets(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { segment := sequence[int(r.Int63())%len(sequence)] - et := NewExecutionTuple(segment, &sequence) + et, err := NewExecutionTuple(segment, &sequence) + require.NoError(b, err) _, _, _ = et.GetStripedOffsets(segment) } }) @@ -602,7 +610,8 @@ func BenchmarkGetStripedOffsetsEven(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { segment := sequence[111233%len(sequence)] - et := NewExecutionTuple(segment, &sequence) + et, err := NewExecutionTuple(segment, &sequence) + require.NoError(b, err) _, _, _ = et.GetStripedOffsets(segment) } }) @@ -631,7 +640,8 @@ func TestGetNewExecutionTupleBesedOnValue(t *testing.T) { segment, err := NewExecutionSegmentFromString(tc.seg) require.NoError(t, err) - et := NewExecutionTuple(segment, &ess) + et, err := NewExecutionTuple(segment, &ess) + require.NoError(t, err) newET := et.GetNewExecutionTupleBasedOnValue(tc.value) require.Equal(t, tc.expected, newET.sequence.String()) }) @@ -778,7 +788,8 @@ func TestNewExecutionTuple(t *testing.T) { for _, testCase := range testCases { testCase := testCase t.Run(fmt.Sprintf("seg:'%s',seq:'%s'", testCase.seg, testCase.seq), func(t *testing.T) { - et := NewExecutionTuple(testCase.seg, testCase.seq) + et, err := NewExecutionTuple(testCase.seg, testCase.seq) + require.NoError(t, err) for scaleValue, result := range testCase.scaleTests { require.Equal(t, result, et.ScaleInt64(scaleValue), "%d->%d", scaleValue, result) } diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 95e48583589..fcd74b11516 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -58,7 +58,9 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta return runner.NewVU(engineOut) }) - et := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) + et, err := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) + require.NoError(t, err) + maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(et)) initializeVUs(ctx, t, logEntry, es, maxVUs) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 47df943dc70..f1056dd0e45 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -50,7 +50,9 @@ func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -60,7 +62,7 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) - err := executor.Run(ctx, engineOut) + err = executor.Run(ctx, engineOut) require.NoError(t, err) entries := logHook.Drain() require.NotEmpty(t, entries) @@ -75,7 +77,9 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestConstantArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -98,7 +102,7 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { } }() var engineOut = make(chan stats.SampleContainer, 1000) - err := executor.Run(ctx, engineOut) + err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) require.Empty(t, logHook.Drain()) @@ -174,10 +178,12 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { t.Run(fmt.Sprintf("segment %s sequence %s", test.segment, test.sequence), func(t *testing.T) { t.Parallel() + et, err := lib.NewExecutionTuple(test.segment, test.sequence) + require.NoError(t, err) es := lib.NewExecutionState(lib.Options{ ExecutionSegment: test.segment, ExecutionSegmentSequence: test.sequence, - }, 10, 50) + }, et, 10, 50) var count int64 var config = getTestConstantArrivalRateConfig() newET := es.ExecutionTuple.GetNewExecutionTupleBasedOnValue(config.MaxVUs.Int64) @@ -221,7 +227,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { }() startTime = time.Now() var engineOut = make(chan stats.SampleContainer, 1000) - err := executor.Run(ctx, engineOut) + err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) require.Empty(t, logHook.Drain()) @@ -243,7 +249,9 @@ func TestArrivalRateCancel(t *testing.T) { var ch = make(chan struct{}) var errCh = make(chan error, 1) var weAreDoneCh = make(chan struct{}) - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { select { diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go index 8e64c258bc9..a3e66122cb0 100644 --- a/lib/executor/constant_looping_vus_test.go +++ b/lib/executor/constant_looping_vus_test.go @@ -44,7 +44,9 @@ func getTestConstantLoopingVUsConfig() ConstantLoopingVUsConfig { func TestConstantLoopingVUsRun(t *testing.T) { t.Parallel() var result sync.Map - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestConstantLoopingVUsConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -56,7 +58,7 @@ func TestConstantLoopingVUsRun(t *testing.T) { }), ) defer cancel() - err := executor.Run(ctx, nil) + err = executor.Run(ctx, nil) require.NoError(t, err) var totalIters uint64 diff --git a/lib/executor/execution_test.go b/lib/executor/execution_test.go index 05fbc99da5f..190631fc414 100644 --- a/lib/executor/execution_test.go +++ b/lib/executor/execution_test.go @@ -38,7 +38,9 @@ import ( func TestExecutionStateVUIDs(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 0, 0) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 0, 0) assert.Equal(t, uint64(1), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(2), es.GetUniqueVUIdentifier()) assert.Equal(t, uint64(3), es.GetUniqueVUIdentifier()) @@ -58,7 +60,9 @@ func TestExecutionStateVUIDs(t *testing.T) { func TestExecutionStateGettingVUsWhenNonAreAvailable(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 0, 0) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 0, 0) logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.WarnLevel}} testLog := logrus.New() testLog.AddHook(logHook) @@ -82,7 +86,9 @@ func TestExecutionStateGettingVUs(t *testing.T) { testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) - es := lib.NewExecutionState(lib.Options{}, 10, 20) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 20) es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { return &minirunner.VU{}, nil }) @@ -144,7 +150,9 @@ func TestExecutionStateGettingVUs(t *testing.T) { func TestMarkStartedPanicsOnSecondRun(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 0, 0) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 0, 0) require.False(t, es.HasStarted()) es.MarkStarted() require.True(t, es.HasStarted()) @@ -153,7 +161,9 @@ func TestMarkStartedPanicsOnSecondRun(t *testing.T) { func TestMarkEnded(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 0, 0) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 0, 0) require.False(t, es.HasEnded()) es.MarkEnded() require.True(t, es.HasEnded()) diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index f5117221feb..385c94563cc 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -87,7 +87,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["someKey"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "10 looping VUs for 1m0s (exec: someFunc, startTime: 1m10s, gracefulStop: 10s)", cm["someKey"].GetDescription(et)) schedReqs := cm["someKey"].GetExecutionRequirements(et) @@ -135,7 +136,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "Up to 30 looping VUs for 3m10s over 2 stages (gracefulRampDown: 10s, startTime: 23s, gracefulStop: 15s)", cm["varloops"].GetDescription(et)) schedReqs := cm["varloops"].GetExecutionRequirements(et) @@ -159,7 +161,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "Up to 10 looping VUs for 10s over 1 stages (gracefulRampDown: 10s)", cm["varloops"].GetDescription(et)) schedReqs := cm["varloops"].GetExecutionRequirements(et) @@ -173,7 +176,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "Up to 10 looping VUs for 20s over 3 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(et)) schedReqs := cm.GetFullExecutionRequirements(et) @@ -187,7 +191,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "Up to 11 looping VUs for 20s over 4 stages (gracefulRampDown: 0s)", cm["varloops"].GetDescription(et)) schedReqs := cm.GetFullExecutionRequirements(et) @@ -214,7 +219,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["ishared"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "22 iterations shared among 12 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(et)) schedReqs := cm["ishared"].GetExecutionRequirements(et) @@ -248,7 +254,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["ipervu"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "23 iterations for each of 13 VUs (maxDuration: 10m0s)", cm["ipervu"].GetDescription(et)) schedReqs := cm["ipervu"].GetExecutionRequirements(et) @@ -274,7 +281,8 @@ var configMapTestCases = []configMapTestCase{ // constant-arrival-rate {`{"carrival": {"type": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) sched := NewConstantArrivalRateConfig("carrival") sched.Rate = null.IntFrom(30) sched.Duration = types.NullDurationFrom(10 * time.Minute) @@ -327,7 +335,8 @@ var configMapTestCases = []configMapTestCase{ assert.Empty(t, cm["varrival"].Validate()) assert.Empty(t, cm.Validate()) - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) assert.Equal(t, "Up to 1.00 iterations/s for 8m0s over 2 stages (maxVUs: 20-50, gracefulStop: 30s)", cm["varrival"].GetDescription(et)) schedReqs := cm["varrival"].GetExecutionRequirements(et) @@ -382,7 +391,8 @@ func TestConfigMapParsingAndValidation(t *testing.T) { func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { t.Parallel() - et := lib.NewExecutionTuple(nil, nil) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) conf := NewVariableLoopingVUsConfig("test") conf.StartVUs = null.IntFrom(4) conf.Stages = []Stage{ diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index 665261a654b..da8f57540f2 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -49,7 +49,9 @@ func getTestExternallyControlledConfig() ExternallyControlledConfig { func TestExternallyControlledRun(t *testing.T) { t.Parallel() var doneIters uint64 - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestExternallyControlledConfig(), es, simpleRunner(func(ctx context.Context) error { diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 9242ebb94a0..41037d45434 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -47,7 +47,9 @@ func getTestPerVUIterationsConfig() PerVUIterationsConfig { func TestPerVUIterationsRun(t *testing.T) { t.Parallel() var result sync.Map - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestPerVUIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -58,7 +60,7 @@ func TestPerVUIterationsRun(t *testing.T) { }), ) defer cancel() - err := executor.Run(ctx, nil) + err = executor.Run(ctx, nil) require.NoError(t, err) var totalIters uint64 @@ -79,7 +81,9 @@ func TestPerVUIterationsRunVariableVU(t *testing.T) { result sync.Map slowVUID int64 ) - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestPerVUIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -98,7 +102,7 @@ func TestPerVUIterationsRunVariableVU(t *testing.T) { }), ) defer cancel() - err := executor.Run(ctx, nil) + err = executor.Run(ctx, nil) require.NoError(t, err) val, ok := result.Load(slowVUID) diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index 5f8b496fb41..61f1a448103 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -47,7 +47,9 @@ func getTestSharedIterationsConfig() SharedIterationsConfig { func TestSharedIterationsRun(t *testing.T) { t.Parallel() var doneIters uint64 - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestSharedIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -56,7 +58,7 @@ func TestSharedIterationsRun(t *testing.T) { }), ) defer cancel() - err := executor.Run(ctx, nil) + err = executor.Run(ctx, nil) require.NoError(t, err) assert.Equal(t, uint64(100), doneIters) } @@ -69,7 +71,9 @@ func TestSharedIterationsRunVariableVU(t *testing.T) { result sync.Map slowVUID int64 ) - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, getTestSharedIterationsConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -89,7 +93,7 @@ func TestSharedIterationsRunVariableVU(t *testing.T) { }), ) defer cancel() - err := executor.Run(ctx, nil) + err = executor.Run(ctx, nil) require.NoError(t, err) var totalIters uint64 diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 8344d820410..2d3858f464c 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -61,7 +61,9 @@ func TestGetPlannedRateChanges0DurationStage(t *testing.T) { }, }, } - changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + changes := config.getPlannedRateChanges(et) require.Equal(t, 2, len(changes)) require.Equal(t, time.Duration(0), changes[0].timeOffset) require.Equal(t, types.NullDurationFrom(time.Millisecond*20), changes[0].tickerPeriod) @@ -113,7 +115,9 @@ func TestGetPlannedRateChangesZeroDurationStart(t *testing.T) { }, } - changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + changes := config.getPlannedRateChanges(et) var expectedTickerPeriod types.Duration for i, change := range changes { switch { @@ -165,7 +169,9 @@ func TestGetPlannedRateChanges(t *testing.T) { }, } - changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + changes := config.getPlannedRateChanges(et) var expectedTickerPeriod types.Duration for i, change := range changes { switch { @@ -205,8 +211,10 @@ func BenchmarkGetPlannedRateChanges(b *testing.B) { } b.RunParallel(func(pb *testing.PB) { + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(b, err) for pb.Next() { - changes := config.getPlannedRateChanges(lib.NewExecutionTuple(nil, nil)) + changes := config.getPlannedRateChanges(et) require.Equal(b, time.Duration(0), changes[0].timeOffset%minIntervalBetweenRateAdjustments, "%+v", changes[0]) @@ -239,7 +247,9 @@ func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -249,7 +259,7 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { ) defer cancel() var engineOut = make(chan stats.SampleContainer, 1000) - err := executor.Run(ctx, engineOut) + err = executor.Run(ctx, engineOut) require.NoError(t, err) entries := logHook.Drain() require.NotEmpty(t, entries) @@ -264,7 +274,9 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { func TestVariableArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, logHook = setupExecutor( t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -295,7 +307,7 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { require.InDelta(t, 50, currentCount, 2) }() var engineOut = make(chan stats.SampleContainer, 1000) - err := executor.Run(ctx, engineOut) + err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) require.Empty(t, logHook.Drain()) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index fa565fc8c9e..fb42d7556a0 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -58,7 +58,9 @@ func TestVariableLoopingVUsRun(t *testing.T) { } var iterCount int64 - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { @@ -114,7 +116,9 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { }, } - es := lib.NewExecutionState(lib.Options{}, 10, 50) + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { From 8cd6f5fd0a4e5678458626b549d8603d98bc0b43 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 31 Mar 2020 19:32:46 +0300 Subject: [PATCH 160/350] Add test and benchmarks for the new ScaleInt64 New benchmark results: BenchmarkExecutionSegmentScale/seq:;segment:/segment.Scale(5)-8 470435558 2.53 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5)-8 967018 1233 ns/op 440 B/op 17 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-8 150010268 7.94 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-8 463766229 2.93 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5523)-8 1000000 1277 ns/op 440 B/op 17 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-8 83739726 12.4 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-8 464901876 3.03 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-8 784173 1309 ns/op 440 B/op 17 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-8 148228074 13.3 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-8 461219998 2.74 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-8 953570 1224 ns/op 440 B/op 17 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-8 117047155 12.6 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-8 850756 1422 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-8 806138 1733 ns/op 632 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-8 154531297 11.8 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-8 926948 1306 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-8 643215 1694 ns/op 632 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-8 147420830 7.77 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-8 1000000 1329 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-8 692818 1745 ns/op 632 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-8 146246756 7.82 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-8 829132 1339 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-8 748573 1760 ns/op 632 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-8 120025989 14.9 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-8 584570 1979 ns/op 512 B/op 28 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-8 333420 3545 ns/op 1432 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-8 151765993 7.75 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-8 519910 2113 ns/op 624 B/op 32 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-8 311260 3682 ns/op 1432 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-8 140018670 9.14 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-8 597290 1980 ns/op 512 B/op 28 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-8 289699 3680 ns/op 1432 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-8 149898025 7.77 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-8 539796 2185 ns/op 672 B/op 33 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-8 347833 3612 ns/op 1432 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-8 122449836 11.3 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-8 610802 2086 ns/op 568 B/op 30 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-8 1831 730963 ns/op 492833 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-8 132629142 12.4 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-8 516368 2294 ns/op 672 B/op 33 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-8 1519 733097 ns/op 492833 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-8 777319 1370 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-8 554041 2184 ns/op 568 B/op 30 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-8 1500 756885 ns/op 492832 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-8 244622 4696 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-8 500317 2334 ns/op 720 B/op 34 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-8 1624 742685 ns/op 492833 B/op 42 allocs/op BenchmarkExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-8 422404 3310 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-8 690877 1675 ns/op 424 B/op 22 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-8 435661 2644 ns/op 1064 B/op 29 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-8 139451774 8.49 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-8 732109 1593 ns/op 424 B/op 22 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-8 448272 2659 ns/op 1064 B/op 29 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-8 147587318 8.26 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-8 840579 1384 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-8 469440 2701 ns/op 1064 B/op 29 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-8 129229272 9.03 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-8 844422 1494 ns/op 376 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-8 435537 2651 ns/op 1064 B/op 29 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-8 88344691 12.1 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-8 733918 1606 ns/op 376 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-8 369414 3192 ns/op 1208 B/op 36 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-8 141027226 9.66 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-8 789291 1544 ns/op 376 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-8 354043 3199 ns/op 1208 B/op 36 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-8 139259516 8.24 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-8 1000000 1396 ns/op 320 B/op 19 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-8 391041 3242 ns/op 1208 B/op 36 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-8 148571696 7.92 ns/op 0 B/op 0 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-8 970454 1550 ns/op 376 B/op 21 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-8 425010 3107 ns/op 1208 B/op 36 allocs/op BenchmarkExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-8 72204832 14.5 ns/op 0 B/op 0 allocs/op --- lib/execution_segment_test.go | 118 ++++++++++++++++++++++++++++++++++ 1 file changed, 118 insertions(+) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 12e019f1047..89b00405783 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -481,6 +481,72 @@ func TestExecutionSegmentScaleConsistency(t *testing.T) { } } +// Ensure that the sum of scaling all execution segments in +// the same sequence with scaling factor M results in M itself. +func TestExecutionTupleScaleConsistency(t *testing.T) { + t.Parallel() + + seed := time.Now().UnixNano() + r := rand.New(rand.NewSource(seed)) + t.Logf("Random source seeded with %d\n", seed) + + const numTests = 10 + for i := 0; i < numTests; i++ { + scale := rand.Int31n(99) + 2 + seq, err := generateRandomSequence(r.Int63n(9)+2, r) + require.NoError(t, err) + + et, err := NewExecutionTuple(seq[0], &seq) + require.NoError(t, err) + t.Run(fmt.Sprintf("%d_%s", scale, seq), func(t *testing.T) { + var total int64 + for _, segment := range seq { + total += et.scaleInt64With(int64(scale), segment) + } + assert.Equal(t, int64(scale), total) + }) + } +} + +func TestExecutionSegmentScaleNoWobble(t *testing.T) { + t.Parallel() + + requireSegmentScaleGreater := func(t *testing.T, et *ExecutionTuple) { + var i, lastResult int64 + for i = 1; i < 1000; i++ { + result := et.ScaleInt64(i) + require.True(t, result >= lastResult, "%d<%d", result, lastResult) + lastResult = result + } + } + + // Baseline full segment test + t.Run("0:1", func(t *testing.T) { + et, err := NewExecutionTuple(nil, nil) + require.NoError(t, err) + requireSegmentScaleGreater(t, et) + }) + + seed := time.Now().UnixNano() + r := rand.New(rand.NewSource(seed)) + t.Logf("Random source seeded with %d\n", seed) + + // Random segments + const numTests = 10 + for i := 0; i < numTests; i++ { + seq, err := generateRandomSequence(r.Int63n(9)+2, r) + require.NoError(t, err) + + es := seq[rand.Intn(len(seq))] + + et, err := NewExecutionTuple(seq[0], &seq) + require.NoError(t, err) + t.Run(es.String(), func(t *testing.T) { + requireSegmentScaleGreater(t, et) + }) + } +} + func TestGetStripedOffsets(t *testing.T) { t.Parallel() testCases := []struct { @@ -805,4 +871,56 @@ func TestNewExecutionTuple(t *testing.T) { } } +func BenchmarkExecutionSegmentScale(b *testing.B) { + testCases := []struct { + seq string + seg string + }{ + {}, + {seg: "0:1"}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3"}, + {seq: "0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.1"}, + {seg: "2/5:4/5"}, + {seg: "2235/5213:4/5"}, // just wanted it to be ugly ;D + } + + for _, tc := range testCases { + tc := tc + b.Run(fmt.Sprintf("seq:%s;segment:%s", tc.seq, tc.seg), func(b *testing.B) { + ess, err := NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(b, err) + segment, err := NewExecutionSegmentFromString(tc.seg) + require.NoError(b, err) + if tc.seg == "" { + segment = nil // specifically for the optimization + } + et, err := NewExecutionTuple(segment, &ess) + require.NoError(b, err) + for _, value := range []int64{5, 5523, 5000000, 67280421310721} { + value := value + b.Run(fmt.Sprintf("segment.Scale(%d)", value), func(b *testing.B) { + for i := 0; i < b.N; i++ { + segment.Scale(value) + } + }) + + b.Run(fmt.Sprintf("et.Scale(%d)", value), func(b *testing.B) { + for i := 0; i < b.N; i++ { + et2, err := NewExecutionTuple(segment, &ess) + require.NoError(b, err) + et2.ScaleInt64(value) + } + }) + + et.ScaleInt64(1) // precache + b.Run(fmt.Sprintf("et.Scale(%d) prefilled", value), func(b *testing.B) { + for i := 0; i < b.N; i++ { + et.ScaleInt64(value) + } + }) + } + }) + } +} + // TODO: test with randomized things From 6a6cb4170079e2a80e1dd5be1e571fc03e708be2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 31 Mar 2020 19:40:52 +0300 Subject: [PATCH 161/350] Move precalculation ExecutionTuple in ConstantArrivalRate Init --- lib/executor/constant_arrival_rate.go | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 58e4024d799..d89071b142d 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -152,7 +152,7 @@ func (carc ConstantArrivalRateConfig) GetExecutionRequirements(et *lib.Execution { TimeOffset: 0, PlannedVUs: uint64(et.ScaleInt64(carc.PreAllocatedVUs.Int64)), - MaxUnplannedVUs: uint64(et.ScaleInt64(carc.MaxVUs.Int64 - carc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(et.ScaleInt64(carc.MaxVUs.Int64) - et.ScaleInt64(carc.PreAllocatedVUs.Int64)), }, { TimeOffset: time.Duration(carc.Duration.Duration + carc.GracefulStop.Duration), PlannedVUs: 0, @@ -165,7 +165,7 @@ func (carc ConstantArrivalRateConfig) GetExecutionRequirements(et *lib.Execution func (carc ConstantArrivalRateConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { - return ConstantArrivalRate{ + return &ConstantArrivalRate{ BaseExecutor: NewBaseExecutor(carc, es, logger), config: carc, }, nil @@ -181,21 +181,28 @@ func (carc ConstantArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { type ConstantArrivalRate struct { *BaseExecutor config ConstantArrivalRateConfig + et *lib.ExecutionTuple } // Make sure we implement the lib.Executor interface. var _ lib.Executor = &ConstantArrivalRate{} +// Init values needed for the execution +func (car *ConstantArrivalRate) Init(ctx context.Context) error { + car.et = car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(car.config.MaxVUs.Int64) + // TODO mvoe the preallocation of VUs here ? + return nil +} + // Run executes a constant number of iterations per second. // // TODO: Reuse the variable arrival rate method? func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen - newET := car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(car.config.MaxVUs.Int64) gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) preAllocatedVUs := car.config.GetPreAllocatedVUs(car.executionState.ExecutionTuple) maxVUs := car.config.GetMaxVUs(car.executionState.ExecutionTuple) - arrivalRate := getScaledArrivalRate(newET.ES, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) + arrivalRate := getScaledArrivalRate(car.et.ES, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() @@ -254,7 +261,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC return math.Min(1, float64(spent)/float64(duration)), right } car.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, car, progresFn) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, &car, progresFn) regDurationDone := regDurationCtx.Done() runIterationBasic := getIterationRunner(car.executionState, car.logger, out) @@ -264,7 +271,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs - start, offsets, _ := newET.GetStripedOffsets(newET.ES) + start, offsets, _ := car.et.GetStripedOffsets(car.et.ES) startTime = time.Now() timer := time.NewTimer(time.Hour * 24) // here the we need the not scaled one From 06a231ed4be24858c943e59d1b65a28b9aedfd9e Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 1 Apr 2020 11:15:42 +0300 Subject: [PATCH 162/350] Optimize case with only 1(full)segment MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit All the memory/allocation drops as it nolonger does any work Unfortunately there is *big* deviation in the time/op part, but I don't want to take anymore time :) name old time/op new time/op delta ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-8 8.70ns ±45% 9.77ns ±60% ~ (p=0.303 n=17+19) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-8 1.93µs ±48% 1.34µs ±32% -30.24% (p=0.000 n=19+17) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-8 25.2ns ±25% 10.3ns ±54% -59.07% (p=0.000 n=19+18) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-8 8.30ns ±43% 8.27ns ±59% ~ (p=0.983 n=19+19) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-8 2.22µs ±10% 1.37µs ±24% -38.24% (p=0.000 n=13+16) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-8 25.8ns ±47% 9.7ns ±37% -62.44% (p=0.000 n=19+18) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-8 8.74ns ±50% 9.09ns ±31% ~ (p=0.754 n=20+19) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-8 2.23µs ±22% 1.32µs ±29% -40.84% (p=0.000 n=17+18) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-8 27.4ns ±29% 11.0ns ±61% -59.82% (p=0.000 n=17+20) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-8 9.49ns ±35% 8.51ns ±28% ~ (p=0.054 n=20+17) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-8 2.14µs ±59% 1.29µs ±40% -39.77% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-8 25.9ns ±53% 11.2ns ±36% -56.72% (p=0.000 n=20+19) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-8 2.64µs ±13% 2.52µs ±36% ~ (p=0.215 n=17+17) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-8 2.42µs ±44% 2.30µs ±34% ~ (p=0.465 n=20+19) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-8 24.9ns ±30% 10.3ns ±28% -58.45% (p=0.000 n=17+17) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-8 2.13µs ±50% 2.22µs ±55% ~ (p=0.820 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-8 2.60µs ±46% 2.00µs ±44% -23.15% (p=0.003 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-8 27.3ns ±60% 10.0ns ±42% -63.40% (p=0.000 n=20+19) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-8 2.34µs ±53% 2.38µs ±54% ~ (p=0.738 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-8 2.58µs ±60% 1.77µs ±47% -31.63% (p=0.001 n=19+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-8 34.1ns ±37% 10.8ns ±23% -68.29% (p=0.000 n=19+18) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-8 2.25µs ±52% 2.46µs ±49% ~ (p=0.324 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-8 2.57µs ±55% 1.90µs ±51% -26.26% (p=0.001 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-8 33.3ns ±26% 10.2ns ±27% -69.30% (p=0.000 n=15+18) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-8 2.91µs ±35% 2.80µs ±47% ~ (p=0.749 n=19+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-8 3.72µs ±63% 3.72µs ±34% ~ (p=0.923 n=20+19) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-8 26.3ns ±13% 28.0ns ±56% ~ (p=0.886 n=17+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-8 2.56µs ±56% 3.01µs ±30% ~ (p=0.071 n=20+17) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-8 3.42µs ±51% 3.68µs ±46% ~ (p=0.478 n=19+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-8 29.6ns ±34% 25.8ns ±59% ~ (p=0.304 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-8 2.39µs ±48% 2.83µs ±15% ~ (p=0.067 n=20+16) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-8 3.48µs ±46% 3.54µs ±30% ~ (p=0.965 n=20+18) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-8 26.8ns ±44% 26.8ns ±49% ~ (p=0.984 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-8 2.58µs ±56% 2.98µs ±35% ~ (p=0.326 n=20+17) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-8 3.55µs ±49% 3.98µs ±32% ~ (p=0.168 n=20+16) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-8 31.1ns ±47% 32.5ns ±37% ~ (p=0.567 n=19+17) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-8 2.39µs ±52% 2.72µs ±48% ~ (p=0.192 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-8 3.72µs ±37% 4.03µs ±38% ~ (p=0.150 n=18+19) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-8 23.4ns ±53% 25.8ns ±54% ~ (p=0.083 n=18+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-8 2.14µs ±56% 2.30µs ±57% ~ (p=0.743 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-8 3.87µs ±56% 3.83µs ±40% ~ (p=1.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-8 24.3ns ±64% 30.7ns ±37% +26.07% (p=0.015 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-8 2.18µs ±52% 2.04µs ±59% ~ (p=0.612 n=19+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-8 4.06µs ±42% 4.21µs ±33% ~ (p=0.748 n=19+17) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-8 28.4ns ±44% 30.2ns ±59% ~ (p=0.641 n=19+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-8 2.76µs ±54% 2.55µs ±52% ~ (p=0.221 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-8 4.46µs ±42% 4.03µs ±43% ~ (p=0.314 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-8 33.1ns ±28% 33.8ns ±21% ~ (p=0.468 n=18+16) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-8 2.95µs ±50% 2.99µs ±52% ~ (p=0.968 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-8 4.41µs ±44% 4.57µs ±52% ~ (p=0.799 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-8 23.9ns ±38% 31.4ns ±67% +31.74% (p=0.001 n=18+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-8 3.24µs ±46% 3.05µs ±52% ~ (p=0.414 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-8 4.64µs ±40% 4.50µs ±44% ~ (p=0.588 n=20+19) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-8 26.3ns ±41% 33.2ns ±55% +26.20% (p=0.001 n=17+19) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-8 3.36µs ±56% 3.26µs ±78% ~ (p=0.398 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-8 4.98µs ±38% 4.54µs ±61% ~ (p=0.301 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-8 26.8ns ±55% 26.1ns ±51% ~ (p=0.878 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-8 3.76µs ±52% 3.77µs ±29% ~ (p=0.493 n=17+16) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-8 4.97µs ±35% 4.46µs ±34% -10.36% (p=0.042 n=19+18) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-8 32.3ns ±68% 28.4ns ±33% ~ (p=0.080 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-8 3.18µs ±52% 3.24µs ±39% ~ (p=0.678 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-8 729µs ± 4% 723µs ± 2% ~ (p=0.068 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-8 26.3ns ±25% 30.0ns ±14% +13.96% (p=0.001 n=16+19) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-8 3.89µs ±28% 3.53µs ±49% ~ (p=0.235 n=19+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-8 736µs ± 5% 734µs ± 9% ~ (p=0.602 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-8 3.30µs ±16% 2.34µs ±55% -29.03% (p=0.000 n=15+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-8 3.37µs ±46% 3.67µs ±45% ~ (p=0.786 n=20+19) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-8 739µs ± 4% 735µs ± 6% ~ (p=0.398 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-8 7.72µs ±37% 6.40µs ±31% -17.10% (p=0.001 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-8 3.65µs ±44% 3.27µs ±47% ~ (p=0.242 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-8 740µs ± 4% 726µs ± 6% -1.83% (p=0.007 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-8 5.49µs ±33% 4.28µs ±45% -22.11% (p=0.001 n=19+20) name old alloc/op new alloc/op delta ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-8 440B ± 0% 320B ± 0% -27.27% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-8 440B ± 0% 320B ± 0% -27.27% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-8 440B ± 0% 320B ± 0% -27.27% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-8 440B ± 0% 320B ± 0% -27.27% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-8 632B ± 0% 512B ± 0% -18.99% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-8 632B ± 0% 512B ± 0% -18.99% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-8 632B ± 0% 512B ± 0% -18.99% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-8 632B ± 0% 512B ± 0% -18.99% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-8 424B ± 0% 424B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-8 1.06kB ± 0% 1.06kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-8 424B ± 0% 424B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-8 1.06kB ± 0% 1.06kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-8 1.06kB ± 0% 1.06kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-8 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-8 1.06kB ± 0% 1.06kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-8 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-8 1.21kB ± 0% 1.21kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-8 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-8 1.21kB ± 0% 1.21kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-8 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-8 1.21kB ± 0% 1.21kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-8 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-8 1.21kB ± 0% 1.21kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-8 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-8 1.43kB ± 0% 1.43kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-8 624B ± 0% 624B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-8 1.43kB ± 0% 1.43kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-8 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-8 1.43kB ± 0% 1.43kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-8 672B ± 0% 672B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-8 1.43kB ± 0% 1.43kB ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-8 568B ± 0% 568B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-8 493kB ± 0% 493kB ± 0% ~ (p=0.592 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-8 672B ± 0% 672B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-8 493kB ± 0% 493kB ± 0% ~ (p=0.107 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-8 568B ± 0% 568B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-8 493kB ± 0% 493kB ± 0% +0.00% (p=0.001 n=16+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-8 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-8 720B ± 0% 720B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-8 493kB ± 0% 493kB ± 0% ~ (p=1.081 n=19+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-8 0.00B 0.00B ~ (all equal) name old allocs/op new allocs/op delta ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-8 17.0 ± 0% 10.0 ± 0% -41.18% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-8 17.0 ± 0% 10.0 ± 0% -41.18% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-8 17.0 ± 0% 10.0 ± 0% -41.18% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-8 17.0 ± 0% 10.0 ± 0% -41.18% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-8 21.0 ± 0% 14.0 ± 0% -33.33% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-8 21.0 ± 0% 14.0 ± 0% -33.33% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-8 21.0 ± 0% 14.0 ± 0% -33.33% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-8 21.0 ± 0% 14.0 ± 0% -33.33% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-8 22.0 ± 0% 22.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-8 29.0 ± 0% 29.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-8 22.0 ± 0% 22.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-8 29.0 ± 0% 29.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-8 29.0 ± 0% 29.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-8 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-8 29.0 ± 0% 29.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-8 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-8 36.0 ± 0% 36.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-8 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-8 36.0 ± 0% 36.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-8 36.0 ± 0% 36.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-8 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-8 36.0 ± 0% 36.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-8 28.0 ± 0% 28.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-8 32.0 ± 0% 32.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-8 28.0 ± 0% 28.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-8 33.0 ± 0% 33.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-8 30.0 ± 0% 30.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-8 33.0 ± 0% 33.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-8 30.0 ± 0% 30.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-8 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-8 34.0 ± 0% 34.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-8 42.0 ± 0% 42.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-8 0.00 0.00 ~ (all equal) --- lib/execution_segment.go | 5 ++++- lib/execution_segment_test.go | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index db34b76a360..88a970e4f97 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -576,6 +576,9 @@ func (et *ExecutionTuple) ScaleInt64(value int64) int64 { if et.esIndex == -1 { return 0 } + if len(et.sequence) == 1 { + return value + } et.once.Do(et.fillCache) offsets := et.offsetsCache[et.esIndex] return scaleInt64(value, offsets[0], offsets[1:], et.lcd) @@ -583,7 +586,7 @@ func (et *ExecutionTuple) ScaleInt64(value int64) int64 { // scaleInt64With scales the provided value based on the ExecutionTuples' // sequence and the segment provided -func (et *ExecutionTuple) scaleInt64With(value int64, es *ExecutionSegment) int64 { +func (et *ExecutionTuple) scaleInt64With(value int64, es *ExecutionSegment) int64 { //nolint:unused start, offsets, lcd := et.GetStripedOffsets(es) return scaleInt64(value, start, offsets, lcd) } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 89b00405783..3513f50cd89 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -906,9 +906,9 @@ func BenchmarkExecutionSegmentScale(b *testing.B) { b.Run(fmt.Sprintf("et.Scale(%d)", value), func(b *testing.B) { for i := 0; i < b.N; i++ { - et2, err := NewExecutionTuple(segment, &ess) + et, err = NewExecutionTuple(segment, &ess) require.NoError(b, err) - et2.ScaleInt64(value) + et.ScaleInt64(value) } }) From c2f3163cbf35867c0ef82d2b9cd807640a3dd2cc Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 1 Apr 2020 13:14:26 +0300 Subject: [PATCH 163/350] Add/remove TODOs --- core/local/local.go | 1 - lib/executor/constant_arrival_rate.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index e0f74bf5921..888b0c5df39 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -59,7 +59,6 @@ var _ lib.ExecutionScheduler = &ExecutionScheduler{} // doesn't initialize the executors and it doesn't initialize or run VUs. func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*ExecutionScheduler, error) { options := runner.GetOptions() - // TODO figure out a way to give it to executionStage that is not terrible et, err := lib.NewExecutionTuple(options.ExecutionSegment, options.ExecutionSegmentSequence) if err != nil { return nil, err diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index d89071b142d..d80d64ed3f1 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -190,7 +190,6 @@ var _ lib.Executor = &ConstantArrivalRate{} // Init values needed for the execution func (car *ConstantArrivalRate) Init(ctx context.Context) error { car.et = car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(car.config.MaxVUs.Int64) - // TODO mvoe the preallocation of VUs here ? return nil } @@ -202,6 +201,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC duration := time.Duration(car.config.Duration.Duration) preAllocatedVUs := car.config.GetPreAllocatedVUs(car.executionState.ExecutionTuple) maxVUs := car.config.GetMaxVUs(car.executionState.ExecutionTuple) + // TODO: refactor and simplify arrivalRate := getScaledArrivalRate(car.et.ES, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() From 4af3174cafc9481f9714bca66dc4bd74524f3efa Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 1 Apr 2020 15:55:33 +0300 Subject: [PATCH 164/350] typo fix MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-Authored-By: Ivan Mirić --- lib/execution_segment.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 88a970e4f97..a25dc9c03d3 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -549,7 +549,7 @@ func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequ et.esIndex = et.find(segment) if et.esIndex == -1 { - return nil, fmt.Errorf("coulnd't find segment %s in sequence %s", segment, sequence) + return nil, fmt.Errorf("couldn't find segment %s in sequence %s", segment, sequence) } return &et, nil } From 22e8ab0c9a020e218044a2f2372d24ba49827786 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 1 Apr 2020 16:25:16 +0300 Subject: [PATCH 165/350] fix golangci issues --- lib/executor/execution_test.go | 7 ++++--- ui/pb/progressbar.go | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/lib/executor/execution_test.go b/lib/executor/execution_test.go index 190631fc414..a038c50612a 100644 --- a/lib/executor/execution_test.go +++ b/lib/executor/execution_test.go @@ -93,9 +93,10 @@ func TestExecutionStateGettingVUs(t *testing.T) { return &minirunner.VU{}, nil }) + var vu lib.VU for i := 0; i < 10; i++ { require.EqualValues(t, i, es.GetInitializedVUsCount()) - vu, err := es.InitializeNewVU(context.Background(), logEntry) + vu, err = es.InitializeNewVU(context.Background(), logEntry) require.NoError(t, err) require.EqualValues(t, i+1, es.GetInitializedVUsCount()) es.ReturnVU(vu, false) @@ -106,7 +107,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { // Test getting initialized VUs is okay :) for i := 0; i < 10; i++ { require.EqualValues(t, i, es.GetCurrentlyActiveVUsCount()) - vu, err := es.GetPlannedVU(logEntry, true) + vu, err = es.GetPlannedVU(logEntry, true) require.NoError(t, err) require.Empty(t, logHook.Drain()) require.NotNil(t, vu) @@ -115,7 +116,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { } // Check that getting 1 more planned VU will error out - vu, err := es.GetPlannedVU(logEntry, true) + vu, err = es.GetPlannedVU(logEntry, true) require.Nil(t, vu) require.Error(t, err) require.Contains(t, err.Error(), "could not get a VU from the buffer in") diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 3878ca4ddc6..24ff9a795d6 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -162,11 +162,11 @@ func (pb *ProgressBar) Modify(options ...ProgressBarOption) { // elements in the terminal output (e.g. for responsive progress // bars). type ProgressBarRender struct { - Color bool + Right []string progress, progressFill, progressPadding string Left, Hijack string status Status - Right []string + Color bool } // Status returns an optionally colorized status string From 4f355f63fa9448d094303014de0061ad159dc6d5 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 9 Apr 2020 17:35:12 +0300 Subject: [PATCH 166/350] Refactor the Engine to process metrics independently from the test Run This allows metric processing to still occur, even after the script run has completed. In practice, this allows you to run k6 with the --linger --no-teardown flags, and execute the teardown() function manually through the REST API, after the main test run has completed. This also fixes https://github.com/loadimpact/k6/issues/1369 - now a second Ctrl+C will _always_ abort k6, albeit in a very abrupt way. On the other hand, the first Ctrl+C would be more graceful than it currently is, since it will now allow teardown() to be executed. --- api/v1/setup_teardown_routes_test.go | 17 +- cmd/run.go | 178 ++++++++++--------- core/engine.go | 248 ++++++++++++++++----------- core/engine_test.go | 213 ++++++++++++----------- core/local/local.go | 16 +- core/local/local_test.go | 26 +-- js/runner_test.go | 8 +- lib/execution.go | 2 +- 8 files changed, 394 insertions(+), 314 deletions(-) diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index f722a55619a..d1bd974ac0b 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -46,7 +46,7 @@ import ( func TestSetupData(t *testing.T) { t.Parallel() - var testCases = []struct { + testCases := []struct { name string script []byte setupRuns [][3]string @@ -154,9 +154,16 @@ func TestSetupData(t *testing.T) { engine, err := core.NewEngine(execScheduler, runner.GetOptions(), logrus.StandardLogger()) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) + globalCtx, globalCancel := context.WithCancel(context.Background()) + runCtx, runCancel := context.WithCancel(globalCtx) + run, wait, err := engine.Init(globalCtx, runCtx) + defer wait() + defer globalCancel() + + require.NoError(t, err) + errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() handler := NewHandler() @@ -185,10 +192,10 @@ func TestSetupData(t *testing.T) { select { case <-time.After(10 * time.Second): - cancel() + runCancel() t.Fatal("Test timed out") case err := <-errC: - cancel() + runCancel() require.NoError(t, err) } }) diff --git a/cmd/run.go b/cmd/run.go index 94cacc29684..bd1ac445eca 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -60,9 +60,10 @@ const ( genericTimeoutErrorCode = 102 genericEngineErrorCode = 103 invalidConfigErrorCode = 104 + externalAbortErrorCode = 105 ) -//TODO: fix this, global variables are not very testable... +// TODO: fix this, global variables are not very testable... //nolint:gochecknoglobals var runType = os.Getenv("K6_TYPE") @@ -94,7 +95,7 @@ a commandline interface for interacting with it.`, k6 run -o influxdb=http://1.2.3.4:8086/k6`[1:], Args: exactArgsWithMsg(1, "arg should either be \"-\", if reading script from stdin, or a path to a script file"), RunE: func(cmd *cobra.Command, args []string) error { - //TODO: disable in quiet mode? + // TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) initBar := pb.New(pb.WithConstLeft(" init")) @@ -143,11 +144,25 @@ a commandline interface for interacting with it.`, return err } - //TODO: don't use a global... or maybe change the logger? + // TODO: don't use a global... or maybe change the logger? logger := logrus.StandardLogger() - ctx, cancel := context.WithCancel(context.Background()) //TODO: move even earlier? - defer cancel() + // We prepare a bunch of contexts: + // - The runCtx is cancelled as soon as the Engine.Run() method finishes, + // and can trigger things like the usage report and end of test summary. + // Crucially, metrics processing by the Engine will still work after this + // context is cancelled! + // - The lingerCtx is cancelled by Ctrl+C, and is used to wait for that + // event when k6 was ran with the --linger option. + // - The globalCtx is cancelled only after we're completely done with the + // test execution and any --linger has been cleared, so that the Engine + // can start winding down its metrics processing. + globalCtx, globalCancel := context.WithCancel(context.Background()) + defer globalCancel() + lingerCtx, lingerCancel := context.WithCancel(globalCtx) + defer lingerCancel() + runCtx, runCancel := context.WithCancel(lingerCtx) + defer runCancel() // Create a local execution scheduler wrapping the runner. printBar(initBar, "execution scheduler") @@ -157,11 +172,18 @@ a commandline interface for interacting with it.`, } executionState := execScheduler.GetState() + + // This is manually triggered after the Engine's Run() has completed, + // and things like a single Ctrl+C don't affect it. We use it to make + // sure that the progressbars finish updating with the latest execution + // state one last ime, after the test run has finished. + progressCtx, progressCancel := context.WithCancel(globalCtx) + defer progressCancel() initBar = execScheduler.GetInitProgressBar() progressBarWG := &sync.WaitGroup{} progressBarWG.Add(1) go func() { - showProgress(ctx, conf, execScheduler, logger) + showProgress(progressCtx, conf, execScheduler, logger) progressBarWG.Done() }() @@ -172,7 +194,7 @@ a commandline interface for interacting with it.`, return err } - //TODO: the engine should just probably have a copy of the config... + // TODO: refactor, the engine should have a copy of the config... // Configure the engine. if conf.NoThresholds.Valid { engine.NoThresholds = conf.NoThresholds.Bool @@ -188,12 +210,12 @@ a commandline interface for interacting with it.`, initBar.Modify(pb.WithConstProgress(0, "Init metric outputs")) for _, out := range conf.Out { t, arg := parseCollector(out) - collector, err := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) - if err != nil { - return err + collector, cerr := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) + if cerr != nil { + return cerr } - if err := collector.Init(); err != nil { - return err + if cerr = collector.Init(); err != nil { + return cerr } engine.Collectors = append(engine.Collectors, collector) } @@ -202,14 +224,14 @@ a commandline interface for interacting with it.`, if address != "" { initBar.Modify(pb.WithConstProgress(0, "Init API server")) go func() { - if err := api.ListenAndServe(address, engine); err != nil { - logger.WithError(err).Warn("Error from API server") + if aerr := api.ListenAndServe(address, engine); err != nil { + logger.WithError(aerr).Warn("Error from API server") } }() } // Write the big banner. - { + { // TODO: rewrite as Engine.GetTestRunDescription() and move out of here out := "-" link := "" @@ -246,81 +268,61 @@ a commandline interface for interacting with it.`, fprintf(stdout, "\n") } - // Run the engine with a cancellable context. - errC := make(chan error) - go func() { - initBar.Modify(pb.WithConstProgress(0, "Init VUs")) - if err := engine.Init(ctx); err != nil { - errC <- err - } else { - initBar.Modify(pb.WithConstProgress(0, "Start test")) - errC <- engine.Run(ctx) - } - }() - // Trap Interrupts, SIGINTs and SIGTERMs. sigC := make(chan os.Signal, 1) signal.Notify(sigC, os.Interrupt, syscall.SIGINT, syscall.SIGTERM) defer signal.Stop(sigC) - mainLoop: - for { - select { - case err := <-errC: - cancel() - if err == nil { - logger.Debug("Engine terminated cleanly") - break mainLoop - } + go func() { + sig := <-sigC + logger.WithField("sig", sig).Debug("Stopping k6 in response to signal...") + lingerCancel() // stop the test run, metric processing is cancelled below + + // If we get a second signal, we immediately exit, so something like + // https://github.com/loadimpact/k6/issues/971 never happens again + sig = <-sigC + logger.WithField("sig", sig).Error("Aborting k6 in response to signal") + globalCancel() // not that it matters, given the following command... + os.Exit(externalAbortErrorCode) + }() - switch e := errors.Cause(err).(type) { - case lib.TimeoutError: - switch e.Place() { - case "setup": - return ExitCode{error: err, Code: setupTimeoutErrorCode, Hint: e.Hint()} - case "teardown": - return ExitCode{error: err, Code: teardownTimeoutErrorCode, Hint: e.Hint()} - default: - return ExitCode{error: err, Code: genericTimeoutErrorCode} - } - default: - //nolint:golint - return ExitCode{error: errors.New("Engine error"), Code: genericEngineErrorCode, Hint: err.Error()} - } - case sig := <-sigC: - logger.WithField("sig", sig).Debug("Exiting in response to signal") - cancel() - //TODO: Actually exit on a second Ctrl+C, even if some of the iterations are stuck. - // This is currently problematic because of https://github.com/loadimpact/k6/issues/971, - // but with uninterruptible iterations it will be even more problematic. - } + // Initialize the engine + initBar.Modify(pb.WithConstProgress(0, "Init VUs")) + engineRun, engineWait, err := engine.Init(globalCtx, runCtx) + if err != nil { + return getExitCodeFromEngine(err) } - var reportCh chan struct{} + // Init has passed successfully, so unless disabled, make sure we send a + // usage report after the context is done. if !conf.NoUsageReport.Bool { - reportCh = make(chan struct{}) + reportDone := make(chan struct{}) go func() { + <-runCtx.Done() _ = reportUsage(execScheduler) - close(reportCh) + close(reportDone) + }() + defer func() { + select { + case <-reportDone: + case <-time.After(3 * time.Second): + } }() } - if quiet || !stdoutTTY { - e := logger.WithFields(logrus.Fields{ - "t": executionState.GetCurrentTestRunDuration(), - "i": executionState.GetFullIterationCount(), - }) - fn := e.Info - if quiet { - fn = e.Debug - } - fn("Test finished") + // Initialize the engine + initBar.Modify(pb.WithConstProgress(0, "Start test")) + if err := engineRun(); err != nil { + return getExitCodeFromEngine(err) } + runCancel() + logger.Debug("Engine terminated cleanly") + progressCancel() progressBarWG.Wait() // Warn if no iterations could be completed. if executionState.GetFullIterationCount() == 0 { - logger.Warn("No data generated, because no script iterations finished, consider making the test duration longer") + logger.Warn("No script iterations finished, consider making the test duration longer") } data := ui.SummaryData{ @@ -357,16 +359,17 @@ a commandline interface for interacting with it.`, } if conf.Linger.Bool { - logger.Info("Linger set; waiting for Ctrl+C...") - <-sigC - } - - if reportCh != nil { select { - case <-reportCh: - case <-time.After(3 * time.Second): + case <-lingerCtx.Done(): + // do nothing, we were interrupted by Ctrl+C already + default: + logger.Info("Linger set; waiting for Ctrl+C...") + <-lingerCtx.Done() } } + globalCancel() // signal the Engine that it should wind down + logger.Debug("Waiting for engine processes to finish...") + engineWait() if engine.IsTainted() { return ExitCode{error: errors.New("some thresholds have failed"), Code: thresholdHaveFailedErrorCode} @@ -375,6 +378,23 @@ a commandline interface for interacting with it.`, }, } +func getExitCodeFromEngine(err error) ExitCode { + switch e := errors.Cause(err).(type) { + case lib.TimeoutError: + switch e.Place() { + case "setup": + return ExitCode{error: err, Code: setupTimeoutErrorCode, Hint: e.Hint()} + case "teardown": + return ExitCode{error: err, Code: teardownTimeoutErrorCode, Hint: e.Hint()} + default: + return ExitCode{error: err, Code: genericTimeoutErrorCode} + } + default: + //nolint:golint + return ExitCode{error: errors.New("Engine error"), Code: genericEngineErrorCode, Hint: err.Error()} + } +} + func reportUsage(execScheduler *local.ExecutionScheduler) error { execState := execScheduler.GetState() executorConfigs := execScheduler.GetExecutorConfigs() @@ -413,7 +433,7 @@ func runCmdFlagSet() *pflag.FlagSet { flags.AddFlagSet(runtimeOptionFlagSet(true)) flags.AddFlagSet(configFlagSet()) - //TODO: Figure out a better way to handle the CLI flags: + // TODO: Figure out a better way to handle the CLI flags: // - the default values are specified in this way so we don't overwrire whatever // was specified via the environment variables // - but we need to manually specify the DefValue, since that's the default value diff --git a/core/engine.go b/core/engine.go index 9717cddb473..d44129f4c09 100644 --- a/core/engine.go +++ b/core/engine.go @@ -45,11 +45,16 @@ const ( BackoffMax = 10 * time.Second ) -// The Engine is the beating heart of K6. +// The Engine is the beating heart of k6. type Engine struct { - runLock sync.Mutex // y tho? TODO: remove? + // TODO: Make most of the stuff here private! And think how to refactor the + // engine to be less stateful... it's currently one big mess of moving + // pieces, and you implicitly first have to call Init() and then Run() - + // maybe we should refactor it so we have a `Session` dauther-object that + // Init() returns? The only problem with doing this is the REST API - it + // expects to be able to get information from the Engine and is initialized + // before the Init() call... - //TODO: make most of the stuff here private! ExecutionScheduler lib.ExecutionScheduler executionState *lib.ExecutionState @@ -59,7 +64,7 @@ type Engine struct { NoSummary bool SummaryExport bool - logger *logrus.Logger + logger *logrus.Entry stopChan chan struct{} Metrics map[string]*stats.Metric @@ -89,7 +94,7 @@ func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) Metrics: make(map[string]*stats.Metric), Samples: make(chan stats.SampleContainer, o.MetricSamplesBufferSize.Int64), stopChan: make(chan struct{}), - logger: logger, + logger: logger.WithField("component", "engine"), } e.thresholds = o.Thresholds @@ -106,128 +111,182 @@ func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) return e, nil } -// Init is used to initialize the execution scheduler. That's a costly operation, since it -// initializes all of the planned VUs and could potentially take a long time. -func (e *Engine) Init(ctx context.Context) error { - return e.ExecutionScheduler.Init(ctx, e.Samples) -} - -func (e *Engine) setRunStatus(status lib.RunStatus) { - for _, c := range e.Collectors { - c.SetRunStatus(status) +// Init is used to initialize the execution scheduler and all metrics processing +// in the engine. The first is a costly operation, since it initializes all of +// the planned VUs and could potentially take a long time. It either returns an +// error immediately, or it returns test Run() and WindDown() functions. +// +// Things to note: +// - The first lambda, Run(), synchronously executes the actual load test. +// - It can be prematurely aborted by cancelling the runCtx - this won't stop +// the metrics collection by the Engine. +// - Stopping the metrics collection can be done at any time after Run() has +// returned by cancelling the globalCtx +// - The second returned lambda can be used to wait for that process to finish. +func (e *Engine) Init(globalCtx, runCtx context.Context) (run func() error, wait func(), err error) { + e.logger.Debug("Initialization starting...") + // TODO: if we ever need metrics processing in the init context, we can move + // this below the other components... or even start them concurrently? + if err := e.ExecutionScheduler.Init(runCtx, e.Samples); err != nil { + return nil, nil, err } -} -func (e *Engine) Run(ctx context.Context) error { - e.runLock.Lock() - defer e.runLock.Unlock() + // TODO: move all of this in a separate struct? see main TODO above + + runSubCtx, runSubCancel := context.WithCancel(runCtx) + + resultCh := make(chan error) + runFn := func() error { + e.logger.Debug("Execution scheduler starting...") + err := e.ExecutionScheduler.Run(globalCtx, runSubCtx, e.Samples) + e.logger.WithError(err).Debug("Execution scheduler terminated") + + select { + case <-runSubCtx.Done(): + // do nothing, the test run was aborted somehow + default: + resultCh <- err // we finished normally, so send the result + } - e.logger.Debug("Engine: Starting with parameters...") + return err + } + waitFn := e.startBackgroundProcesses(globalCtx, runCtx, resultCh, runSubCancel) + return runFn, waitFn, nil +} - collectorwg := sync.WaitGroup{} - collectorctx, collectorcancel := context.WithCancel(context.Background()) +// This starts a bunch of goroutines to process metrics, thresholds, and set the +// test run status when it ends. It returns a function that can be used after +// the provided context is called, to wait for the complete winding down of all +// started goroutines. +func (e *Engine) startBackgroundProcesses( //nolint:funlen + globalCtx, runCtx context.Context, runResult <-chan error, runSubCancel func(), +) (wait func()) { + processes := new(sync.WaitGroup) + // Spin up all configured collectors for _, collector := range e.Collectors { - collectorwg.Add(1) + processes.Add(1) go func(collector lib.Collector) { - collector.Run(collectorctx) - collectorwg.Done() + collector.Run(globalCtx) + processes.Done() }(collector) } - subctx, subcancel := context.WithCancel(context.Background()) - subwg := sync.WaitGroup{} + // Siphon and handle all produced metric samples + processes.Add(1) + go func() { + defer processes.Done() + e.processMetrics(globalCtx) + }() - // Run metrics emission. - subwg.Add(1) + // Run VU metrics emission, only while the test is running. + // TODO: move? this seems like something the ExecutionScheduler should emit... + processes.Add(1) go func() { - e.runMetricsEmission(subctx) - e.logger.Debug("Engine: Emission terminated") - subwg.Done() + defer processes.Done() + e.logger.Debug("Starting emission of VU metrics...") + e.runMetricsEmission(runCtx) + e.logger.Debug("Metrics emission terminated") }() - // Run thresholds. + // Update the test run status when the test finishes + processes.Add(1) + thresholdAbortChan := make(chan struct{}) + go func() { + defer processes.Done() + select { + case err := <-runResult: + if err != nil { + e.logger.WithError(err).Debug("run: execution scheduler returned an error") + e.setRunStatus(lib.RunStatusAbortedSystem) + } else { + e.logger.Debug("run: execution scheduler terminated") + e.setRunStatus(lib.RunStatusFinished) + } + case <-runCtx.Done(): + e.logger.Debug("run: context expired; exiting...") + e.setRunStatus(lib.RunStatusAbortedUser) + case <-e.stopChan: + runSubCancel() + e.logger.Debug("run: stopped by user; exiting...") + e.setRunStatus(lib.RunStatusAbortedUser) + case <-thresholdAbortChan: + e.logger.Debug("run: stopped by thresholds; exiting...") + runSubCancel() + e.setRunStatus(lib.RunStatusAbortedThreshold) + } + }() + + // Run thresholds, if not disabled. if !e.NoThresholds { - subwg.Add(1) + processes.Add(1) go func() { - e.runThresholds(subctx, subcancel) - e.logger.Debug("Engine: Thresholds terminated") - subwg.Done() + defer processes.Done() + defer e.logger.Debug("Engine: Thresholds terminated") + ticker := time.NewTicker(ThresholdsRate) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + if e.processThresholds() { + close(thresholdAbortChan) + return + } + case <-runCtx.Done(): + return + } + } }() } - // Run the execution scheduler. - errC := make(chan error) - subwg.Add(1) - go func() { - errC <- e.ExecutionScheduler.Run(subctx, e.Samples) - e.logger.Debug("Engine: Execution scheduler terminated") - subwg.Done() - }() + return processes.Wait +} +func (e *Engine) processMetrics(globalCtx context.Context) { sampleContainers := []stats.SampleContainer{} - defer func() { - // Shut down subsystems. - subcancel() - // Process samples until the subsystems have shut down. - // Filter out samples produced past the end of a test. - go func() { - if errC != nil { - <-errC - errC = nil - } - subwg.Wait() - close(e.Samples) - }() + defer func() { + // Process any remaining metrics in the pipeline, by this point Run() + // has already finished and nothing else should be producing metrics. + e.logger.Debug("Metrics processing winding down...") + close(e.Samples) for sc := range e.Samples { sampleContainers = append(sampleContainers, sc) } - e.processSamples(sampleContainers) - // Process final thresholds. if !e.NoThresholds { - e.processThresholds(nil) + e.processThresholds() // Process the thresholds one final time } - - // Finally, shut down collector. - collectorcancel() - collectorwg.Wait() }() ticker := time.NewTicker(CollectRate) + defer ticker.Stop() + + e.logger.Debug("Metrics processing started...") for { select { case <-ticker.C: if len(sampleContainers) > 0 { e.processSamples(sampleContainers) - sampleContainers = []stats.SampleContainer{} + sampleContainers = []stats.SampleContainer{} // TODO: optimize? } case sc := <-e.Samples: sampleContainers = append(sampleContainers, sc) - case err := <-errC: - errC = nil - if err != nil { - e.logger.WithError(err).Debug("run: execution scheduler returned an error") - e.setRunStatus(lib.RunStatusAbortedSystem) - return err - } - e.logger.Debug("run: execution scheduler terminated") - return nil - case <-ctx.Done(): - e.logger.Debug("run: context expired; exiting...") - e.setRunStatus(lib.RunStatusAbortedUser) - return nil - case <-e.stopChan: - e.logger.Debug("run: stopped by user; exiting...") - e.setRunStatus(lib.RunStatusAbortedUser) - return nil + case <-globalCtx.Done(): + return } } } +func (e *Engine) setRunStatus(status lib.RunStatus) { + for _, c := range e.Collectors { + c.SetRunStatus(status) + } +} + func (e *Engine) IsTainted() bool { return e.thresholdsTainted } @@ -282,24 +341,11 @@ func (e *Engine) emitMetrics() { }}) } -func (e *Engine) runThresholds(ctx context.Context, abort func()) { - ticker := time.NewTicker(ThresholdsRate) - for { - select { - case <-ticker.C: - e.processThresholds(abort) - case <-ctx.Done(): - return - } - } -} - -func (e *Engine) processThresholds(abort func()) { +func (e *Engine) processThresholds() (shouldAbort bool) { e.MetricsLock.Lock() defer e.MetricsLock.Unlock() t := e.executionState.GetCurrentTestRunDuration() - abortOnFail := false e.thresholdsTainted = false for _, m := range e.Metrics { @@ -318,17 +364,13 @@ func (e *Engine) processThresholds(abort func()) { e.logger.WithField("m", m.Name).Debug("Thresholds failed") m.Tainted = null.BoolFrom(true) e.thresholdsTainted = true - if !abortOnFail && m.Thresholds.Abort { - abortOnFail = true + if m.Thresholds.Abort { + shouldAbort = true } } } - if abortOnFail && abort != nil { - //TODO: When sending this status we get a 422 Unprocessable Entity - e.setRunStatus(lib.RunStatusAbortedThreshold) - abort() - } + return shouldAbort } func (e *Engine) processSamplesForMetrics(sampleCointainers []stats.SampleContainer) { diff --git a/core/engine_test.go b/core/engine_test.go index 7b711629d77..beeb289538d 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -50,12 +50,16 @@ import ( const isWindows = runtime.GOOS == "windows" // Wrapper around NewEngine that applies a logger and manages the options. -func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts lib.Options) *Engine { //nolint: golint +func newTestEngine( //nolint:golint + t *testing.T, runCtx context.Context, runner lib.Runner, opts lib.Options, +) (engine *Engine, run func() error, wait func()) { if runner == nil { runner = &minirunner.MiniRunner{} } - if ctx == nil { - ctx = context.Background() + globalCtx, globalCancel := context.WithCancel(context.Background()) + var runCancel func() + if runCtx == nil { + runCtx, runCancel = context.WithCancel(globalCtx) } newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ @@ -72,12 +76,19 @@ func newTestEngine(t *testing.T, ctx context.Context, runner lib.Runner, opts li execScheduler, err := local.NewExecutionScheduler(runner, logger) require.NoError(t, err) - engine, err := NewEngine(execScheduler, opts, logger) + engine, err = NewEngine(execScheduler, opts, logger) require.NoError(t, err) - require.NoError(t, engine.Init(ctx)) + run, waitFn, err := engine.Init(globalCtx, runCtx) + require.NoError(t, err) - return engine + return engine, run, func() { + if runCancel != nil { + runCancel() + } + globalCancel() + waitFn() + } } func TestNewEngine(t *testing.T) { @@ -87,21 +98,32 @@ func TestNewEngine(t *testing.T) { func TestEngineRun(t *testing.T) { logrus.SetLevel(logrus.DebugLevel) t.Run("exits with context", func(t *testing.T) { - duration := 100 * time.Millisecond - e := newTestEngine(t, nil, nil, lib.Options{}) + done := make(chan struct{}) + runner := &minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + <-ctx.Done() + close(done) + return nil + }} + duration := 100 * time.Millisecond ctx, cancel := context.WithTimeout(context.Background(), duration) defer cancel() + + _, run, wait := newTestEngine(t, ctx, runner, lib.Options{}) + defer wait() + startTime := time.Now() - assert.NoError(t, e.Run(ctx)) + assert.NoError(t, run()) assert.WithinDuration(t, startTime.Add(duration), time.Now(), 100*time.Millisecond) + <-done }) t.Run("exits with executor", func(t *testing.T) { - e := newTestEngine(t, nil, nil, lib.Options{ + e, run, wait := newTestEngine(t, nil, nil, lib.Options{ VUs: null.IntFrom(10), Iterations: null.IntFrom(100), }) - assert.NoError(t, e.Run(context.Background())) + defer wait() + assert.NoError(t, run()) assert.Equal(t, uint64(100), e.ExecutionScheduler.GetState().GetFullIterationCount()) }) // Make sure samples are discarded after context close (using "cutoff" timestamp in local.go) @@ -119,7 +141,7 @@ func TestEngineRun(t *testing.T) { }} ctx, cancel := context.WithCancel(context.Background()) - e := newTestEngine(t, ctx, runner, lib.Options{ + e, run, wait := newTestEngine(t, ctx, runner, lib.Options{ VUs: null.IntFrom(1), Iterations: null.IntFrom(1), }) @@ -128,10 +150,11 @@ func TestEngineRun(t *testing.T) { e.Collectors = []lib.Collector{c} errC := make(chan error) - go func() { errC <- e.Run(ctx) }() + go func() { errC <- run() }() <-signalChan cancel() assert.NoError(t, <-errC) + wait() found := 0 for _, s := range c.Samples { @@ -146,25 +169,27 @@ func TestEngineRun(t *testing.T) { } func TestEngineAtTime(t *testing.T) { - e := newTestEngine(t, nil, nil, lib.Options{ + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) + defer cancel() + _, run, wait := newTestEngine(t, ctx, nil, lib.Options{ VUs: null.IntFrom(2), Duration: types.NullDurationFrom(20 * time.Second), }) + defer wait() - ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) - defer cancel() - assert.NoError(t, e.Run(ctx)) + assert.NoError(t, run()) } func TestEngineStopped(t *testing.T) { - e := newTestEngine(t, nil, nil, lib.Options{ + ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) + defer cancel() + e, run, wait := newTestEngine(t, ctx, nil, lib.Options{ VUs: null.IntFrom(1), Duration: types.NullDurationFrom(20 * time.Second), }) + defer wait() - ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) - defer cancel() - assert.NoError(t, e.Run(ctx)) + assert.NoError(t, run()) assert.Equal(t, false, e.IsStopped(), "engine should be running") e.Stop() assert.Equal(t, true, e.IsStopped(), "engine should be stopped") @@ -178,12 +203,13 @@ func TestEngineCollector(t *testing.T) { return nil }} - e := newTestEngine(t, nil, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) + e, run, wait := newTestEngine(t, nil, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) c := &dummy.Collector{} e.Collectors = []lib.Collector{c} - assert.NoError(t, e.Run(context.Background())) + assert.NoError(t, run()) + wait() cSamples := []stats.Sample{} for _, sample := range c.Samples { @@ -206,7 +232,8 @@ func TestEngine_processSamples(t *testing.T) { metric := stats.New("my_metric", stats.Gauge) t.Run("metric", func(t *testing.T) { - e := newTestEngine(t, nil, nil, lib.Options{}) + e, _, wait := newTestEngine(t, nil, nil, lib.Options{}) + defer wait() e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, @@ -218,11 +245,12 @@ func TestEngine_processSamples(t *testing.T) { ths, err := stats.NewThresholds([]string{`1+1==2`}) assert.NoError(t, err) - e := newTestEngine(t, nil, nil, lib.Options{ + e, _, wait := newTestEngine(t, nil, nil, lib.Options{ Thresholds: map[string]stats.Thresholds{ "my_metric{a:1}": ths, }, }) + defer wait() sms := e.submetrics["my_metric"] assert.Len(t, sms, 1) @@ -238,60 +266,54 @@ func TestEngine_processSamples(t *testing.T) { }) } -func TestEngine_runThresholds(t *testing.T) { +func TestEngineThresholdsWillAbort(t *testing.T) { metric := stats.New("my_metric", stats.Gauge) - thresholds := make(map[string]stats.Thresholds, 1) ths, err := stats.NewThresholds([]string{"1+1==3"}) assert.NoError(t, err) + ths.Thresholds[0].AbortOnFail = true - t.Run("aborted", func(t *testing.T) { - ths.Thresholds[0].AbortOnFail = true - thresholds[metric.Name] = ths - e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + thresholds := map[string]stats.Thresholds{metric.Name: ths} - e.processSamples( - []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, - ) + e, _, wait := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + defer wait() - ctx, cancel := context.WithCancel(context.Background()) - aborted := false - - cancelFunc := func() { - cancel() - aborted = true - } + e.processSamples( + []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, + ) + assert.True(t, e.processThresholds()) +} - e.runThresholds(ctx, cancelFunc) +func TestEngineAbortedByThresholds(t *testing.T) { + metric := stats.New("my_metric", stats.Gauge) - assert.True(t, aborted) - }) + ths, err := stats.NewThresholds([]string{"1+1==3"}) + assert.NoError(t, err) + ths.Thresholds[0].AbortOnFail = true - t.Run("canceled", func(t *testing.T) { - ths.Abort = false - thresholds[metric.Name] = ths - e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + thresholds := map[string]stats.Thresholds{metric.Name: ths} - e.processSamples( - []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, - ) + done := make(chan struct{}) + runner := &minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + out <- stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})} + <-ctx.Done() + close(done) + return nil + }} - ctx, cancel := context.WithCancel(context.Background()) - cancel() + _, run, wait := newTestEngine(t, nil, runner, lib.Options{Thresholds: thresholds}) + defer wait() - done := make(chan struct{}) - go func() { - defer close(done) - e.runThresholds(ctx, cancel) - }() + go func() { + assert.NoError(t, run()) + }() - select { - case <-done: - return - case <-time.After(1 * time.Second): - assert.Fail(t, "Test should have completed within a second") - } - }) + select { + case <-done: + return + case <-time.After(10 * time.Second): + assert.Fail(t, "Test should have completed within 10 seconds") + } } func TestEngine_processThresholds(t *testing.T) { @@ -313,6 +335,7 @@ func TestEngine_processThresholds(t *testing.T) { } for name, data := range testdata { + name, data := name, data t.Run(name, func(t *testing.T) { thresholds := make(map[string]stats.Thresholds, len(data.ths)) for m, srcs := range data.ths { @@ -322,24 +345,15 @@ func TestEngine_processThresholds(t *testing.T) { thresholds[m] = ths } - e := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + e, _, wait := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + defer wait() e.processSamples( []stats.SampleContainer{stats.Sample{Metric: metric, Value: 1.25, Tags: stats.IntoSampleTags(&map[string]string{"a": "1"})}}, ) - abortCalled := false - - abortFunc := func() { - abortCalled = true - } - - e.processThresholds(abortFunc) - + assert.Equal(t, data.abort, e.processThresholds()) assert.Equal(t, data.pass, !e.IsTainted()) - if data.abort { - assert.True(t, abortCalled) - } }) } } @@ -430,9 +444,7 @@ func TestSentReceivedMetrics(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - engine := newTestEngine(t, ctx, r, lib.Options{ + engine, run, wait := newTestEngine(t, nil, r, lib.Options{ Iterations: null.IntFrom(tc.Iterations), VUs: null.IntFrom(tc.VUs), Hosts: tb.Dialer.Hosts, @@ -445,7 +457,7 @@ func TestSentReceivedMetrics(t *testing.T) { engine.Collectors = []lib.Collector{collector} errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -453,6 +465,7 @@ func TestSentReceivedMetrics(t *testing.T) { case err := <-errC: require.NoError(t, err) } + wait() checkData := func(name string, expected int64) float64 { data := getMetricSum(collector, name) @@ -566,10 +579,7 @@ func TestRunTags(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - engine := newTestEngine(t, ctx, r, lib.Options{ + engine, run, wait := newTestEngine(t, nil, r, lib.Options{ Iterations: null.IntFrom(3), VUs: null.IntFrom(2), Hosts: tb.Dialer.Hosts, @@ -577,12 +587,13 @@ func TestRunTags(t *testing.T) { SystemTags: &stats.DefaultSystemTagSet, InsecureSkipTLSVerify: null.BoolFrom(true), }) + defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -661,17 +672,16 @@ func TestSetupTeardownThresholds(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - engine := newTestEngine(t, ctx, runner, lib.Options{ + engine, run, wait := newTestEngine(t, nil, runner, lib.Options{ SystemTags: &stats.DefaultSystemTagSet, SetupTimeout: types.NullDurationFrom(3 * time.Second), TeardownTimeout: types.NullDurationFrom(3 * time.Second), VUs: null.IntFrom(3), }) + defer wait() errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -725,15 +735,14 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - engine := newTestEngine(t, ctx, runner, lib.Options{}) + engine, run, wait := newTestEngine(t, nil, runner, lib.Options{}) + defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -832,16 +841,14 @@ func TestMetricsEmission(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - engine := newTestEngine(t, ctx, runner, runner.GetOptions()) + engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) + defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -921,13 +928,11 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { ) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - engine := newTestEngine(t, ctx, runner, runner.GetOptions()) + engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) + defer wait() errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): t.Fatal("Test timed out") diff --git a/core/local/local.go b/core/local/local.go index 888b0c5df39..5931907891b 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -337,7 +337,7 @@ func (e *ExecutionScheduler) runExecutor( // Run the ExecutionScheduler, funneling all generated metric samples through the supplied // out channel. -func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error { +func (e *ExecutionScheduler) Run(globalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error { executorsCount := len(e.executors) logger := e.logger.WithField("phase", "local-execution-scheduler-run") e.initProgress.Modify(pb.WithConstLeft("Run")) @@ -349,7 +349,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam select { case <-e.state.ResumeNotify(): // continue - case <-ctx.Done(): + case <-runCtx.Done(): return nil } } @@ -362,7 +362,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam runResults := make(chan error, executorsCount) // nil values are successful runs - runCtx, cancel := context.WithCancel(ctx) + runSubCtx, cancel := context.WithCancel(runCtx) defer cancel() // just in case, and to shut up go vet... // Run setup() before any executors, if it's not disabled @@ -370,7 +370,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam logger.Debug("Running setup()") e.state.SetExecutionStatus(lib.ExecutionStatusSetup) e.initProgress.Modify(pb.WithConstProgress(1, "setup()")) - if err := e.runner.Setup(runCtx, engineOut); err != nil { + if err := e.runner.Setup(runSubCtx, engineOut); err != nil { logger.WithField("error", err).Debug("setup() aborted by error") return err } @@ -381,7 +381,7 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam logger.Debug("Start all executors...") e.state.SetExecutionStatus(lib.ExecutionStatusRunning) for _, exec := range e.executors { - go e.runExecutor(runCtx, runResults, engineOut, exec) + go e.runExecutor(runSubCtx, runResults, engineOut, exec) } // Wait for all executors to finish @@ -399,7 +399,11 @@ func (e *ExecutionScheduler) Run(ctx context.Context, engineOut chan<- stats.Sam if !e.options.NoTeardown.Bool { logger.Debug("Running teardown()") e.state.SetExecutionStatus(lib.ExecutionStatusTeardown) - if err := e.runner.Teardown(ctx, engineOut); err != nil { + e.initProgress.Modify(pb.WithConstProgress(1, "teardown()")) + + // We run teardown() with the global context, so it isn't interrupted by + // aborts caused by thresholds or even Ctrl+C (unless used twice). + if err := e.runner.Teardown(globalCtx, engineOut); err != nil { logger.WithField("error", err).Debug("teardown() aborted by error") return err } diff --git a/core/local/local_test.go b/core/local/local_test.go index f420e546564..edd64851ecc 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -92,7 +92,7 @@ func TestExecutionSchedulerRun(t *testing.T) { defer cancel() err := make(chan error, 1) - go func() { err <- execScheduler.Run(ctx, samples) }() + go func() { err <- execScheduler.Run(ctx, ctx, samples) }() assert.NoError(t, <-err) } @@ -114,7 +114,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{}) err := make(chan error, 1) - go func() { err <- execScheduler.Run(ctx, samples) }() + go func() { err <- execScheduler.Run(ctx, ctx, samples) }() defer cancel() <-setupC <-teardownC @@ -128,7 +128,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { } ctx, cancel, execScheduler, samples := newTestExecutionScheduler(t, runner, nil, lib.Options{}) defer cancel() - assert.EqualError(t, execScheduler.Run(ctx, samples), "setup error") + assert.EqualError(t, execScheduler.Run(ctx, ctx, samples), "setup error") }) t.Run("Don't Run Setup", func(t *testing.T) { runner := &minirunner.MiniRunner{ @@ -145,7 +145,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { Iterations: null.IntFrom(1), }) defer cancel() - assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") + assert.EqualError(t, execScheduler.Run(ctx, ctx, samples), "teardown error") }) t.Run("Teardown Error", func(t *testing.T) { @@ -163,7 +163,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { }) defer cancel() - assert.EqualError(t, execScheduler.Run(ctx, samples), "teardown error") + assert.EqualError(t, execScheduler.Run(ctx, ctx, samples), "teardown error") }) t.Run("Don't Run Teardown", func(t *testing.T) { runner := &minirunner.MiniRunner{ @@ -180,7 +180,7 @@ func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { Iterations: null.IntFrom(1), }) defer cancel() - assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) }) } @@ -224,7 +224,7 @@ func TestExecutionSchedulerStages(t *testing.T) { Stages: data.Stages, }) defer cancel() - assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) assert.True(t, execScheduler.GetState().GetCurrentTestRunDuration() >= data.Duration) }) } @@ -249,7 +249,7 @@ func TestExecutionSchedulerEndTime(t *testing.T) { assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") @@ -276,7 +276,7 @@ func TestExecutionSchedulerRuntimeErrors(t *testing.T) { assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") @@ -313,7 +313,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { assert.True(t, isFinal) startTime := time.Now() - assert.NoError(t, execScheduler.Run(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) runTime := time.Since(startTime) assert.True(t, runTime > 1*time.Second, "test did not take 1s") assert.True(t, runTime < 10*time.Second, "took more than 10 seconds") @@ -357,7 +357,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { samples := make(chan stats.SampleContainer, 300) require.NoError(t, execScheduler.Init(ctx, samples)) - require.NoError(t, execScheduler.Run(ctx, samples)) + require.NoError(t, execScheduler.Run(ctx, ctx, samples)) assert.Equal(t, uint64(100), execScheduler.GetState().GetFullIterationCount()) assert.Equal(t, uint64(0), execScheduler.GetState().GetPartialIterationCount()) @@ -382,7 +382,7 @@ func TestExecutionSchedulerIsRunning(t *testing.T) { state := execScheduler.GetState() err := make(chan error) - go func() { err <- execScheduler.Run(ctx, nil) }() + go func() { err <- execScheduler.Run(ctx, ctx, nil) }() for !state.HasStarted() { time.Sleep(10 * time.Microsecond) } @@ -558,7 +558,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { sampleContainers := make(chan stats.SampleContainer) go func() { require.NoError(t, execScheduler.Init(ctx, sampleContainers)) - assert.NoError(t, execScheduler.Run(ctx, sampleContainers)) + assert.NoError(t, execScheduler.Run(ctx, ctx, sampleContainers)) close(done) }() diff --git a/js/runner_test.go b/js/runner_test.go index 8a87eabcfdf..dfc21bafc1c 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -47,7 +47,7 @@ import ( k6metrics "github.com/loadimpact/k6/js/modules/k6/metrics" "github.com/loadimpact/k6/js/modules/k6/ws" "github.com/loadimpact/k6/lib" - _ "github.com/loadimpact/k6/lib/executor" //TODO: figure out something better + _ "github.com/loadimpact/k6/lib/executor" // TODO: figure out something better "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/lib/types" @@ -284,13 +284,14 @@ func TestSetupDataIsolation(t *testing.T) { require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) - require.NoError(t, engine.Init(ctx)) + run, wait, err := engine.Init(ctx, ctx) + require.NoError(t, err) collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} errC := make(chan error) - go func() { errC <- engine.Run(ctx) }() + go func() { errC <- run() }() select { case <-time.After(10 * time.Second): @@ -300,6 +301,7 @@ func TestSetupDataIsolation(t *testing.T) { cancel() require.NoError(t, err) require.False(t, engine.IsTainted()) + wait() } var count int for _, s := range collector.Samples { diff --git a/lib/execution.go b/lib/execution.go index b6dd2deff34..415f72724c1 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -62,7 +62,7 @@ type ExecutionScheduler interface { // Run the ExecutionScheduler, funneling the generated metric samples // through the supplied out channel. - Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error + Run(lobalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error // Pause a test, or start/resume it. To check if a test is paused, use // GetState().IsPaused(). From 05f54089c30bb327122aadc20a1a5076b5940137 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 9 Apr 2020 17:58:07 +0300 Subject: [PATCH 167/350] Fix a racy test --- core/engine_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/engine_test.go b/core/engine_test.go index beeb289538d..bf62868bfd5 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -929,7 +929,6 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { require.NoError(t, err) engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) - defer wait() errC := make(chan error) go func() { errC <- run() }() @@ -938,6 +937,7 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { t.Fatal("Test timed out") case err := <-errC: require.NoError(t, err) + wait() require.False(t, engine.IsTainted()) } }) From de645da1ca36aeff2612cea9496bbb0007de4b70 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 9 Apr 2020 19:29:29 +0300 Subject: [PATCH 168/350] Fix more data races in tests --- core/engine_test.go | 6 +++--- js/runner_test.go | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/engine_test.go b/core/engine_test.go index bf62868bfd5..7d3c7d9479f 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -587,7 +587,6 @@ func TestRunTags(t *testing.T) { SystemTags: &stats.DefaultSystemTagSet, InsecureSkipTLSVerify: null.BoolFrom(true), }) - defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} @@ -601,6 +600,7 @@ func TestRunTags(t *testing.T) { case err := <-errC: require.NoError(t, err) } + wait() systemMetrics := []*stats.Metric{ metrics.VUs, metrics.VUsMax, metrics.Iterations, metrics.IterationDuration, @@ -736,7 +736,6 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { require.NoError(t, err) engine, run, wait := newTestEngine(t, nil, runner, lib.Options{}) - defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} @@ -749,6 +748,7 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { t.Fatal("Test timed out") case err := <-errC: require.NoError(t, err) + wait() require.False(t, engine.IsTainted()) } @@ -842,7 +842,6 @@ func TestMetricsEmission(t *testing.T) { require.NoError(t, err) engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) - defer wait() collector := &dummy.Collector{} engine.Collectors = []lib.Collector{collector} @@ -855,6 +854,7 @@ func TestMetricsEmission(t *testing.T) { t.Fatal("Test timed out") case err := <-errC: require.NoError(t, err) + wait() require.False(t, engine.IsTainted()) } diff --git a/js/runner_test.go b/js/runner_test.go index dfc21bafc1c..11309f96da7 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -300,8 +300,8 @@ func TestSetupDataIsolation(t *testing.T) { case err := <-errC: cancel() require.NoError(t, err) - require.False(t, engine.IsTainted()) wait() + require.False(t, engine.IsTainted()) } var count int for _, s := range collector.Samples { From f2ca5b08beff01637aeba8d0e8d1ad27bf321374 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 14 Apr 2020 10:34:29 +0300 Subject: [PATCH 169/350] Address code review comments --- core/engine.go | 23 +++++++++-------------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/core/engine.go b/core/engine.go index d44129f4c09..efea7b69373 100644 --- a/core/engine.go +++ b/core/engine.go @@ -35,14 +35,9 @@ import ( ) const ( - TickRate = 1 * time.Millisecond - MetricsRate = 1 * time.Second - CollectRate = 50 * time.Millisecond - ThresholdsRate = 2 * time.Second - ShutdownTimeout = 10 * time.Second - - BackoffAmount = 50 * time.Millisecond - BackoffMax = 10 * time.Second + metricsRate = 1 * time.Second + collectRate = 50 * time.Millisecond + thresholdsRate = 2 * time.Second ) // The Engine is the beating heart of k6. @@ -114,7 +109,7 @@ func NewEngine(ex lib.ExecutionScheduler, o lib.Options, logger *logrus.Logger) // Init is used to initialize the execution scheduler and all metrics processing // in the engine. The first is a costly operation, since it initializes all of // the planned VUs and could potentially take a long time. It either returns an -// error immediately, or it returns test Run() and WindDown() functions. +// error immediately, or it returns test run() and wait() functions. // // Things to note: // - The first lambda, Run(), synchronously executes the actual load test. @@ -223,7 +218,7 @@ func (e *Engine) startBackgroundProcesses( //nolint:funlen go func() { defer processes.Done() defer e.logger.Debug("Engine: Thresholds terminated") - ticker := time.NewTicker(ThresholdsRate) + ticker := time.NewTicker(thresholdsRate) defer ticker.Stop() for { @@ -262,7 +257,7 @@ func (e *Engine) processMetrics(globalCtx context.Context) { } }() - ticker := time.NewTicker(CollectRate) + ticker := time.NewTicker(collectRate) defer ticker.Stop() e.logger.Debug("Metrics processing started...") @@ -307,7 +302,7 @@ func (e *Engine) IsStopped() bool { } func (e *Engine) runMetricsEmission(ctx context.Context) { - ticker := time.NewTicker(MetricsRate) + ticker := time.NewTicker(metricsRate) for { select { case <-ticker.C: @@ -322,7 +317,7 @@ func (e *Engine) emitMetrics() { t := time.Now() executionState := e.ExecutionScheduler.GetState() - e.processSamples([]stats.SampleContainer{stats.ConnectedSamples{ + e.Samples <- stats.ConnectedSamples{ Samples: []stats.Sample{ { Time: t, @@ -338,7 +333,7 @@ func (e *Engine) emitMetrics() { }, Tags: e.Options.RunTags, Time: t, - }}) + } } func (e *Engine) processThresholds() (shouldAbort bool) { From 85d8826a0a395bf1bb26808c5e187f46195c80bd Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 14 Apr 2020 10:35:03 +0300 Subject: [PATCH 170/350] Add a test that ensures teardown() is ran even on aborted test runs --- core/engine_test.go | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/core/engine_test.go b/core/engine_test.go index 7d3c7d9479f..38e421f5248 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -943,3 +943,35 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { }) } } + +func TestEngineRunsTeardownEvenAfterTestRunIsAborted(t *testing.T) { + testMetric := stats.New("teardown_metric", stats.Counter) + + ctx, cancel := context.WithCancel(context.Background()) + + runner := &minirunner.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + cancel() // we cancel the runCtx immediately after the test starts + return nil + }, + TeardownFn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + out <- stats.Sample{Metric: testMetric, Value: 1} + return nil + }, + } + + e, run, wait := newTestEngine(t, ctx, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) + c := &dummy.Collector{} + e.Collectors = []lib.Collector{c} + + assert.NoError(t, run()) + wait() + + var count float64 + for _, sample := range c.Samples { + if sample.Metric == testMetric { + count += sample.Value + } + } + assert.Equal(t, 1.0, count) +} From 3f95a23913564480a594958fd455435d167d0833 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 14 Apr 2020 15:35:01 +0300 Subject: [PATCH 171/350] Fix typos and wrong comments --- cmd/run.go | 6 +++--- lib/execution.go | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index bd1ac445eca..0995f78ae4d 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -148,7 +148,7 @@ a commandline interface for interacting with it.`, logger := logrus.StandardLogger() // We prepare a bunch of contexts: - // - The runCtx is cancelled as soon as the Engine.Run() method finishes, + // - The runCtx is cancelled as soon as the Engine's run() lambda finishes, // and can trigger things like the usage report and end of test summary. // Crucially, metrics processing by the Engine will still work after this // context is cancelled! @@ -176,7 +176,7 @@ a commandline interface for interacting with it.`, // This is manually triggered after the Engine's Run() has completed, // and things like a single Ctrl+C don't affect it. We use it to make // sure that the progressbars finish updating with the latest execution - // state one last ime, after the test run has finished. + // state one last time, after the test run has finished. progressCtx, progressCancel := context.WithCancel(globalCtx) defer progressCancel() initBar = execScheduler.GetInitProgressBar() @@ -309,7 +309,7 @@ a commandline interface for interacting with it.`, }() } - // Initialize the engine + // Start the test run initBar.Modify(pb.WithConstProgress(0, "Start test")) if err := engineRun(); err != nil { return getExitCodeFromEngine(err) diff --git a/lib/execution.go b/lib/execution.go index 415f72724c1..0e29536028d 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -62,7 +62,7 @@ type ExecutionScheduler interface { // Run the ExecutionScheduler, funneling the generated metric samples // through the supplied out channel. - Run(lobalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error + Run(globalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error // Pause a test, or start/resume it. To check if a test is paused, use // GetState().IsPaused(). From a8d1085f5c35d9a7eb0d64482fe949fffcf2bdad Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 14 Apr 2020 15:53:58 +0300 Subject: [PATCH 172/350] Fix VariableArrivalRate not working well (#1285) The previous implementation worked basically as follows: A list of changes to the arrival rate is generated where there is a minimum time between these changes (currently 250ms). This means that for any ramp-up/down there is a step every 250ms (more or less). After this is generated a goroutine will read that list and send the change on a channel at the appropriate time. During ramp-up/down this is every 250ms (more or less). Another goroutine will be receiving those changes and resetting a timer to the new value. And here is where the problem lies: If the arrival rate is more then 1 iteration each 250ms this means that it will never trigger the start of a VU. The extreme example is having ramp-up to 4 iterations per second - this is exactly 1 iteration each 250ms, which means that for the whole duration up to end of the ramp-up there will be zero iterations started. In order to completely remove this, I went the step further. The way we separate the interval in small pieces reminded me of integrals and is one of the very easy cases for using integrals. So I just calculate the integral of the function that is the number of VUs over time. The answer is how many iterations need to be done there, so if I reverse this and instead calculate for how much time I will get 1 I get when the first iteration should start. I can do that for any iteration number and if there is no result then obviously we can get that :D. This also has the awesome side effect that if we split the execution in 10 we only need to calculate 1/10th of the integrals on each instance. --- lib/execution_segment.go | 4 + lib/executor/constant_arrival_rate_test.go | 29 +- lib/executor/variable_arrival_rate.go | 315 ++++++----- lib/executor/variable_arrival_rate_test.go | 583 ++++++++++++++------- 4 files changed, 561 insertions(+), 370 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index a25dc9c03d3..a4a6cf27136 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -517,6 +517,10 @@ type ExecutionTuple struct { // TODO rename once *sync.Once } +func (et *ExecutionTuple) String() string { + return fmt.Sprintf("%s in %s", et.ES, et.sequence) +} + func fillSequence(sequence ExecutionSegmentSequence) ExecutionSegmentSequence { if sequence[0].from.Cmp(zeroRat) != 0 { es := newExecutionSegment(zeroRat, sequence[0].from) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index f1056dd0e45..20e8a4df49c 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -38,6 +38,23 @@ import ( "github.com/loadimpact/k6/stats" ) +func newExecutionSegmentFromString(str string) *lib.ExecutionSegment { + r, err := lib.NewExecutionSegmentFromString(str) + if err != nil { + panic(err) + } + return r +} + +func newExecutionSegmentSequenceFromString(str string) *lib.ExecutionSegmentSequence { + r, err := lib.NewExecutionSegmentSequenceFromString(str) + + if err != nil { + panic(err) + } + return &r +} + func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { return ConstantArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), @@ -109,18 +126,6 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { } func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { - newExecutionSegmentFromString := func(str string) *lib.ExecutionSegment { - r, err := lib.NewExecutionSegmentFromString(str) - require.NoError(t, err) - return r - } - - newExecutionSegmentSequenceFromString := func(str string) *lib.ExecutionSegmentSequence { - r, err := lib.NewExecutionSegmentSequenceFromString(str) - require.NoError(t, err) - return &r - } - var tests = []struct { segment *lib.ExecutionSegment sequence *lib.ExecutionSegmentSequence diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 7fdd1969014..283e2bf9318 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -24,7 +24,6 @@ import ( "context" "fmt" "math" - "math/big" "sync/atomic" "time" @@ -39,10 +38,6 @@ import ( const variableArrivalRateType = "variable-arrival-rate" -// How often we can make arrival rate adjustments when processing stages -// TODO: make configurable, in some bounds? -const minIntervalBetweenRateAdjustments = 250 * time.Millisecond - func init() { lib.RegisterExecutorConfigType( variableArrivalRateType, @@ -155,109 +150,13 @@ func (varc VariableArrivalRateConfig) GetExecutionRequirements(et *lib.Execution } } -type rateChange struct { - // At what time should the rate below be applied. - timeOffset time.Duration - // Equals 1/rate: if rate was "1/5s", then this value, which is intended to - // be passed to time.NewTicker(), will be 5s. There's a special case when - // the rate is 0, for which we'll set Valid=false. That's because 0 isn't a - // valid ticker period and shouldn't be passed to time.NewTicker(). Instead, - // an empty or stopped ticker should be used. - tickerPeriod types.NullDuration -} - -// A helper method to generate the plan how the rate changes would happen. -func (varc VariableArrivalRateConfig) getPlannedRateChanges(et *lib.ExecutionTuple) []rateChange { - timeUnit := time.Duration(varc.TimeUnit.Duration) - // Important note for accuracy: we must work with and scale only the - // rational numbers, never the raw target values directly. It matters most - // for the accuracy of the intermediate rate change values, but it's - // important even here. - // - // Say we have a desired rate growth from 1/sec to 2/sec over 1 minute, and - // we split the test into two segments of 20% and 80%. If we used the whole - // numbers for scaling, then the instance executing the first segment won't - // ever do even a single request, since scale(20%, 1) would be 0, whereas - // the rational value for scale(20%, 1/sec) is 0.2/sec, or rather 1/5sec... - currentRate := getScaledArrivalRate(et.ES, varc.StartRate.Int64, timeUnit) - - rateChanges := []rateChange{} - timeFromStart := time.Duration(0) - - var tArrivalRate = new(big.Rat) - var tArrivalRateStep = new(big.Rat) - var stepCoef = new(big.Rat) - for _, stage := range varc.Stages { - stageTargetRate := getScaledArrivalRate(et.ES, stage.Target.Int64, timeUnit) - stageDuration := time.Duration(stage.Duration.Duration) - - if currentRate.Cmp(stageTargetRate) == 0 { - // We don't have to do anything but update the time offset - // if the rate wasn't changed in this stage - timeFromStart += stageDuration - continue - } - - // Handle 0-duration stages, i.e. instant rate jumps - if stageDuration == 0 { - // check if the last set change is for the same time and overwrite it - if len(rateChanges) > 0 && rateChanges[len(rateChanges)-1].timeOffset == timeFromStart { - rateChanges[len(rateChanges)-1].tickerPeriod = getTickerPeriod(stageTargetRate) - } else { - rateChanges = append(rateChanges, rateChange{ - timeOffset: timeFromStart, - tickerPeriod: getTickerPeriod(stageTargetRate), - }) - } - currentRate = stageTargetRate - continue - } - // Basically, find out how many regular intervals with size of at least - // minIntervalBetweenRateAdjustments are in the stage's duration, and - // then use that number to calculate the actual step. All durations have - // nanosecond precision, so there isn't any actual loss of precision... - stepNumber := (stageDuration / minIntervalBetweenRateAdjustments) - if stepNumber > 1 { - rateDiff := new(big.Rat).Sub(stageTargetRate, currentRate) - stepInterval := stageDuration / stepNumber - for t := stepInterval; ; t += stepInterval { - if stageDuration-t < minIntervalBetweenRateAdjustments { - break - } - - tArrivalRate.Add( - currentRate, - tArrivalRateStep.Mul( - rateDiff, - stepCoef.SetFrac64(int64(t), int64(stageDuration)), - ), - ) - - rateChanges = append(rateChanges, rateChange{ - timeOffset: timeFromStart + t, - tickerPeriod: getTickerPeriod(tArrivalRate), - }) - } - } - timeFromStart += stageDuration - rateChanges = append(rateChanges, rateChange{ - timeOffset: timeFromStart, - tickerPeriod: getTickerPeriod(stageTargetRate), - }) - currentRate = stageTargetRate - } - - return rateChanges -} - // NewExecutor creates a new VariableArrivalRate executor func (varc VariableArrivalRateConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { return VariableArrivalRate{ - BaseExecutor: NewBaseExecutor(varc, es, logger), - config: varc, - plannedRateChanges: varc.getPlannedRateChanges(es.ExecutionTuple), + BaseExecutor: NewBaseExecutor(varc, es, logger), + config: varc, }, nil } @@ -271,59 +170,129 @@ func (varc VariableArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { //TODO: combine with the ConstantArrivalRate? type VariableArrivalRate struct { *BaseExecutor - config VariableArrivalRateConfig - plannedRateChanges []rateChange + config VariableArrivalRateConfig } // Make sure we implement the lib.Executor interface. var _ lib.Executor = &VariableArrivalRate{} -// streamRateChanges is a helper method that emits rate change events at their -// proper time. -func (varr VariableArrivalRate) streamRateChanges(ctx context.Context, startTime time.Time) <-chan rateChange { - ch := make(chan rateChange) - go func() { - for _, step := range varr.plannedRateChanges { - offsetDiff := step.timeOffset - time.Since(startTime) - if offsetDiff > 0 { // wait until time of event arrives - select { - case <-ctx.Done(): - return // exit if context is cancelled - case <-time.After(offsetDiff): //TODO: reuse a timer? - // do nothing - } +// cal calculates the transtitions between stages and gives the next full value produced by the +// stages. In this explanation we are talking about events and in practice those events are starting +// of an iteration, but could really be anything that needs to occur at a constant or linear rate. +// +// The basic idea is that we make a graph with the X axis being time and the Y axis being +// events/s we know that the area of the figure between the graph and the X axis is equal to the +// amount of events done - we multiply time by events per time so we get events ... +// Mathematics :). +// +// Lets look at a simple example - lets say we start with 2 events and the first stage is 5 +// seconds to 2 events/s and then we have a second stage for 5 second that goes up to 3 events +// (using small numbers because ... well it is easier :D). This will look something like: +// ^ +// 7| +// 6| +// 5| +// 4| +// 3| ,-+ +// 2|----+-' | +// 1| | | +// +----+----+----------------------------------> +// 0s 5s 10s +// TODO: bigger and more stages +// +// Now the question is when(where on the graph) does the first event happen? Well in this simple +// case it is easy it will be at 0.5 seconds as we are doing 2 events/s. If we want to know when +// event n will happen we need to calculate n = 2 * x, where x is the time it will happen, so we +// need to calculate x = n/2as we are interested in the time, x. +// So if we just had a constant function for each event n we can calculate n/2 and find out when +// it needs to start. +// As we can see though the graph changes as stages change. But we can calculate how many events +// each stage will have, again it is the area from the start of the stage to it's end and between +// the graph and the X axis. So in this case we know that the first stage will have 10 full events +// in it and no more or less. So we are trying to find out when the 12 event will happen the answer +// will be after the 5th second. +// +// The graph doesn't show this well but we are ramping up linearly (we could possibly add +// other ramping up/down functions later). So at 7.5 seconds for example we should be doing 2.5 +// events/s. You could start slicing the graph constantly and in this way to represent the ramping +// up/down as a multiple constant functions, and you will get mostly okayish results. But here is +// where calculus comes into play. Calculus gives us a way of exactly calculate the area for any +// given function and linear ramp up/downs just happen to be pretty easy(actual math prove in +// https://github.com/loadimpact/k6/issues/1299#issuecomment-575661084). +// +// One tricky last point is what happens if stage only completes 9.8 events? Let's say that the +// first stage above was 4.9 seconds long 2 * 4.9 is 9.8, we have 9 events and .8 of an event, what +// do with do with that? Well the 10th even will happen in the next stage (if any) and will happen +// when the are from the start till time x is 0.2 (instead of 1) as 0.2 + 0.8 is 10. So the 12th for +// example will be when the area is 2.2 as 9.8+2.2. So we just carry this around. +// +// So in the end what calis doing is to get formulas which will tell it when +// a given event n in order will happen. It helps itself by knowing that in a given +// stage will do some given amount (the area of the stage) events and if we past that one we +// know we are not in that stage. +// +// The specific implementation here can only go forward and does incorporate +// the striping algorithm from the lib.ExecutionTuple for additional speed up but this could +// possibly be refactored if need for this arises. +func (varc VariableArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time.Duration) { + start, offsets, _ := et.GetStripedOffsets(et.ES) + li := -1 + // TODO: move this to a utility function, or directly what GetStripedOffsets uses once we see everywhere we will use it + next := func() int64 { + li++ + return offsets[li%len(offsets)] + } + defer close(ch) // TODO: maybe this is not a good design - closing a channel we get + var ( + stageStart time.Duration + timeUnit = float64(varc.TimeUnit.Duration) + doneSoFar, endCount, to, dur float64 + from = float64(varc.StartRate.ValueOrZero()) / timeUnit + // start .. starts at 0 but the algorithm works with area so we need to start from 1 not 0 + i = float64(start + 1) + ) + + for _, stage := range varc.Stages { + to = float64(stage.Target.ValueOrZero()) / timeUnit + dur = float64(stage.Duration.Duration) + if from != to { // ramp up/down + endCount += dur * ((to-from)/2 + from) + for ; i <= endCount; i += float64(next()) { + // TODO: try to twist this in a way to be able to get i (the only changing part) + // somewhere where it is less in the middle of the equation + x := (from*dur - math.Sqrt(dur*(from*from*dur+2*(i-doneSoFar)*(to-from)))) / (from - to) + + ch <- time.Duration(x) + stageStart } - select { - case <-ctx.Done(): - return // exit if context is cancelled - case ch <- step: // send the step + } else { + endCount += dur * to + for ; i <= endCount; i += float64(next()) { + ch <- time.Duration((i-doneSoFar)/to) + stageStart } } - }() - return ch + doneSoFar = endCount + from = to + stageStart += time.Duration(stage.Duration.Duration) + } } // Run executes a variable number of iterations per second. func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen - segment := varr.executionState.Options.ExecutionSegment + segment := varr.executionState.ExecutionTuple.ES gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) preAllocatedVUs := varr.config.GetPreAllocatedVUs(varr.executionState.ExecutionTuple) maxVUs := varr.config.GetMaxVUs(varr.executionState.ExecutionTuple) + // TODO: refactor and simplify timeUnit := time.Duration(varr.config.TimeUnit.Duration) startArrivalRate := getScaledArrivalRate(segment, varr.config.StartRate.Int64, timeUnit) - maxUnscaledRate := getStagesUnscaledMaxTarget(varr.config.StartRate.Int64, varr.config.Stages) maxArrivalRatePerSec, _ := getArrivalRatePerSec(getScaledArrivalRate(segment, maxUnscaledRate, timeUnit)).Float64() startTickerPeriod := getTickerPeriod(startArrivalRate) startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) defer cancel() - ticker := &time.Ticker{} - if startTickerPeriod.Valid { - ticker = time.NewTicker(time.Duration(startTickerPeriod.Duration)) - } // Make sure the log and the progress bar have accurate information varr.logger.WithFields(logrus.Fields{ @@ -346,7 +315,8 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := varr.executionState.GetPlannedVU(varr.logger, true) + var vu lib.VU + vu, err = varr.executionState.GetPlannedVU(varr.logger, true) if err != nil { return err } @@ -354,15 +324,14 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample vus <- vu } - tickerPeriod := new(int64) - *tickerPeriod = int64(startTickerPeriod.Duration) + tickerPeriod := int64(startTickerPeriod.Duration) vusFmt := pb.GetFixedLengthIntFormat(maxVUs) itersFmt := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" progresFn := func() (float64, []string) { currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) - currentTickerPeriod := atomic.LoadInt64(tickerPeriod) + currentTickerPeriod := atomic.LoadInt64(&tickerPeriod) vusInBuffer := uint64(len(vus)) progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) @@ -386,49 +355,59 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample return math.Min(1, float64(spent)/float64(duration)), right } + varr.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) regDurationDone := regDurationCtx.Done() - runIterationBasic := getIterationRunner(varr.executionState, varr.logger, out) - runIteration := func(vu lib.VU) { - runIterationBasic(maxDurationCtx, vu) - vus <- vu - } + runIteration := getIterationRunner(varr.executionState, varr.logger, out) remainingUnplannedVUs := maxVUs - preAllocatedVUs - rateChangesStream := varr.streamRateChanges(maxDurationCtx, startTime) - for { + var timer = time.NewTimer(time.Hour) + var start = time.Now() + var ch = make(chan time.Duration, 10) // buffer 10 iteration times ahead + var prevTime time.Duration + go varr.config.cal(varr.executionState.ExecutionTuple, ch) + for nextTime := range ch { select { - case rateChange := <-rateChangesStream: - newPeriod := rateChange.tickerPeriod - ticker.Stop() - if newPeriod.Valid { - ticker = time.NewTicker(time.Duration(newPeriod.Duration)) - } - atomic.StoreInt64(tickerPeriod, int64(newPeriod.Duration)) - case <-ticker.C: - select { - case vu := <-vus: - // ideally, we get the VU from the buffer without any issues - go runIteration(vu) - default: - if remainingUnplannedVUs == 0 { - //TODO: emit an error metric? - varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) - break - } - vu, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) - if err != nil { - return err - } - remainingUnplannedVUs-- - atomic.AddUint64(&initialisedVUs, 1) - go runIteration(vu) - } case <-regDurationDone: return nil + default: + } + atomic.StoreInt64(&tickerPeriod, int64(nextTime-prevTime)) + prevTime = nextTime + b := time.Until(start.Add(nextTime)) + if b > 0 { // TODO: have a minimal ? + timer.Reset(b) + select { + case <-timer.C: + case <-regDurationDone: + return nil + } + } + + var vu lib.VU + select { + case vu = <-vus: + // ideally, we get the VU from the buffer without any issues + default: + if remainingUnplannedVUs == 0 { + //TODO: emit an error metric? + varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) + continue + } + vu, err = varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) + if err != nil { + return err + } + remainingUnplannedVUs-- + atomic.AddUint64(&initialisedVUs, 1) } + go func(vu lib.VU) { + runIteration(maxDurationCtx, vu) + vus <- vu + }(vu) } + return nil } diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 2d3858f464c..51eeaceda6e 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -22,6 +22,7 @@ package executor import ( "context" + "fmt" "math/big" "sync" "sync/atomic" @@ -29,6 +30,7 @@ import ( "time" "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" null "gopkg.in/guregu/null.v3" @@ -37,191 +39,6 @@ import ( "github.com/loadimpact/k6/stats" ) -func TestGetPlannedRateChanges0DurationStage(t *testing.T) { - t.Parallel() - var config = VariableArrivalRateConfig{ - TimeUnit: types.NullDurationFrom(time.Second), - StartRate: null.IntFrom(0), - Stages: []Stage{ - { - Duration: types.NullDurationFrom(0), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(0), - Target: null.IntFrom(100), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(100), - }, - }, - } - et, err := lib.NewExecutionTuple(nil, nil) - require.NoError(t, err) - changes := config.getPlannedRateChanges(et) - require.Equal(t, 2, len(changes)) - require.Equal(t, time.Duration(0), changes[0].timeOffset) - require.Equal(t, types.NullDurationFrom(time.Millisecond*20), changes[0].tickerPeriod) - - require.Equal(t, time.Minute, changes[1].timeOffset) - require.Equal(t, types.NullDurationFrom(time.Millisecond*10), changes[1].tickerPeriod) -} - -// helper function to calculate the expected rate change at a given time -func calculateTickerPeriod(current, start, duration time.Duration, from, to int64) types.Duration { - var coef = big.NewRat( - (current - start).Nanoseconds(), - duration.Nanoseconds(), - ) - - var oneRat = new(big.Rat).Mul(big.NewRat(from-to, 1), coef) - oneRat = new(big.Rat).Sub(big.NewRat(from, 1), oneRat) - oneRat = new(big.Rat).Mul(big.NewRat(int64(time.Second), 1), new(big.Rat).Inv(oneRat)) - return types.Duration(new(big.Int).Div(oneRat.Num(), oneRat.Denom()).Int64()) -} - -func TestGetPlannedRateChangesZeroDurationStart(t *testing.T) { - // TODO: Make multiple of those tests - t.Parallel() - var config = VariableArrivalRateConfig{ - TimeUnit: types.NullDurationFrom(time.Second), - StartRate: null.IntFrom(0), - Stages: []Stage{ - { - Duration: types.NullDurationFrom(0), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(0), - Target: null.IntFrom(100), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(100), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(0), - }, - }, - } - - et, err := lib.NewExecutionTuple(nil, nil) - require.NoError(t, err) - changes := config.getPlannedRateChanges(et) - var expectedTickerPeriod types.Duration - for i, change := range changes { - switch { - case change.timeOffset == 0: - expectedTickerPeriod = types.Duration(20 * time.Millisecond) - case change.timeOffset == time.Minute*1: - expectedTickerPeriod = types.Duration(10 * time.Millisecond) - case change.timeOffset < time.Minute*3: - expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 2*time.Minute, time.Minute, 100, 0) - case change.timeOffset == time.Minute*3: - expectedTickerPeriod = 0 - default: - t.Fatalf("this shouldn't happen %d index %+v", i, change) - } - require.Equal(t, time.Duration(0), - change.timeOffset%minIntervalBetweenRateAdjustments, "%d index %+v", i, change) - require.Equal(t, change.tickerPeriod.Duration, expectedTickerPeriod, "%d index %+v", i, change) - } -} - -func TestGetPlannedRateChanges(t *testing.T) { - // TODO: Make multiple of those tests - t.Parallel() - var config = VariableArrivalRateConfig{ - TimeUnit: types.NullDurationFrom(time.Second), - StartRate: null.IntFrom(0), - Stages: []Stage{ - { - Duration: types.NullDurationFrom(2 * time.Minute), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(50), - }, - { - Duration: types.NullDurationFrom(time.Minute), - Target: null.IntFrom(100), - }, - { - Duration: types.NullDurationFrom(0), - Target: null.IntFrom(200), - }, - - { - Duration: types.NullDurationFrom(time.Second * 23), - Target: null.IntFrom(50), - }, - }, - } - - et, err := lib.NewExecutionTuple(nil, nil) - require.NoError(t, err) - changes := config.getPlannedRateChanges(et) - var expectedTickerPeriod types.Duration - for i, change := range changes { - switch { - case change.timeOffset <= time.Minute*2: - expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 0, time.Minute*2, 0, 50) - case change.timeOffset < time.Minute*4: - expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, time.Minute*3, time.Minute, 50, 100) - case change.timeOffset == time.Minute*4: - expectedTickerPeriod = types.Duration(5 * time.Millisecond) - default: - expectedTickerPeriod = calculateTickerPeriod(change.timeOffset, 4*time.Minute, 23*time.Second, 200, 50) - } - require.Equal(t, time.Duration(0), - change.timeOffset%minIntervalBetweenRateAdjustments, "%d index %+v", i, change) - require.Equal(t, change.tickerPeriod.Duration, expectedTickerPeriod, "%d index %+v", i, change) - } -} - -func BenchmarkGetPlannedRateChanges(b *testing.B) { - var config = VariableArrivalRateConfig{ - TimeUnit: types.NullDurationFrom(time.Second), - StartRate: null.IntFrom(0), - Stages: []Stage{ - { - Duration: types.NullDurationFrom(5 * time.Minute), - Target: null.IntFrom(5000), - }, - { - Duration: types.NullDurationFrom(50 * time.Minute), - Target: null.IntFrom(5000), - }, - { - Duration: types.NullDurationFrom(5 * time.Minute), - Target: null.IntFrom(0), - }, - }, - } - - b.RunParallel(func(pb *testing.PB) { - et, err := lib.NewExecutionTuple(nil, nil) - require.NoError(b, err) - for pb.Next() { - changes := config.getPlannedRateChanges(et) - - require.Equal(b, time.Duration(0), - changes[0].timeOffset%minIntervalBetweenRateAdjustments, "%+v", changes[0]) - } - }) -} - func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { return VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), @@ -294,17 +111,15 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { time.Sleep(time.Second) currentCount = atomic.SwapInt64(&count, 0) - require.InDelta(t, 10, currentCount, 1) + assert.InDelta(t, 10, currentCount, 1) time.Sleep(time.Second) currentCount = atomic.SwapInt64(&count, 0) - // this is highly dependant on minIntervalBetweenRateAdjustments - // TODO find out why this isn't 30 and fix it - require.InDelta(t, 23, currentCount, 2) + assert.InDelta(t, 30, currentCount, 2) time.Sleep(time.Second) currentCount = atomic.SwapInt64(&count, 0) - require.InDelta(t, 50, currentCount, 2) + assert.InDelta(t, 50, currentCount, 2) }() var engineOut = make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) @@ -312,3 +127,391 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { require.NoError(t, err) require.Empty(t, logHook.Drain()) } + +func TestVariableArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { + t.Parallel() + var count int64 + var now = time.Now() + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + var expectedTimes = []time.Duration{ + time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6} + var ctx, cancel, executor, logHook = setupExecutor( + t, VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(time.Second * 6), + Target: null.IntFrom(1), + }, + { + Duration: types.NullDurationFrom(time.Second * 0), + Target: null.IntFrom(0), + }, + { + Duration: types.NullDurationFrom(time.Second * 1), + Target: null.IntFrom(0), + }, + }, + PreAllocatedVUs: null.IntFrom(10), + MaxVUs: null.IntFrom(20), + }, + es, + simpleRunner(func(ctx context.Context) error { + current := atomic.AddInt64(&count, 1) + if !assert.True(t, int(current) <= len(expectedTimes)) { + return nil + } + expectedTime := expectedTimes[current-1] + assert.WithinDuration(t, + now.Add(expectedTime), + time.Now(), + time.Millisecond*100, + "%d expectedTime %s", current, expectedTime, + ) + return nil + }), + ) + defer cancel() + var engineOut = make(chan stats.SampleContainer, 1000) + err = executor.Run(ctx, engineOut) + require.NoError(t, err) + require.Equal(t, int64(len(expectedTimes)), count) + require.Empty(t, logHook.Drain()) +} + +func mustNewExecutionTuple(seg *lib.ExecutionSegment, seq *lib.ExecutionSegmentSequence) *lib.ExecutionTuple { + et, err := lib.NewExecutionTuple(seg, seq) + if err != nil { + panic(err) + } + return et +} + +func TestVariableArrivalRateCal(t *testing.T) { + t.Parallel() + + var ( + defaultTimeUnit = time.Second + config = VariableArrivalRateConfig{ + StartRate: null.IntFrom(0), + Stages: []Stage{ // TODO make this even bigger and longer .. will need more time + { + Duration: types.NullDurationFrom(time.Second * 5), + Target: null.IntFrom(1), + }, + { + Duration: types.NullDurationFrom(time.Second * 1), + Target: null.IntFrom(1), + }, + { + Duration: types.NullDurationFrom(time.Second * 5), + Target: null.IntFrom(0), + }, + }, + } + ) + + testCases := []struct { + expectedTimes []time.Duration + et *lib.ExecutionTuple + timeUnit time.Duration + }{ + { + expectedTimes: []time.Duration{time.Millisecond * 3162, time.Millisecond * 4472, time.Millisecond * 5500, time.Millisecond * 6527, time.Millisecond * 7837, time.Second * 11}, + et: mustNewExecutionTuple(nil, nil), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 4472, time.Millisecond * 7837}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), nil), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 4472, time.Millisecond * 7837}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1")), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 4472, time.Millisecond * 7837}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("1/3:2/3"), nil), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 4472, time.Millisecond * 7837}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), nil), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 3162, time.Millisecond * 6527}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 4472, time.Millisecond * 7837}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("1/3:2/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 5500, time.Millisecond * 11000}, + et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + }, + { + expectedTimes: []time.Duration{time.Millisecond * 1825, time.Millisecond * 2581, time.Millisecond * 3162, time.Millisecond * 3651, time.Millisecond * 4082, time.Millisecond * 4472, + time.Millisecond * 4830, time.Millisecond * 5166, time.Millisecond * 5499, time.Millisecond * 5833, time.Millisecond * 6169, time.Millisecond * 6527, + time.Millisecond * 6917, time.Millisecond * 7348, time.Millisecond * 7837, time.Millisecond * 8418, time.Millisecond * 9174, time.Millisecond * 10999}, + et: mustNewExecutionTuple(nil, nil), + timeUnit: time.Second / 3, // three times as fast + }, + // TODO: extend more + } + + for _, testCase := range testCases { + et := testCase.et + expectedTimes := testCase.expectedTimes + config.TimeUnit = types.NewNullDuration(testCase.timeUnit, true) + if testCase.timeUnit == 0 { + config.TimeUnit = types.NewNullDuration(defaultTimeUnit, true) + } + + t.Run(fmt.Sprintf("%s timeunit %s", et, config.TimeUnit), func(t *testing.T) { + var ch = make(chan time.Duration) + go config.cal(et, ch) + var changes = make([]time.Duration, 0, len(expectedTimes)) + for c := range ch { + changes = append(changes, c) + } + assert.Equal(t, len(expectedTimes), len(changes)) + for i, expectedTime := range expectedTimes { + require.True(t, i < len(changes)) + change := changes[i] + assert.InEpsilon(t, expectedTime, change, 0.001, "%s %s", expectedTime, change) + } + }) + } +} + +func BenchmarkCal(b *testing.B) { + for _, t := range []time.Duration{ + time.Second, time.Minute, + } { + t := t + b.Run(t.String(), func(b *testing.B) { + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(50), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(t), + Target: null.IntFrom(49), + }, + { + Duration: types.NullDurationFrom(t), + Target: null.IntFrom(50), + }, + }, + } + et := mustNewExecutionTuple(nil, nil) + + b.ResetTimer() + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + var ch = make(chan time.Duration, 20) + go config.cal(et, ch) + for c := range ch { + _ = c + } + } + }) + }) + } +} + +func BenchmarkCalRat(b *testing.B) { + for _, t := range []time.Duration{ + time.Second, time.Minute, + } { + t := t + b.Run(t.String(), func(b *testing.B) { + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(50), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(t), + Target: null.IntFrom(49), + }, + { + Duration: types.NullDurationFrom(t), + Target: null.IntFrom(50), + }, + }, + } + et := mustNewExecutionTuple(nil, nil) + + b.ResetTimer() + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + var ch = make(chan time.Duration, 20) + go config.calRat(et, ch) + for c := range ch { + _ = c + } + } + }) + }) + } +} + +func TestCompareCalImplementation(t *testing.T) { + t.Parallel() + // This test checks that the cal and calRat implementation get roughly similar numbers + // in my experiment the difference is 1(nanosecond) in 7 case for the whole test + // the duration is 1 second for each stage as calRat takes way longer - a longer better test can + // be done when/if it's performance is improved + var config = VariableArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(0), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(200), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(200), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(2000), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(2000), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(300), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(300), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(1333), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(1334), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(1334), + }, + }, + } + + et := mustNewExecutionTuple(nil, nil) + var chRat = make(chan time.Duration, 20) + var ch = make(chan time.Duration, 20) + go config.calRat(et, chRat) + go config.cal(et, ch) + count := 0 + var diff int + for c := range ch { + count++ + cRat := <-chRat + if !assert.InDelta(t, c, cRat, 1, "%d", count) { + diff++ + } + } + require.Equal(t, 0, diff) +} + +// calRat code here is just to check how accurate the cal implemenattion is +// there are no other tests for it so it depends on the test of cal that it is actually accurate :D + +//nolint:gochecknoglobals +var two = big.NewRat(2, 1) + +// from https://groups.google.com/forum/#!topic/golang-nuts/aIcDf8T-Png +func sqrtRat(x *big.Rat) *big.Rat { + var z, a, b big.Rat + var ns, ds big.Int + ni, di := x.Num(), x.Denom() + z.SetFrac(ns.Rsh(ni, uint(ni.BitLen())/2), ds.Rsh(di, uint(di.BitLen())/2)) + for i := 10; i > 0; i-- { //TODO: better termination + a.Sub(a.Mul(&z, &z), x) + f, _ := a.Float64() + if f == 0 { + break + } + // fmt.Println(x, z, i) + z.Sub(&z, b.Quo(&a, b.Mul(two, &z))) + } + return &z +} + +// This implementation is just for reference and accuracy testing +func (varc VariableArrivalRateConfig) calRat(et *lib.ExecutionTuple, ch chan<- time.Duration) { + defer close(ch) + + start, offsets, _ := et.GetStripedOffsets(et.ES) + li := -1 + next := func() int64 { + li++ + return offsets[li%len(offsets)] + } + iRat := big.NewRat(start+1, 1) + + var carry = big.NewRat(0, 1) + var doneSoFar = big.NewRat(0, 1) + var endCount = big.NewRat(0, 1) + curr := varc.StartRate.ValueOrZero() + var base time.Duration + for _, stage := range varc.Stages { + target := stage.Target.ValueOrZero() + if target != curr { + var ( + from = big.NewRat(curr, int64(time.Second)) + to = big.NewRat(target, int64(time.Second)) + dur = big.NewRat(time.Duration(stage.Duration.Duration).Nanoseconds(), 1) + ) + // precalcualations :) + toMinusFrom := new(big.Rat).Sub(to, from) + fromSquare := new(big.Rat).Mul(from, from) + durMulSquare := new(big.Rat).Mul(dur, fromSquare) + fromMulDur := new(big.Rat).Mul(from, dur) + oneOverToMinusFrom := new(big.Rat).Inv(toMinusFrom) + + endCount.Add(endCount, + new(big.Rat).Mul( + dur, + new(big.Rat).Add(new(big.Rat).Mul(toMinusFrom, big.NewRat(1, 2)), from))) + for ; endCount.Cmp(iRat) >= 0; iRat.Add(iRat, big.NewRat(next(), 1)) { + // even with all of this optimizations sqrtRat is taking so long this is still + // extremely slow ... :( + buf := new(big.Rat).Sub(iRat, doneSoFar) + buf.Mul(buf, two) + buf.Mul(buf, toMinusFrom) + buf.Add(buf, durMulSquare) + buf.Mul(buf, dur) + buf.Sub(fromMulDur, sqrtRat(buf)) + buf.Mul(buf, oneOverToMinusFrom) + + r, _ := buf.Float64() + ch <- base + time.Duration(-r) // the minus is because we don't deive by from-to but by to-from above + } + } else { + step := big.NewRat(int64(time.Second), target) + first := big.NewRat(0, 1) + first.Sub(first, carry) + endCount.Add(endCount, new(big.Rat).Mul(big.NewRat(target, 1), big.NewRat(time.Duration(stage.Duration.Duration).Nanoseconds(), time.Duration(varc.TimeUnit.Duration).Nanoseconds()))) + + for ; endCount.Cmp(iRat) >= 0; iRat.Add(iRat, big.NewRat(next(), 1)) { + res := new(big.Rat).Sub(iRat, doneSoFar) // this can get next added to it but will need to change the above for .. so + r, _ := res.Mul(res, step).Float64() + ch <- base + time.Duration(r) + first.Add(first, step) + } + } + doneSoFar.Set(endCount) // copy + curr = target + base += time.Duration(stage.Duration.Duration) + } +} From 1840d083db6d8b27f536a403686f6ff5c73671fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 11 Mar 2020 18:55:37 +0100 Subject: [PATCH 173/350] Fix typos --- core/engine.go | 6 +++--- js/runner_test.go | 2 +- lib/execution.go | 2 +- lib/executor/execution_test.go | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/engine.go b/core/engine.go index efea7b69373..2ff7717b60f 100644 --- a/core/engine.go +++ b/core/engine.go @@ -368,9 +368,9 @@ func (e *Engine) processThresholds() (shouldAbort bool) { return shouldAbort } -func (e *Engine) processSamplesForMetrics(sampleCointainers []stats.SampleContainer) { - for _, sampleCointainer := range sampleCointainers { - samples := sampleCointainer.GetSamples() +func (e *Engine) processSamplesForMetrics(sampleContainers []stats.SampleContainer) { + for _, sampleContainer := range sampleContainers { + samples := sampleContainer.GetSamples() if len(samples) == 0 { continue diff --git a/js/runner_test.go b/js/runner_test.go index 11309f96da7..9aad87c2292 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -1392,7 +1392,7 @@ func TestInitContextForbidden(t *testing.T) { } } -func TestArchiveRunningIntegraty(t *testing.T) { +func TestArchiveRunningIntegrity(t *testing.T) { tb := httpmultibin.NewHTTPMultiBin(t) defer tb.Cleanup() diff --git a/lib/execution.go b/lib/execution.go index 0e29536028d..210745f7826 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -527,7 +527,7 @@ func (es *ExecutionState) ResumeNotify() <-chan struct{} { // If modifyActiveVUCount is true, the method would also increment the counter // for active VUs. In most cases, that's the desired behavior, but some // executors might have to retrieve their reserved VUs without using them -// immediately - for example, the the externally-controlled executor when the +// immediately - for example, the externally-controlled executor when the // configured maxVUs number is greater than the configured starting VUs. func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount bool) (VU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { diff --git a/lib/executor/execution_test.go b/lib/executor/execution_test.go index a038c50612a..0db52ba85d6 100644 --- a/lib/executor/execution_test.go +++ b/lib/executor/execution_test.go @@ -126,7 +126,7 @@ func TestExecutionStateGettingVUs(t *testing.T) { require.Contains(t, entry.Message, "Could not get a VU from the buffer for ") } - // Test getting uninitiazed vus will work + // Test getting uninitialized vus will work for i := 0; i < 10; i++ { require.EqualValues(t, 10+i, es.GetCurrentlyActiveVUsCount()) vu, err = es.GetUnplannedVU(context.Background(), logEntry) From 20b69f398a8fe261635e978e9c794ffa3b70a910 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 13:38:59 +0100 Subject: [PATCH 174/350] Refactor VU context handling, introduce VU activation This cleans up how context was being handled for purposes of interruption, and introduces a VU activation method that also handles de-activation (i.e. returning the VU to the pool) via a callback passed during execution. See #1283 --- core/local/local.go | 31 ++++++------- js/runner.go | 96 +++++++++++++++------------------------ lib/execution.go | 24 ++++------ lib/executor/helpers.go | 10 ++-- lib/executor/vu_handle.go | 37 ++++++++------- lib/executors.go | 7 +-- lib/runner.go | 41 +++++++++++------ 7 files changed, 114 insertions(+), 132 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 5931907891b..7b1ab217664 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -148,24 +148,20 @@ func (e *ExecutionScheduler) GetExecutionPlan() []lib.ExecutionStep { return e.executionPlan } -// initVU is just a helper method that's used to both initialize the planned VUs +// initVU is a helper method that's used to both initialize the planned VUs // in the Init() method, and also passed to executors so they can initialize // any unplanned VUs themselves. -// TODO: actually use the context... func (e *ExecutionScheduler) initVU( - _ context.Context, logger *logrus.Entry, engineOut chan<- stats.SampleContainer, -) (lib.VU, error) { - vu, err := e.runner.NewVU(engineOut) - if err != nil { - return nil, fmt.Errorf("error while initializing a VU: '%s'", err) - } - + samplesOut chan<- stats.SampleContainer, logger *logrus.Entry, +) (lib.InitializedVU, error) { // Get the VU ID here, so that the VUs are (mostly) ordered by their // number in the channel buffer vuID := e.state.GetUniqueVUIdentifier() - if err := vu.Reconfigure(int64(vuID)); err != nil { - return nil, fmt.Errorf("error while reconfiguring VU #%d: '%s'", vuID, err) + vu, err := e.runner.NewVU(int64(vuID), samplesOut) + if err != nil { + return nil, fmt.Errorf("error while initializing VU #%d: '%s'", vuID, err) } + logger.Debugf("Initialized VU #%d", vuID) return vu, nil } @@ -191,7 +187,8 @@ func (e *ExecutionScheduler) getRunStats() string { } func (e *ExecutionScheduler) initVUsConcurrently( - ctx context.Context, engineOut chan<- stats.SampleContainer, count uint64, concurrency int, logger *logrus.Entry, + ctx context.Context, samplesOut chan<- stats.SampleContainer, count uint64, + concurrency int, logger *logrus.Entry, ) chan error { doneInits := make(chan error, count) // poor man's early-return waitgroup limiter := make(chan struct{}) @@ -199,7 +196,7 @@ func (e *ExecutionScheduler) initVUsConcurrently( for i := 0; i < concurrency; i++ { go func() { for range limiter { - newVU, err := e.initVU(ctx, logger, engineOut) + newVU, err := e.initVU(samplesOut, logger) if err == nil { e.state.AddInitializedVU(newVU) } @@ -224,7 +221,7 @@ func (e *ExecutionScheduler) initVUsConcurrently( // Init concurrently initializes all of the planned VUs and then sequentially // initializes all of the configured executors. -func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error { +func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.SampleContainer) error { logger := e.logger.WithField("phase", "local-execution-scheduler-init") vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) @@ -237,7 +234,7 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa defer cancel() e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) - doneInits := e.initVUsConcurrently(subctx, engineOut, vusToInitialize, runtime.NumCPU(), logger) + doneInits := e.initVUsConcurrently(subctx, samplesOut, vusToInitialize, runtime.NumCPU(), logger) initializedVUs := new(uint64) vusFmt := pb.GetFixedLengthIntFormat(int64(vusToInitialize)) @@ -264,8 +261,8 @@ func (e *ExecutionScheduler) Init(ctx context.Context, engineOut chan<- stats.Sa } } - e.state.SetInitVUFunc(func(ctx context.Context, logger *logrus.Entry) (lib.VU, error) { - return e.initVU(ctx, logger, engineOut) + e.state.SetInitVUFunc(func(ctx context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { + return e.initVU(samplesOut, logger) }) e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) diff --git a/js/runner.go b/js/runner.go index d3374266f56..11059769e42 100644 --- a/js/runner.go +++ b/js/runner.go @@ -28,7 +28,6 @@ import ( "net/http" "net/http/cookiejar" "strconv" - "sync" "time" "github.com/dop251/goja" @@ -114,15 +113,17 @@ func (r *Runner) MakeArchive() *lib.Archive { return r.Bundle.makeArchive() } -func (r *Runner) NewVU(samplesOut chan<- stats.SampleContainer) (lib.VU, error) { - vu, err := r.newVU(samplesOut) +// NewVU returns a new initialized VU. +func (r *Runner) NewVU(id int64, samplesOut chan<- stats.SampleContainer) (lib.InitializedVU, error) { + vu, err := r.newVU(id, samplesOut) if err != nil { return nil, err } - return lib.VU(vu), nil + return lib.InitializedVU(vu), nil } -func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { +// nolint:funlen +func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, error) { // Instantiate a new bundle, make a VU out of it. bi, err := r.Bundle.Instantiate() if err != nil { @@ -185,6 +186,7 @@ func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { } vu := &VU{ + ID: id, BundleInstance: *bi, Runner: r, Transport: transport, @@ -194,8 +196,8 @@ func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { Console: r.console, BPool: bpool.NewBufferPool(100), Samples: samplesOut, - m: &sync.Mutex{}, } + vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) common.BindToGlobal(vu.Runtime, map[string]interface{}{ "open": func() { @@ -206,11 +208,6 @@ func (r *Runner) newVU(samplesOut chan<- stats.SampleContainer) (*VU, error) { }, }) - // Give the VU an initial sense of identity. - if err := vu.Reconfigure(0); err != nil { - return nil, err - } - return vu, nil } @@ -301,7 +298,7 @@ func (r *Runner) SetOptions(opts lib.Options) error { // 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, out chan<- stats.SampleContainer, name string, arg interface{}) (goja.Value, error) { - vu, err := r.newVU(out) + vu, err := r.newVU(0, out) if err != nil { return goja.Undefined(), err } @@ -356,13 +353,14 @@ func (r *Runner) timeoutErrorDuration(stage string) time.Duration { type VU struct { BundleInstance - Runner *Runner - Transport *http.Transport - Dialer *netext.Dialer - CookieJar *cookiejar.Jar - TLSConfig *tls.Config - ID int64 - Iteration int64 + Runner *Runner + RunContext *context.Context + Transport *http.Transport + Dialer *netext.Dialer + CookieJar *cookiejar.Jar + TLSConfig *tls.Config + ID int64 + Iteration int64 Console *console BPool *bpool.BufferPool @@ -370,51 +368,31 @@ type VU struct { Samples chan<- stats.SampleContainer setupData goja.Value - - // A VU will track the last context it was called with for cancellation. - // Note that interruptTrackedCtx is the context that is currently being tracked, while - // interruptCancel cancels an unrelated context that terminates the tracking goroutine - // without triggering an interrupt (for if the context changes). - // There are cleaner ways of handling the interruption problem, but this is a hot path that - // needs to be called thousands of times per second, which rules out anything that spawns a - // goroutine per call. - interruptTrackedCtx context.Context - interruptCancel context.CancelFunc - - m *sync.Mutex } -// Verify that VU implements lib.VU -var _ lib.VU = &VU{} +// Verify that interfaces are implemented +var _ lib.ActiveVU = &VU{} +var _ lib.InitializedVU = &VU{} -func (u *VU) Reconfigure(id int64) error { - u.ID = id - u.Iteration = 0 - u.Runtime.Set("__VU", u.ID) - return nil -} +// Activate the VU so it will be able to run code +func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { + u.Runtime.ClearInterrupt() + u.RunContext = ¶ms.RunContext + // u.Env = params.Env -func (u *VU) RunOnce(ctx context.Context) error { - u.m.Lock() - defer u.m.Unlock() - // Track the context and interrupt JS execution if it's cancelled. - if u.interruptTrackedCtx != ctx { - interCtx, interCancel := context.WithCancel(context.Background()) - if u.interruptCancel != nil { - u.interruptCancel() + go func() { + <-params.RunContext.Done() + u.Runtime.Interrupt(errInterrupt) + if params.DeactivateCallback != nil { + params.DeactivateCallback() } - u.interruptCancel = interCancel - u.interruptTrackedCtx = ctx - defer interCancel() - go func() { - select { - case <-interCtx.Done(): - case <-ctx.Done(): - u.Runtime.Interrupt(errInterrupt) - } - }() - } + }() + + return lib.ActiveVU(u) +} +// RunOnce runs the default function once. +func (u *VU) RunOnce() error { // Unmarshall the setupData only the first time for each VU so that VUs are isolated but we // still don't use too much CPU in the middle test if u.setupData == nil { @@ -430,7 +408,7 @@ func (u *VU) RunOnce(ctx context.Context) error { } // Call the default function. - _, isFullIteration, totalTime, err := u.runFn(ctx, u.Runner.defaultGroup, true, u.Default, u.setupData) + _, isFullIteration, totalTime, err := u.runFn(*u.RunContext, u.Runner.defaultGroup, true, u.Default, u.setupData) // If MinIterationDuration is specified and the iteration wasn't cancelled // and was less than it, sleep for the remainder diff --git a/lib/execution.go b/lib/execution.go index 210745f7826..d8f4b73eb02 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -58,11 +58,11 @@ type ExecutionScheduler interface { GetExecutors() []Executor // Init initializes all executors, including all of their needed VUs. - Init(ctx context.Context, engineOut chan<- stats.SampleContainer) error + Init(ctx context.Context, samplesOut chan<- stats.SampleContainer) error // Run the ExecutionScheduler, funneling the generated metric samples // through the supplied out channel. - Run(globalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error + Run(globalCtx, runCtx context.Context, samplesOut chan<- stats.SampleContainer) error // Pause a test, or start/resume it. To check if a test is paused, use // GetState().IsPaused(). @@ -175,7 +175,7 @@ type ExecutionState struct { // directly with the channel. These methods will emit a warning or can even // return an error if retrieving a VU takes more than // MaxTimeToWaitForPlannedVU. - vus chan VU + vus chan InitializedVU // The current VU ID, used for the __VU execution context variable. Use the // GetUniqueVUIdentifier() to get unique values for each VU, starting from 1 @@ -280,7 +280,7 @@ func NewExecutionState(options Options, et *ExecutionTuple, maxPlannedVUs, maxPo return &ExecutionState{ Options: options, - vus: make(chan VU, maxPossibleVUs), + vus: make(chan InitializedVU, maxPossibleVUs), executionStatus: new(uint32), currentVUIdentifier: new(uint64), @@ -529,7 +529,7 @@ func (es *ExecutionState) ResumeNotify() <-chan struct{} { // executors might have to retrieve their reserved VUs without using them // immediately - for example, the externally-controlled executor when the // configured maxVUs number is greater than the configured starting VUs. -func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount bool) (VU, error) { +func (es *ExecutionState) GetPlannedVU(logger *logrus.Entry, modifyActiveVUCount bool) (InitializedVU, error) { for i := 1; i <= MaxRetriesGetPlannedVU; i++ { select { case vu := <-es.vus: @@ -566,7 +566,7 @@ func (es *ExecutionState) SetInitVUFunc(initVUFunc InitVUFunc) { // Executors are trusted to correctly declare their needs (via their // GetExecutionRequirements() methods) and then to never ask for more VUs than // they have specified in those requirements. -func (es *ExecutionState) GetUnplannedVU(ctx context.Context, logger *logrus.Entry) (VU, error) { +func (es *ExecutionState) GetUnplannedVU(ctx context.Context, logger *logrus.Entry) (InitializedVU, error) { remVUs := atomic.AddInt64(es.uninitializedUnplannedVUs, -1) if remVUs < 0 { logger.Debug("Reusing a previously initialized unplanned VU") @@ -575,16 +575,12 @@ func (es *ExecutionState) GetUnplannedVU(ctx context.Context, logger *logrus.Ent } logger.Debug("Initializing an unplanned VU, this may affect test results") - vu, err := es.InitializeNewVU(ctx, logger) - if err == nil { - es.ModCurrentlyActiveVUsCount(+1) - } - return vu, err + return es.InitializeNewVU(ctx, logger) } // InitializeNewVU creates and returns a brand new VU, updating the relevant // tracking counters. -func (es *ExecutionState) InitializeNewVU(ctx context.Context, logger *logrus.Entry) (VU, error) { +func (es *ExecutionState) InitializeNewVU(ctx context.Context, logger *logrus.Entry) (InitializedVU, error) { if es.initVUFunc == nil { return nil, fmt.Errorf("initVUFunc wasn't set in the execution state") } @@ -598,14 +594,14 @@ func (es *ExecutionState) InitializeNewVU(ctx context.Context, logger *logrus.En // AddInitializedVU is a helper function that adds VUs into the buffer and // increases the initialized VUs counter. -func (es *ExecutionState) AddInitializedVU(vu VU) { +func (es *ExecutionState) AddInitializedVU(vu InitializedVU) { es.vus <- vu es.ModInitializedVUsCount(+1) } // ReturnVU is a helper function that puts VUs back into the buffer and // decreases the active VUs counter. -func (es *ExecutionState) ReturnVU(vu VU, wasActive bool) { +func (es *ExecutionState) ReturnVU(vu InitializedVU, wasActive bool) { es.vus <- vu if wasActive { es.ModCurrentlyActiveVUsCount(-1) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 34355d8fb72..fa8162250a8 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -32,7 +32,6 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats" ) func sumStagesDuration(stages []Stage) (result time.Duration) { @@ -81,10 +80,10 @@ func validateStages(stages []Stage) []error { // // TODO: emit the end-of-test iteration metrics here (https://github.com/loadimpact/k6/issues/1250) func getIterationRunner( - executionState *lib.ExecutionState, logger *logrus.Entry, _ chan<- stats.SampleContainer, -) func(context.Context, lib.VU) { - return func(ctx context.Context, vu lib.VU) { - err := vu.RunOnce(ctx) + executionState *lib.ExecutionState, logger *logrus.Entry, +) func(context.Context, lib.ActiveVU) { + return func(ctx context.Context, vu lib.ActiveVU) { + err := vu.RunOnce() //TODO: track (non-ramp-down) errors from script iterations as a metric, // and have a default threshold that will abort the script when the error @@ -101,7 +100,6 @@ func getIterationRunner( } else { logger.Error(err.Error()) } - //TODO: investigate context cancelled errors } //TODO: move emission of end-of-iteration metrics here? diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index c97e221096a..10d2e3f5971 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -37,8 +37,8 @@ import ( type vuHandle struct { mutex *sync.RWMutex parentCtx context.Context - getVU func() (lib.VU, error) - returnVU func(lib.VU) + getVU func() (lib.InitializedVU, error) + returnVU func(lib.InitializedVU) canStartIter chan struct{} @@ -48,7 +48,7 @@ type vuHandle struct { } func newStoppedVUHandle( - parentCtx context.Context, getVU func() (lib.VU, error), returnVU func(lib.VU), logger *logrus.Entry, + parentCtx context.Context, getVU func() (lib.InitializedVU, error), returnVU func(lib.InitializedVU), logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) @@ -101,15 +101,11 @@ func (vh *vuHandle) hardStop() { //TODO: simplify this somehow - I feel like there should be a better way to //implement this logic... maybe with sync.Cond? -func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.VU)) { +func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.ActiveVU)) { executorDone := vh.parentCtx.Done() - var vu lib.VU - defer func() { - if vu != nil { - vh.returnVU(vu) - } - }() + var vu lib.ActiveVU + var deactivateVU func() mainLoop: for { @@ -127,12 +123,9 @@ mainLoop: return default: // We're not running, but the executor isn't done yet, so we wait - // for either one of those conditions. But before that, we'll return - // our VU to the pool, if we have it. - if vu != nil { - vh.returnVU(vu) - vu = nil - } + // for either one of those conditions. But before that, clear + // the VU reference to ensure we get a fresh one below. + vu = nil select { case <-canStartIter: // continue on, we were unblocked... @@ -154,13 +147,19 @@ mainLoop: default: } - // Ensure we have a VU + // Ensure we have an active VU if vu == nil { - freshVU, err := vh.getVU() + initVU, err := vh.getVU() if err != nil { return } - vu = freshVU + deactivateVU = func() { + vh.returnVU(initVU) + } + vu = initVU.Activate(&lib.VUActivationParams{ + RunContext: ctx, + DeactivateCallback: deactivateVU, + }) } runIter(ctx, vu) diff --git a/lib/executors.go b/lib/executors.go index b9a110a2aed..7e65b74d221 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -29,10 +29,11 @@ import ( "sync" "time" - "github.com/loadimpact/k6/stats" - "github.com/loadimpact/k6/ui/pb" "github.com/sirupsen/logrus" null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" ) //TODO: remove globals and use some type of explicit dependency injection? @@ -110,7 +111,7 @@ type ExecutorConfig interface { // InitVUFunc is just a shorthand so we don't have to type the function // signature every time. -type InitVUFunc func(context.Context, *logrus.Entry) (VU, error) +type InitVUFunc func(context.Context, *logrus.Entry) (InitializedVU, error) // Executor is the interface all executors should implement type Executor interface { diff --git a/lib/runner.go b/lib/runner.go index 41b68285198..ff5e389e281 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -26,6 +26,32 @@ import ( "github.com/loadimpact/k6/stats" ) +// ActiveVU represents an actively running virtual user. +type ActiveVU interface { + // Runs the VU once. The only way to interrupt the execution is to cancel + // the context given to InitializedVU.Activate() + RunOnce() error +} + +// InitializedVU represents a virtual user ready for work. It needs to be +// activated (i.e. given a context) before it can actually be used. Activation +// also requires a callback function, which will be called when the supplied +// context is done. That way, VUs can be returned to a pool and reused. +type InitializedVU interface { + // Fully activate the VU so it will be able to run code + Activate(*VUActivationParams) ActiveVU +} + +// VUActivationParams are supplied by each executor when it retrieves a VU from +// the buffer pool and activates it for use. +type VUActivationParams struct { + RunContext context.Context + DeactivateCallback func() + // Env map[string]string + // Tags map[string]string + // Exec null.String +} + // 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. The Runner doesn't actually *do* anything in @@ -42,8 +68,7 @@ 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. - //TODO: pass context.Context, so initialization can be killed properly... - NewVU(out chan<- stats.SampleContainer) (VU, error) + NewVU(id int64, out chan<- stats.SampleContainer) (InitializedVU, error) // Runs pre-test setup, if applicable. Setup(ctx context.Context, out chan<- stats.SampleContainer) error @@ -66,15 +91,3 @@ type Runner interface { GetOptions() Options SetOptions(opts Options) error } - -// A VU is a Virtual User, that can be scheduled by an Executor. -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) 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. - //TODO: support reconfiguring of env vars, tags and exec - Reconfigure(id int64) error -} From 03ddf43e1afd7b68c0d2dd09e8b039eee474c5fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 13:41:20 +0100 Subject: [PATCH 175/350] Refactor per-vu-iterations executor for new VU activation --- lib/executor/per_vu_iterations.go | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 30eb6f34f8a..6ddcf88f0fa 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -194,11 +194,20 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(pvi.executionState, pvi.logger, out) + runIteration := getIterationRunner(pvi.executionState, pvi.logger) - handleVU := func(vu lib.VU) { + handleVU := func(initVU lib.InitializedVU) { defer activeVUs.Done() - defer pvi.executionState.ReturnVU(vu, true) + + ctx, cancel := context.WithCancel(maxDurationCtx) + defer cancel() + + vu := initVU.Activate(&lib.VUActivationParams{ + RunContext: ctx, + DeactivateCallback: func() { + pvi.executionState.ReturnVU(initVU, true) + }, + }) for i := int64(0); i < iterations; i++ { select { @@ -213,13 +222,13 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta } for i := int64(0); i < numVUs; i++ { - vu, err := pvi.executionState.GetPlannedVU(pvi.logger, true) + initializedVU, err := pvi.executionState.GetPlannedVU(pvi.logger, true) if err != nil { cancel() return err } activeVUs.Add(1) - go handleVU(vu) + go handleVU(initializedVU) } return nil From 5b48445ec64feb531075c5dd686cc8b2659c2e1d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 13:44:10 +0100 Subject: [PATCH 176/350] Refactor variable-looping-vus executor for new VU activation --- lib/executor/variable_looping_vus.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b2965303a6d..0d1c1f96d7e 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -529,21 +529,21 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo activeVUs := &sync.WaitGroup{} defer activeVUs.Wait() - runIteration := getIterationRunner(vlv.executionState, vlv.logger, out) - getVU := func() (lib.VU, error) { - vu, err := vlv.executionState.GetPlannedVU(vlv.logger, true) + runIteration := getIterationRunner(vlv.executionState, vlv.logger) + getVU := func() (lib.InitializedVU, error) { + initVU, err := vlv.executionState.GetPlannedVU(vlv.logger, true) if err != nil { cancel() - } else { - activeVUs.Add(1) - atomic.AddInt64(activeVUsCount, 1) + return nil, err } - return vu, err + activeVUs.Add(1) + atomic.AddInt64(activeVUsCount, 1) + return initVU, nil } - returnVU := func(vu lib.VU) { - vlv.executionState.ReturnVU(vu, true) - atomic.AddInt64(activeVUsCount, -1) + returnVU := func(initVU lib.InitializedVU) { activeVUs.Done() + atomic.AddInt64(activeVUsCount, -1) + vlv.executionState.ReturnVU(initVU, true) } vuHandles := make([]*vuHandle, maxVUs) From c529e1ee74667d52897b764ff00928d7a04401e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 11:43:02 +0100 Subject: [PATCH 177/350] Refactor shared-iterations executor for new VU activation --- lib/executor/shared_iterations.go | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 04af864484a..5992234cc04 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -193,12 +193,21 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(si.executionState, si.logger, out) + runIteration := getIterationRunner(si.executionState, si.logger) attemptedIters := new(uint64) - handleVU := func(vu lib.VU) { + handleVU := func(initVU lib.InitializedVU) { defer activeVUs.Done() - defer si.executionState.ReturnVU(vu, true) + + ctx, cancel := context.WithCancel(maxDurationCtx) + defer cancel() + + vu := initVU.Activate(&lib.VUActivationParams{ + RunContext: ctx, + DeactivateCallback: func() { + si.executionState.ReturnVU(initVU, true) + }, + }) for { select { @@ -219,13 +228,13 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta } for i := int64(0); i < numVUs; i++ { - vu, err := si.executionState.GetPlannedVU(si.logger, true) + initVU, err := si.executionState.GetPlannedVU(si.logger, true) if err != nil { cancel() return err } activeVUs.Add(1) - go handleVU(vu) + go handleVU(initVU) } return nil From 25c8ee9658d75348bb59526c474333828a517380 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 11:51:49 +0100 Subject: [PATCH 178/350] Refactor constant-looping-vus executor for new VU activation --- lib/executor/constant_looping_vus.go | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index bb3f86b8ef5..8eb49618c74 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -174,12 +174,21 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo defer activeVUs.Wait() regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(clv.executionState, clv.logger, out) + runIteration := getIterationRunner(clv.executionState, clv.logger) - handleVU := func(vu lib.VU) { - defer clv.executionState.ReturnVU(vu, true) + handleVU := func(initVU lib.InitializedVU) { defer activeVUs.Done() + ctx, cancel := context.WithCancel(maxDurationCtx) + defer cancel() + + vu := initVU.Activate(&lib.VUActivationParams{ + RunContext: ctx, + DeactivateCallback: func() { + clv.executionState.ReturnVU(initVU, true) + }, + }) + for { select { case <-regDurationDone: @@ -192,13 +201,13 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } for i := int64(0); i < numVUs; i++ { - vu, err := clv.executionState.GetPlannedVU(clv.logger, true) + initVU, err := clv.executionState.GetPlannedVU(clv.logger, true) if err != nil { cancel() return err } activeVUs.Add(1) - go handleVU(vu) + go handleVU(initVU) } return nil From 2a2b4b60bdf8028bc424409d3d81631326be1ddd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 13:34:24 +0100 Subject: [PATCH 179/350] Refactor externally-controlled executor for new VU activation --- lib/executor/externally_controlled.go | 31 +++++++++++++-------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index a91ea45e475..e87191f8275 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -331,8 +331,8 @@ func (mex *ExternallyControlled) stopWhenDurationIsReached(ctx context.Context, // executing their current iterations before returning. type manualVUHandle struct { *vuHandle - vu lib.VU - wg *sync.WaitGroup + initVU lib.InitializedVU + wg *sync.WaitGroup // This is the cancel of the local context, used to kill its goroutine when // we reduce the number of MaxVUs, so that the Go GC can clean up the VU. @@ -340,16 +340,17 @@ type manualVUHandle struct { } func newManualVUHandle( - parentCtx context.Context, state *lib.ExecutionState, localActiveVUsCount *int64, vu lib.VU, logger *logrus.Entry, + parentCtx context.Context, state *lib.ExecutionState, + localActiveVUsCount *int64, initVU lib.InitializedVU, logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} - getVU := func() (lib.VU, error) { + getVU := func() (lib.InitializedVU, error) { wg.Add(1) state.ModCurrentlyActiveVUsCount(+1) atomic.AddInt64(localActiveVUsCount, +1) - return vu, nil + return initVU, nil } - returnVU := func(_ lib.VU) { + returnVU := func(_ lib.InitializedVU) { state.ModCurrentlyActiveVUsCount(-1) atomic.AddInt64(localActiveVUsCount, -1) wg.Done() @@ -357,7 +358,7 @@ func newManualVUHandle( ctx, cancel := context.WithCancel(parentCtx) return &manualVUHandle{ vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, logger), - vu: vu, + initVU: initVU, wg: &wg, cancelVU: cancel, } @@ -376,7 +377,7 @@ type externallyControlledRunState struct { vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused - runIteration func(context.Context, lib.VU) // a helper closure function that runs a single iteration + runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration } // retrieveStartMaxVUs gets and initializes the (scaled) number of MaxVUs @@ -385,12 +386,12 @@ type externallyControlledRunState struct { // for us. func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { for i := int64(0); i < rs.startMaxVUs; i++ { // get the initial planned VUs from the common buffer - vu, vuGetErr := rs.executor.executionState.GetPlannedVU(rs.executor.logger, false) + initVU, vuGetErr := rs.executor.executionState.GetPlannedVU(rs.executor.logger, false) if vuGetErr != nil { return vuGetErr } vuHandle := newManualVUHandle( - rs.ctx, rs.executor.executionState, rs.activeVUsCount, vu, rs.executor.logger.WithField("vuNum", i), + rs.ctx, rs.executor.executionState, rs.activeVUsCount, initVU, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle @@ -443,12 +444,12 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern return rs.ctx.Err() default: // do nothing } - vu, vuInitErr := executionState.InitializeNewVU(rs.ctx, rs.executor.logger) + initVU, vuInitErr := executionState.InitializeNewVU(rs.ctx, rs.executor.logger) if vuInitErr != nil { return vuInitErr } vuHandle := newManualVUHandle( - rs.ctx, executionState, rs.activeVUsCount, vu, rs.executor.logger.WithField("vuNum", i), + rs.ctx, executionState, rs.activeVUsCount, initVU, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) @@ -474,9 +475,7 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern rs.vuHandles[i].cancelVU() if i < rs.startMaxVUs { // return the initial planned VUs to the common buffer - executionState.ReturnVU(rs.vuHandles[i].vu, false) - } else { - executionState.ModInitializedVUsCount(-1) + executionState.ReturnVU(rs.vuHandles[i].initVU, false) } rs.vuHandles[i] = nil } @@ -522,7 +521,7 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats currentlyPaused: false, activeVUsCount: new(int64), maxVUs: new(int64), - runIteration: getIterationRunner(mex.executionState, mex.logger, out), + runIteration: getIterationRunner(mex.executionState, mex.logger), } *runState.maxVUs = startMaxVUs if err = runState.retrieveStartMaxVUs(); err != nil { From 721295add2ded17736b536c31e6db3264763584d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 16 Mar 2020 18:15:49 +0100 Subject: [PATCH 180/350] Refactor constant-arrival-rate executor for new VU activation --- lib/executor/constant_arrival_rate.go | 45 ++++++++++++++++----------- 1 file changed, 27 insertions(+), 18 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index d80d64ed3f1..2e269f994fb 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -216,26 +216,33 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC }).Debug("Starting executor run...") // Pre-allocate the VUs local shared buffer - vus := make(chan lib.VU, maxVUs) + activeVUs := make(chan lib.ActiveVU, maxVUs) - initialisedVUs := uint64(0) + activeVUsCount := uint64(0) // Make sure we put planned and unplanned VUs back in the global // buffer, and as an extra incentive, this replaces a waitgroup. defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} - for i := uint64(0); i < initialisedVUs; i++ { - car.executionState.ReturnVU(<-vus, true) + for i := uint64(0); i < activeVUsCount; i++ { + vu := <-activeVUs + car.executionState.ReturnVU(vu.(lib.InitializedVU), false) } }() + activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { + activeVU := initVU.Activate(&lib.VUActivationParams{RunContext: maxDurationCtx}) + car.executionState.ModCurrentlyActiveVUsCount(+1) + activeVUsCount++ + return activeVU + } + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - vu, err := car.executionState.GetPlannedVU(car.logger, true) + initVU, err := car.executionState.GetPlannedVU(car.logger, false) if err != nil { return err } - initialisedVUs++ - vus <- vu + activeVUs <- activateVU(initVU) } vusFmt := pb.GetFixedLengthIntFormat(maxVUs) @@ -243,10 +250,10 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0)+" iters/s", arrivalRatePerSec) progresFn := func() (float64, []string) { spent := time.Since(startTime) - currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) - vusInBuffer := uint64(len(vus)) + currActiveVUs := atomic.LoadUint64(&activeVUsCount) + vusInBuffer := uint64(len(activeVUs)) progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", - currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) + currActiveVUs-vusInBuffer, currActiveVUs) right := []string{progVUs, duration.String(), progIters} @@ -264,10 +271,13 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC go trackProgress(ctx, maxDurationCtx, regDurationCtx, &car, progresFn) regDurationDone := regDurationCtx.Done() - runIterationBasic := getIterationRunner(car.executionState, car.logger, out) - runIteration := func(vu lib.VU) { - runIterationBasic(maxDurationCtx, vu) - vus <- vu + runIterationBasic := getIterationRunner(car.executionState, car.logger) + runIteration := func(vu lib.ActiveVU) { + ctx, cancel := context.WithCancel(maxDurationCtx) + defer cancel() + + runIterationBasic(ctx, vu) + activeVUs <- vu } remainingUnplannedVUs := maxVUs - preAllocatedVUs @@ -288,7 +298,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC select { case <-timer.C: select { - case vu := <-vus: + case vu := <-activeVUs: // ideally, we get the VU from the buffer without any issues go runIteration(vu) default: @@ -297,13 +307,12 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } - vu, err := car.executionState.GetUnplannedVU(maxDurationCtx, car.logger) + initVU, err := car.executionState.GetUnplannedVU(maxDurationCtx, car.logger) if err != nil { return err } remainingUnplannedVUs-- - atomic.AddUint64(&initialisedVUs, 1) - go runIteration(vu) + go runIteration(activateVU(initVU)) } case <-regDurationDone: return nil From f5968ae89b81ffa59b6632960cfe5c44a783370f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 17 Mar 2020 10:27:09 +0100 Subject: [PATCH 181/350] Refactor variable-arrival-rate executor for new VU activation --- lib/executor/variable_arrival_rate.go | 50 ++++++++++++++++----------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 283e2bf9318..4062cd16f75 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -301,27 +301,33 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample }).Debug("Starting executor run...") // Pre-allocate the VUs local shared buffer - vus := make(chan lib.VU, maxVUs) + activeVUs := make(chan lib.ActiveVU, maxVUs) - initialisedVUs := uint64(0) + activeVUsCount := uint64(0) // Make sure we put back planned and unplanned VUs back in the global // buffer, and as an extra incentive, this replaces a waitgroup. defer func() { // no need for atomics, since initialisedVUs is mutated only in the select{} - for i := uint64(0); i < initialisedVUs; i++ { - varr.executionState.ReturnVU(<-vus, true) + for i := uint64(0); i < activeVUsCount; i++ { + vu := <-activeVUs + varr.executionState.ReturnVU(vu.(lib.InitializedVU), false) } }() + activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { + activeVU := initVU.Activate(&lib.VUActivationParams{RunContext: maxDurationCtx}) + varr.executionState.ModCurrentlyActiveVUsCount(+1) + atomic.AddUint64(&activeVUsCount, 1) + return activeVU + } + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { - var vu lib.VU - vu, err = varr.executionState.GetPlannedVU(varr.logger, true) + initVU, err := varr.executionState.GetPlannedVU(varr.logger, false) if err != nil { return err } - initialisedVUs++ - vus <- vu + activeVUs <- activateVU(initVU) } tickerPeriod := int64(startTickerPeriod.Duration) @@ -330,11 +336,11 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample itersFmt := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" progresFn := func() (float64, []string) { - currentInitialisedVUs := atomic.LoadUint64(&initialisedVUs) + currActiveVUs := atomic.LoadUint64(&activeVUsCount) currentTickerPeriod := atomic.LoadInt64(&tickerPeriod) - vusInBuffer := uint64(len(vus)) + vusInBuffer := uint64(len(activeVUs)) progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", - currentInitialisedVUs-vusInBuffer, currentInitialisedVUs) + currActiveVUs-vusInBuffer, currActiveVUs) itersPerSec := 0.0 if currentTickerPeriod > 0 { @@ -360,7 +366,14 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) regDurationDone := regDurationCtx.Done() - runIteration := getIterationRunner(varr.executionState, varr.logger, out) + runIterationBasic := getIterationRunner(varr.executionState, varr.logger) + runIteration := func(vu lib.ActiveVU) { + ctx, cancel := context.WithCancel(maxDurationCtx) + defer cancel() + + runIterationBasic(ctx, vu) + activeVUs <- vu + } remainingUnplannedVUs := maxVUs - preAllocatedVUs @@ -387,9 +400,9 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample } } - var vu lib.VU + var vu lib.ActiveVU select { - case vu = <-vus: + case vu = <-activeVUs: // ideally, we get the VU from the buffer without any issues default: if remainingUnplannedVUs == 0 { @@ -397,17 +410,14 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) continue } - vu, err = varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) + initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) if err != nil { return err } + vu = activateVU(initVU) remainingUnplannedVUs-- - atomic.AddUint64(&initialisedVUs, 1) } - go func(vu lib.VU) { - runIteration(maxDurationCtx, vu) - vus <- vu - }(vu) + go runIteration(vu) } return nil } From 6b88d930f5369fec56f8480509d37dd85b88ca2f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 17 Mar 2020 11:43:42 +0100 Subject: [PATCH 182/350] Fix tests --- js/console_test.go | 19 +-- js/http_bench_test.go | 10 +- js/module_loading_test.go | 51 ++++---- js/modules/k6/marshalling_test.go | 12 +- js/runner_test.go | 159 ++++++++++++++++--------- lib/executor/common_test.go | 4 +- lib/executor/execution_test.go | 15 +-- lib/executor/vu_handle.go | 3 +- lib/testutils/minirunner/minirunner.go | 32 ++--- 9 files changed, 186 insertions(+), 119 deletions(-) diff --git a/js/console_test.go b/js/console_test.go index aa0e137a51d..43dd4c8ef78 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -123,14 +123,17 @@ func TestConsole(t *testing.T) { assert.NoError(t, err) samples := make(chan stats.SampleContainer, 100) - vu, err := r.newVU(samples) + initVU, err := r.newVU(1, samples) assert.NoError(t, err) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + logger, hook := logtest.NewNullLogger() logger.Level = logrus.DebugLevel - vu.Console.Logger = logger + jsVU := vu.(*VU) + jsVU.Console.Logger = logger - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(t, err) entry := hook.LastEntry() @@ -215,13 +218,15 @@ func TestFileConsole(t *testing.T) { assert.NoError(t, err) samples := make(chan stats.SampleContainer, 100) - vu, err := r.newVU(samples) + initVU, err := r.newVU(1, samples) assert.NoError(t, err) - vu.Console.Logger.Level = logrus.DebugLevel - hook := logtest.NewLocal(vu.Console.Logger) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + jsVU := vu.(*VU) + jsVU.Console.Logger.Level = logrus.DebugLevel + hook := logtest.NewLocal(jsVU.Console.Logger) - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(t, err) // Test if the file was created. diff --git a/js/http_bench_test.go b/js/http_bench_test.go index 413bc317e27..d0bde93b1ef 100644 --- a/js/http_bench_test.go +++ b/js/http_bench_test.go @@ -4,11 +4,12 @@ import ( "context" "testing" + "github.com/stretchr/testify/assert" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" - "gopkg.in/guregu/null.v3" ) func BenchmarkHTTPRequests(b *testing.B) { @@ -40,13 +41,14 @@ func BenchmarkHTTPRequests(b *testing.B) { <-ch } }() - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) if !assert.NoError(b, err) { return } + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) b.StartTimer() for i := 0; i < b.N; i++ { - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(b, err) } } diff --git a/js/module_loading_test.go b/js/module_loading_test.go index 1ac8121b350..e0183de5350 100644 --- a/js/module_loading_test.go +++ b/js/module_loading_test.go @@ -26,11 +26,12 @@ import ( "os" "testing" + "github.com/spf13/afero" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/stats" - "github.com/spf13/afero" - "github.com/stretchr/testify/require" ) func newDevNullSampleChannel() chan stats.SampleContainer { @@ -111,9 +112,11 @@ func TestLoadOnceGlobalVars(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) + + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -156,9 +159,10 @@ func TestLoadExportsIsUsableInModule(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -200,9 +204,10 @@ func TestLoadDoesntBreakHTTPGet(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -241,15 +246,17 @@ func TestLoadGlobalVarsAreNotSharedBetweenVUs(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) // run a second VU - vu, err = r.NewVU(ch) + initVU, err = r.NewVU(2, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu = initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -298,9 +305,10 @@ func TestLoadCycle(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -357,9 +365,10 @@ func TestLoadCycleBinding(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -418,9 +427,10 @@ func TestBrowserified(t *testing.T) { t.Run(name, func(t *testing.T) { ch := make(chan stats.SampleContainer, 100) defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -456,9 +466,10 @@ func TestLoadingUnexistingModuleDoesntPanic(t *testing.T) { t.Run(name, func(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } diff --git a/js/modules/k6/marshalling_test.go b/js/modules/k6/marshalling_test.go index 0b05f070a97..f8065e5b44b 100644 --- a/js/modules/k6/marshalling_test.go +++ b/js/modules/k6/marshalling_test.go @@ -26,14 +26,15 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestSetupDataMarshalling(t *testing.T) { @@ -129,14 +130,15 @@ func TestSetupDataMarshalling(t *testing.T) { require.NoError(t, err) - samples := make(chan stats.SampleContainer, 100) + samples := make(chan<- stats.SampleContainer, 100) if !assert.NoError(t, runner.Setup(context.Background(), samples)) { return } - vu, err := runner.NewVU(samples) + initVU, err := runner.NewVU(1, samples) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } } diff --git a/js/runner_test.go b/js/runner_test.go index 9aad87c2292..15f398fd7cb 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -39,6 +39,12 @@ import ( "github.com/sirupsen/logrus" + 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" + "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js/common" @@ -53,11 +59,6 @@ import ( "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/dummy" - 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" ) func TestRunnerNew(t *testing.T) { @@ -69,14 +70,15 @@ func TestRunnerNew(t *testing.T) { assert.NoError(t, err) t.Run("NewVU", func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) assert.NoError(t, err) - vuc, ok := vu.(*VU) + vuc, ok := initVU.(*VU) assert.True(t, ok) assert.Equal(t, int64(0), vuc.Runtime.Get("counter").Export()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) t.Run("RunOnce", func(t *testing.T) { - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(t, err) assert.Equal(t, int64(1), vuc.Runtime.Get("counter").Export()) }) @@ -160,9 +162,10 @@ func TestOptionsSettingToScript(t *testing.T) { require.Equal(t, newOptions, r.GetOptions()) samples := make(chan stats.SampleContainer, 100) - vu, err := r.NewVU(samples) + initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } }) @@ -202,12 +205,15 @@ func TestOptionsPropagationToScript(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { + r := r t.Run(name, func(t *testing.T) { samples := make(chan stats.SampleContainer, 100) - vu, err := r.NewVU(samples) + initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } }) @@ -324,15 +330,17 @@ func testSetupDataHelper(t *testing.T, data string) { testdata := map[string]*Runner{"Source": r1} for name, r := range testdata { + r := r t.Run(name, func(t *testing.T) { samples := make(chan stats.SampleContainer, 100) if !assert.NoError(t, r.Setup(context.Background(), samples)) { return } - vu, err := r.NewVU(samples) + initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } }) @@ -388,9 +396,10 @@ func TestConsoleInInitContext(t *testing.T) { r := r t.Run(name, func(t *testing.T) { samples := make(chan stats.SampleContainer, 100) - vu, err := r.NewVU(samples) + initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } }) @@ -462,9 +471,10 @@ func TestRunnerIntegrationImports(t *testing.T) { for name, r := range testdata { r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -488,8 +498,9 @@ func TestVURunContext(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.newVU(make(chan stats.SampleContainer, 100)) + vu, err := r.newVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } @@ -509,7 +520,8 @@ func TestVURunContext(t *testing.T) { assert.Equal(t, vu.Transport, state.Transport) } }) - err = vu.RunOnce(context.Background()) + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = activeVU.RunOnce() assert.NoError(t, err) assert.True(t, fnCalled, "fn() not called") }) @@ -543,10 +555,11 @@ func TestVURunInterrupt(t *testing.T) { } }() - vu, err := r.newVU(samples) + vu, err := r.newVU(1, samples) require.NoError(t, err) - err = vu.RunOnce(ctx) + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) + err = activeVU.RunOnce() assert.Error(t, err) assert.True(t, strings.HasPrefix(err.Error(), "context cancelled at ")) }) @@ -581,16 +594,17 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { }() var wg sync.WaitGroup - vu, err := r.newVU(samples) + initVU, err := r.newVU(1, samples) require.NoError(t, err) for i := 0; i < 1000; i++ { wg.Add(1) newCtx, newCancel := context.WithCancel(ctx) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: newCtx}) ch := make(chan struct{}) go func() { defer wg.Done() close(ch) - vuErr := vu.RunOnce(newCtx) + vuErr := vu.RunOnce() assert.Error(t, vuErr) assert.Contains(t, vuErr.Error(), "context cancelled") }() @@ -625,7 +639,7 @@ func TestVUIntegrationGroups(t *testing.T) { for name, r := range testdata { r := r t.Run(name, func(t *testing.T) { - vu, err := r.newVU(make(chan stats.SampleContainer, 100)) + vu, err := r.newVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } @@ -650,7 +664,8 @@ 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()) + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = activeVU.RunOnce() assert.NoError(t, err) assert.True(t, fnOuterCalled, "fnOuter() not called") assert.True(t, fnInnerCalled, "fnInner() not called") @@ -673,14 +688,16 @@ func TestVUIntegrationMetrics(t *testing.T) { testdata := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range testdata { + r := r t.Run(name, func(t *testing.T) { samples := make(chan stats.SampleContainer, 100) - vu, err := r.newVU(samples) + vu, err := r.newVU(1, samples) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = activeVU.RunOnce() assert.NoError(t, err) sampleCount := 0 for i, sampleC := range stats.GetBufferedSamples(samples) { @@ -742,14 +759,17 @@ func TestVUIntegrationInsecureRequests(t *testing.T) { require.NoError(t, err) runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() if data.errMsg != "" { require.Error(t, err) assert.Contains(t, err.Error(), data.errMsg) @@ -786,12 +806,14 @@ func TestVUIntegrationBlacklistOption(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "IP (10.1.2.3) is in a blacklisted range (10.0.0.0/8)") }) @@ -823,11 +845,12 @@ func TestVUIntegrationBlacklistScript(t *testing.T) { for name, r := range runners { r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "IP (10.1.2.3) is in a blacklisted range (10.0.0.0/8)") }) @@ -867,13 +890,15 @@ func TestVUIntegrationHosts(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() if !assert.NoError(t, err) { return } @@ -937,14 +962,16 @@ func TestVUIntegrationTLSConfig(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() if data.errMsg != "" { require.Error(t, err) assert.Contains(t, err.Error(), data.errMsg) @@ -981,13 +1008,15 @@ func TestVUIntegrationHTTP2(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { samples := make(chan stats.SampleContainer, 100) - vu, err := r.NewVU(samples) + initVU, err := r.NewVU(1, samples) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() assert.NoError(t, err) protoFound := false @@ -1010,9 +1039,10 @@ func TestVUIntegrationOpenFunctionError(t *testing.T) { `) assert.NoError(t, err) - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) assert.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() assert.Error(t, err) assert.Contains(t, err.Error(), "only available to init code") } @@ -1054,13 +1084,15 @@ func TestVUIntegrationCookiesReset(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) for i := 0; i < 2; i++ { - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(t, err) } }) @@ -1109,16 +1141,18 @@ func TestVUIntegrationCookiesNoReset(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() assert.NoError(t, err) - err = vu.RunOnce(context.Background()) + err = vu.RunOnce() assert.NoError(t, err) }) } @@ -1142,13 +1176,15 @@ func TestVUIntegrationVUID(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1234, make(chan stats.SampleContainer, 100)) if !assert.NoError(t, err) { return } - assert.NoError(t, vu.Reconfigure(1234)) - err = vu.RunOnce(context.Background()) + + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() assert.NoError(t, err) }) } @@ -1238,11 +1274,13 @@ func TestVUIntegrationClientCerts(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "remote error: tls: bad certificate") } @@ -1284,10 +1322,12 @@ func TestVUIntegrationClientCerts(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { - vu, err := r.NewVU(make(chan stats.SampleContainer, 100)) + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - err := vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err := vu.RunOnce() assert.NoError(t, err) } }) @@ -1424,13 +1464,15 @@ func TestArchiveRunningIntegrity(t *testing.T) { runners := map[string]*Runner{"Source": r1, "Archive": r2} for name, r := range runners { + r := r t.Run(name, func(t *testing.T) { ch := make(chan stats.SampleContainer, 100) err = r.Setup(context.Background(), ch) require.NoError(t, err) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) - err = vu.RunOnce(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + err = vu.RunOnce() require.NoError(t, err) }) } @@ -1504,12 +1546,13 @@ func TestStuffNotPanicking(t *testing.T) { require.NoError(t, err) ch := make(chan stats.SampleContainer, 1000) - vu, err := r.NewVU(ch) + initVU, err := r.NewVU(1, ch) require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) errC := make(chan error) - go func() { errC <- vu.RunOnce(ctx) }() + go func() { errC <- vu.RunOnce() }() select { case <-time.After(15 * time.Second): diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index fcd74b11516..3a355a26f0d 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -54,8 +54,8 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) - es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.VU, error) { - return runner.NewVU(engineOut) + es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { + return runner.NewVU(1, engineOut) }) et, err := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) diff --git a/lib/executor/execution_test.go b/lib/executor/execution_test.go index 0db52ba85d6..39470cd36ad 100644 --- a/lib/executor/execution_test.go +++ b/lib/executor/execution_test.go @@ -28,12 +28,13 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils" - "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" ) func TestExecutionStateVUIDs(t *testing.T) { @@ -89,11 +90,11 @@ func TestExecutionStateGettingVUs(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 20) - es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.VU, error) { + es.SetInitVUFunc(func(_ context.Context, _ *logrus.Entry) (lib.InitializedVU, error) { return &minirunner.VU{}, nil }) - var vu lib.VU + var vu lib.InitializedVU for i := 0; i < 10; i++ { require.EqualValues(t, i, es.GetInitializedVUsCount()) vu, err = es.InitializeNewVU(context.Background(), logEntry) @@ -128,13 +129,13 @@ func TestExecutionStateGettingVUs(t *testing.T) { // Test getting uninitialized vus will work for i := 0; i < 10; i++ { - require.EqualValues(t, 10+i, es.GetCurrentlyActiveVUsCount()) + require.EqualValues(t, 10+i, es.GetInitializedVUsCount()) vu, err = es.GetUnplannedVU(context.Background(), logEntry) require.NoError(t, err) require.Empty(t, logHook.Drain()) require.NotNil(t, vu) - require.EqualValues(t, 10+i+1, es.GetCurrentlyActiveVUsCount()) require.EqualValues(t, 10+i+1, es.GetInitializedVUsCount()) + require.EqualValues(t, 10, es.GetCurrentlyActiveVUsCount()) } // Check that getting 1 more unplanned VU will error out diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 10d2e3f5971..3f94a50753a 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -48,7 +48,8 @@ type vuHandle struct { } func newStoppedVUHandle( - parentCtx context.Context, getVU func() (lib.InitializedVU, error), returnVU func(lib.InitializedVU), logger *logrus.Entry, + parentCtx context.Context, getVU func() (lib.InitializedVU, error), + returnVU func(lib.InitializedVU), logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index ab7c2babdb1..19f2620cfd2 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -30,8 +30,9 @@ import ( // Ensure mock implementations conform to the interfaces. var ( - _ lib.Runner = &MiniRunner{} - _ lib.VU = &VU{} + _ lib.Runner = &MiniRunner{} + _ lib.InitializedVU = &VU{} + _ lib.ActiveVU = &VU{} ) // MiniRunner partially implements the lib.Runner interface, but instead of @@ -56,7 +57,7 @@ func (r MiniRunner) MakeArchive() *lib.Archive { } // NewVU returns a new VU with an incremental ID. -func (r *MiniRunner) NewVU(out chan<- stats.SampleContainer) (lib.VU, error) { +func (r *MiniRunner) NewVU(_ int64, out chan<- stats.SampleContainer) (lib.InitializedVU, error) { nextVUNum := atomic.AddInt64(&r.NextVUID, 1) return &VU{R: r, Out: out, ID: nextVUNum - 1}, nil } @@ -109,14 +110,21 @@ func (r *MiniRunner) SetOptions(opts lib.Options) error { // VU is a mock VU, spawned by a MiniRunner. type VU struct { - R *MiniRunner - Out chan<- stats.SampleContainer - ID int64 - Iteration int64 + R *MiniRunner + RunContext *context.Context + Out chan<- stats.SampleContainer + ID int64 + Iteration int64 +} + +// Activate the VU so it will be able to run code +func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { + vu.RunContext = ¶ms.RunContext + return lib.ActiveVU(vu) } // RunOnce runs the mock default function once, incrementing its iteration. -func (vu VU) RunOnce(ctx context.Context) error { +func (vu *VU) RunOnce() error { if vu.R.Fn == nil { return nil } @@ -125,15 +133,9 @@ func (vu VU) RunOnce(ctx context.Context) error { Vu: vu.ID, Iteration: vu.Iteration, } - newctx := lib.WithState(ctx, state) + newctx := lib.WithState(*vu.RunContext, state) vu.Iteration++ return vu.R.Fn(newctx, vu.Out) } - -// Reconfigure changes the VU ID. -func (vu *VU) Reconfigure(id int64) error { - vu.ID = id - return nil -} From b963308620e51b0bfc969957012e6820e87cd59c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 7 Apr 2020 11:23:33 +0200 Subject: [PATCH 183/350] Defer cancel all RunContexts in tests to release deactivation goroutines This is a gotcha of the API that could easily leak goroutines... I'm hoping this will resolve all timeout failures in CI. --- js/console_test.go | 8 ++- js/http_bench_test.go | 4 +- js/module_loading_test.go | 36 +++++++++---- js/modules/k6/marshalling_test.go | 4 +- js/runner_test.go | 89 +++++++++++++++++++++++-------- 5 files changed, 105 insertions(+), 36 deletions(-) diff --git a/js/console_test.go b/js/console_test.go index 43dd4c8ef78..54327bcc35d 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -126,7 +126,9 @@ func TestConsole(t *testing.T) { initVU, err := r.newVU(1, samples) assert.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) logger, hook := logtest.NewNullLogger() logger.Level = logrus.DebugLevel @@ -221,7 +223,9 @@ func TestFileConsole(t *testing.T) { initVU, err := r.newVU(1, samples) assert.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) jsVU := vu.(*VU) jsVU.Console.Logger.Level = logrus.DebugLevel hook := logtest.NewLocal(jsVU.Console.Logger) diff --git a/js/http_bench_test.go b/js/http_bench_test.go index d0bde93b1ef..63e92dfc836 100644 --- a/js/http_bench_test.go +++ b/js/http_bench_test.go @@ -45,7 +45,9 @@ func BenchmarkHTTPRequests(b *testing.B) { if !assert.NoError(b, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) b.StartTimer() for i := 0; i < b.N; i++ { err = vu.RunOnce() diff --git a/js/module_loading_test.go b/js/module_loading_test.go index e0183de5350..daee1431330 100644 --- a/js/module_loading_test.go +++ b/js/module_loading_test.go @@ -114,7 +114,9 @@ func TestLoadOnceGlobalVars(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) require.NoError(t, err) err = vu.RunOnce() require.NoError(t, err) @@ -160,7 +162,9 @@ func TestLoadExportsIsUsableInModule(t *testing.T) { ch := newDevNullSampleChannel() defer close(ch) initVU, err := r.NewVU(1, ch) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) require.NoError(t, err) err = vu.RunOnce() require.NoError(t, err) @@ -206,7 +210,9 @@ func TestLoadDoesntBreakHTTPGet(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -248,14 +254,18 @@ func TestLoadGlobalVarsAreNotSharedBetweenVUs(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) // run a second VU initVU, err = r.NewVU(2, ch) require.NoError(t, err) - vu = initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel = context.WithCancel(context.Background()) + defer cancel() + vu = initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -307,7 +317,9 @@ func TestLoadCycle(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -367,7 +379,9 @@ func TestLoadCycleBinding(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -429,7 +443,9 @@ func TestBrowserified(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -468,7 +484,9 @@ func TestLoadingUnexistingModuleDoesntPanic(t *testing.T) { defer close(ch) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) diff --git a/js/modules/k6/marshalling_test.go b/js/modules/k6/marshalling_test.go index f8065e5b44b..8a55527ba7f 100644 --- a/js/modules/k6/marshalling_test.go +++ b/js/modules/k6/marshalling_test.go @@ -137,7 +137,9 @@ func TestSetupDataMarshalling(t *testing.T) { } initVU, err := runner.NewVU(1, samples) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } diff --git a/js/runner_test.go b/js/runner_test.go index 15f398fd7cb..3ddac98941c 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -76,7 +76,9 @@ func TestRunnerNew(t *testing.T) { assert.True(t, ok) assert.Equal(t, int64(0), vuc.Runtime.Get("counter").Export()) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) t.Run("RunOnce", func(t *testing.T) { err = vu.RunOnce() assert.NoError(t, err) @@ -164,7 +166,9 @@ func TestOptionsSettingToScript(t *testing.T) { samples := make(chan stats.SampleContainer, 100) initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } @@ -211,8 +215,9 @@ func TestOptionsPropagationToScript(t *testing.T) { initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } @@ -339,7 +344,9 @@ func testSetupDataHelper(t *testing.T, data string) { } initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } @@ -398,7 +405,9 @@ func TestConsoleInInitContext(t *testing.T) { samples := make(chan stats.SampleContainer, 100) initVU, err := r.NewVU(1, samples) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } @@ -473,7 +482,9 @@ func TestRunnerIntegrationImports(t *testing.T) { t.Run(name, func(t *testing.T) { initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) @@ -520,7 +531,9 @@ func TestVURunContext(t *testing.T) { assert.Equal(t, vu.Transport, state.Transport) } }) - activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) err = activeVU.RunOnce() assert.NoError(t, err) assert.True(t, fnCalled, "fn() not called") @@ -664,7 +677,9 @@ func TestVUIntegrationGroups(t *testing.T) { assert.Equal(t, "my group", g.Parent.Name) assert.Equal(t, r.GetDefaultGroup(), g.Parent.Parent) }) - activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) err = activeVU.RunOnce() assert.NoError(t, err) assert.True(t, fnOuterCalled, "fnOuter() not called") @@ -696,7 +711,9 @@ func TestVUIntegrationMetrics(t *testing.T) { return } - activeVU := vu.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) err = activeVU.RunOnce() assert.NoError(t, err) sampleCount := 0 @@ -768,7 +785,9 @@ func TestVUIntegrationInsecureRequests(t *testing.T) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() if data.errMsg != "" { require.Error(t, err) @@ -812,7 +831,9 @@ func TestVUIntegrationBlacklistOption(t *testing.T) { if !assert.NoError(t, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "IP (10.1.2.3) is in a blacklisted range (10.0.0.0/8)") @@ -849,7 +870,9 @@ func TestVUIntegrationBlacklistScript(t *testing.T) { if !assert.NoError(t, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "IP (10.1.2.3) is in a blacklisted range (10.0.0.0/8)") @@ -897,7 +920,9 @@ func TestVUIntegrationHosts(t *testing.T) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() if !assert.NoError(t, err) { return @@ -970,7 +995,9 @@ func TestVUIntegrationTLSConfig(t *testing.T) { if !assert.NoError(t, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() if data.errMsg != "" { require.Error(t, err) @@ -1015,7 +1042,9 @@ func TestVUIntegrationHTTP2(t *testing.T) { if !assert.NoError(t, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() assert.NoError(t, err) @@ -1041,7 +1070,9 @@ func TestVUIntegrationOpenFunctionError(t *testing.T) { initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) assert.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() assert.Error(t, err) assert.Contains(t, err.Error(), "only available to init code") @@ -1090,7 +1121,9 @@ func TestVUIntegrationCookiesReset(t *testing.T) { if !assert.NoError(t, err) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) for i := 0; i < 2; i++ { err = vu.RunOnce() assert.NoError(t, err) @@ -1148,7 +1181,9 @@ func TestVUIntegrationCookiesNoReset(t *testing.T) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() assert.NoError(t, err) @@ -1183,7 +1218,9 @@ func TestVUIntegrationVUID(t *testing.T) { return } - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() assert.NoError(t, err) }) @@ -1279,7 +1316,9 @@ func TestVUIntegrationClientCerts(t *testing.T) { r.Logger, _ = logtest.NewNullLogger() initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() require.Error(t, err) assert.Contains(t, err.Error(), "remote error: tls: bad certificate") @@ -1326,7 +1365,9 @@ func TestVUIntegrationClientCerts(t *testing.T) { t.Run(name, func(t *testing.T) { initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) if assert.NoError(t, err) { - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err := vu.RunOnce() assert.NoError(t, err) } @@ -1471,7 +1512,9 @@ func TestArchiveRunningIntegrity(t *testing.T) { require.NoError(t, err) initVU, err := r.NewVU(1, ch) require.NoError(t, err) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: context.Background()}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() require.NoError(t, err) }) From 2e1ef94cb142611fed037a3d105ed70c10eca6bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 31 Mar 2020 17:17:02 +0200 Subject: [PATCH 184/350] Use a separate ActiveVU struct Resolves https://github.com/loadimpact/k6/pull/1368#discussion_r400153280 --- js/console_test.go | 4 ++-- js/runner.go | 30 +++++++++++++++++------------- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/js/console_test.go b/js/console_test.go index 54327bcc35d..f25a575b850 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -132,7 +132,7 @@ func TestConsole(t *testing.T) { logger, hook := logtest.NewNullLogger() logger.Level = logrus.DebugLevel - jsVU := vu.(*VU) + jsVU := vu.(*ActiveVU) jsVU.Console.Logger = logger err = vu.RunOnce() @@ -226,7 +226,7 @@ func TestFileConsole(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) - jsVU := vu.(*VU) + jsVU := vu.(*ActiveVU) jsVU.Console.Logger.Level = logrus.DebugLevel hook := logtest.NewLocal(jsVU.Console.Logger) diff --git a/js/runner.go b/js/runner.go index 11059769e42..56afcf97a24 100644 --- a/js/runner.go +++ b/js/runner.go @@ -353,14 +353,13 @@ func (r *Runner) timeoutErrorDuration(stage string) time.Duration { type VU struct { BundleInstance - Runner *Runner - RunContext *context.Context - Transport *http.Transport - Dialer *netext.Dialer - CookieJar *cookiejar.Jar - TLSConfig *tls.Config - ID int64 - Iteration int64 + Runner *Runner + Transport *http.Transport + Dialer *netext.Dialer + CookieJar *cookiejar.Jar + TLSConfig *tls.Config + ID int64 + Iteration int64 Console *console BPool *bpool.BufferPool @@ -371,13 +370,18 @@ type VU struct { } // Verify that interfaces are implemented -var _ lib.ActiveVU = &VU{} +var _ lib.ActiveVU = &ActiveVU{} var _ lib.InitializedVU = &VU{} +// ActiveVU holds a VU and its activation parameters +type ActiveVU struct { + *VU + *lib.VUActivationParams +} + // Activate the VU so it will be able to run code func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() - u.RunContext = ¶ms.RunContext // u.Env = params.Env go func() { @@ -388,11 +392,11 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { } }() - return lib.ActiveVU(u) + return &ActiveVU{u, params} } // RunOnce runs the default function once. -func (u *VU) RunOnce() error { +func (u *ActiveVU) RunOnce() error { // Unmarshall the setupData only the first time for each VU so that VUs are isolated but we // still don't use too much CPU in the middle test if u.setupData == nil { @@ -408,7 +412,7 @@ func (u *VU) RunOnce() error { } // Call the default function. - _, isFullIteration, totalTime, err := u.runFn(*u.RunContext, u.Runner.defaultGroup, true, u.Default, u.setupData) + _, isFullIteration, totalTime, err := u.runFn(u.RunContext, u.Runner.defaultGroup, true, u.Default, u.setupData) // If MinIterationDuration is specified and the iteration wasn't cancelled // and was less than it, sleep for the remainder From 8fb23c31c15424a679eb12faba589b2506ec9d47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 1 Apr 2020 11:50:42 +0200 Subject: [PATCH 185/350] Synchronize ActiveVU.RunOnce calls This avoids panics with concurrent RunOnce calls. It essentially brings back a2e1c6f4e. Resolves https://github.com/loadimpact/k6/pull/1368#discussion_r400156411 --- js/runner.go | 6 ++++++ js/runner_test.go | 9 +++++---- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/js/runner.go b/js/runner.go index 56afcf97a24..f0468f44032 100644 --- a/js/runner.go +++ b/js/runner.go @@ -28,6 +28,7 @@ import ( "net/http" "net/http/cookiejar" "strconv" + "sync" "time" "github.com/dop251/goja" @@ -196,6 +197,7 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, Console: r.console, BPool: bpool.NewBufferPool(100), Samples: samplesOut, + RunMutex: &sync.Mutex{}, } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) @@ -360,6 +362,7 @@ type VU struct { TLSConfig *tls.Config ID int64 Iteration int64 + RunMutex *sync.Mutex Console *console BPool *bpool.BufferPool @@ -397,6 +400,9 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { // RunOnce runs the default function once. func (u *ActiveVU) RunOnce() error { + u.RunMutex.Lock() + defer u.RunMutex.Unlock() + // Unmarshall the setupData only the first time for each VU so that VUs are isolated but we // still don't use too much CPU in the middle test if u.setupData == nil { diff --git a/js/runner_test.go b/js/runner_test.go index 3ddac98941c..08e70709681 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -597,10 +597,9 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { for name, r := range testdata { name, r := name, r t.Run(name, func(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Minute) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() samples := make(chan stats.SampleContainer, 100) - defer close(samples) go func() { for range samples { } @@ -612,10 +611,12 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { for i := 0; i < 1000; i++ { wg.Add(1) newCtx, newCancel := context.WithCancel(ctx) - vu := initVU.Activate(&lib.VUActivationParams{RunContext: newCtx}) + vu := initVU.Activate(&lib.VUActivationParams{ + RunContext: newCtx, + DeactivateCallback: func() { wg.Done() }, + }) ch := make(chan struct{}) go func() { - defer wg.Done() close(ch) vuErr := vu.RunOnce() assert.Error(t, vuErr) From 5ab0d34bcab7f0f05015accc575412638be51452 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 3 Apr 2020 17:16:27 +0200 Subject: [PATCH 186/350] Fix active VU tracking in variable-looping-vus This is not good, for several reasons: - "Active VUs" are tracked in three different places: the activeVUs WG used for synchronizing VUs with the executor's Run(), the activeVUsCount used for progress/UI, and the global active VUs count in ExecutionState shared by all executors. Ideally there should be a single API that unifies this and preferably doesn't allow direct access to ExecutionState. - getVU increments the local active VU count (and in contrast those must be decremented in returnVU), even though VUs aren't technically "activated" until vuHandle.start() is called. Fixing this would involve changes to vuHandle to maybe also handle an additional activateVU lambda. - This change moves the global ExecutionState VU count handling to handleNewScheduledVUs(), mainly so that tests like TestVariableLoopingVUsRampDownNoWobble can access it, but this is not a good reason, and the split active VU tracking will likely cause weird issues when run with other executors. The refactor in point 1 above would likely help here. --- lib/executor/variable_looping_vus.go | 16 +++++++++------- lib/executor/variable_looping_vus_test.go | 2 +- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 0d1c1f96d7e..62b12f1e411 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -531,19 +531,19 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo runIteration := getIterationRunner(vlv.executionState, vlv.logger) getVU := func() (lib.InitializedVU, error) { - initVU, err := vlv.executionState.GetPlannedVU(vlv.logger, true) + initVU, err := vlv.executionState.GetPlannedVU(vlv.logger, false) if err != nil { cancel() - return nil, err + } else { + activeVUs.Add(1) + atomic.AddInt64(activeVUsCount, 1) } - activeVUs.Add(1) - atomic.AddInt64(activeVUsCount, 1) - return initVU, nil + return initVU, err } returnVU := func(initVU lib.InitializedVU) { - activeVUs.Done() + vlv.executionState.ReturnVU(initVU, false) atomic.AddInt64(activeVUsCount, -1) - vlv.executionState.ReturnVU(initVU, true) + activeVUs.Done() } vuHandles := make([]*vuHandle, maxVUs) @@ -563,10 +563,12 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo if newScheduledVUs > currentScheduledVUs { for vuNum := currentScheduledVUs; vuNum < newScheduledVUs; vuNum++ { vuHandles[vuNum].start() + vlv.executionState.ModCurrentlyActiveVUsCount(+1) } } else { for vuNum := newScheduledVUs; vuNum < currentScheduledVUs; vuNum++ { vuHandles[vuNum].gracefulStop() + vlv.executionState.ModCurrentlyActiveVUsCount(-1) } } currentScheduledVUs = newScheduledVUs diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index fb42d7556a0..e3279385dff 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -92,7 +92,7 @@ func TestVariableLoopingVUsRun(t *testing.T) { require.NoError(t, <-errCh) assert.Equal(t, []int64{5, 3, 0}, result) - assert.Equal(t, int64(29), iterCount) + assert.Equal(t, int64(29), atomic.LoadInt64(&iterCount)) } // Ensure there's no wobble of VUs during graceful ramp-down, without segments. From e66f7e6795daac6ef0a43a6d1e172b9593e26306 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 7 Apr 2020 13:13:45 +0200 Subject: [PATCH 187/350] Handle VU ID counter in tests with ExecutionState This aligns it more with how this is done in regular execution. Resolves https://github.com/loadimpact/k6/pull/1368#discussion_r398551808 --- lib/executor/common_test.go | 2 +- lib/testutils/minirunner/minirunner.go | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 3a355a26f0d..2bd27d09600 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -55,7 +55,7 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta logEntry := logrus.NewEntry(testLog) es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { - return runner.NewVU(1, engineOut) + return runner.NewVU(int64(es.GetUniqueVUIdentifier()), engineOut) }) et, err := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index 19f2620cfd2..e5d18a5a281 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -22,7 +22,6 @@ package minirunner import ( "context" - "sync/atomic" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" @@ -57,9 +56,8 @@ func (r MiniRunner) MakeArchive() *lib.Archive { } // NewVU returns a new VU with an incremental ID. -func (r *MiniRunner) NewVU(_ int64, out chan<- stats.SampleContainer) (lib.InitializedVU, error) { - nextVUNum := atomic.AddInt64(&r.NextVUID, 1) - return &VU{R: r, Out: out, ID: nextVUNum - 1}, nil +func (r *MiniRunner) NewVU(id int64, out chan<- stats.SampleContainer) (lib.InitializedVU, error) { + return &VU{R: r, Out: out, ID: id}, nil } // Setup calls the supplied mock setup() function, if present. From 476ccca0fc6e930171072784de0196c631ec9f2f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 14 Apr 2020 16:20:45 +0200 Subject: [PATCH 188/350] Synchronize VU deactivation with executors This ensures executors wait for all VU deactivation goroutines to complete. Resolves: - https://github.com/loadimpact/k6/pull/1368#discussion_r400156411 - https://github.com/loadimpact/k6/pull/1368#discussion_r400036864 - https://github.com/loadimpact/k6/pull/1368#discussion_r407501463 --- lib/executor/constant_arrival_rate.go | 23 +++++++++++----- lib/executor/constant_arrival_rate_test.go | 1 + lib/executor/constant_looping_vus.go | 3 +-- lib/executor/constant_looping_vus_test.go | 5 ++-- lib/executor/externally_controlled_test.go | 6 ++--- lib/executor/per_vu_iterations.go | 3 +-- lib/executor/per_vu_iterations_test.go | 1 + lib/executor/shared_iterations.go | 3 +-- lib/executor/variable_arrival_rate.go | 21 ++++++++++----- lib/executor/variable_arrival_rate_test.go | 5 ++-- lib/executor/variable_looping_vus_test.go | 2 +- lib/testutils/minirunner/minirunner.go | 31 +++++++++++++++------- 12 files changed, 67 insertions(+), 37 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 2e269f994fb..7f1adcee4a9 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -25,6 +25,7 @@ import ( "fmt" "math" "math/big" + "sync" "sync/atomic" "time" @@ -217,22 +218,30 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC // Pre-allocate the VUs local shared buffer activeVUs := make(chan lib.ActiveVU, maxVUs) - activeVUsCount := uint64(0) - // Make sure we put planned and unplanned VUs back in the global - // buffer, and as an extra incentive, this replaces a waitgroup. + + activeVUsWg := &sync.WaitGroup{} + defer activeVUsWg.Wait() + // Make sure we put planned and unplanned VUs back in the global buffer defer func() { - // no need for atomics, since initialisedVUs is mutated only in the select{} - for i := uint64(0); i < activeVUsCount; i++ { + currActiveVUs := atomic.LoadUint64(&activeVUsCount) + for i := uint64(0); i < currActiveVUs; i++ { vu := <-activeVUs car.executionState.ReturnVU(vu.(lib.InitializedVU), false) } }() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { - activeVU := initVU.Activate(&lib.VUActivationParams{RunContext: maxDurationCtx}) + activeVUsWg.Add(1) + activeVU := initVU.Activate(&lib.VUActivationParams{ + RunContext: maxDurationCtx, + DeactivateCallback: func() { + car.executionState.ReturnVU(initVU, true) + activeVUsWg.Done() + }, + }) car.executionState.ModCurrentlyActiveVUsCount(+1) - activeVUsCount++ + atomic.AddUint64(&activeVUsCount, 1) return activeVU } diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 20e8a4df49c..5b300a94917 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -57,6 +57,7 @@ func newExecutionSegmentSequenceFromString(str string) *lib.ExecutionSegmentSequ func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { return ConstantArrivalRateConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), Rate: null.IntFrom(50), Duration: types.NullDurationFrom(5 * time.Second), diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 8eb49618c74..76c31eeb916 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -177,8 +177,6 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo runIteration := getIterationRunner(clv.executionState, clv.logger) handleVU := func(initVU lib.InitializedVU) { - defer activeVUs.Done() - ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -186,6 +184,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo RunContext: ctx, DeactivateCallback: func() { clv.executionState.ReturnVU(initVU, true) + activeVUs.Done() }, }) diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go index a3e66122cb0..d45751e0ecd 100644 --- a/lib/executor/constant_looping_vus_test.go +++ b/lib/executor/constant_looping_vus_test.go @@ -36,8 +36,9 @@ import ( func getTestConstantLoopingVUsConfig() ConstantLoopingVUsConfig { return ConstantLoopingVUsConfig{ - VUs: null.IntFrom(10), - Duration: types.NullDurationFrom(1 * time.Second), + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, + VUs: null.IntFrom(10), + Duration: types.NullDurationFrom(1 * time.Second), } } diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index da8f57540f2..fde53426880 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -48,7 +48,7 @@ func getTestExternallyControlledConfig() ExternallyControlledConfig { func TestExternallyControlledRun(t *testing.T) { t.Parallel() - var doneIters uint64 + doneIters := new(uint64) et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) @@ -56,7 +56,7 @@ func TestExternallyControlledRun(t *testing.T) { t, getTestExternallyControlledConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(200 * time.Millisecond) - atomic.AddUint64(&doneIters, 1) + atomic.AddUint64(doneIters, 1) return nil }), ) @@ -107,6 +107,6 @@ func TestExternallyControlledRun(t *testing.T) { <-doneCh wg.Wait() require.NoError(t, <-errCh) - assert.Equal(t, uint64(50), doneIters) + assert.Equal(t, uint64(50), atomic.LoadUint64(doneIters)) assert.Equal(t, []int64{2, 4, 8, 0}, resultVUCount) } diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 6ddcf88f0fa..92236d7d6ed 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -197,8 +197,6 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta runIteration := getIterationRunner(pvi.executionState, pvi.logger) handleVU := func(initVU lib.InitializedVU) { - defer activeVUs.Done() - ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -206,6 +204,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta RunContext: ctx, DeactivateCallback: func() { pvi.executionState.ReturnVU(initVU, true) + activeVUs.Done() }, }) diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 41037d45434..45d6b63b2db 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -37,6 +37,7 @@ import ( func getTestPerVUIterationsConfig() PerVUIterationsConfig { return PerVUIterationsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, VUs: null.IntFrom(10), Iterations: null.IntFrom(100), MaxDuration: types.NullDurationFrom(3 * time.Second), diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 5992234cc04..1fbad262d00 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -197,8 +197,6 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) handleVU := func(initVU lib.InitializedVU) { - defer activeVUs.Done() - ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -206,6 +204,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta RunContext: ctx, DeactivateCallback: func() { si.executionState.ReturnVU(initVU, true) + activeVUs.Done() }, }) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 4062cd16f75..0a6b69c0084 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -24,6 +24,7 @@ import ( "context" "fmt" "math" + "sync" "sync/atomic" "time" @@ -302,20 +303,28 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample // Pre-allocate the VUs local shared buffer activeVUs := make(chan lib.ActiveVU, maxVUs) - activeVUsCount := uint64(0) - // Make sure we put back planned and unplanned VUs back in the global - // buffer, and as an extra incentive, this replaces a waitgroup. + // Make sure we put planned and unplanned VUs back in the global buffer defer func() { - // no need for atomics, since initialisedVUs is mutated only in the select{} - for i := uint64(0); i < activeVUsCount; i++ { + currActiveVUs := atomic.LoadUint64(&activeVUsCount) + for i := uint64(0); i < currActiveVUs; i++ { vu := <-activeVUs varr.executionState.ReturnVU(vu.(lib.InitializedVU), false) } }() + activeVUsWg := &sync.WaitGroup{} + defer activeVUsWg.Wait() + activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { - activeVU := initVU.Activate(&lib.VUActivationParams{RunContext: maxDurationCtx}) + activeVUsWg.Add(1) + activeVU := initVU.Activate(&lib.VUActivationParams{ + RunContext: maxDurationCtx, + DeactivateCallback: func() { + varr.executionState.ReturnVU(initVU, true) + activeVUsWg.Done() + }, + }) varr.executionState.ModCurrentlyActiveVUsCount(+1) atomic.AddUint64(&activeVUsCount, 1) return activeVU diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 51eeaceda6e..8eea969adce 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -41,8 +41,9 @@ import ( func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { return VariableArrivalRateConfig{ - TimeUnit: types.NullDurationFrom(time.Second), - StartRate: null.IntFrom(10), + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, + TimeUnit: types.NullDurationFrom(time.Second), + StartRate: null.IntFrom(10), Stages: []Stage{ { Duration: types.NullDurationFrom(time.Second * 1), diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index e3279385dff..0f2f232918e 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -130,7 +130,7 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { sampleTimes := []time.Duration{ 100 * time.Millisecond, - 3400 * time.Millisecond, + 3000 * time.Millisecond, } const rampDownSamples = 50 diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index e5d18a5a281..9b2c73320e9 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -31,7 +31,7 @@ import ( var ( _ lib.Runner = &MiniRunner{} _ lib.InitializedVU = &VU{} - _ lib.ActiveVU = &VU{} + _ lib.ActiveVU = &ActiveVU{} ) // MiniRunner partially implements the lib.Runner interface, but instead of @@ -108,21 +108,32 @@ func (r *MiniRunner) SetOptions(opts lib.Options) error { // VU is a mock VU, spawned by a MiniRunner. type VU struct { - R *MiniRunner - RunContext *context.Context - Out chan<- stats.SampleContainer - ID int64 - Iteration int64 + R *MiniRunner + Out chan<- stats.SampleContainer + ID int64 + Iteration int64 +} + +// ActiveVU holds a VU and its activation parameters +type ActiveVU struct { + *VU + *lib.VUActivationParams } // Activate the VU so it will be able to run code func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { - vu.RunContext = ¶ms.RunContext - return lib.ActiveVU(vu) + go func() { + <-params.RunContext.Done() + if params.DeactivateCallback != nil { + params.DeactivateCallback() + } + }() + + return &ActiveVU{vu, params} } // RunOnce runs the mock default function once, incrementing its iteration. -func (vu *VU) RunOnce() error { +func (vu *ActiveVU) RunOnce() error { if vu.R.Fn == nil { return nil } @@ -131,7 +142,7 @@ func (vu *VU) RunOnce() error { Vu: vu.ID, Iteration: vu.Iteration, } - newctx := lib.WithState(*vu.RunContext, state) + newctx := lib.WithState(vu.RunContext, state) vu.Iteration++ From d6f5dd00b765325e88701a96381de5611c6f7444 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 16 Apr 2020 14:42:23 +0300 Subject: [PATCH 189/350] CI updates --- .circleci/config.yml | 3 +++ Dockerfile | 4 ++-- appveyor.yml | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 74d93b100fe..8625d47b7f6 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -103,6 +103,9 @@ jobs: if [ "${CIRCLE_BRANCH}" == "master" ]; then docker tag loadimpact/k6 loadimpact/k6:master docker push loadimpact/k6:master + elif [ "${CIRCLE_BRANCH}" == "new-schedulers" ]; then + docker tag loadimpact/k6 loadimpact/k6:new-executors + docker push loadimpact/k6:new-executors elif [[ "${CIRCLE_TAG}" =~ ^v[0-9]+(\.[0-9]+)*$ ]]; then docker tag loadimpact/k6 loadimpact/k6:${CIRCLE_TAG:1} docker push loadimpact/k6:latest diff --git a/Dockerfile b/Dockerfile index 89d250b7221..4b548b7bd2b 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,10 +1,10 @@ -FROM golang:1.13-alpine as builder +FROM golang:1.14-alpine as builder WORKDIR $GOPATH/src/github.com/loadimpact/k6 ADD . . RUN apk --no-cache add git RUN CGO_ENABLED=0 go install -a -trimpath -ldflags "-s -w -X github.com/loadimpact/k6/lib/consts.VersionDetails=$(date -u +"%FT%T%z")/$(git describe --always --long --dirty)" -FROM alpine:3.10 +FROM alpine:3.11 RUN apk add --no-cache ca-certificates && \ adduser -D -u 12345 -g 12345 k6 COPY --from=builder /go/bin/k6 /usr/bin/k6 diff --git a/appveyor.yml b/appveyor.yml index f71ab8d5286..24090f366e9 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -10,7 +10,7 @@ environment: # specific to go VERSION: "%APPVEYOR_REPO_TAG_NAME:v=%" GOPATH: c:\gopath - GOVERSION: 1.14 + GOVERSION: 1.14.2 GOMAXPROCS: 2 CGO_ENABLED: '0' GOARCH: amd64 From 6ee9e3e6186cf1f9e2310bf1feed05001e8bc5bd Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 16 Apr 2020 15:12:44 +0300 Subject: [PATCH 190/350] Fix a minor bug --- core/engine.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/engine.go b/core/engine.go index efea7b69373..3e53069669b 100644 --- a/core/engine.go +++ b/core/engine.go @@ -317,7 +317,8 @@ func (e *Engine) emitMetrics() { t := time.Now() executionState := e.ExecutionScheduler.GetState() - e.Samples <- stats.ConnectedSamples{ + // TODO: optimize and move this, it shouldn't call processSamples() directly + e.processSamples([]stats.SampleContainer{stats.ConnectedSamples{ Samples: []stats.Sample{ { Time: t, @@ -333,7 +334,7 @@ func (e *Engine) emitMetrics() { }, Tags: e.Options.RunTags, Time: t, - } + }}) } func (e *Engine) processThresholds() (shouldAbort bool) { From b3b2a6e753a66b48626f2c43baf8e0360e6f8113 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 16 Apr 2020 11:08:52 +0200 Subject: [PATCH 191/350] Unexport runMutex, move to ActiveVU Resolves: - https://github.com/loadimpact/k6/pull/1368#discussion_r406811906 - https://github.com/loadimpact/k6/pull/1368#discussion_r413765333 --- js/runner.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/js/runner.go b/js/runner.go index f0468f44032..e372c2f6997 100644 --- a/js/runner.go +++ b/js/runner.go @@ -197,7 +197,6 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, Console: r.console, BPool: bpool.NewBufferPool(100), Samples: samplesOut, - RunMutex: &sync.Mutex{}, } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) @@ -362,7 +361,6 @@ type VU struct { TLSConfig *tls.Config ID int64 Iteration int64 - RunMutex *sync.Mutex Console *console BPool *bpool.BufferPool @@ -378,6 +376,7 @@ var _ lib.InitializedVU = &VU{} // ActiveVU holds a VU and its activation parameters type ActiveVU struct { + runMutex *sync.Mutex *VU *lib.VUActivationParams } @@ -395,13 +394,13 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { } }() - return &ActiveVU{u, params} + return &ActiveVU{&sync.Mutex{}, u, params} } // RunOnce runs the default function once. func (u *ActiveVU) RunOnce() error { - u.RunMutex.Lock() - defer u.RunMutex.Unlock() + u.runMutex.Lock() + defer u.runMutex.Unlock() // Unmarshall the setupData only the first time for each VU so that VUs are isolated but we // still don't use too much CPU in the middle test From eb13303fce6d8bc28378adf7adb2553ac2d82609 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 23 Apr 2020 20:15:39 +0300 Subject: [PATCH 192/350] Fix constant-arrival-rate executor hanging See https://github.com/loadimpact/k6/pull/1368#discussion_r414004705 --- lib/executor/constant_arrival_rate.go | 29 ++++++++++++--------------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 7f1adcee4a9..8f441316e45 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -207,9 +207,6 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) - defer cancel() - // Make sure the log and the progress bar have accurate information car.logger.WithFields(logrus.Fields{ "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, @@ -222,12 +219,16 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC activeVUsWg := &sync.WaitGroup{} defer activeVUsWg.Wait() - // Make sure we put planned and unplanned VUs back in the global buffer + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + + // Make sure all VUs aren't executing iterations anymore, for the cancel() + // above to deactivate them. defer func() { - currActiveVUs := atomic.LoadUint64(&activeVUsCount) - for i := uint64(0); i < currActiveVUs; i++ { - vu := <-activeVUs - car.executionState.ReturnVU(vu.(lib.InitializedVU), false) + // activeVUsCount is modified only in the loop below, which is done here + for i := uint64(0); i < activeVUsCount; i++ { + <-activeVUs } }() @@ -279,13 +280,9 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC car.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, &car, progresFn) - regDurationDone := regDurationCtx.Done() runIterationBasic := getIterationRunner(car.executionState, car.logger) runIteration := func(vu lib.ActiveVU) { - ctx, cancel := context.WithCancel(maxDurationCtx) - defer cancel() - - runIterationBasic(ctx, vu) + runIterationBasic(maxDurationCtx, vu) activeVUs <- vu } @@ -302,7 +299,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC )).Duration) for li, gi := 0, start; ; li, gi = li+1, gi+offsets[li%len(offsets)] { - var t = notScaledTickerPeriod*time.Duration(gi) - time.Since(startTime) + t := notScaledTickerPeriod*time.Duration(gi) - time.Since(startTime) timer.Reset(t) select { case <-timer.C: @@ -312,7 +309,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC go runIteration(vu) default: if remainingUnplannedVUs == 0 { - //TODO: emit an error metric? + // TODO: emit an error metric? car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) break } @@ -323,7 +320,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC remainingUnplannedVUs-- go runIteration(activateVU(initVU)) } - case <-regDurationDone: + case <-regDurationCtx.Done(): return nil } } From d01e2c195647b4ade57577a93e320ff9d6dd2fdb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 24 Apr 2020 13:22:00 +0200 Subject: [PATCH 193/350] Change defer order in variable-arrival-rate executor This does the same changes as the commit before for the constant-arrival-rate executor, to avoid any potential hanging, since both executors behave similarly. --- lib/executor/variable_arrival_rate.go | 26 +++++++++++--------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 0a6b69c0084..4689e9666c2 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -292,30 +292,29 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample maxArrivalRatePerSec, _ := getArrivalRatePerSec(getScaledArrivalRate(segment, maxUnscaledRate, timeUnit)).Float64() startTickerPeriod := getTickerPeriod(startArrivalRate) - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) - defer cancel() - // Make sure the log and the progress bar have accurate information varr.logger.WithFields(logrus.Fields{ "maxVUs": maxVUs, "preAllocatedVUs": preAllocatedVUs, "duration": duration, "numStages": len(varr.config.Stages), "startTickerPeriod": startTickerPeriod.Duration, "type": varr.config.GetType(), }).Debug("Starting executor run...") + activeVUsWg := &sync.WaitGroup{} + defer activeVUsWg.Wait() + + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + defer cancel() + // Pre-allocate the VUs local shared buffer activeVUs := make(chan lib.ActiveVU, maxVUs) activeVUsCount := uint64(0) - // Make sure we put planned and unplanned VUs back in the global buffer + // Make sure all VUs aren't executing iterations anymore, for the cancel() + // above to deactivate them. defer func() { - currActiveVUs := atomic.LoadUint64(&activeVUsCount) - for i := uint64(0); i < currActiveVUs; i++ { - vu := <-activeVUs - varr.executionState.ReturnVU(vu.(lib.InitializedVU), false) + for i := uint64(0); i < activeVUsCount; i++ { + <-activeVUs } }() - activeVUsWg := &sync.WaitGroup{} - defer activeVUsWg.Wait() - activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ @@ -377,10 +376,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample regDurationDone := regDurationCtx.Done() runIterationBasic := getIterationRunner(varr.executionState, varr.logger) runIteration := func(vu lib.ActiveVU) { - ctx, cancel := context.WithCancel(maxDurationCtx) - defer cancel() - - runIterationBasic(ctx, vu) + runIterationBasic(maxDurationCtx, vu) activeVUs <- vu } From aac9d810efaa023b04a7983781124562ac8b82b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 24 Apr 2020 13:40:20 +0200 Subject: [PATCH 194/350] Bring back context cancelled TODO Resolves https://github.com/loadimpact/k6/pull/1368#discussion_r414411191 but unresolves https://github.com/loadimpact/k6/pull/1368#discussion_r400160573 :) --- lib/executor/helpers.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index fa8162250a8..62e98e2c683 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -85,7 +85,7 @@ func getIterationRunner( return func(ctx context.Context, vu lib.ActiveVU) { err := vu.RunOnce() - //TODO: track (non-ramp-down) errors from script iterations as a metric, + // TODO: track (non-ramp-down) errors from script iterations as a metric, // and have a default threshold that will abort the script when the error // rate exceeds a certain percentage @@ -100,9 +100,10 @@ func getIterationRunner( } else { logger.Error(err.Error()) } + // TODO: investigate context cancelled errors } - //TODO: move emission of end-of-iteration metrics here? + // TODO: move emission of end-of-iteration metrics here? executionState.AddFullIterations(1) } } From a8ac0e572a835053189579d31ae861405f436ec3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 24 Apr 2020 11:32:29 +0200 Subject: [PATCH 195/350] Skip TestCDNJS See #1408 --- loader/cdnjs_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/loader/cdnjs_test.go b/loader/cdnjs_test.go index 55de95eadba..848129e7ee0 100644 --- a/loader/cdnjs_test.go +++ b/loader/cdnjs_test.go @@ -30,6 +30,8 @@ import ( ) func TestCDNJS(t *testing.T) { + t.Skip("skipped to avoid inconsistent API responses") + paths := map[string]struct { parts []string src string From a52c5029b62962493e4147040335a1c01f75b151 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 24 Apr 2020 13:47:46 +0200 Subject: [PATCH 196/350] Appease the godot linter --- js/runner.go | 2 +- lib/testutils/minirunner/minirunner.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/js/runner.go b/js/runner.go index e372c2f6997..2302abc8bc1 100644 --- a/js/runner.go +++ b/js/runner.go @@ -381,7 +381,7 @@ type ActiveVU struct { *lib.VUActivationParams } -// Activate the VU so it will be able to run code +// Activate the VU so it will be able to run code. func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() // u.Env = params.Env diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index 9b2c73320e9..cc6197c4bf8 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -120,7 +120,7 @@ type ActiveVU struct { *lib.VUActivationParams } -// Activate the VU so it will be able to run code +// Activate the VU so it will be able to run code. func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { go func() { <-params.RunContext.Done() From 91bb3fb0eeb449425d5be48614f48c7a4dc5f2f5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 27 Apr 2020 10:28:22 +0200 Subject: [PATCH 197/350] Move runMutex to js.VU Tests that share a VU like TestVURunInterruptDoesntPanic need to access the same mutex instance, otherwise it panics while reading setupData. This could be fixed in the test itself, but protecting against panics should be transparent to user code. --- js/runner.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/js/runner.go b/js/runner.go index 2302abc8bc1..a32b06c5197 100644 --- a/js/runner.go +++ b/js/runner.go @@ -197,6 +197,7 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, Console: r.console, BPool: bpool.NewBufferPool(100), Samples: samplesOut, + runMutex: sync.Mutex{}, } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) @@ -367,6 +368,7 @@ type VU struct { Samples chan<- stats.SampleContainer + runMutex sync.Mutex setupData goja.Value } @@ -376,7 +378,6 @@ var _ lib.InitializedVU = &VU{} // ActiveVU holds a VU and its activation parameters type ActiveVU struct { - runMutex *sync.Mutex *VU *lib.VUActivationParams } @@ -394,7 +395,7 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { } }() - return &ActiveVU{&sync.Mutex{}, u, params} + return &ActiveVU{u, params} } // RunOnce runs the default function once. From 82443a39fd045609a0548bb2413d29a46eba44b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 27 Apr 2020 10:53:15 +0200 Subject: [PATCH 198/350] Fix lint CI step This is still crappy, but it seems to produce base commits more reliably. --- .circleci/config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 45c6eb94e73..764cad19494 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -53,7 +53,7 @@ jobs: name: Run golangci-lint command: | export PATH="$GOPATH/bin:$PATH" - basecommit=$(git log --first-parent --pretty=format:'%H %D' | egrep '/(master|new-schedulers)' | head -1 | cut -d' ' -f1) + basecommit=$(git log --decorate | grep '^commit.*origin/' | sed -n 2p | cut -d' ' -f2) echo "basecommit=$basecommit" golangci-lint run --out-format=tab --new-from-rev "$basecommit" ./... From 27e4f0ef79a3fa7544db1739211156bbaa1495c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 27 Apr 2020 18:27:55 +0200 Subject: [PATCH 199/350] Silence nestif linter --- api/v1/status_routes.go | 5 +++-- lib/executor/helpers.go | 2 +- ui/pb/progressbar.go | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index 01368a27bfb..c3c3e8b44aa 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -26,10 +26,11 @@ import ( "net/http" "github.com/julienschmidt/httprouter" + "github.com/manyminds/api2go/jsonapi" + "github.com/loadimpact/k6/api/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/executor" - "github.com/manyminds/api2go/jsonapi" ) func HandleGetStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { @@ -71,7 +72,7 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par return } - if status.Stopped { + if status.Stopped { //nolint:nestif engine.Stop() } else { if status.Paused.Valid { diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 34355d8fb72..a781d423467 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -55,7 +55,7 @@ func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 // A helper function to avoid code duplication func validateStages(stages []Stage) []error { var errors []error - if len(stages) == 0 { + if len(stages) == 0 { //nolint:nestif errors = append(errors, fmt.Errorf("at least one stage has to be specified")) } else { for i, s := range stages { diff --git a/ui/pb/progressbar.go b/ui/pb/progressbar.go index 24ff9a795d6..90bf830abc0 100644 --- a/ui/pb/progressbar.go +++ b/ui/pb/progressbar.go @@ -245,7 +245,7 @@ func (pb *ProgressBar) Render(maxLeft, widthDelta int) ProgressBarRender { width := Clampf(float64(pb.width+widthDelta), minWidth, DefaultWidth) pb.width = int(width) - if pb.width > minWidth { + if pb.width > minWidth { //nolint:nestif space := pb.width - 2 filled := int(float64(space) * progress) From af18a3af6dcb4b44884b694ae895e83ef6b44b1c Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 10:28:08 +0300 Subject: [PATCH 200/350] use ExecutionTuple --- lib/executor/executors_test.go | 4 ++-- lib/executor/variable_looping_vus.go | 15 +++++++-------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 385c94563cc..4b21a23f56c 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -428,13 +428,13 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 18 * time.Second, PlannedVUs: 4}, {TimeOffset: 20 * time.Second, PlannedVUs: 1}, } - rawStepsNoZeroEnd := conf.getRawExecutionSteps(nil, false) + rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) assert.Equal(t, expRawStepsNoZeroEnd, rawStepsNoZeroEnd) endOffset, isFinal := lib.GetEndOffset(rawStepsNoZeroEnd) assert.Equal(t, 20*time.Second, endOffset) assert.Equal(t, false, isFinal) - rawStepsZeroEnd := conf.getRawExecutionSteps(nil, true) + rawStepsZeroEnd := conf.getRawExecutionSteps(et, true) assert.Equal(t, append(expRawStepsNoZeroEnd, lib.ExecutionStep{TimeOffset: 23 * time.Second, PlannedVUs: 0}), rawStepsZeroEnd, diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b2965303a6d..13fc1a356aa 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -187,7 +187,7 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // // More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 //nolint:funlen -func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegment, zeroEnd bool) []lib.ExecutionStep { +func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple, zeroEnd bool) []lib.ExecutionStep { // For accurate results, calculations are done with the unscaled values, and // the values are scaled only before we add them to the steps result slice fromVUs := vlvc.StartVUs.Int64 @@ -200,7 +200,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme } // Reserve the scaled StartVUs at the beginning - prevScaledVUs := es.Scale(vlvc.StartVUs.Int64) + prevScaledVUs := et.ES.Scale(vlvc.StartVUs.Int64) steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} timeFromStart := time.Duration(0) totalDuration := time.Duration(0) @@ -221,7 +221,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme // Handle 0-duration stages, i.e. instant VU jumps if stageDuration == 0 { fromVUs = stageEndVUs - prevScaledVUs = es.Scale(stageEndVUs) + prevScaledVUs = et.ES.Scale(stageEndVUs) steps = append(steps, lib.ExecutionStep{ TimeOffset: timeFromStart, PlannedVUs: uint64(prevScaledVUs), @@ -254,7 +254,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme stepGlobalVUs := fromVUs + int64( math.Round((float64(t)*float64(stageEndVUs-fromVUs))/float64(stageDuration)), ) - stepScaledVus := es.Scale(stepGlobalVUs) + stepScaledVus := et.ES.Scale(stepGlobalVUs) if stepScaledVus == prevScaledVUs { // only add steps when there's a change in the number of VUs @@ -272,7 +272,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(es *lib.ExecutionSegme } fromVUs = stageEndVUs - prevScaledVUs = es.Scale(stageEndVUs) + prevScaledVUs = et.ES.Scale(stageEndVUs) timeFromStart += stageDuration steps = append(steps, lib.ExecutionStep{ TimeOffset: timeFromStart, @@ -436,7 +436,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu // - If the last stage's target is more than 0, the VUs at the end of the // executor's life will have more time to finish their last iterations. func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { - steps := vlvc.getRawExecutionSteps(et.ES, false) + steps := vlvc.getRawExecutionSteps(et, false) executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) // Handle graceful ramp-downs, if we have them @@ -484,8 +484,7 @@ var _ lib.Executor = &VariableLoopingVUs{} // and see what happens)... :/ so maybe see how it can be split? // nolint:funlen,gocognit func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { - segment := vlv.executionState.Options.ExecutionSegment - rawExecutionSteps := vlv.config.getRawExecutionSteps(segment, true) + rawExecutionSteps := vlv.config.getRawExecutionSteps(vlv.executionState.ExecutionTuple, true) regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) if !isFinal { return fmt.Errorf("%s expected raw end offset at %s to be final", vlv.config.GetName(), regularDuration) From 4c59d5c102ca53364aef2ded898d2aaff1251bb1 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 11:34:48 +0300 Subject: [PATCH 201/350] Add test with execution segmeng 0:1/3 --- lib/executor/executors_test.go | 100 ------------ lib/executor/variable_looping_vus.go | 9 +- lib/executor/variable_looping_vus_test.go | 181 ++++++++++++++++++++++ 3 files changed, 182 insertions(+), 108 deletions(-) diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 4b21a23f56c..d3eaac4e016 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -388,103 +388,3 @@ func TestConfigMapParsingAndValidation(t *testing.T) { }) } } - -func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { - t.Parallel() - et, err := lib.NewExecutionTuple(nil, nil) - require.NoError(t, err) - conf := NewVariableLoopingVUsConfig("test") - conf.StartVUs = null.IntFrom(4) - conf.Stages = []Stage{ - {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, - {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, - {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, - {Target: null.IntFrom(1), Duration: types.NullDurationFrom(4 * time.Second)}, - {Target: null.IntFrom(4), Duration: types.NullDurationFrom(3 * time.Second)}, - {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, - {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, - {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, - } - - expRawStepsNoZeroEnd := []lib.ExecutionStep{ - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 5}, - {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 3 * time.Second, PlannedVUs: 5}, - {TimeOffset: 4 * time.Second, PlannedVUs: 4}, - {TimeOffset: 5 * time.Second, PlannedVUs: 3}, - {TimeOffset: 6 * time.Second, PlannedVUs: 2}, - {TimeOffset: 7 * time.Second, PlannedVUs: 1}, - {TimeOffset: 8 * time.Second, PlannedVUs: 2}, - {TimeOffset: 9 * time.Second, PlannedVUs: 3}, - {TimeOffset: 10 * time.Second, PlannedVUs: 4}, - {TimeOffset: 11 * time.Second, PlannedVUs: 5}, - {TimeOffset: 12 * time.Second, PlannedVUs: 4}, - {TimeOffset: 13 * time.Second, PlannedVUs: 3}, - {TimeOffset: 14 * time.Second, PlannedVUs: 2}, - {TimeOffset: 15 * time.Second, PlannedVUs: 1}, - {TimeOffset: 16 * time.Second, PlannedVUs: 2}, - {TimeOffset: 17 * time.Second, PlannedVUs: 3}, - {TimeOffset: 18 * time.Second, PlannedVUs: 4}, - {TimeOffset: 20 * time.Second, PlannedVUs: 1}, - } - rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) - assert.Equal(t, expRawStepsNoZeroEnd, rawStepsNoZeroEnd) - endOffset, isFinal := lib.GetEndOffset(rawStepsNoZeroEnd) - assert.Equal(t, 20*time.Second, endOffset) - assert.Equal(t, false, isFinal) - - rawStepsZeroEnd := conf.getRawExecutionSteps(et, true) - assert.Equal(t, - append(expRawStepsNoZeroEnd, lib.ExecutionStep{TimeOffset: 23 * time.Second, PlannedVUs: 0}), - rawStepsZeroEnd, - ) - endOffset, isFinal = lib.GetEndOffset(rawStepsZeroEnd) - assert.Equal(t, 23*time.Second, endOffset) - assert.Equal(t, true, isFinal) - - // GracefulStop and GracefulRampDown equal to the default 30 sec - assert.Equal(t, []lib.ExecutionStep{ - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 5}, - {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 33 * time.Second, PlannedVUs: 5}, - {TimeOffset: 42 * time.Second, PlannedVUs: 4}, - {TimeOffset: 50 * time.Second, PlannedVUs: 1}, - {TimeOffset: 53 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(et)) - - // Try a longer GracefulStop than the GracefulRampDown - conf.GracefulStop = types.NullDurationFrom(80 * time.Second) - assert.Equal(t, []lib.ExecutionStep{ - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 5}, - {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 33 * time.Second, PlannedVUs: 5}, - {TimeOffset: 42 * time.Second, PlannedVUs: 4}, - {TimeOffset: 50 * time.Second, PlannedVUs: 1}, - {TimeOffset: 103 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(et)) - - // Try a much shorter GracefulStop than the GracefulRampDown - conf.GracefulStop = types.NullDurationFrom(3 * time.Second) - assert.Equal(t, []lib.ExecutionStep{ - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 5}, - {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 26 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(et)) - - // Try a zero GracefulStop - conf.GracefulStop = types.NullDurationFrom(0 * time.Second) - assert.Equal(t, []lib.ExecutionStep{ - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 5}, - {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 23 * time.Second, PlannedVUs: 0}, - }, conf.GetExecutionRequirements(et)) - - // Try a zero GracefulStop and GracefulRampDown, i.e. raw steps with 0 end cap - conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) - assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) -} diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 13fc1a356aa..dabbd22a1cb 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -234,9 +234,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // every minIntervalBetweenVUAdjustments. No floats or ratios, // since nanoseconds should be good enough for anyone... :) stepInterval := stageDuration / time.Duration(stageVUAbsDiff) - if stepInterval < minIntervalBetweenVUAdjustments { - stepInterval = minIntervalBetweenVUAdjustments - } // Loop through the potential steps, adding an item to the // result only when there's a change in the number of VUs. @@ -246,11 +243,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // important that the scaling via the execution segment should // happen AFTER the rest of the calculations have been done and // we've rounded the global "global" number of VUs. - for t := stepInterval; ; t += stepInterval { // Skip the first step, since we've already added that - if time.Duration(abs(int64(stageDuration-t))) < minIntervalBetweenVUAdjustments { - // Skip the last step of the stage, add it below to correct any minor clock skew - break - } + for t := stepInterval; t < stageDuration; t += stepInterval { // Skip the first step, since we've already added that stepGlobalVUs := fromVUs + int64( math.Round((float64(t)*float64(stageEndVUs-fromVUs))/float64(stageDuration)), ) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index fb42d7556a0..4d6c33035ae 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -165,3 +165,184 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { } assert.Equal(t, []int64{10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}, vuChanges) } + +func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { + t.Parallel() + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + conf := NewVariableLoopingVUsConfig("test") + conf.StartVUs = null.IntFrom(4) + conf.Stages = []Stage{ + {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, + } + + expRawStepsNoZeroEnd := []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 3 * time.Second, PlannedVUs: 5}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 3}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 8 * time.Second, PlannedVUs: 2}, + {TimeOffset: 9 * time.Second, PlannedVUs: 3}, + {TimeOffset: 10 * time.Second, PlannedVUs: 4}, + {TimeOffset: 11 * time.Second, PlannedVUs: 5}, + {TimeOffset: 12 * time.Second, PlannedVUs: 4}, + {TimeOffset: 13 * time.Second, PlannedVUs: 3}, + {TimeOffset: 14 * time.Second, PlannedVUs: 2}, + {TimeOffset: 15 * time.Second, PlannedVUs: 1}, + {TimeOffset: 16 * time.Second, PlannedVUs: 2}, + {TimeOffset: 17 * time.Second, PlannedVUs: 3}, + {TimeOffset: 18 * time.Second, PlannedVUs: 4}, + {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + } + rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) + assert.Equal(t, expRawStepsNoZeroEnd, rawStepsNoZeroEnd) + endOffset, isFinal := lib.GetEndOffset(rawStepsNoZeroEnd) + assert.Equal(t, 20*time.Second, endOffset) + assert.Equal(t, false, isFinal) + + rawStepsZeroEnd := conf.getRawExecutionSteps(et, true) + assert.Equal(t, + append(expRawStepsNoZeroEnd, lib.ExecutionStep{TimeOffset: 23 * time.Second, PlannedVUs: 0}), + rawStepsZeroEnd, + ) + endOffset, isFinal = lib.GetEndOffset(rawStepsZeroEnd) + assert.Equal(t, 23*time.Second, endOffset) + assert.Equal(t, true, isFinal) + + // GracefulStop and GracefulRampDown equal to the default 30 sec + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 50 * time.Second, PlannedVUs: 1}, + {TimeOffset: 53 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a longer GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(80 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 50 * time.Second, PlannedVUs: 1}, + {TimeOffset: 103 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a much shorter GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(3 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a zero GracefulStop + conf.GracefulStop = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 23 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a zero GracefulStop and GracefulRampDown, i.e. raw steps with 0 end cap + conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) +} + +func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { + t.Parallel() + et, err := lib.NewExecutionTuple(newExecutionSegmentFromString("0:1/3"), nil) + require.NoError(t, err) + conf := NewVariableLoopingVUsConfig("test") + conf.StartVUs = null.IntFrom(4) + conf.Stages = []Stage{ + {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, + } + + expRawStepsNoZeroEnd := []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 2 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 18 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + } + rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) + assert.Equal(t, expRawStepsNoZeroEnd, rawStepsNoZeroEnd) + endOffset, isFinal := lib.GetEndOffset(rawStepsNoZeroEnd) + assert.Equal(t, 20*time.Second, endOffset) + assert.Equal(t, true, isFinal) + + rawStepsZeroEnd := conf.getRawExecutionSteps(et, true) + assert.Equal(t, expRawStepsNoZeroEnd, rawStepsZeroEnd) + endOffset, isFinal = lib.GetEndOffset(rawStepsZeroEnd) + assert.Equal(t, 20*time.Second, endOffset) + assert.Equal(t, true, isFinal) + + // GracefulStop and GracefulRampDown equal to the default 30 sec + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 42 * time.Second, PlannedVUs: 1}, + {TimeOffset: 50 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a longer GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(80 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 42 * time.Second, PlannedVUs: 1}, + {TimeOffset: 50 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a much shorter GracefulStop than the GracefulRampDown + conf.GracefulStop = types.NullDurationFrom(3 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a zero GracefulStop + conf.GracefulStop = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 23 * time.Second, PlannedVUs: 0}, + }, conf.GetExecutionRequirements(et)) + + // Try a zero GracefulStop and GracefulRampDown, i.e. raw steps with 0 end cap + conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) + assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) +} From 721a47e6072ccfec05774c1f867a7b286a8968f2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 12:16:16 +0300 Subject: [PATCH 202/350] Fix not adding steps with the same amount of VUs --- lib/executor/variable_looping_vus.go | 40 +++++++++-------------- lib/executor/variable_looping_vus_test.go | 2 -- 2 files changed, 16 insertions(+), 26 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index dabbd22a1cb..23ddc135ac1 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -39,10 +39,6 @@ import ( const variableLoopingVUsType = "variable-looping-vus" -// How often we can make VU adjustments when processing stages -// TODO: make configurable, in some bounds? -const minIntervalBetweenVUAdjustments = 100 * time.Millisecond - func init() { lib.RegisterExecutorConfigType( variableLoopingVUsType, @@ -200,11 +196,18 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } // Reserve the scaled StartVUs at the beginning - prevScaledVUs := et.ES.Scale(vlvc.StartVUs.Int64) + prevScaledVUs := et.ScaleInt64(vlvc.StartVUs.Int64) steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} timeFromStart := time.Duration(0) totalDuration := time.Duration(0) + addStep := func(step lib.ExecutionStep) { + if len(steps) == 0 || steps[len(steps)-1].PlannedVUs != step.PlannedVUs { + steps = append(steps, step) + prevScaledVUs = int64(step.PlannedVUs) + } + } + for _, stage := range vlvc.Stages { stageEndVUs := stage.Target.Int64 stageDuration := time.Duration(stage.Duration.Duration) @@ -221,18 +224,14 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // Handle 0-duration stages, i.e. instant VU jumps if stageDuration == 0 { fromVUs = stageEndVUs - prevScaledVUs = et.ES.Scale(stageEndVUs) - steps = append(steps, lib.ExecutionStep{ + addStep(lib.ExecutionStep{ TimeOffset: timeFromStart, - PlannedVUs: uint64(prevScaledVUs), + PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), }) continue } - // For each stage, limit any VU adjustments between the previous - // number of VUs and the stage's target to happen at most once - // every minIntervalBetweenVUAdjustments. No floats or ratios, - // since nanoseconds should be good enough for anyone... :) + // No floats or ratios,since nanoseconds should be good enough for anyone... :) stepInterval := stageDuration / time.Duration(stageVUAbsDiff) // Loop through the potential steps, adding an item to the @@ -247,29 +246,22 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple stepGlobalVUs := fromVUs + int64( math.Round((float64(t)*float64(stageEndVUs-fromVUs))/float64(stageDuration)), ) - stepScaledVus := et.ES.Scale(stepGlobalVUs) - - if stepScaledVus == prevScaledVUs { - // only add steps when there's a change in the number of VUs - continue - } // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() - steps = append(steps, lib.ExecutionStep{ + addStep(lib.ExecutionStep{ TimeOffset: timeFromStart + t, - PlannedVUs: uint64(stepScaledVus), + PlannedVUs: uint64(et.ScaleInt64(stepGlobalVUs)), }) - prevScaledVUs = stepScaledVus } fromVUs = stageEndVUs - prevScaledVUs = et.ES.Scale(stageEndVUs) timeFromStart += stageDuration - steps = append(steps, lib.ExecutionStep{ + + addStep(lib.ExecutionStep{ TimeOffset: timeFromStart, - PlannedVUs: uint64(prevScaledVUs), + PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), }) } diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 4d6c33035ae..ba010f7c7cc 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -286,7 +286,6 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { expRawStepsNoZeroEnd := []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 2 * time.Second, PlannedVUs: 2}, {TimeOffset: 4 * time.Second, PlannedVUs: 1}, {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, @@ -294,7 +293,6 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { {TimeOffset: 12 * time.Second, PlannedVUs: 1}, {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, - {TimeOffset: 18 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, } rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) From 44512c79952b572feb324a589858f0eb9e79a030 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 12:44:37 +0300 Subject: [PATCH 203/350] Simplify and probably stabilize the calculation of the steps --- lib/executor/variable_looping_vus.go | 65 +++++++--------------------- 1 file changed, 16 insertions(+), 49 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 23ddc135ac1..2b14e19c9b6 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -23,7 +23,6 @@ package executor import ( "context" "fmt" - "math" "sync" "sync/atomic" "time" @@ -188,13 +187,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // the values are scaled only before we add them to the steps result slice fromVUs := vlvc.StartVUs.Int64 - abs := func(n int64) int64 { // sigh... - if n < 0 { - return -n - } - return n - } - // Reserve the scaled StartVUs at the beginning prevScaledVUs := et.ScaleInt64(vlvc.StartVUs.Int64) steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} @@ -213,47 +205,22 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple stageDuration := time.Duration(stage.Duration.Duration) totalDuration += stageDuration - stageVUAbsDiff := abs(stageEndVUs - fromVUs) - if stageVUAbsDiff == 0 { - // We don't have to do anything but update the time offset - // if the number of VUs wasn't changed in this stage - timeFromStart += stageDuration - continue - } - - // Handle 0-duration stages, i.e. instant VU jumps - if stageDuration == 0 { - fromVUs = stageEndVUs - addStep(lib.ExecutionStep{ - TimeOffset: timeFromStart, - PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), - }) - continue - } - - // No floats or ratios,since nanoseconds should be good enough for anyone... :) - stepInterval := stageDuration / time.Duration(stageVUAbsDiff) - - // Loop through the potential steps, adding an item to the - // result only when there's a change in the number of VUs. - // - // IMPORTANT: we have to be very careful of rounding errors, - // both from the step duration and from the VUs. It's especially - // important that the scaling via the execution segment should - // happen AFTER the rest of the calculations have been done and - // we've rounded the global "global" number of VUs. - for t := stepInterval; t < stageDuration; t += stepInterval { // Skip the first step, since we've already added that - stepGlobalVUs := fromVUs + int64( - math.Round((float64(t)*float64(stageEndVUs-fromVUs))/float64(stageDuration)), - ) - - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() - - addStep(lib.ExecutionStep{ - TimeOffset: timeFromStart + t, - PlannedVUs: uint64(et.ScaleInt64(stepGlobalVUs)), - }) + stageVUDiff := stageEndVUs - fromVUs + if stageDuration != 0 && stageVUDiff != 0 { + var sign int64 = 1 + if stageVUDiff < 0 { + sign = -1 + } + // Loop through the potential steps, adding an item to the + // result only when there's a change in the number of VUs. + for i := sign; i != stageVUDiff; i += sign { // Skip the first step, since we've already added that + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + addStep(lib.ExecutionStep{ + TimeOffset: timeFromStart + (stageDuration*time.Duration(i))/time.Duration(stageVUDiff), + PlannedVUs: uint64(et.ScaleInt64(fromVUs + i)), + }) + } } fromVUs = stageEndVUs From 7402ee048747c730d9e159468e2b84ed5bf5c6f2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 12:51:39 +0300 Subject: [PATCH 204/350] drop unneded variable --- lib/executor/variable_looping_vus.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 2b14e19c9b6..448b0680a72 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -191,7 +191,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple prevScaledVUs := et.ScaleInt64(vlvc.StartVUs.Int64) steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} timeFromStart := time.Duration(0) - totalDuration := time.Duration(0) addStep := func(step lib.ExecutionStep) { if len(steps) == 0 || steps[len(steps)-1].PlannedVUs != step.PlannedVUs { @@ -203,7 +202,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple for _, stage := range vlvc.Stages { stageEndVUs := stage.Target.Int64 stageDuration := time.Duration(stage.Duration.Duration) - totalDuration += stageDuration stageVUDiff := stageEndVUs - fromVUs if stageDuration != 0 && stageVUDiff != 0 { @@ -234,7 +232,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple if zeroEnd && steps[len(steps)-1].PlannedVUs != 0 { // If the last PlannedVUs value wasn't 0, add a last step with 0 - steps = append(steps, lib.ExecutionStep{TimeOffset: totalDuration, PlannedVUs: 0}) + steps = append(steps, lib.ExecutionStep{TimeOffset: timeFromStart, PlannedVUs: 0}) } return steps } From e4607a1698ee8b6d034a3470ba8c54f1573bc93b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 10 Apr 2020 13:31:10 +0300 Subject: [PATCH 205/350] Add one test with executionTuples for looping vus --- lib/executor/variable_looping_vus_test.go | 173 ++++++++++++++++++++++ 1 file changed, 173 insertions(+) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index ba010f7c7cc..52fb7974f85 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -344,3 +344,176 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { conf.GracefulRampDown = types.NullDurationFrom(0 * time.Second) assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) } + +func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) { + t.Parallel() + + conf := NewVariableLoopingVUsConfig("test") + conf.StartVUs = null.IntFrom(4) + conf.Stages = []Stage{ + {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(4 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, + } + /* + + Graph of the above: + ^ + 8 | + 7 | + 6 | + + 5 |/ \ + + 4 + \ / \ +-+ + 3 | \ / \ / | + 2 | \ / \ / | + 1 | + + +--+ + 0 +-------------------------------------------------------------> + 0123456789012345678901234567890 + + */ + + testCases := []struct { + expectedSteps []lib.ExecutionStep + et *lib.ExecutionTuple + }{ + { + et: mustNewExecutionTuple(nil, nil), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 5}, + {TimeOffset: 2 * time.Second, PlannedVUs: 6}, + {TimeOffset: 3 * time.Second, PlannedVUs: 5}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 3}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 8 * time.Second, PlannedVUs: 2}, + {TimeOffset: 9 * time.Second, PlannedVUs: 3}, + {TimeOffset: 10 * time.Second, PlannedVUs: 4}, + {TimeOffset: 11 * time.Second, PlannedVUs: 5}, + {TimeOffset: 12 * time.Second, PlannedVUs: 4}, + {TimeOffset: 13 * time.Second, PlannedVUs: 3}, + {TimeOffset: 14 * time.Second, PlannedVUs: 2}, + {TimeOffset: 15 * time.Second, PlannedVUs: 1}, + {TimeOffset: 16 * time.Second, PlannedVUs: 2}, + {TimeOffset: 17 * time.Second, PlannedVUs: 3}, + {TimeOffset: 18 * time.Second, PlannedVUs: 4}, + {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), nil), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1")), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("1/3:2/3"), nil), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), nil), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 2}, + {TimeOffset: 5 * time.Second, PlannedVUs: 1}, + {TimeOffset: 10 * time.Second, PlannedVUs: 2}, + {TimeOffset: 13 * time.Second, PlannedVUs: 1}, + {TimeOffset: 18 * time.Second, PlannedVUs: 2}, + {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("1/3:2/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 1 * time.Second, PlannedVUs: 2}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 2}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 16 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + { + et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 2}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 9 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 17 * time.Second, PlannedVUs: 1}, + {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + }, + }, + } + + for _, testCase := range testCases { + et := testCase.et + expectedSteps := testCase.expectedSteps + + t.Run(et.String(), func(t *testing.T) { + rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) + assert.Equal(t, expectedSteps, rawStepsNoZeroEnd) + }) + } +} From 42f04759620f325fb995f6ae0676de13c9cc11c7 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 13 Apr 2020 16:19:18 +0300 Subject: [PATCH 206/350] delete preveScaledVUs and 'optimize' addStep ;) --- lib/executor/variable_looping_vus.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 448b0680a72..b54e9f2b892 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -188,14 +188,12 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple fromVUs := vlvc.StartVUs.Int64 // Reserve the scaled StartVUs at the beginning - prevScaledVUs := et.ScaleInt64(vlvc.StartVUs.Int64) - steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(prevScaledVUs)}} + steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(et.ScaleInt64(vlvc.StartVUs.Int64))}} timeFromStart := time.Duration(0) addStep := func(step lib.ExecutionStep) { - if len(steps) == 0 || steps[len(steps)-1].PlannedVUs != step.PlannedVUs { + if steps[len(steps)-1].PlannedVUs != step.PlannedVUs { steps = append(steps, step) - prevScaledVUs = int64(step.PlannedVUs) } } From 71c088f5eec35727540430ddda9965b1ef778d44 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 14 Apr 2020 23:41:28 +0300 Subject: [PATCH 207/350] add benchmark MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name time/op VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 461µs ± 1% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 5.31ms ± 2% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 463µs ± 2% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 5.28ms ± 1% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 311µs ± 2% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.22ms ± 2% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 209µs ± 4% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 2.82ms ± 3% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 354µs ± 3% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 4.70ms ± 4% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 9.77ms ± 3% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 60.4ms ± 3% name alloc/op VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 1.07MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 14.6MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 1.07MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 14.6MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 254kB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.62MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 49.1kB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 1.38MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 352kB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 5.83MB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 352kB ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 5.83MB ± 0% name allocs/op VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 20.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 31.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 20.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 31.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 15.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 26.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 11.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 21.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 16.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 27.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 16.0 ± 0% VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 27.0 ± 0% --- lib/executor/variable_looping_vus_test.go | 65 +++++++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 52fb7974f85..cea9b7a28a4 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -22,6 +22,8 @@ package executor import ( "context" + "encoding/json" + "fmt" "sync/atomic" "testing" "time" @@ -517,3 +519,66 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) }) } } + +func BenchmarkVarriableArrivalRateGetRawExecutionSteps(b *testing.B) { + testCases := []struct { + seq string + seg string + }{ + {}, + {seg: "0:1"}, + {seq: "0,0.3,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.3"}, + {seq: "0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1", seg: "0:0.1"}, + {seg: "2/5:4/5"}, + {seg: "2235/5213:4/5"}, // just wanted it to be ugly ;D + } + + stageCases := []struct { + name string + stages string + }{ + { + name: "normal", + stages: `[{"duration":"5m", "target":5000},{"duration":"5m", "target":5000},{"duration":"5m", "target":10000},{"duration":"5m", "target":10000}]`, + }, { + name: "rollercoaster", + stages: `[{"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}, + {"duration":"5m", "target":5000},{"duration":"5m", "target":0}]`, + }, + } + for _, tc := range testCases { + tc := tc + b.Run(fmt.Sprintf("seq:%s;segment:%s", tc.seq, tc.seg), func(b *testing.B) { + ess, err := lib.NewExecutionSegmentSequenceFromString(tc.seq) + require.NoError(b, err) + segment, err := lib.NewExecutionSegmentFromString(tc.seg) + require.NoError(b, err) + if tc.seg == "" { + segment = nil // specifically for the optimization + } + et, err := lib.NewExecutionTuple(segment, &ess) + require.NoError(b, err) + for _, stageCase := range stageCases { + var st []Stage + require.NoError(b, json.Unmarshal([]byte(stageCase.stages), &st)) + vlvc := VariableLoopingVUsConfig{ + Stages: st, + } + b.Run(stageCase.name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = vlvc.getRawExecutionSteps(et, false) + } + }) + } + }) + } +} From 96e503fe610244c21d23acc9fe62adf30a1c9725 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 14 Apr 2020 22:37:22 +0300 Subject: [PATCH 208/350] Rewrite the varriable looping vus to fix a bug and speed it up MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously the variable arrival rate instead of stopping VUs right away after they need to stopped them on the next step down. This is also not how the previous stages worked and is not how it was designed to work in the first place so it needed to be fixed. The speed up comes from the fact that we now skip some calculations if the test is not ran with a full execution segment. Unfortunately this is slower for the full execution segment as there we still do all the calculations but now we additionally do more stuff, maybe some more optimizations can be done name old time/op new time/op delta VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 461µs ± 1% 571µs ± 3% +23.70% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 5.31ms ± 2% 6.19ms ± 2% +16.69% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 463µs ± 2% 569µs ± 2% +22.93% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 5.28ms ± 1% 6.21ms ± 2% +17.71% (p=0.000 n=19+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 311µs ± 2% 163µs ± 2% -47.38% (p=0.000 n=19+19) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.22ms ± 2% 2.10ms ± 2% -50.13% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 209µs ± 4% 44µs ± 3% -78.90% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 2.82ms ± 3% 0.68ms ± 2% -75.98% (p=0.000 n=20+19) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 354µs ± 3% 217µs ± 2% -38.76% (p=0.000 n=19+18) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 4.70ms ± 4% 2.71ms ± 2% -42.24% (p=0.000 n=20+19) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 9.77ms ± 3% 0.21ms ± 2% -97.90% (p=0.000 n=19+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 60.4ms ± 3% 2.5ms ± 2% -95.86% (p=0.000 n=20+20) name old alloc/op new alloc/op delta VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 1.07MB ± 0% 1.07MB ± 0% ~ (all equal) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 14.6MB ± 0% 14.6MB ± 0% ~ (p=0.121 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 1.07MB ± 0% 1.07MB ± 0% +0.01% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 14.6MB ± 0% 14.6MB ± 0% +0.00% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 254kB ± 0% 254kB ± 0% +0.04% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.62MB ± 0% 4.62MB ± 0% +0.00% (p=0.000 n=20+19) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 49.1kB ± 0% 49.3kB ± 0% +0.24% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 1.38MB ± 0% 1.38MB ± 0% +0.01% (p=0.000 n=19+19) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 352kB ± 0% 352kB ± 0% +0.04% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 5.83MB ± 0% 5.83MB ± 0% +0.00% (p=0.000 n=19+17) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 352kB ± 0% 352kB ± 0% +0.04% (p=0.000 n=16+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 5.83MB ± 0% 5.83MB ± 0% +0.00% (p=0.000 n=20+19) name old allocs/op new allocs/op delta VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/normal-8 20.0 ± 0% 20.0 ± 0% ~ (all equal) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:/rollercoaster-8 31.0 ± 0% 31.0 ± 0% ~ (all equal) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/normal-8 20.0 ± 0% 22.0 ± 0% +10.00% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 31.0 ± 0% 33.0 ± 0% +6.45% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 15.0 ± 0% 19.0 ± 0% +26.67% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 26.0 ± 0% 30.0 ± 0% +15.38% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 11.0 ± 0% 16.0 ± 0% +45.45% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 21.0 ± 0% 26.0 ± 0% +23.81% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 16.0 ± 0% 19.0 ± 0% +18.75% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 27.0 ± 0% 30.0 ± 0% +11.11% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 16.0 ± 0% 19.0 ± 0% +18.75% (p=0.000 n=20+20) VarriableArrivalRateGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 27.0 ± 0% 30.0 ± 0% +11.11% (p=0.000 n=20+20) --- lib/executor/variable_looping_vus.go | 76 +++++++--- lib/executor/variable_looping_vus_test.go | 172 ++++++++++++++-------- 2 files changed, 165 insertions(+), 83 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b54e9f2b892..dfbb30935a1 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -189,7 +189,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // Reserve the scaled StartVUs at the beginning steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(et.ScaleInt64(vlvc.StartVUs.Int64))}} - timeFromStart := time.Duration(0) + var timeTillEnd time.Duration addStep := func(step lib.ExecutionStep) { if steps[len(steps)-1].PlannedVUs != step.PlannedVUs { @@ -197,40 +197,72 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } } + start, offsets, _ := et.GetStripedOffsets(et.ES) + var localIndex int64 // this is the index of the vu for this execution segment + next := func(sign int64) int64 { + r := offsets[int(localIndex)%len(offsets)] + localIndex += sign + return r + } + i := start + 1 // this is the index for the full execution segment for _, stage := range vlvc.Stages { stageEndVUs := stage.Target.Int64 stageDuration := time.Duration(stage.Duration.Duration) + timeTillEnd += stageDuration stageVUDiff := stageEndVUs - fromVUs - if stageDuration != 0 && stageVUDiff != 0 { - var sign int64 = 1 - if stageVUDiff < 0 { - sign = -1 + switch { + case stageDuration == 0: + addStep(lib.ExecutionStep{ + TimeOffset: timeTillEnd, + PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), + }) + case stageVUDiff != 0: + // Get the index to the start if they are not there + if i > fromVUs { + for ; i > fromVUs; i -= next(-1) { + if localIndex == 0 { // we want ot enter for this index but not actually go below 0 + break + } + } + } else { + for ; i < fromVUs; i += next(1) { // <= test + } } - // Loop through the potential steps, adding an item to the - // result only when there's a change in the number of VUs. - for i := sign; i != stageVUDiff; i += sign { // Skip the first step, since we've already added that - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() - addStep(lib.ExecutionStep{ - TimeOffset: timeFromStart + (stageDuration*time.Duration(i))/time.Duration(stageVUDiff), - PlannedVUs: uint64(et.ScaleInt64(fromVUs + i)), - }) + + if i > stageEndVUs { // ramp down + // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it + // it will just go to it + for ; i > stageEndVUs; i -= next(-1) { + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + addStep(lib.ExecutionStep{ + TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), + PlannedVUs: uint64(localIndex), + }) + if localIndex == 0 { // we want ot enter for this index but not actually go below 0 + break + } + } + } else { + // here we want the emit for the last one as this case it actually should emit that + // we start it + for ; i <= stageEndVUs; i += next(1) { + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + addStep(lib.ExecutionStep{ + TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), + PlannedVUs: uint64(localIndex + 1), + }) + } } } - fromVUs = stageEndVUs - timeFromStart += stageDuration - - addStep(lib.ExecutionStep{ - TimeOffset: timeFromStart, - PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), - }) } if zeroEnd && steps[len(steps)-1].PlannedVUs != 0 { // If the last PlannedVUs value wasn't 0, add a last step with 0 - steps = append(steps, lib.ExecutionStep{TimeOffset: timeFromStart, PlannedVUs: 0}) + steps = append(steps, lib.ExecutionStep{TimeOffset: timeTillEnd, PlannedVUs: 0}) } return steps } diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index cea9b7a28a4..4d90128543e 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -189,19 +189,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 3 * time.Second, PlannedVUs: 5}, - {TimeOffset: 4 * time.Second, PlannedVUs: 4}, - {TimeOffset: 5 * time.Second, PlannedVUs: 3}, - {TimeOffset: 6 * time.Second, PlannedVUs: 2}, - {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 5}, + {TimeOffset: 3 * time.Second, PlannedVUs: 4}, + {TimeOffset: 4 * time.Second, PlannedVUs: 3}, + {TimeOffset: 5 * time.Second, PlannedVUs: 2}, + {TimeOffset: 6 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 2}, {TimeOffset: 9 * time.Second, PlannedVUs: 3}, {TimeOffset: 10 * time.Second, PlannedVUs: 4}, {TimeOffset: 11 * time.Second, PlannedVUs: 5}, - {TimeOffset: 12 * time.Second, PlannedVUs: 4}, - {TimeOffset: 13 * time.Second, PlannedVUs: 3}, - {TimeOffset: 14 * time.Second, PlannedVUs: 2}, - {TimeOffset: 15 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 4}, + {TimeOffset: 12 * time.Second, PlannedVUs: 3}, + {TimeOffset: 13 * time.Second, PlannedVUs: 2}, + {TimeOffset: 14 * time.Second, PlannedVUs: 1}, {TimeOffset: 16 * time.Second, PlannedVUs: 2}, {TimeOffset: 17 * time.Second, PlannedVUs: 3}, {TimeOffset: 18 * time.Second, PlannedVUs: 4}, @@ -227,8 +227,8 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 33 * time.Second, PlannedVUs: 5}, - {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 32 * time.Second, PlannedVUs: 5}, + {TimeOffset: 41 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 53 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -239,8 +239,8 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 33 * time.Second, PlannedVUs: 5}, - {TimeOffset: 42 * time.Second, PlannedVUs: 4}, + {TimeOffset: 32 * time.Second, PlannedVUs: 5}, + {TimeOffset: 41 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 103 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -288,12 +288,12 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { expRawStepsNoZeroEnd := []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, } @@ -313,7 +313,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 42 * time.Second, PlannedVUs: 1}, + {TimeOffset: 41 * time.Second, PlannedVUs: 1}, {TimeOffset: 50 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -322,7 +322,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 42 * time.Second, PlannedVUs: 1}, + {TimeOffset: 41 * time.Second, PlannedVUs: 1}, {TimeOffset: 50 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -347,7 +347,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) } -func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) { +func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { t.Parallel() conf := NewVariableLoopingVUsConfig("test") @@ -361,6 +361,12 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) {Target: null.IntFrom(4), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(0 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(3 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(4), Duration: types.NullDurationFrom(4 * time.Second)}, } /* @@ -369,13 +375,13 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) 8 | 7 | 6 | + - 5 |/ \ + - 4 + \ / \ +-+ - 3 | \ / \ / | - 2 | \ / \ / | - 1 | + + +--+ - 0 +-------------------------------------------------------------> - 0123456789012345678901234567890 + 5 |/ \ + +--+ + 4 + \ / \ +-+ | | * + 3 | \ / \ / | | | / + 2 | \ / \ / | | | + / + 1 | + + +--+ |/ \ / + 0 +-------------------------+---+------------------------------> + 01234567890123456789012345678901234567890 */ @@ -389,23 +395,33 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 3 * time.Second, PlannedVUs: 5}, - {TimeOffset: 4 * time.Second, PlannedVUs: 4}, - {TimeOffset: 5 * time.Second, PlannedVUs: 3}, - {TimeOffset: 6 * time.Second, PlannedVUs: 2}, - {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 5}, + {TimeOffset: 3 * time.Second, PlannedVUs: 4}, + {TimeOffset: 4 * time.Second, PlannedVUs: 3}, + {TimeOffset: 5 * time.Second, PlannedVUs: 2}, + {TimeOffset: 6 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 2}, {TimeOffset: 9 * time.Second, PlannedVUs: 3}, {TimeOffset: 10 * time.Second, PlannedVUs: 4}, {TimeOffset: 11 * time.Second, PlannedVUs: 5}, - {TimeOffset: 12 * time.Second, PlannedVUs: 4}, - {TimeOffset: 13 * time.Second, PlannedVUs: 3}, - {TimeOffset: 14 * time.Second, PlannedVUs: 2}, - {TimeOffset: 15 * time.Second, PlannedVUs: 1}, + {TimeOffset: 11 * time.Second, PlannedVUs: 4}, + {TimeOffset: 12 * time.Second, PlannedVUs: 3}, + {TimeOffset: 13 * time.Second, PlannedVUs: 2}, + {TimeOffset: 14 * time.Second, PlannedVUs: 1}, {TimeOffset: 16 * time.Second, PlannedVUs: 2}, {TimeOffset: 17 * time.Second, PlannedVUs: 3}, {TimeOffset: 18 * time.Second, PlannedVUs: 4}, {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + {TimeOffset: 23 * time.Second, PlannedVUs: 5}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 27 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 2}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, + {TimeOffset: 31 * time.Second, PlannedVUs: 1}, + {TimeOffset: 32 * time.Second, PlannedVUs: 2}, + {TimeOffset: 33 * time.Second, PlannedVUs: 3}, + {TimeOffset: 34 * time.Second, PlannedVUs: 4}, }, }, { @@ -413,14 +429,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, { @@ -428,14 +449,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, { @@ -443,14 +469,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, { @@ -458,25 +489,36 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, { et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 2}, - {TimeOffset: 5 * time.Second, PlannedVUs: 1}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, {TimeOffset: 10 * time.Second, PlannedVUs: 2}, - {TimeOffset: 13 * time.Second, PlannedVUs: 1}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, {TimeOffset: 18 * time.Second, PlannedVUs: 2}, {TimeOffset: 20 * time.Second, PlannedVUs: 1}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 27 * time.Second, PlannedVUs: 1}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, + {TimeOffset: 31 * time.Second, PlannedVUs: 1}, + {TimeOffset: 34 * time.Second, PlannedVUs: 2}, }, }, { @@ -484,14 +526,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, - {TimeOffset: 15 * time.Second, PlannedVUs: 0}, + {TimeOffset: 11 * time.Second, PlannedVUs: 1}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 2}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 28 * time.Second, PlannedVUs: 1}, + {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, { @@ -499,12 +546,15 @@ func TestVariableLoopingVUsConfigExecutionPlanExecutionTupleTests(t *testing.T) expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 2 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 2 * time.Second, PlannedVUs: 1}, + {TimeOffset: 5 * time.Second, PlannedVUs: 0}, {TimeOffset: 9 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 13 * time.Second, PlannedVUs: 0}, {TimeOffset: 17 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, + {TimeOffset: 23 * time.Second, PlannedVUs: 1}, + {TimeOffset: 26 * time.Second, PlannedVUs: 0}, + {TimeOffset: 33 * time.Second, PlannedVUs: 1}, }, }, } From 89d5b736e7d4b5b9820ea465bc0bd32f0cbdb9a4 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 15 Apr 2020 17:48:18 +0300 Subject: [PATCH 209/350] Rename variable looping vus benchmark --- lib/executor/variable_looping_vus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 4d90128543e..86008c9b8ea 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -570,7 +570,7 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { } } -func BenchmarkVarriableArrivalRateGetRawExecutionSteps(b *testing.B) { +func BenchmarkVarriableLoopingVUsGetRawExecutionSteps(b *testing.B) { testCases := []struct { seq string seg string From 1bce1febff8d83be7fb90f99a1a2eaa83377fa75 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 15 Apr 2020 19:01:07 +0300 Subject: [PATCH 210/350] Add a bunch of corner case tests for variable looping vus --- lib/executor/variable_looping_vus_test.go | 140 ++++++++++++++++++++++ 1 file changed, 140 insertions(+) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 86008c9b8ea..1e25f68a75b 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -570,6 +570,146 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { } } +func TestVarriableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { + t.Parallel() + + testCases := []struct { + name string + expectedSteps []lib.ExecutionStep + et *lib.ExecutionTuple + stages []Stage + start int64 + }{ + { + name: "going up then down straight away", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 2}, + {TimeOffset: 0 * time.Second, PlannedVUs: 5}, + {TimeOffset: 0 * time.Second, PlannedVUs: 4}, + {TimeOffset: 1 * time.Second, PlannedVUs: 3}, + }, + stages: []Stage{ + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(0 * time.Second)}, + {Target: null.IntFrom(3), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + start: 2, + }, + { + name: "jump up then go up again", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 3}, + {TimeOffset: 1 * time.Second, PlannedVUs: 4}, + {TimeOffset: 2 * time.Second, PlannedVUs: 5}, + }, + stages: []Stage{ + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + start: 3, + }, + { + name: "up down up down", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 1 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 2}, + {TimeOffset: 2 * time.Second, PlannedVUs: 1}, + {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 5 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 6 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + }, + stages: []Stage{ + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + }, + { + name: "up down up down in half", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 1 * time.Second, PlannedVUs: 1}, + {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 5 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + }, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/2"), nil), + stages: []Stage{ + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + }, + { + name: "up down up down in the other half", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 2 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 0}, + {TimeOffset: 6 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + }, + et: mustNewExecutionTuple(newExecutionSegmentFromString("1/2:1"), nil), + stages: []Stage{ + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + }, + { + name: "up down up down in with nothing", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + }, + et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), + stages: []Stage{ + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + }, + { + name: "up down up down in with funky sequence", // panics if there are no localIndex == 0 guards + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 1 * time.Second, PlannedVUs: 1}, + {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 5 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + }, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1/2,2/3,1")), + stages: []Stage{ + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, + }, + }, + } + + for _, testCase := range testCases { + conf := NewVariableLoopingVUsConfig("test") + conf.StartVUs = null.IntFrom(testCase.start) + conf.Stages = testCase.stages + et := testCase.et + if et == nil { + et = mustNewExecutionTuple(nil, nil) + } + expectedSteps := testCase.expectedSteps + + t.Run(testCase.name, func(t *testing.T) { + rawStepsNoZeroEnd := conf.getRawExecutionSteps(et, false) + assert.Equal(t, expectedSteps, rawStepsNoZeroEnd) + }) + } + +} + func BenchmarkVarriableLoopingVUsGetRawExecutionSteps(b *testing.B) { testCases := []struct { seq string From b75355c31cb28a8055facd80a2adc2b4b195360f Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 15 Apr 2020 20:46:01 +0300 Subject: [PATCH 211/350] Varriable->Variable --- lib/executor/variable_looping_vus_test.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 1e25f68a75b..42b741ee053 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -570,7 +570,7 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { } } -func TestVarriableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { +func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { t.Parallel() testCases := []struct { @@ -707,10 +707,9 @@ func TestVarriableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { assert.Equal(t, expectedSteps, rawStepsNoZeroEnd) }) } - } -func BenchmarkVarriableLoopingVUsGetRawExecutionSteps(b *testing.B) { +func BenchmarkVariableLoopingVUsGetRawExecutionSteps(b *testing.B) { testCases := []struct { seq string seg string From 8faff961134636895586aa436f69e8dd7b5cf9e8 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 16 Apr 2020 11:37:24 +0300 Subject: [PATCH 212/350] Fix going back if the offsets are different --- lib/executor/variable_looping_vus.go | 8 ++++++-- lib/executor/variable_looping_vus_test.go | 20 ++++++++++++++++++++ 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index dfbb30935a1..cea07156057 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -199,8 +199,12 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple start, offsets, _ := et.GetStripedOffsets(et.ES) var localIndex int64 // this is the index of the vu for this execution segment - next := func(sign int64) int64 { - r := offsets[int(localIndex)%len(offsets)] + next := func(sign int64) (r int64) { + if sign == 1 { + r = offsets[int(localIndex)%len(offsets)] + } else { + r = offsets[int(localIndex-1)%len(offsets)] + } localIndex += sign return r } diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 42b741ee053..ba49a768863 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -690,6 +690,26 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, }, }, + { + name: "strange", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 1 * time.Second, PlannedVUs: 1}, + {TimeOffset: 5 * time.Second, PlannedVUs: 2}, + {TimeOffset: 8 * time.Second, PlannedVUs: 3}, + {TimeOffset: 11 * time.Second, PlannedVUs: 4}, + {TimeOffset: 15 * time.Second, PlannedVUs: 5}, + {TimeOffset: 18 * time.Second, PlannedVUs: 6}, + {TimeOffset: 23 * time.Second, PlannedVUs: 7}, + {TimeOffset: 35 * time.Second, PlannedVUs: 8}, + {TimeOffset: 44 * time.Second, PlannedVUs: 9}, + }, + et: mustNewExecutionTuple(newExecutionSegmentFromString("0:0.3"), newExecutionSegmentSequenceFromString("0,0.3,0.6,0.9,1")), + stages: []Stage{ + {Target: null.IntFrom(20), Duration: types.NullDurationFrom(20 * time.Second)}, + {Target: null.IntFrom(30), Duration: types.NullDurationFrom(30 * time.Second)}, + }, + }, } for _, testCase := range testCases { From eea43fc373b8b0a4659f9507c960a9463d5f1c96 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 16 Apr 2020 11:40:22 +0300 Subject: [PATCH 213/350] don't use switch use ifs --- lib/executor/variable_looping_vus.go | 75 +++++++++++++++------------- 1 file changed, 39 insertions(+), 36 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index cea07156057..8620586abff 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -215,51 +215,54 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple timeTillEnd += stageDuration stageVUDiff := stageEndVUs - fromVUs - switch { - case stageDuration == 0: + if stageVUDiff == 0 { + continue + } + if stageDuration == 0 { addStep(lib.ExecutionStep{ TimeOffset: timeTillEnd, PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), }) - case stageVUDiff != 0: - // Get the index to the start if they are not there - if i > fromVUs { - for ; i > fromVUs; i -= next(-1) { - if localIndex == 0 { // we want ot enter for this index but not actually go below 0 - break - } - } - } else { - for ; i < fromVUs; i += next(1) { // <= test + fromVUs = stageEndVUs + continue + } + // Get the index to the start if they are not there + if i > fromVUs { + for ; i > fromVUs; i -= next(-1) { + if localIndex == 0 { // we want ot enter for this index but not actually go below 0 + break } } + } else { + for ; i < fromVUs; i += next(1) { // <= test + } + } - if i > stageEndVUs { // ramp down - // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it - // it will just go to it - for ; i > stageEndVUs; i -= next(-1) { - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() - addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), - PlannedVUs: uint64(localIndex), - }) - if localIndex == 0 { // we want ot enter for this index but not actually go below 0 - break - } - } - } else { - // here we want the emit for the last one as this case it actually should emit that - // we start it - for ; i <= stageEndVUs; i += next(1) { - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() - addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), - PlannedVUs: uint64(localIndex + 1), - }) + if i > stageEndVUs { // ramp down + // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it + // it will just go to it + for ; i > stageEndVUs; i -= next(-1) { + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + addStep(lib.ExecutionStep{ + TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), + PlannedVUs: uint64(localIndex), + }) + if localIndex == 0 { // we want ot enter for this index but not actually go below 0 + break } } + } else { + // here we want the emit for the last one as this case it actually should emit that + // we start it + for ; i <= stageEndVUs; i += next(1) { + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() + addStep(lib.ExecutionStep{ + TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), + PlannedVUs: uint64(localIndex + 1), + }) + } } fromVUs = stageEndVUs } From 35d4a5e1f6c34f9b3ea6d3f9fbb5e06873c49d19 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 23 Apr 2020 12:33:47 +0300 Subject: [PATCH 214/350] Refactor getRawExecutionSteps to be more logical MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This has no (or questionable) performance difference for the benchmarks name old time/op new time/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 560µs ± 2% 560µs ± 3% ~ (p=0.563 n=19+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 6.02ms ± 2% 5.97ms ± 2% -0.82% (p=0.017 n=20+18) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 562µs ± 2% 561µs ± 2% ~ (p=0.647 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 5.93ms ± 3% 5.94ms ± 3% ~ (p=0.904 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 162µs ± 2% 161µs ± 2% ~ (p=0.247 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 2.07ms ± 1% 2.07ms ± 1% ~ (p=0.678 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 43.2µs ± 4% 43.5µs ± 2% ~ (p=0.108 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 655µs ± 1% 654µs ± 2% ~ (p=0.613 n=18+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 214µs ± 2% 215µs ± 2% ~ (p=0.383 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 2.63ms ± 2% 2.64ms ± 2% ~ (p=0.547 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 202µs ± 1% 202µs ± 2% ~ (p=0.678 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 2.42ms ± 2% 2.43ms ± 2% ~ (p=0.565 n=20+20) name old alloc/op new alloc/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 1.07MB ± 0% 1.07MB ± 0% -0.00% (p=0.037 n=16+17) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 14.6MB ± 0% 14.6MB ± 0% ~ (p=0.983 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 1.07MB ± 0% 1.07MB ± 0% ~ (p=0.096 n=20+15) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 14.6MB ± 0% 14.6MB ± 0% ~ (p=0.495 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 254kB ± 0% 254kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.62MB ± 0% 4.62MB ± 0% ~ (p=0.796 n=20+18) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 49.3kB ± 0% 49.3kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 1.38MB ± 0% 1.38MB ± 0% ~ (p=0.566 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 352kB ± 0% 352kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 5.83MB ± 0% 5.83MB ± 0% ~ (p=0.772 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 352kB ± 0% 352kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 5.83MB ± 0% 5.83MB ± 0% +0.00% (p=0.013 n=20+19) name old allocs/op new allocs/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 20.0 ± 0% 20.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 31.0 ± 0% 31.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 22.0 ± 0% 22.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 33.0 ± 0% 33.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 30.0 ± 0% 30.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 16.0 ± 0% 16.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 26.0 ± 0% 26.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 30.0 ± 0% 30.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 19.0 ± 0% 19.0 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 30.0 ± 0% 30.0 ± 0% ~ (all equal) --- lib/executor/variable_looping_vus.go | 102 +++++++----- lib/executor/variable_looping_vus_test.go | 193 +++++++++++++++++++++- 2 files changed, 256 insertions(+), 39 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 8620586abff..2bd36fdf292 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -187,8 +187,12 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // the values are scaled only before we add them to the steps result slice fromVUs := vlvc.StartVUs.Int64 + start, offsets, lcd := et.GetStripedOffsets(et.ES) + var index = segmentedIndex{start: start, lcd: lcd, offsets: offsets} + index.goTo(vlvc.StartVUs.Int64) // Reserve the scaled StartVUs at the beginning - steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(et.ScaleInt64(vlvc.StartVUs.Int64))}} + steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(index.local)}} + var timeTillEnd time.Duration addStep := func(step lib.ExecutionStep) { @@ -197,18 +201,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } } - start, offsets, _ := et.GetStripedOffsets(et.ES) - var localIndex int64 // this is the index of the vu for this execution segment - next := func(sign int64) (r int64) { - if sign == 1 { - r = offsets[int(localIndex)%len(offsets)] - } else { - r = offsets[int(localIndex-1)%len(offsets)] - } - localIndex += sign - return r - } - i := start + 1 // this is the index for the full execution segment for _, stage := range vlvc.Stages { stageEndVUs := stage.Target.Int64 stageDuration := time.Duration(stage.Duration.Duration) @@ -219,48 +211,38 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple continue } if stageDuration == 0 { - addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd, - PlannedVUs: uint64(et.ScaleInt64(stageEndVUs)), - }) + index.goTo(stageEndVUs) + addStep(lib.ExecutionStep{TimeOffset: timeTillEnd, PlannedVUs: uint64(index.local)}) fromVUs = stageEndVUs continue } - // Get the index to the start if they are not there - if i > fromVUs { - for ; i > fromVUs; i -= next(-1) { - if localIndex == 0 { // we want ot enter for this index but not actually go below 0 - break - } - } - } else { - for ; i < fromVUs; i += next(1) { // <= test - } - } - if i > stageEndVUs { // ramp down + if index.global > stageEndVUs { // ramp down // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it // it will just go to it - for ; i > stageEndVUs; i -= next(-1) { + for ; index.global > stageEndVUs; index.prev() { + if index.global > fromVUs { + continue + } // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), - PlannedVUs: uint64(localIndex), + TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global)/stageVUDiff), + PlannedVUs: uint64(index.local - 1), }) - if localIndex == 0 { // we want ot enter for this index but not actually go below 0 - break - } } } else { // here we want the emit for the last one as this case it actually should emit that // we start it - for ; i <= stageEndVUs; i += next(1) { + for ; index.global <= stageEndVUs; index.next() { + if index.global < fromVUs { + continue + } // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - (stageDuration*time.Duration((stageEndVUs-i)))/time.Duration(stageVUDiff), - PlannedVUs: uint64(localIndex + 1), + TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global)/stageVUDiff), + PlannedVUs: uint64(index.local), }) } } @@ -274,6 +256,50 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple return steps } +type segmentedIndex struct { // TODO: rename ... although this is probably the best name so far :D + start, lcd int64 + offsets []int64 + local, global int64 +} + +func (s *segmentedIndex) next() { + if s.local == 0 { + s.global += s.start + 1 + } else { + s.global += s.offsets[int(s.local-1)%len(s.offsets)] + } + s.local++ +} + +func (s *segmentedIndex) prev() { + if s.local == 1 { + s.global -= s.start + 1 + } else { + s.global -= s.offsets[int(s.local-2)%len(s.offsets)] + } + s.local-- +} + +func (s *segmentedIndex) goTo(value int64) { // TODO optimize + var gi int64 + s.local = (value / s.lcd) * int64(len(s.offsets)) + s.global = s.local / int64(len(s.offsets)) * s.lcd // TODO optimize ? + i := s.start + for ; i < value%s.lcd; gi, i = gi+1, i+s.offsets[gi] { + s.local++ + } + + if gi > 0 { + s.global += i - s.offsets[gi-1] + } else if s.local > 0 { + s.global -= s.offsets[len(s.offsets)-1] - s.start + } + + if s.local > 0 { + s.global++ // this is to fix the fact it starts from 0 + } +} + // If the graceful ramp-downs are enabled, we need to reserve any VUs that may // potentially have to finish running iterations when we're scaling their number // down. This would prevent attempts from other executors to use them while the diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index ba49a768863..5c8856b1cd1 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -79,7 +79,7 @@ func TestVariableLoopingVUsRun(t *testing.T) { sampleTimes := []time.Duration{ 500 * time.Millisecond, 1000 * time.Millisecond, - 700 * time.Millisecond, + 800 * time.Millisecond, } errCh := make(chan error) @@ -791,3 +791,194 @@ func BenchmarkVariableLoopingVUsGetRawExecutionSteps(b *testing.B) { }) } } + +func TestSegmentedIndex(t *testing.T) { + // TODO ... more structure ? + t.Run("full", func(t *testing.T) { + s := segmentedIndex{start: 0, lcd: 1, offsets: []int64{1}} + + s.next() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 2, s.local) + + s.next() + assert.EqualValues(t, 3, s.global) + assert.EqualValues(t, 3, s.local) + + s.prev() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 2, s.local) + + s.prev() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 2, s.local) + }) + + t.Run("half", func(t *testing.T) { + s := segmentedIndex{start: 0, lcd: 2, offsets: []int64{2}} + + s.next() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.next() + assert.EqualValues(t, 3, s.global) + assert.EqualValues(t, 2, s.local) + + s.next() + assert.EqualValues(t, 5, s.global) + assert.EqualValues(t, 3, s.local) + + s.prev() + assert.EqualValues(t, 3, s.global) + assert.EqualValues(t, 2, s.local) + + s.prev() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 1, s.global) + assert.EqualValues(t, 1, s.local) + }) + + t.Run("the other half", func(t *testing.T) { + s := segmentedIndex{start: 1, lcd: 2, offsets: []int64{2}} + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.next() + assert.EqualValues(t, 4, s.global) + assert.EqualValues(t, 2, s.local) + + s.next() + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 3, s.local) + + s.prev() + assert.EqualValues(t, 4, s.global) + assert.EqualValues(t, 2, s.local) + + s.prev() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + }) + + t.Run("strange", func(t *testing.T) { + s := segmentedIndex{start: 1, lcd: 7, offsets: []int64{4, 3}} + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.next() + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.next() + assert.EqualValues(t, 9, s.global) + assert.EqualValues(t, 3, s.local) + + s.prev() + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.prev() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + + s.next() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.goTo(6) + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.goTo(5) + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.goTo(7) + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.goTo(8) + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.goTo(9) + assert.EqualValues(t, 9, s.global) + assert.EqualValues(t, 3, s.local) + + s.prev() + assert.EqualValues(t, 6, s.global) + assert.EqualValues(t, 2, s.local) + + s.prev() + assert.EqualValues(t, 2, s.global) + assert.EqualValues(t, 1, s.local) + + s.prev() + assert.EqualValues(t, 0, s.global) + assert.EqualValues(t, 0, s.local) + }) +} From d0a8ac5645713250671abd0f04c322e30dc8cd68 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 23 Apr 2020 13:37:12 +0300 Subject: [PATCH 215/350] Optimize getRawExecutionSteps by preallocating MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 560µs ± 3% 302µs ± 3% -46.11% (p=0.000 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 5.97ms ± 2% 2.80ms ± 3% -53.04% (p=0.000 n=18+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 561µs ± 2% 303µs ± 3% -45.93% (p=0.000 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 5.94ms ± 3% 2.81ms ± 3% -52.63% (p=0.000 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 161µs ± 2% 86µs ± 4% -46.52% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 2.07ms ± 1% 1.08ms ± 3% -47.87% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 43.5µs ± 2% 30.0µs ± 3% -31.11% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 654µs ± 2% 331µs ± 2% -49.46% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 215µs ± 2% 114µs ± 4% -46.86% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 2.64ms ± 2% 1.33ms ± 3% -49.72% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 202µs ± 2% 111µs ± 3% -45.27% (p=0.000 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 2.43ms ± 2% 1.21ms ± 3% -50.08% (p=0.000 n=20+20) name old alloc/op new alloc/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 1.07MB ± 0% 0.25MB ± 0% -77.10% (p=0.000 n=17+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 14.6MB ± 0% 2.6MB ± 0% -81.87% (p=0.000 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 1.07MB ± 0% 0.25MB ± 0% -77.09% (p=0.000 n=15+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 14.6MB ± 0% 2.6MB ± 0% -81.87% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 254kB ± 0% 74kB ± 0% -70.93% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 4.62MB ± 0% 0.79MB ± 0% -82.80% (p=0.000 n=18+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 49.3kB ± 0% 24.7kB ± 0% -49.86% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 1.38MB ± 0% 0.27MB ± 0% -80.47% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 352kB ± 0% 98kB ± 0% -72.06% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 5.83MB ± 0% 1.06MB ± 0% -81.88% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 352kB ± 0% 90kB ± 0% -74.39% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 5.83MB ± 0% 0.98MB ± 0% -83.14% (p=0.000 n=19+20) name old allocs/op new allocs/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 20.0 ± 0% 1.0 ± 0% -95.00% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 31.0 ± 0% 1.0 ± 0% -96.77% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 22.0 ± 0% 3.0 ± 0% -86.36% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 33.0 ± 0% 3.0 ± 0% -90.91% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 19.0 ± 0% 5.0 ± 0% -73.68% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 30.0 ± 0% 5.0 ± 0% -83.33% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 16.0 ± 0% 6.0 ± 0% -62.50% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 26.0 ± 0% 6.0 ± 0% -76.92% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 19.0 ± 0% 4.0 ± 0% -78.95% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 30.0 ± 0% 4.0 ± 0% -86.67% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 19.0 ± 0% 4.0 ± 0% -78.95% (p=0.000 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 30.0 ± 0% 4.0 ± 0% -86.67% (p=0.000 n=20+20) --- lib/executor/variable_looping_vus.go | 47 +++++++++++++++++++++------- 1 file changed, 36 insertions(+), 11 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 2bd36fdf292..d1619522ce2 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -183,18 +183,16 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 //nolint:funlen func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple, zeroEnd bool) []lib.ExecutionStep { - // For accurate results, calculations are done with the unscaled values, and - // the values are scaled only before we add them to the steps result slice - fromVUs := vlvc.StartVUs.Int64 - - start, offsets, lcd := et.GetStripedOffsets(et.ES) - var index = segmentedIndex{start: start, lcd: lcd, offsets: offsets} - index.goTo(vlvc.StartVUs.Int64) + var ( + timeTillEnd time.Duration + fromVUs = vlvc.StartVUs.Int64 + start, offsets, lcd = et.GetStripedOffsets(et.ES) + index = segmentedIndex{start: start, lcd: lcd, offsets: offsets} + ) + index.goTo(fromVUs) + var steps = make([]lib.ExecutionStep, 0, vlvc.precalculateTheRequiredSteps(et, zeroEnd)) // Reserve the scaled StartVUs at the beginning - steps := []lib.ExecutionStep{{TimeOffset: 0, PlannedVUs: uint64(index.local)}} - - var timeTillEnd time.Duration - + steps = append(steps, lib.ExecutionStep{TimeOffset: 0, PlannedVUs: uint64(index.local)}) addStep := func(step lib.ExecutionStep) { if steps[len(steps)-1].PlannedVUs != step.PlannedVUs { steps = append(steps, step) @@ -300,6 +298,33 @@ func (s *segmentedIndex) goTo(value int64) { // TODO optimize } } +func absInt64(a int64) int64 { + if a < 0 { + return -a + } + return a +} + +func (vlvc VariableLoopingVUsConfig) precalculateTheRequiredSteps(et *lib.ExecutionTuple, zeroEnd bool) int { + p := et.ScaleInt64(vlvc.StartVUs.Int64) + var result int64 + result++ // for the first one + + if zeroEnd { + result++ // for the last one - this one can be more then needed + } + for _, stage := range vlvc.Stages { + stageEndVUs := et.ScaleInt64(stage.Target.Int64) + if stage.Duration.Duration == 0 { + result++ + } else { + result += absInt64(p - stageEndVUs) + } + p = stageEndVUs + } + return int(result) +} + // If the graceful ramp-downs are enabled, we need to reserve any VUs that may // potentially have to finish running iterations when we're scaling their number // down. This would prevent attempts from other executors to use them while the From df30a710a2bce7474c040852025a6b77a1857bc2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 23 Apr 2020 13:49:20 +0300 Subject: [PATCH 216/350] fix TestVariableLoopingVUsRampDownNoWobble --- lib/executor/variable_looping_vus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 5c8856b1cd1..328679be1b2 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -132,7 +132,7 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { sampleTimes := []time.Duration{ 100 * time.Millisecond, - 3400 * time.Millisecond, + 3200 * time.Millisecond, } const rampDownSamples = 50 From c60f01e8a8fb63ef50e5f95597ed636fbe728b01 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 24 Apr 2020 17:46:09 +0300 Subject: [PATCH 217/350] Don't stop VUs right away when stepping down but instead on the "next" step This reverts the "fix" from 1eb3e7ab as apperantly this is how it previously worked and is also what we want. --- lib/executor/variable_looping_vus.go | 2 +- lib/executor/variable_looping_vus_test.go | 166 ++++++++++++---------- 2 files changed, 94 insertions(+), 74 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index d1619522ce2..da3c85e3d6d 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -225,7 +225,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global)/stageVUDiff), + TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global+1)/stageVUDiff), PlannedVUs: uint64(index.local - 1), }) } diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 328679be1b2..5c69f80dd95 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -189,19 +189,19 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 2 * time.Second, PlannedVUs: 5}, - {TimeOffset: 3 * time.Second, PlannedVUs: 4}, - {TimeOffset: 4 * time.Second, PlannedVUs: 3}, - {TimeOffset: 5 * time.Second, PlannedVUs: 2}, - {TimeOffset: 6 * time.Second, PlannedVUs: 1}, + {TimeOffset: 3 * time.Second, PlannedVUs: 5}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 3}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 2}, {TimeOffset: 9 * time.Second, PlannedVUs: 3}, {TimeOffset: 10 * time.Second, PlannedVUs: 4}, {TimeOffset: 11 * time.Second, PlannedVUs: 5}, - {TimeOffset: 11 * time.Second, PlannedVUs: 4}, - {TimeOffset: 12 * time.Second, PlannedVUs: 3}, - {TimeOffset: 13 * time.Second, PlannedVUs: 2}, - {TimeOffset: 14 * time.Second, PlannedVUs: 1}, + {TimeOffset: 12 * time.Second, PlannedVUs: 4}, + {TimeOffset: 13 * time.Second, PlannedVUs: 3}, + {TimeOffset: 14 * time.Second, PlannedVUs: 2}, + {TimeOffset: 15 * time.Second, PlannedVUs: 1}, {TimeOffset: 16 * time.Second, PlannedVUs: 2}, {TimeOffset: 17 * time.Second, PlannedVUs: 3}, {TimeOffset: 18 * time.Second, PlannedVUs: 4}, @@ -227,8 +227,8 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 32 * time.Second, PlannedVUs: 5}, - {TimeOffset: 41 * time.Second, PlannedVUs: 4}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 53 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -239,8 +239,8 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 32 * time.Second, PlannedVUs: 5}, - {TimeOffset: 41 * time.Second, PlannedVUs: 4}, + {TimeOffset: 33 * time.Second, PlannedVUs: 5}, + {TimeOffset: 42 * time.Second, PlannedVUs: 4}, {TimeOffset: 50 * time.Second, PlannedVUs: 1}, {TimeOffset: 103 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -288,12 +288,12 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { expRawStepsNoZeroEnd := []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, } @@ -313,7 +313,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 41 * time.Second, PlannedVUs: 1}, + {TimeOffset: 42 * time.Second, PlannedVUs: 1}, {TimeOffset: 50 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -322,7 +322,7 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 41 * time.Second, PlannedVUs: 1}, + {TimeOffset: 42 * time.Second, PlannedVUs: 1}, {TimeOffset: 50 * time.Second, PlannedVUs: 0}, }, conf.GetExecutionRequirements(et)) @@ -395,19 +395,19 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 4}, {TimeOffset: 1 * time.Second, PlannedVUs: 5}, {TimeOffset: 2 * time.Second, PlannedVUs: 6}, - {TimeOffset: 2 * time.Second, PlannedVUs: 5}, - {TimeOffset: 3 * time.Second, PlannedVUs: 4}, - {TimeOffset: 4 * time.Second, PlannedVUs: 3}, - {TimeOffset: 5 * time.Second, PlannedVUs: 2}, - {TimeOffset: 6 * time.Second, PlannedVUs: 1}, + {TimeOffset: 3 * time.Second, PlannedVUs: 5}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 3}, + {TimeOffset: 6 * time.Second, PlannedVUs: 2}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 2}, {TimeOffset: 9 * time.Second, PlannedVUs: 3}, {TimeOffset: 10 * time.Second, PlannedVUs: 4}, {TimeOffset: 11 * time.Second, PlannedVUs: 5}, - {TimeOffset: 11 * time.Second, PlannedVUs: 4}, - {TimeOffset: 12 * time.Second, PlannedVUs: 3}, - {TimeOffset: 13 * time.Second, PlannedVUs: 2}, - {TimeOffset: 14 * time.Second, PlannedVUs: 1}, + {TimeOffset: 12 * time.Second, PlannedVUs: 4}, + {TimeOffset: 13 * time.Second, PlannedVUs: 3}, + {TimeOffset: 14 * time.Second, PlannedVUs: 2}, + {TimeOffset: 15 * time.Second, PlannedVUs: 1}, {TimeOffset: 16 * time.Second, PlannedVUs: 2}, {TimeOffset: 17 * time.Second, PlannedVUs: 3}, {TimeOffset: 18 * time.Second, PlannedVUs: 4}, @@ -416,8 +416,8 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 27 * time.Second, PlannedVUs: 1}, {TimeOffset: 28 * time.Second, PlannedVUs: 2}, - {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 29 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 1}, + {TimeOffset: 30 * time.Second, PlannedVUs: 0}, {TimeOffset: 31 * time.Second, PlannedVUs: 1}, {TimeOffset: 32 * time.Second, PlannedVUs: 2}, {TimeOffset: 33 * time.Second, PlannedVUs: 3}, @@ -429,18 +429,18 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, @@ -449,18 +449,18 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, @@ -469,18 +469,18 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, @@ -489,18 +489,18 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, @@ -508,15 +508,15 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 2}, - {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 5 * time.Second, PlannedVUs: 1}, {TimeOffset: 10 * time.Second, PlannedVUs: 2}, - {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 13 * time.Second, PlannedVUs: 1}, {TimeOffset: 18 * time.Second, PlannedVUs: 2}, {TimeOffset: 20 * time.Second, PlannedVUs: 1}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 27 * time.Second, PlannedVUs: 1}, - {TimeOffset: 29 * time.Second, PlannedVUs: 0}, + {TimeOffset: 30 * time.Second, PlannedVUs: 0}, {TimeOffset: 31 * time.Second, PlannedVUs: 1}, {TimeOffset: 34 * time.Second, PlannedVUs: 2}, }, @@ -526,18 +526,18 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 1 * time.Second, PlannedVUs: 2}, - {TimeOffset: 3 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 1}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, {TimeOffset: 8 * time.Second, PlannedVUs: 1}, {TimeOffset: 11 * time.Second, PlannedVUs: 2}, - {TimeOffset: 11 * time.Second, PlannedVUs: 1}, - {TimeOffset: 14 * time.Second, PlannedVUs: 0}, + {TimeOffset: 12 * time.Second, PlannedVUs: 1}, + {TimeOffset: 15 * time.Second, PlannedVUs: 0}, {TimeOffset: 16 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 2}, {TimeOffset: 26 * time.Second, PlannedVUs: 0}, {TimeOffset: 28 * time.Second, PlannedVUs: 1}, - {TimeOffset: 28 * time.Second, PlannedVUs: 0}, + {TimeOffset: 29 * time.Second, PlannedVUs: 0}, {TimeOffset: 32 * time.Second, PlannedVUs: 1}, }, }, @@ -546,10 +546,10 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 1}, {TimeOffset: 2 * time.Second, PlannedVUs: 2}, - {TimeOffset: 2 * time.Second, PlannedVUs: 1}, - {TimeOffset: 5 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 6 * time.Second, PlannedVUs: 0}, {TimeOffset: 9 * time.Second, PlannedVUs: 1}, - {TimeOffset: 13 * time.Second, PlannedVUs: 0}, + {TimeOffset: 14 * time.Second, PlannedVUs: 0}, {TimeOffset: 17 * time.Second, PlannedVUs: 1}, {TimeOffset: 20 * time.Second, PlannedVUs: 0}, {TimeOffset: 23 * time.Second, PlannedVUs: 1}, @@ -585,8 +585,8 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 2}, {TimeOffset: 0 * time.Second, PlannedVUs: 5}, - {TimeOffset: 0 * time.Second, PlannedVUs: 4}, - {TimeOffset: 1 * time.Second, PlannedVUs: 3}, + {TimeOffset: 1 * time.Second, PlannedVUs: 4}, + {TimeOffset: 2 * time.Second, PlannedVUs: 3}, }, stages: []Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(0 * time.Second)}, @@ -612,12 +612,12 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 0}, {TimeOffset: 1 * time.Second, PlannedVUs: 1}, {TimeOffset: 2 * time.Second, PlannedVUs: 2}, - {TimeOffset: 2 * time.Second, PlannedVUs: 1}, - {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 1}, + {TimeOffset: 4 * time.Second, PlannedVUs: 0}, {TimeOffset: 5 * time.Second, PlannedVUs: 1}, {TimeOffset: 6 * time.Second, PlannedVUs: 2}, - {TimeOffset: 6 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 7 * time.Second, PlannedVUs: 1}, + {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -631,9 +631,9 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 0}, {TimeOffset: 1 * time.Second, PlannedVUs: 1}, - {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 0}, {TimeOffset: 5 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/2"), nil), stages: []Stage{ @@ -648,9 +648,9 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 0}, {TimeOffset: 2 * time.Second, PlannedVUs: 1}, - {TimeOffset: 2 * time.Second, PlannedVUs: 0}, + {TimeOffset: 3 * time.Second, PlannedVUs: 0}, {TimeOffset: 6 * time.Second, PlannedVUs: 1}, - {TimeOffset: 6 * time.Second, PlannedVUs: 0}, + {TimeOffset: 7 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("1/2:1"), nil), stages: []Stage{ @@ -678,9 +678,9 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { expectedSteps: []lib.ExecutionStep{ {TimeOffset: 0 * time.Second, PlannedVUs: 0}, {TimeOffset: 1 * time.Second, PlannedVUs: 1}, - {TimeOffset: 3 * time.Second, PlannedVUs: 0}, + {TimeOffset: 4 * time.Second, PlannedVUs: 0}, {TimeOffset: 5 * time.Second, PlannedVUs: 1}, - {TimeOffset: 7 * time.Second, PlannedVUs: 0}, + {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1/2,2/3,1")), stages: []Stage{ @@ -710,6 +710,26 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {Target: null.IntFrom(30), Duration: types.NullDurationFrom(30 * time.Second)}, }, }, + { + name: "more up and down", + expectedSteps: []lib.ExecutionStep{ + {TimeOffset: 0 * time.Second, PlannedVUs: 0}, + {TimeOffset: 1 * time.Second, PlannedVUs: 1}, + {TimeOffset: 2 * time.Second, PlannedVUs: 2}, + {TimeOffset: 3 * time.Second, PlannedVUs: 3}, + {TimeOffset: 4 * time.Second, PlannedVUs: 4}, + {TimeOffset: 5 * time.Second, PlannedVUs: 5}, + {TimeOffset: 6 * time.Second, PlannedVUs: 4}, + {TimeOffset: 7 * time.Second, PlannedVUs: 3}, + {TimeOffset: 8 * time.Second, PlannedVUs: 2}, + {TimeOffset: 9 * time.Second, PlannedVUs: 1}, + {TimeOffset: 10 * time.Second, PlannedVUs: 0}, + }, + stages: []Stage{ + {Target: null.IntFrom(5), Duration: types.NullDurationFrom(5 * time.Second)}, + {Target: null.IntFrom(0), Duration: types.NullDurationFrom(5 * time.Second)}, + }, + }, } for _, testCase := range testCases { From 70275a0ae4db88052ca8156abb74c5ab20d60051 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 24 Apr 2020 18:14:03 +0300 Subject: [PATCH 218/350] drop two ifs that are no longer needed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name old time/op new time/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 302µs ± 3% 299µs ± 2% -0.83% (p=0.018 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 2.80ms ± 3% 2.80ms ± 4% ~ (p=0.879 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 303µs ± 3% 302µs ± 3% ~ (p=0.550 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 2.81ms ± 3% 2.80ms ± 2% ~ (p=0.444 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 86.3µs ± 4% 86.7µs ± 2% ~ (p=0.718 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 1.08ms ± 3% 1.07ms ± 3% ~ (p=0.166 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 30.0µs ± 3% 29.8µs ± 3% ~ (p=0.175 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 331µs ± 2% 329µs ± 3% ~ (p=0.369 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 114µs ± 4% 115µs ± 4% ~ (p=0.158 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 1.33ms ± 3% 1.32ms ± 3% ~ (p=0.771 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 111µs ± 3% 110µs ± 4% ~ (p=0.057 n=19+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 1.21ms ± 3% 1.20ms ± 2% ~ (p=0.096 n=20+20) name old alloc/op new alloc/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 246kB ± 0% 246kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 2.65MB ± 0% 2.65MB ± 0% ~ (p=0.736 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 246kB ± 0% 246kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 2.65MB ± 0% 2.65MB ± 0% ~ (p=0.468 n=20+19) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 73.8kB ± 0% 73.8kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 795kB ± 0% 795kB ± 0% ~ (p=0.304 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 24.7kB ± 0% 24.7kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 270kB ± 0% 270kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 98.4kB ± 0% 98.4kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 1.06MB ± 0% 1.06MB ± 0% ~ (p=0.985 n=20+20) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 90.3kB ± 0% 90.3kB ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 983kB ± 0% 983kB ± 0% ~ (p=0.081 n=20+19) name old allocs/op new allocs/op delta VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/normal-8 1.00 ± 0% 1.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:/rollercoaster-8 1.00 ± 0% 1.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/normal-8 3.00 ± 0% 3.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:0:1/rollercoaster-8 3.00 ± 0% 3.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/normal-8 5.00 ± 0% 5.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/rollercoaster-8 5.00 ± 0% 5.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/normal-8 6.00 ± 0% 6.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/rollercoaster-8 6.00 ± 0% 6.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/normal-8 4.00 ± 0% 4.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2/5:4/5/rollercoaster-8 4.00 ± 0% 4.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/normal-8 4.00 ± 0% 4.00 ± 0% ~ (all equal) VariableLoopingVUsGetRawExecutionSteps/seq:;segment:2235/5213:4/5/rollercoaster-8 4.00 ± 0% 4.00 ± 0% ~ (all equal) --- lib/executor/variable_looping_vus.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index da3c85e3d6d..b4b5bb1c87f 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -219,9 +219,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it // it will just go to it for ; index.global > stageEndVUs; index.prev() { - if index.global > fromVUs { - continue - } // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ @@ -233,9 +230,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple // here we want the emit for the last one as this case it actually should emit that // we start it for ; index.global <= stageEndVUs; index.next() { - if index.global < fromVUs { - continue - } // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ From 964b477c2edbea29755cad105fc6dd73d72cfb5c Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 24 Apr 2020 19:01:45 +0300 Subject: [PATCH 219/350] rename local and global to scaled and unscaled --- lib/executor/variable_looping_vus.go | 58 ++++---- lib/executor/variable_looping_vus_test.go | 172 +++++++++++----------- 2 files changed, 114 insertions(+), 116 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b4b5bb1c87f..7f0c1f376bd 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -192,7 +192,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple index.goTo(fromVUs) var steps = make([]lib.ExecutionStep, 0, vlvc.precalculateTheRequiredSteps(et, zeroEnd)) // Reserve the scaled StartVUs at the beginning - steps = append(steps, lib.ExecutionStep{TimeOffset: 0, PlannedVUs: uint64(index.local)}) + steps = append(steps, lib.ExecutionStep{TimeOffset: 0, PlannedVUs: uint64(index.scaled)}) addStep := func(step lib.ExecutionStep) { if steps[len(steps)-1].PlannedVUs != step.PlannedVUs { steps = append(steps, step) @@ -210,31 +210,29 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } if stageDuration == 0 { index.goTo(stageEndVUs) - addStep(lib.ExecutionStep{TimeOffset: timeTillEnd, PlannedVUs: uint64(index.local)}) + addStep(lib.ExecutionStep{TimeOffset: timeTillEnd, PlannedVUs: uint64(index.scaled)}) fromVUs = stageEndVUs continue } - if index.global > stageEndVUs { // ramp down + if index.unscaled > stageEndVUs { // ramp down // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it // it will just go to it - for ; index.global > stageEndVUs; index.prev() { + for ; index.unscaled > stageEndVUs; index.prev() { // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global+1)/stageVUDiff), - PlannedVUs: uint64(index.local - 1), + TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled+1)/stageVUDiff), + PlannedVUs: uint64(index.scaled - 1), }) } } else { - // here we want the emit for the last one as this case it actually should emit that - // we start it - for ; index.global <= stageEndVUs; index.next() { + for ; index.unscaled <= stageEndVUs; index.next() { // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.global)/stageVUDiff), - PlannedVUs: uint64(index.local), + TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled)/stageVUDiff), + PlannedVUs: uint64(index.scaled), }) } } @@ -249,46 +247,46 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } type segmentedIndex struct { // TODO: rename ... although this is probably the best name so far :D - start, lcd int64 - offsets []int64 - local, global int64 + start, lcd int64 + offsets []int64 + scaled, unscaled int64 } func (s *segmentedIndex) next() { - if s.local == 0 { - s.global += s.start + 1 + if s.scaled == 0 { + s.unscaled += s.start + 1 } else { - s.global += s.offsets[int(s.local-1)%len(s.offsets)] + s.unscaled += s.offsets[int(s.scaled-1)%len(s.offsets)] } - s.local++ + s.scaled++ } func (s *segmentedIndex) prev() { - if s.local == 1 { - s.global -= s.start + 1 + if s.scaled == 1 { + s.unscaled -= s.start + 1 } else { - s.global -= s.offsets[int(s.local-2)%len(s.offsets)] + s.unscaled -= s.offsets[int(s.scaled-2)%len(s.offsets)] } - s.local-- + s.scaled-- } func (s *segmentedIndex) goTo(value int64) { // TODO optimize var gi int64 - s.local = (value / s.lcd) * int64(len(s.offsets)) - s.global = s.local / int64(len(s.offsets)) * s.lcd // TODO optimize ? + s.scaled = (value / s.lcd) * int64(len(s.offsets)) + s.unscaled = s.scaled / int64(len(s.offsets)) * s.lcd // TODO optimize ? i := s.start for ; i < value%s.lcd; gi, i = gi+1, i+s.offsets[gi] { - s.local++ + s.scaled++ } if gi > 0 { - s.global += i - s.offsets[gi-1] - } else if s.local > 0 { - s.global -= s.offsets[len(s.offsets)-1] - s.start + s.unscaled += i - s.offsets[gi-1] + } else if s.scaled > 0 { + s.unscaled -= s.offsets[len(s.offsets)-1] - s.start } - if s.local > 0 { - s.global++ // this is to fix the fact it starts from 0 + if s.scaled > 0 { + s.unscaled++ // this is to fix the fact it starts from 0 } } diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 5c69f80dd95..89a9101866e 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -818,187 +818,187 @@ func TestSegmentedIndex(t *testing.T) { s := segmentedIndex{start: 0, lcd: 1, offsets: []int64{1}} s.next() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.next() - assert.EqualValues(t, 3, s.global) - assert.EqualValues(t, 3, s.local) + assert.EqualValues(t, 3, s.unscaled) + assert.EqualValues(t, 3, s.scaled) s.prev() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.prev() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 2, s.scaled) }) t.Run("half", func(t *testing.T) { s := segmentedIndex{start: 0, lcd: 2, offsets: []int64{2}} s.next() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.next() - assert.EqualValues(t, 3, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 3, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.next() - assert.EqualValues(t, 5, s.global) - assert.EqualValues(t, 3, s.local) + assert.EqualValues(t, 5, s.unscaled) + assert.EqualValues(t, 3, s.scaled) s.prev() - assert.EqualValues(t, 3, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 3, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.prev() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 1, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 1, s.unscaled) + assert.EqualValues(t, 1, s.scaled) }) t.Run("the other half", func(t *testing.T) { s := segmentedIndex{start: 1, lcd: 2, offsets: []int64{2}} s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.next() - assert.EqualValues(t, 4, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 4, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.next() - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 3, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 3, s.scaled) s.prev() - assert.EqualValues(t, 4, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 4, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.prev() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) }) t.Run("strange", func(t *testing.T) { s := segmentedIndex{start: 1, lcd: 7, offsets: []int64{4, 3}} s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.next() - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.next() - assert.EqualValues(t, 9, s.global) - assert.EqualValues(t, 3, s.local) + assert.EqualValues(t, 9, s.unscaled) + assert.EqualValues(t, 3, s.scaled) s.prev() - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.prev() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) s.next() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.goTo(6) - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.goTo(5) - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.goTo(7) - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.goTo(8) - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.goTo(9) - assert.EqualValues(t, 9, s.global) - assert.EqualValues(t, 3, s.local) + assert.EqualValues(t, 9, s.unscaled) + assert.EqualValues(t, 3, s.scaled) s.prev() - assert.EqualValues(t, 6, s.global) - assert.EqualValues(t, 2, s.local) + assert.EqualValues(t, 6, s.unscaled) + assert.EqualValues(t, 2, s.scaled) s.prev() - assert.EqualValues(t, 2, s.global) - assert.EqualValues(t, 1, s.local) + assert.EqualValues(t, 2, s.unscaled) + assert.EqualValues(t, 1, s.scaled) s.prev() - assert.EqualValues(t, 0, s.global) - assert.EqualValues(t, 0, s.local) + assert.EqualValues(t, 0, s.unscaled) + assert.EqualValues(t, 0, s.scaled) }) } From 37492ac0896d2d2c72e0ea26841ee9bd4d747682 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 24 Apr 2020 19:25:37 +0300 Subject: [PATCH 220/350] misc changes and comment fixes/additions --- lib/executor/variable_looping_vus.go | 38 +++++++++++++++------------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 7f0c1f376bd..b4d983cf605 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -181,21 +181,20 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // 00000000001111111111222 (t/10) // // More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 -//nolint:funlen func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple, zeroEnd bool) []lib.ExecutionStep { var ( timeTillEnd time.Duration fromVUs = vlvc.StartVUs.Int64 start, offsets, lcd = et.GetStripedOffsets(et.ES) + steps = make([]lib.ExecutionStep, 0, vlvc.precalculateTheRequiredSteps(et, zeroEnd)) index = segmentedIndex{start: start, lcd: lcd, offsets: offsets} ) - index.goTo(fromVUs) - var steps = make([]lib.ExecutionStep, 0, vlvc.precalculateTheRequiredSteps(et, zeroEnd)) + // Reserve the scaled StartVUs at the beginning - steps = append(steps, lib.ExecutionStep{TimeOffset: 0, PlannedVUs: uint64(index.scaled)}) - addStep := func(step lib.ExecutionStep) { - if steps[len(steps)-1].PlannedVUs != step.PlannedVUs { - steps = append(steps, step) + steps = append(steps, lib.ExecutionStep{TimeOffset: 0, PlannedVUs: uint64(index.goTo(fromVUs))}) + addStep := func(timeOffset time.Duration, plannedVUs uint64) { + if steps[len(steps)-1].PlannedVUs != plannedVUs { + steps = append(steps, lib.ExecutionStep{TimeOffset: timeOffset, PlannedVUs: plannedVUs}) } } @@ -209,8 +208,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple continue } if stageDuration == 0 { - index.goTo(stageEndVUs) - addStep(lib.ExecutionStep{TimeOffset: timeTillEnd, PlannedVUs: uint64(index.scaled)}) + addStep(timeTillEnd, uint64(index.goTo(stageEndVUs))) fromVUs = stageEndVUs continue } @@ -221,19 +219,21 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple for ; index.unscaled > stageEndVUs; index.prev() { // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() - addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled+1)/stageVUDiff), - PlannedVUs: uint64(index.scaled - 1), - }) + addStep( // this is the time that we should go up 1 if we are ramping up + // but we are ramping down so we should go 1 down, but because we want to not + // stop VUs immediately we stop it on the next unscaled VU's time + timeTillEnd-time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled+1)/stageVUDiff), + uint64(index.scaled-1), + ) } } else { for ; index.unscaled <= stageEndVUs; index.next() { // VU reservation for gracefully ramping down is handled as a // separate method: reserveVUsForGracefulRampDowns() - addStep(lib.ExecutionStep{ - TimeOffset: timeTillEnd - time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled)/stageVUDiff), - PlannedVUs: uint64(index.scaled), - }) + addStep( + timeTillEnd-time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled)/stageVUDiff), + uint64(index.scaled), + ) } } fromVUs = stageEndVUs @@ -270,7 +270,7 @@ func (s *segmentedIndex) prev() { s.scaled-- } -func (s *segmentedIndex) goTo(value int64) { // TODO optimize +func (s *segmentedIndex) goTo(value int64) int64 { // TODO optimize var gi int64 s.scaled = (value / s.lcd) * int64(len(s.offsets)) s.unscaled = s.scaled / int64(len(s.offsets)) * s.lcd // TODO optimize ? @@ -288,6 +288,8 @@ func (s *segmentedIndex) goTo(value int64) { // TODO optimize if s.scaled > 0 { s.unscaled++ // this is to fix the fact it starts from 0 } + + return s.scaled } func absInt64(a int64) int64 { From c03b5e0490d58321fd9711e1ba264a67104e3c22 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Sat, 25 Apr 2020 15:06:52 +0300 Subject: [PATCH 221/350] Steal some documentation and implementation ideas from the more readable branch --- lib/executor/variable_looping_vus.go | 62 ++++++++++++++++++---------- 1 file changed, 40 insertions(+), 22 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b4d983cf605..09a572b2122 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -213,13 +213,14 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple continue } + // VU reservation for gracefully ramping down is handled as a + // separate method: reserveVUsForGracefulRampDowns() if index.unscaled > stageEndVUs { // ramp down // here we don't want to emit for the equal to stageEndVUs as it doesn't go below it // it will just go to it for ; index.unscaled > stageEndVUs; index.prev() { - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() - addStep( // this is the time that we should go up 1 if we are ramping up + addStep( + // this is the time that we should go up 1 if we are ramping up // but we are ramping down so we should go 1 down, but because we want to not // stop VUs immediately we stop it on the next unscaled VU's time timeTillEnd-time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled+1)/stageVUDiff), @@ -228,8 +229,6 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple } } else { for ; index.unscaled <= stageEndVUs; index.next() { - // VU reservation for gracefully ramping down is handled as a - // separate method: reserveVUsForGracefulRampDowns() addStep( timeTillEnd-time.Duration(int64(stageDuration)*(stageEndVUs-index.unscaled)/stageVUDiff), uint64(index.scaled), @@ -249,44 +248,63 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple type segmentedIndex struct { // TODO: rename ... although this is probably the best name so far :D start, lcd int64 offsets []int64 - scaled, unscaled int64 + scaled, unscaled int64 // for both the first element(vu) is 1 not 0 } +// goes to the next scaled index and move the unscaled one accordingly func (s *segmentedIndex) next() { - if s.scaled == 0 { - s.unscaled += s.start + 1 - } else { - s.unscaled += s.offsets[int(s.scaled-1)%len(s.offsets)] + if s.scaled == 0 { // the 1 element(VU) is at the start + s.unscaled += s.start + 1 // the first element of the start 0, but the here we need it to be 1 so we add 1 + } else { // if we are not at the first element we need to go through the offsets, looping over them + s.unscaled += s.offsets[int(s.scaled-1)%len(s.offsets)] // slice's index start at 0 ours start at 1 } s.scaled++ } +// prev goest to the previous scaled value and sets the unscaled one accordingly +// calling prev when s.scaled == 0 is undefined func (s *segmentedIndex) prev() { - if s.scaled == 1 { - s.unscaled -= s.start + 1 - } else { - s.unscaled -= s.offsets[int(s.scaled-2)%len(s.offsets)] + if s.scaled == 1 { // we are the first need to go to the 0th element which means we need to remove the start + s.unscaled -= s.start + 1 // this could've been just settign to 0 + } else { // not at the first element - need to get the previously added offset so + s.unscaled -= s.offsets[int(s.scaled-2)%len(s.offsets)] // slice's index start 0 our start at 1 } s.scaled-- } +// goTo sets the scaled index to it's biggest value for which the corresponding unscaled index is +// is smaller or equal to value func (s *segmentedIndex) goTo(value int64) int64 { // TODO optimize var gi int64 - s.scaled = (value / s.lcd) * int64(len(s.offsets)) - s.unscaled = s.scaled / int64(len(s.offsets)) * s.lcd // TODO optimize ? + // Because of the cyclical nature of the striping algorithm (with a cycle + // length of LCD, the least common denominator), when scaling large values + // (i.e. many multiples of the LCD), we can quickly calculate how many times + // the cycle repeats. + wholeCycles := (value / s.lcd) + // So we can set some approximate initial values quickly, since we also know + // precisely how many scaled values there are per cycle length. + s.scaled = wholeCycles * int64(len(s.offsets)) + s.unscaled = wholeCycles*s.lcd + s.start + 1 // our indexes are from 1 the start is from 0 + // Approach the final value using the slow algorithm with the step by step loop + // TODO: this can be optimized by another array with size offsets that instead of the offsets + // from the previous is the offset from either 0 or start i := s.start for ; i < value%s.lcd; gi, i = gi+1, i+s.offsets[gi] { s.scaled++ + s.unscaled += s.offsets[gi] } - if gi > 0 { - s.unscaled += i - s.offsets[gi-1] - } else if s.scaled > 0 { - s.unscaled -= s.offsets[len(s.offsets)-1] - s.start + if gi > 0 { // there were more values after the wholecycles + // the last offset actually shouldn't have been added + s.unscaled -= s.offsets[gi-1] + } else if s.scaled > 0 { // we didn't actually have more values after the wholecycles but we still had some + // in this case the unscaled value needs to move back by the last offset as it would've been + // the one to get it from the value it needs to be to it's current one + s.unscaled -= s.offsets[len(s.offsets)-1] } - if s.scaled > 0 { - s.unscaled++ // this is to fix the fact it starts from 0 + if s.scaled == 0 { + s.unscaled = 0 // we would've added the start and 1 } return s.scaled From fb145c2cee4ea2c5182066bf2c61b7746e740199 Mon Sep 17 00:00:00 2001 From: na-- Date: Mon, 27 Apr 2020 17:45:51 +0300 Subject: [PATCH 222/350] Add a randomized test for the variable-looping-vus requirements (#1413) --- lib/execution_segment_test.go | 52 ++++----- lib/executor/variable_looping_vus_test.go | 124 +++++++++++++++++++++- 2 files changed, 142 insertions(+), 34 deletions(-) diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 3513f50cd89..7f430db8a62 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -237,6 +237,7 @@ func TestExecutionTupleScale(t *testing.T) { require.Equal(t, int64(1), et.ScaleInt64(2)) require.Equal(t, int64(1), et.ScaleInt64(3)) } + func TestBigScale(t *testing.T) { es := new(ExecutionSegment) ess, err := NewExecutionSegmentSequenceFromString("0,7/20,7/10,1") @@ -431,29 +432,24 @@ func TestExecutionSegmentStringSequences(t *testing.T) { // Return a randomly distributed sequence of n amount of // execution segments whose length totals 1. -func generateRandomSequence(n int64, r *rand.Rand) (ExecutionSegmentSequence, error) { +func generateRandomSequence(t testing.TB, n, m int64, r *rand.Rand) ExecutionSegmentSequence { var err error - var ess = ExecutionSegmentSequence(make([]*ExecutionSegment, n)) - var numerators = make([]int64, n) + ess := ExecutionSegmentSequence(make([]*ExecutionSegment, n)) + numerators := make([]int64, n) var denominator int64 for i := int64(0); i < n; i++ { - for numerators[i] == 0 { - numerators[i] = r.Int63n(n) - denominator += numerators[i] - } + numerators[i] = 1 + r.Int63n(m) + denominator += numerators[i] } - ess[0], err = NewExecutionSegment(big.NewRat(0, 1), big.NewRat(numerators[0], denominator)) - if err != nil { - return nil, err - } - for i := int64(1); i < n; i++ { - ess[i], err = NewExecutionSegment(ess[i-1].to, new(big.Rat).Add(big.NewRat(numerators[i], denominator), ess[i-1].to)) - if err != nil { - return nil, err - } + from := big.NewRat(0, 1) + for i := int64(0); i < n; i++ { + to := new(big.Rat).Add(big.NewRat(numerators[i], denominator), from) + ess[i], err = NewExecutionSegment(from, to) + require.NoError(t, err) + from = to } - return ess, nil + return ess } // Ensure that the sum of scaling all execution segments in @@ -468,8 +464,7 @@ func TestExecutionSegmentScaleConsistency(t *testing.T) { const numTests = 10 for i := 0; i < numTests; i++ { scale := rand.Int31n(99) + 2 - seq, err := generateRandomSequence(r.Int63n(9)+2, r) - require.NoError(t, err) + seq := generateRandomSequence(t, r.Int63n(9)+2, 100, r) t.Run(fmt.Sprintf("%d_%s", scale, seq), func(t *testing.T) { var total int64 @@ -493,8 +488,7 @@ func TestExecutionTupleScaleConsistency(t *testing.T) { const numTests = 10 for i := 0; i < numTests; i++ { scale := rand.Int31n(99) + 2 - seq, err := generateRandomSequence(r.Int63n(9)+2, r) - require.NoError(t, err) + seq := generateRandomSequence(t, r.Int63n(9)+2, 200, r) et, err := NewExecutionTuple(seq[0], &seq) require.NoError(t, err) @@ -534,8 +528,7 @@ func TestExecutionSegmentScaleNoWobble(t *testing.T) { // Random segments const numTests = 10 for i := 0; i < numTests; i++ { - seq, err := generateRandomSequence(r.Int63n(9)+2, r) - require.NoError(t, err) + seq := generateRandomSequence(t, r.Int63n(9)+2, 100, r) es := seq[rand.Intn(len(seq))] @@ -628,15 +621,14 @@ func TestSequenceLCD(t *testing.T) { } func BenchmarkGetStripedOffsets(b *testing.B) { - var lengths = [...]int64{10, 100} + lengths := [...]int64{10, 100} const seed = 777 r := rand.New(rand.NewSource(seed)) for _, length := range lengths { length := length b.Run(fmt.Sprintf("length%d,seed%d", length, seed), func(b *testing.B) { - sequence, err := generateRandomSequence(length, r) - require.NoError(b, err) + sequence := generateRandomSequence(b, length, 100, r) b.ResetTimer() for i := 0; i < b.N; i++ { segment := sequence[int(r.Int63())%len(sequence)] @@ -649,11 +641,11 @@ func BenchmarkGetStripedOffsets(b *testing.B) { } func BenchmarkGetStripedOffsetsEven(b *testing.B) { - var lengths = [...]int64{10, 100, 1000} + lengths := [...]int64{10, 100, 1000} generateSequence := func(n int64) ExecutionSegmentSequence { var err error - var ess = ExecutionSegmentSequence(make([]*ExecutionSegment, n)) - var numerators = make([]int64, n) + ess := ExecutionSegmentSequence(make([]*ExecutionSegment, n)) + numerators := make([]int64, n) var denominator int64 for i := int64(0); i < n; i++ { numerators[i] = 1 // nice and simple :) @@ -731,7 +723,7 @@ func mustNewExecutionSegmentSequence(str string) *ExecutionSegmentSequence { } func TestNewExecutionTuple(t *testing.T) { - var testCases = []struct { + testCases := []struct { seg *ExecutionSegment seq *ExecutionSegmentSequence scaleTests map[int64]int64 diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 89a9101866e..92bcc481560 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -24,6 +24,8 @@ import ( "context" "encoding/json" "fmt" + "math/big" + "math/rand" "sync/atomic" "testing" "time" @@ -63,7 +65,7 @@ func TestVariableLoopingVUsRun(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, _ = setupExecutor( + ctx, cancel, executor, _ := setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { // Sleeping for a weird duration somewhat offset from the @@ -85,7 +87,7 @@ func TestVariableLoopingVUsRun(t *testing.T) { errCh := make(chan error) go func() { errCh <- executor.Run(ctx, nil) }() - var result = make([]int64, len(sampleTimes)) + result := make([]int64, len(sampleTimes)) for i, d := range sampleTimes { time.Sleep(d) result[i] = es.GetCurrentlyActiveVUsCount() @@ -121,7 +123,7 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, _ = setupExecutor( + ctx, cancel, executor, _ := setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { time.Sleep(1 * time.Second) @@ -139,7 +141,7 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { errCh := make(chan error) go func() { errCh <- executor.Run(ctx, nil) }() - var result = make([]int64, len(sampleTimes)+rampDownSamples) + result := make([]int64, len(sampleTimes)+rampDownSamples) for i, d := range sampleTimes { time.Sleep(d) result[i] = es.GetCurrentlyActiveVUsCount() @@ -1002,3 +1004,117 @@ func TestSegmentedIndex(t *testing.T) { assert.EqualValues(t, 0, s.scaled) }) } + +// TODO: delete in favor of lib.generateRandomSequence() after +// https://github.com/loadimpact/k6/issues/1302 is done (can't import now due to +// import loops...) +func generateRandomSequence(t testing.TB, n, m int64, r *rand.Rand) lib.ExecutionSegmentSequence { + var err error + ess := lib.ExecutionSegmentSequence(make([]*lib.ExecutionSegment, n)) + numerators := make([]int64, n) + var denominator int64 + for i := int64(0); i < n; i++ { + numerators[i] = 1 + r.Int63n(m) + denominator += numerators[i] + } + from := big.NewRat(0, 1) + for i := int64(0); i < n; i++ { + to := new(big.Rat).Add(big.NewRat(numerators[i], denominator), from) + ess[i], err = lib.NewExecutionSegment(from, to) + require.NoError(t, err) + from = to + } + + return ess +} + +func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { + t.Parallel() + + seed := time.Now().UnixNano() + r := rand.New(rand.NewSource(seed)) + t.Logf("Random source seeded with %d\n", seed) + + const ( + numTests = 10 + maxStages = 10 + minStageDuration = 1 * time.Second + maxStageDuration = 10 * time.Minute + maxVUs = 300 + segmentSeqMaxLen = 15 + maxNumerator = 300 + ) + getTestConfig := func(name string) VariableLoopingVUsConfig { + stagesCount := 1 + r.Int31n(maxStages) + stages := make([]Stage, stagesCount) + for s := int32(0); s < stagesCount; s++ { + dur := time.Duration(r.Int63n(int64(maxStageDuration - minStageDuration))).Round(time.Second) + stages[s] = Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} + } + + c := NewVariableLoopingVUsConfig(name) + c.GracefulRampDown = types.NullDurationFrom(0) + c.GracefulStop = types.NullDurationFrom(0) + c.StartVUs = null.IntFrom(r.Int63n(maxVUs)) + c.Stages = stages + return c + } + + subtractChildSteps := func(t *testing.T, parent, child []lib.ExecutionStep) { + t.Logf("subtractChildSteps()") + for _, step := range child { + t.Logf(" child planned VUs for time offset %s: %d", step.TimeOffset, step.PlannedVUs) + } + sub := uint64(0) + ci := 0 + for pi, p := range parent { + // We iterate over all parent steps and match them to child steps. + // Once we have a match, we remove the child step's plannedVUs from + // the parent steps until a new match, when we adjust the subtracted + // amount again. + if p.TimeOffset > child[ci].TimeOffset && ci != len(child)-1 { + t.Errorf("ERR Could not match child offset %s with any parent time offset", child[ci].TimeOffset) + } + if p.TimeOffset == child[ci].TimeOffset { + t.Logf("Setting sub to %d at t=%s", child[ci].PlannedVUs, child[ci].TimeOffset) + sub = child[ci].PlannedVUs + if ci != len(child)-1 { + ci++ + } + } + t.Logf("Subtracting %d VUs (out of %d) at t=%s", sub, p.PlannedVUs, p.TimeOffset) + parent[pi].PlannedVUs -= sub + } + } + + for i := 0; i < numTests; i++ { + name := fmt.Sprintf("random%02d", i) + t.Run(name, func(t *testing.T) { + c := getTestConfig(name) + ranSeqLen := 2 + r.Int63n(segmentSeqMaxLen-1) + t.Logf("Config: %#v, ranSeqLen: %d", c, ranSeqLen) + randomSequence := generateRandomSequence(t, ranSeqLen, maxNumerator, r) + t.Logf("Random sequence: %s", randomSequence) + fullSeg, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + fullRawSteps := c.getRawExecutionSteps(fullSeg, false) + + for _, step := range fullRawSteps { + t.Logf("original planned VUs for time offset %s: %d", step.TimeOffset, step.PlannedVUs) + } + + for s := 0; s < len(randomSequence); s++ { + et, err := lib.NewExecutionTuple(randomSequence[s], &randomSequence) + require.NoError(t, err) + segRawSteps := c.getRawExecutionSteps(et, false) + subtractChildSteps(t, fullRawSteps, segRawSteps) + } + + for _, step := range fullRawSteps { + if step.PlannedVUs != 0 { + t.Errorf("ERR Remaining planned VUs for time offset %s are not 0 but %d", step.TimeOffset, step.PlannedVUs) + } + } + }) + } +} From 6e8686181280d6312ecb51e331e4a0f8165c182b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Sat, 25 Apr 2020 15:36:33 +0300 Subject: [PATCH 223/350] Fix wrongly preallocating the offsetsCache by 1 smaller MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit As we have the start in that offset and also the offset to go from the previous cycle to the next it should be the numerator + 1. name old time/op new time/op delta GetStripedOffsets/length10,seed777-24 12.3µs ±22% 11.4µs ±11% -7.22% (p=0.020 n=20+17) GetStripedOffsets/length100,seed777-24 2.73ms ±12% 2.67ms ±16% ~ (p=0.461 n=20+20) GetStripedOffsetsEven/length10-24 6.51µs ± 3% 6.00µs ± 4% -7.83% (p=0.000 n=20+20) GetStripedOffsetsEven/length100-24 99.3µs ± 2% 92.6µs ± 2% -6.75% (p=0.000 n=20+20) GetStripedOffsetsEven/length1000-24 7.20ms ± 1% 6.76ms ± 1% -6.07% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-24 4.69ns ± 0% 4.69ns ± 0% ~ (p=0.199 n=19+18) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-24 1.04µs ± 3% 1.02µs ± 2% -1.78% (p=0.000 n=20+19) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-24 6.85ns ± 0% 6.86ns ± 0% ~ (p=0.851 n=16+15) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-24 4.69ns ± 0% 4.69ns ± 0% ~ (p=0.440 n=13+12) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-24 1.03µs ± 3% 1.03µs ± 3% ~ (p=0.784 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-24 6.85ns ± 0% 6.85ns ± 0% ~ (p=0.392 n=16+12) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-24 4.69ns ± 0% 4.69ns ± 0% ~ (p=0.308 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-24 1.04µs ± 3% 1.03µs ± 3% ~ (p=0.457 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-24 6.86ns ± 1% 6.86ns ± 0% ~ (p=0.594 n=20+16) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-24 4.69ns ± 0% 4.69ns ± 0% ~ (p=0.100 n=15+15) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-24 1.03µs ± 3% 1.03µs ± 2% ~ (p=0.075 n=20+20) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-24 6.85ns ± 1% 6.85ns ± 0% ~ (p=0.481 n=19+12) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-24 1.64µs ± 2% 1.65µs ± 2% ~ (p=0.194 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-24 1.67µs ± 4% 1.68µs ± 3% ~ (p=0.208 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-24 6.85ns ± 0% 6.85ns ± 0% ~ (p=0.114 n=19+19) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-24 1.66µs ± 2% 1.64µs ± 3% -1.26% (p=0.003 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-24 1.66µs ± 3% 1.65µs ± 3% ~ (p=0.420 n=18+19) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-24 6.85ns ± 0% 6.85ns ± 0% ~ (p=0.401 n=17+18) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-24 1.65µs ± 3% 1.66µs ± 2% ~ (p=0.206 n=20+19) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-24 1.66µs ± 3% 1.66µs ± 4% ~ (p=0.433 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-24 6.85ns ± 0% 6.85ns ± 0% ~ (p=0.883 n=17+15) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-24 1.65µs ± 3% 1.65µs ± 3% ~ (p=0.794 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-24 1.64µs ± 3% 1.65µs ± 4% ~ (p=0.155 n=20+20) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-24 6.85ns ± 0% 6.85ns ± 0% ~ (p=0.087 n=16+15) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-24 2.10µs ± 3% 2.12µs ± 3% ~ (p=0.208 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-24 4.03µs ± 1% 3.47µs ± 4% -14.02% (p=0.000 n=16+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-24 27.3ns ± 1% 27.3ns ± 1% ~ (p=0.514 n=19+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-24 2.02µs ± 3% 2.02µs ± 3% ~ (p=0.651 n=20+19) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-24 4.10µs ± 3% 3.43µs ± 2% -16.35% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-24 25.4ns ± 1% 26.0ns ± 0% +2.21% (p=0.000 n=19+16) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-24 1.76µs ± 3% 1.73µs ± 3% -1.65% (p=0.001 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-24 4.07µs ± 4% 3.47µs ± 2% -14.77% (p=0.000 n=20+19) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-24 24.2ns ± 0% 24.2ns ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-24 1.94µs ± 2% 1.92µs ± 2% -0.95% (p=0.008 n=19+19) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-24 4.09µs ± 3% 3.50µs ± 3% -14.38% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-24 26.5ns ± 1% 26.4ns ± 1% -0.23% (p=0.013 n=18+19) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-24 1.94µs ± 3% 1.97µs ± 2% +1.28% (p=0.004 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-24 5.04µs ± 2% 4.46µs ± 1% -11.49% (p=0.000 n=20+18) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-24 26.0ns ± 1% 26.0ns ± 1% ~ (p=0.700 n=18+17) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-24 1.96µs ± 2% 1.92µs ± 3% -1.66% (p=0.000 n=20+19) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-24 5.02µs ± 2% 4.54µs ± 3% -9.61% (p=0.000 n=20+19) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-24 26.0ns ± 1% 26.0ns ± 1% ~ (p=0.625 n=19+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-24 1.74µs ± 3% 1.73µs ± 2% ~ (p=0.365 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-24 4.99µs ± 2% 4.54µs ± 3% -9.05% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-24 24.2ns ± 0% 24.2ns ± 0% ~ (p=0.085 n=18+14) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-24 1.89µs ± 3% 1.89µs ± 2% ~ (p=0.776 n=20+19) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-24 4.95µs ± 4% 4.51µs ± 3% -8.91% (p=0.000 n=19+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-24 26.4ns ± 1% 26.4ns ± 1% ~ (p=0.985 n=19+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-24 2.53µs ± 3% 2.54µs ± 3% ~ (p=0.331 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-24 5.09µs ± 4% 4.89µs ± 3% -3.95% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-24 22.9ns ± 0% 22.9ns ± 0% ~ (p=0.644 n=12+19) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-24 2.75µs ± 3% 2.74µs ± 3% ~ (p=0.449 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-24 5.15µs ± 3% 4.92µs ± 2% -4.36% (p=0.000 n=20+19) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-24 26.0ns ± 0% 26.0ns ± 0% ~ (p=0.794 n=16+17) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-24 2.54µs ± 3% 2.53µs ± 2% ~ (p=0.311 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-24 5.11µs ± 3% 4.91µs ± 4% -4.06% (p=0.000 n=19+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-24 24.2ns ± 0% 24.2ns ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-24 2.85µs ± 4% 2.91µs ± 4% +1.94% (p=0.001 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-24 5.15µs ± 3% 4.88µs ± 3% -5.21% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-24 24.4ns ± 1% 24.4ns ± 1% ~ (p=0.349 n=19+18) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-24 2.66µs ± 3% 2.68µs ± 3% ~ (p=0.071 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-24 1.62ms ± 5% 1.31ms ± 2% -19.01% (p=0.000 n=20+18) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-24 27.3ns ± 1% 27.3ns ± 1% ~ (p=0.059 n=18+18) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-24 2.98µs ± 3% 3.00µs ± 4% ~ (p=0.449 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-24 1.61ms ± 3% 1.31ms ± 2% -18.68% (p=0.000 n=19+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-24 2.25µs ± 0% 2.25µs ± 0% ~ (p=0.657 n=20+18) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-24 2.77µs ± 3% 2.76µs ± 4% ~ (p=0.588 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-24 1.62ms ± 3% 1.32ms ± 3% -18.93% (p=0.000 n=19+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-24 8.72µs ± 0% 8.72µs ± 0% ~ (p=0.167 n=16+16) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-24 3.10µs ± 3% 3.08µs ± 3% ~ (p=0.280 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-24 1.60ms ± 3% 1.32ms ± 4% -17.54% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-24 5.45µs ± 0% 5.46µs ± 1% +0.14% (p=0.034 n=19+20) name old alloc/op new alloc/op delta GetStripedOffsets/length10,seed777-24 2.57kB ±14% 2.41kB ±11% -6.26% (p=0.018 n=20+20) GetStripedOffsets/length100,seed777-24 105kB ±11% 103kB ±11% ~ (p=0.565 n=20+20) GetStripedOffsetsEven/length10-24 1.86kB ± 0% 1.78kB ± 0% -4.31% (p=0.000 n=20+20) GetStripedOffsetsEven/length100-24 10.3kB ± 0% 9.5kB ± 0% -7.76% (p=0.000 n=20+20) GetStripedOffsetsEven/length1000-24 83.6kB ± 0% 75.6kB ± 0% -9.57% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-24 424B ± 0% 424B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-24 1.06kB ± 0% 0.96kB ± 0% -9.77% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-24 424B ± 0% 424B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-24 1.06kB ± 0% 0.96kB ± 0% -9.77% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-24 1.06kB ± 0% 0.96kB ± 0% -9.77% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-24 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-24 1.06kB ± 0% 0.96kB ± 0% -9.77% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-24 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-24 1.21kB ± 0% 1.13kB ± 0% -6.62% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-24 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-24 1.21kB ± 0% 1.13kB ± 0% -6.62% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-24 320B ± 0% 320B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-24 1.21kB ± 0% 1.13kB ± 0% -6.62% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-24 376B ± 0% 376B ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-24 1.21kB ± 0% 1.13kB ± 0% -6.62% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-24 1.43kB ± 0% 1.39kB ± 0% -2.79% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-24 624B ± 0% 624B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-24 1.43kB ± 0% 1.39kB ± 0% -2.79% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-24 512B ± 0% 512B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-24 1.43kB ± 0% 1.39kB ± 0% -2.79% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-24 672B ± 0% 672B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-24 1.43kB ± 0% 1.39kB ± 0% -2.79% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-24 568B ± 0% 568B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-24 493kB ± 0% 222kB ± 0% -54.85% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-24 672B ± 0% 672B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-24 493kB ± 0% 222kB ± 0% -54.85% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-24 568B ± 0% 568B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-24 493kB ± 0% 222kB ± 0% -54.85% (p=0.000 n=18+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-24 0.00B 0.00B ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-24 720B ± 0% 720B ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-24 493kB ± 0% 222kB ± 0% -54.85% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-24 0.00B 0.00B ~ (all equal) name old allocs/op new allocs/op delta GetStripedOffsets/length10,seed777-24 43.0 ± 7% 39.1 ± 7% -9.18% (p=0.000 n=20+20) GetStripedOffsets/length100,seed777-24 232 ± 6% 229 ± 5% ~ (p=0.203 n=20+20) GetStripedOffsetsEven/length10-24 47.0 ± 0% 37.0 ± 0% -21.28% (p=0.000 n=20+20) GetStripedOffsetsEven/length100-24 239 ± 0% 139 ± 0% -41.84% (p=0.000 n=20+20) GetStripedOffsetsEven/length1000-24 2.05k ± 0% 1.05k ± 0% -48.76% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:/segment.Scale(5)-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)-24 10.0 ± 0% 10.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5523)-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)-24 10.0 ± 0% 10.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(5000000)-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)-24 10.0 ± 0% 10.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/segment.Scale(67280421310721)-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)-24 10.0 ± 0% 10.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)-24 14.0 ± 0% 14.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5523)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)-24 14.0 ± 0% 14.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(5000000)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)-24 14.0 ± 0% 14.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/segment.Scale(67280421310721)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)-24 14.0 ± 0% 14.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:0:1/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5)-24 22.0 ± 0% 22.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)-24 29.0 ± 0% 22.0 ± 0% -24.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5523)-24 22.0 ± 0% 22.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)-24 29.0 ± 0% 22.0 ± 0% -24.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(5000000)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)-24 29.0 ± 0% 22.0 ± 0% -24.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/segment.Scale(67280421310721)-24 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)-24 29.0 ± 0% 22.0 ± 0% -24.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.3,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.3/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5)-24 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)-24 36.0 ± 0% 26.0 ± 0% -27.78% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5523)-24 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)-24 36.0 ± 0% 26.0 ± 0% -27.78% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(5000000)-24 19.0 ± 0% 19.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)-24 36.0 ± 0% 26.0 ± 0% -27.78% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/segment.Scale(67280421310721)-24 21.0 ± 0% 21.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)-24 36.0 ± 0% 26.0 ± 0% -27.78% (p=0.000 n=20+20) ExecutionSegmentScale/seq:0,0.1,0.2,0.3,0.4,0.5,0.6,0.7,0.8,0.9,1;segment:0:0.1/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5)-24 28.0 ± 0% 28.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5523)-24 32.0 ± 0% 32.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(5000000)-24 28.0 ± 0% 28.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/segment.Scale(67280421310721)-24 33.0 ± 0% 33.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2/5:4/5/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5)-24 30.0 ± 0% 30.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5523)-24 33.0 ± 0% 33.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5523)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(5000000)-24 30.0 ± 0% 30.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(5000000)_prefilled-24 0.00 0.00 ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/segment.Scale(67280421310721)-24 34.0 ± 0% 34.0 ± 0% ~ (all equal) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)-24 42.0 ± 0% 39.0 ± 0% -7.14% (p=0.000 n=20+20) ExecutionSegmentScale/seq:;segment:2235/5213:4/5/et.Scale(67280421310721)_prefilled-24 0.00 0.00 ~ (all equal) --- lib/execution_segment.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index a4a6cf27136..1d0410052b7 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -608,7 +608,7 @@ func (et *ExecutionTuple) fillCache() { et.offsetsCache = make([][]int64, len(et.sequence)) for i := range et.offsetsCache { - et.offsetsCache[i] = make([]int64, 0, wrapper.slice[i].numerator) + et.offsetsCache[i] = make([]int64, 0, wrapper.slice[i].numerator+1) } var prev = make([]int64, len(et.sequence)) From ad190b2874c8df4bf765c51a604530ac35e06227 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 28 Apr 2020 16:53:31 +0300 Subject: [PATCH 224/350] Fix TestSumRandomSegmentSequenceMatchesNoSegment --- lib/executor/variable_looping_vus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 92bcc481560..d0a39ab873c 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -1048,7 +1048,7 @@ func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { stagesCount := 1 + r.Int31n(maxStages) stages := make([]Stage, stagesCount) for s := int32(0); s < stagesCount; s++ { - dur := time.Duration(r.Int63n(int64(maxStageDuration - minStageDuration))).Round(time.Second) + dur := (minStageDuration + time.Duration(r.Int63n(int64(maxStageDuration-minStageDuration)))).Round(time.Second) stages[s] = Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} } From 5e7457b77e370a27e44d1ded7c7afdd0f0430b97 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 28 Apr 2020 17:02:30 +0300 Subject: [PATCH 225/350] Fix checking the wrong error when initing collectors --- cmd/run.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/run.go b/cmd/run.go index 0995f78ae4d..e27a1cc5496 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -214,7 +214,7 @@ a commandline interface for interacting with it.`, if cerr != nil { return cerr } - if cerr = collector.Init(); err != nil { + if cerr = collector.Init(); cerr != nil { return cerr } engine.Collectors = append(engine.Collectors, collector) From 79e1438b6c6a3c9c75cd30495a784a19d3d4cddf Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 29 Apr 2020 10:38:47 +0300 Subject: [PATCH 226/350] =?UTF-8?q?GetNewExecutionTupleBasedOnValue=20shou?= =?UTF-8?q?ld=20return=20the=20same=20ExecutionTup=E2=80=A6=20(#1420)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-Authored-By: Ivan Mirić --- lib/execution_segment.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 1d0410052b7..7440d4b7f85 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -650,13 +650,17 @@ func (et *ExecutionTuple) GetStripedOffsets(segment *ExecutionSegment) (int64, [ // original, if that segmetn would've been with length 0 then it is nil, and obviously isn't part of // the sequence. func (et *ExecutionTuple) GetNewExecutionTupleBasedOnValue(value int64) *ExecutionTuple { + et.once.Do(et.fillCache) + if value != 0 && value%et.lcd == 0 { // the value is perfectly divisible so we will get the same tuple + return et + } + var ( newESS = make(ExecutionSegmentSequence, 0, len(et.sequence)) // this can be smaller newES *ExecutionSegment esIndex = -1 + prev int64 ) - et.once.Do(et.fillCache) - var prev int64 for i := range et.sequence { offsets := et.offsetsCache[i] newValue := scaleInt64(value, offsets[0], offsets[1:], et.lcd) From 4892443f9b90b82c6dc61669b1a787c65efbb6a0 Mon Sep 17 00:00:00 2001 From: na-- Date: Wed, 29 Apr 2020 17:40:16 +0300 Subject: [PATCH 227/350] Properly synchronize VU deactivation and fix some tests (#1424) --- js/runner.go | 35 +++++++++++++++++++++++++++-------- js/runner_test.go | 7 +++---- 2 files changed, 30 insertions(+), 12 deletions(-) diff --git a/js/runner.go b/js/runner.go index a32b06c5197..bfa8c6f9da6 100644 --- a/js/runner.go +++ b/js/runner.go @@ -28,7 +28,6 @@ import ( "net/http" "net/http/cookiejar" "strconv" - "sync" "time" "github.com/dop251/goja" @@ -197,7 +196,6 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, Console: r.console, BPool: bpool.NewBufferPool(100), Samples: samplesOut, - runMutex: sync.Mutex{}, } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) @@ -368,18 +366,20 @@ type VU struct { Samples chan<- stats.SampleContainer - runMutex sync.Mutex setupData goja.Value } // Verify that interfaces are implemented -var _ lib.ActiveVU = &ActiveVU{} -var _ lib.InitializedVU = &VU{} +var ( + _ lib.ActiveVU = &ActiveVU{} + _ lib.InitializedVU = &VU{} +) // ActiveVU holds a VU and its activation parameters type ActiveVU struct { *VU *lib.VUActivationParams + busy chan struct{} } // Activate the VU so it will be able to run code. @@ -387,21 +387,40 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() // u.Env = params.Env + avu := &ActiveVU{ + VU: u, + VUActivationParams: params, + busy: make(chan struct{}, 1), + } + go func() { + // Wait for the run context to be over <-params.RunContext.Done() + // Interrupt the JS runtime u.Runtime.Interrupt(errInterrupt) + // Wait for the VU to stop running, if it was, and prevent it from + // running again for this activation + avu.busy <- struct{}{} + if params.DeactivateCallback != nil { params.DeactivateCallback() } }() - return &ActiveVU{u, params} + return avu } // RunOnce runs the default function once. func (u *ActiveVU) RunOnce() error { - u.runMutex.Lock() - defer u.runMutex.Unlock() + select { + case <-u.RunContext.Done(): + return u.RunContext.Err() // we are done, return + case u.busy <- struct{}{}: + // nothing else can run now, and the VU cannot be deactivated + } + defer func() { + <-u.busy // unlock deactivation again + }() // Unmarshall the setupData only the first time for each VU so that VUs are isolated but we // still don't use too much CPU in the middle test diff --git a/js/runner_test.go b/js/runner_test.go index 92182933cb4..15d24d1d80d 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -32,7 +32,6 @@ import ( "net" "net/http" "os" - "strings" "sync" "testing" "time" @@ -559,7 +558,7 @@ func TestVURunInterrupt(t *testing.T) { for name, r := range testdata { name, r := name, r t.Run(name, func(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + ctx, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) defer cancel() samples := make(chan stats.SampleContainer, 100) defer close(samples) @@ -574,7 +573,7 @@ func TestVURunInterrupt(t *testing.T) { activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) err = activeVU.RunOnce() assert.Error(t, err) - assert.True(t, strings.HasPrefix(err.Error(), "context cancelled at ")) + assert.Contains(t, err.Error(), "context cancelled") }) } } @@ -625,8 +624,8 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { <-ch time.Sleep(time.Millisecond * 1) // NOTE: increase this in case of problems ;) newCancel() + wg.Wait() } - wg.Wait() }) } } From d2a6f8a1c124eef815112a88abdee8851e0a64d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 29 Apr 2020 16:52:12 +0200 Subject: [PATCH 228/350] Ignore funlen linter for SharedIterations.Run --- lib/executor/shared_iterations.go | 1 + 1 file changed, 1 insertion(+) diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 1fbad262d00..1346de5ae3b 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -155,6 +155,7 @@ func (sic SharedIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { // Run executes a specific total number of iterations, which are all shared by // the configured VUs. +// nolint:funlen func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := si.config.GetVUs(si.executionState.ExecutionTuple) iterations := si.config.GetIterations(si.executionState.ExecutionTuple) From 62b519e8b3d0ed4b731cd13724edaa6675baddde Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 29 Apr 2020 17:01:25 +0200 Subject: [PATCH 229/350] Fix TestVariableLoopingVUsRampDownNoWobble We should be at 10 VUs at exactly t=3s, so the additional 100ms of head start is enough. --- lib/executor/variable_looping_vus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index 4ebed71ba2c..c67794b6fbf 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -134,7 +134,7 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { sampleTimes := []time.Duration{ 100 * time.Millisecond, - 3200 * time.Millisecond, + 3000 * time.Millisecond, } const rampDownSamples = 50 From f7cf5a6ac948d392c553e8f2282bac8cbc67ebd1 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Apr 2020 12:38:22 +0300 Subject: [PATCH 230/350] Fix TestVURunInterrupt by creating the context just before it's used Previously what was happening that the context could timeout by the time RunOnce is called which will lead to it seeing that and exiting with the context error (timeout in this case). This is easily fixable by creating the context just before it is used. Which also makes it possible to decrease the context timeout significantly, as it now doesn't need to last newVU for example. --- js/runner_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/js/runner_test.go b/js/runner_test.go index 15d24d1d80d..ad88b95e941 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -558,8 +558,6 @@ func TestVURunInterrupt(t *testing.T) { for name, r := range testdata { name, r := name, r t.Run(name, func(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) - defer cancel() samples := make(chan stats.SampleContainer, 100) defer close(samples) go func() { @@ -570,6 +568,8 @@ func TestVURunInterrupt(t *testing.T) { vu, err := r.newVU(1, samples) require.NoError(t, err) + ctx, cancel := context.WithTimeout(context.Background(), 20*time.Millisecond) + defer cancel() activeVU := vu.Activate(&lib.VUActivationParams{RunContext: ctx}) err = activeVU.RunOnce() assert.Error(t, err) From 30bd86326de996e0cfd04c13a2903280303b67ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 30 Apr 2020 12:53:02 +0200 Subject: [PATCH 231/350] Add VU deactivation synchronization to MiniRunner This replicates the approach used for js.ActiveVU in 4892443f. --- lib/testutils/minirunner/minirunner.go | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index cc6197c4bf8..3dc88b9a462 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -118,18 +118,30 @@ type VU struct { type ActiveVU struct { *VU *lib.VUActivationParams + busy chan struct{} } // Activate the VU so it will be able to run code. func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { + avu := &ActiveVU{ + VU: vu, + VUActivationParams: params, + busy: make(chan struct{}, 1), + } + go func() { <-params.RunContext.Done() + + // Wait for the VU to stop running, if it was, and prevent it from + // running again for this activation + avu.busy <- struct{}{} + if params.DeactivateCallback != nil { params.DeactivateCallback() } }() - return &ActiveVU{vu, params} + return avu } // RunOnce runs the mock default function once, incrementing its iteration. @@ -138,6 +150,16 @@ func (vu *ActiveVU) RunOnce() error { return nil } + select { + case <-vu.RunContext.Done(): + return vu.RunContext.Err() // we are done, return + case vu.busy <- struct{}{}: + // nothing else can run now, and the VU cannot be deactivated + } + defer func() { + <-vu.busy // unlock deactivation again + }() + state := &lib.State{ Vu: vu.ID, Iteration: vu.Iteration, From a644c134ba85aa9901def9f7c3443c35e5542780 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 28 Apr 2020 13:19:52 +0300 Subject: [PATCH 232/350] Use the striping algorithm for shared iterations Unfortunately because of the way the ExecutorConfig interface is designed I can't just cache some of the results which will result in some probably unneeded amount of calculation. This will need either a change to the interface or for ExecutionTuple to cache the results of GetNewExecutionTupleBasedOnValue, but I find it unlikely this to be noticeable outside of extreme examples closes #1308 --- core/local/local_test.go | 3 ++- lib/executor/executors_test.go | 40 +++++++++++++++++++++++++++++++ lib/executor/shared_iterations.go | 30 ++++++++++++++++++----- 3 files changed, 66 insertions(+), 7 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index edd64851ecc..c0ab0e24f57 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -736,7 +736,8 @@ func TestNewExecutionSchedulerHasWork(t *testing.T) { import http from 'k6/http'; export let options = { - executionSegment: "2/4:3/4", + executionSegment: "3/4:1", + executionSegmentSequence: "0,1/4,2/4,3/4,1", execution: { shared_iters1: { type: "shared-iterations", diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index d3eaac4e016..9e8421fc9e0 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -232,6 +232,46 @@ var configMapTestCases = []configMapTestCase{ totalReqs := cm.GetFullExecutionRequirements(et) assert.Equal(t, schedReqs, totalReqs) + + et = mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")) + assert.Equal(t, "8 iterations shared among 4 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(et)) + + schedReqs = cm["ishared"].GetExecutionRequirements(et) + endOffset, isFinal = lib.GetEndOffset(schedReqs) + assert.Equal(t, 130*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(4), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(4), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs = cm.GetFullExecutionRequirements(et) + assert.Equal(t, schedReqs, totalReqs) + + et = mustNewExecutionTuple(newExecutionSegmentFromString("1/3:2/3"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")) + assert.Equal(t, "7 iterations shared among 4 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(et)) + + schedReqs = cm["ishared"].GetExecutionRequirements(et) + endOffset, isFinal = lib.GetEndOffset(schedReqs) + assert.Equal(t, 130*time.Second, endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(4), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(4), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs = cm.GetFullExecutionRequirements(et) + assert.Equal(t, schedReqs, totalReqs) + + et = mustNewExecutionTuple(newExecutionSegmentFromString("12/13:1"), + newExecutionSegmentSequenceFromString("0,1/13,2/13,3/13,4/13,5/13,6/13,7/13,8/13,9/13,10/13,11/13,12/13,1")) + assert.Equal(t, "0 iterations shared among 0 VUs (maxDuration: 1m40s, gracefulStop: 30s)", cm["ishared"].GetDescription(et)) + + schedReqs = cm["ishared"].GetExecutionRequirements(et) + endOffset, isFinal = lib.GetEndOffset(schedReqs) + assert.Equal(t, time.Duration(0), endOffset) + assert.Equal(t, true, isFinal) + assert.Equal(t, uint64(0), lib.GetMaxPlannedVUs(schedReqs)) + assert.Equal(t, uint64(0), lib.GetMaxPossibleVUs(schedReqs)) + + totalReqs = cm.GetFullExecutionRequirements(et) + assert.Equal(t, schedReqs, totalReqs) }}, }, {`{"ishared": {"type": "shared-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 04af864484a..6fe83bf11de 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -72,12 +72,13 @@ var _ lib.ExecutorConfig = &SharedIterationsConfig{} // GetVUs returns the scaled VUs for the executor. func (sic SharedIterationsConfig) GetVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(sic.VUs.Int64) + return et.ScaleInt64(sic.VUs.Int64) } // GetIterations returns the scaled iteration count for the executor. func (sic SharedIterationsConfig) GetIterations(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(sic.Iterations.Int64) + // Optimize this by probably changing the whole Config API + return et.GetNewExecutionTupleBasedOnValue(sic.VUs.Int64).ScaleInt64(sic.Iterations.Int64) } // GetDescription returns a human-readable description of the executor options @@ -116,10 +117,20 @@ func (sic SharedIterationsConfig) Validate() []error { // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. func (sic SharedIterationsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { + vus := sic.GetVUs(et) + if vus == 0 { + return []lib.ExecutionStep{ + { + TimeOffset: 0, + PlannedVUs: 0, + }, + } + } + return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(sic.GetVUs(et)), + PlannedVUs: uint64(vus), }, { TimeOffset: time.Duration(sic.MaxDuration.Duration + sic.GracefulStop.Duration), @@ -132,7 +143,7 @@ func (sic SharedIterationsConfig) GetExecutionRequirements(et *lib.ExecutionTupl func (sic SharedIterationsConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { - return SharedIterations{ + return &SharedIterations{ BaseExecutor: NewBaseExecutor(sic, es, logger), config: sic, }, nil @@ -143,6 +154,7 @@ func (sic SharedIterationsConfig) NewExecutor( type SharedIterations struct { *BaseExecutor config SharedIterationsConfig + et *lib.ExecutionTuple } // Make sure we implement the lib.Executor interface. @@ -153,11 +165,17 @@ func (sic SharedIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { return sic.GetVUs(et) > 0 && sic.GetIterations(et) > 0 } +// Init values needed for the execution +func (si *SharedIterations) Init(ctx context.Context) error { + si.et = si.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(si.config.VUs.Int64) + return nil +} + // Run executes a specific total number of iterations, which are all shared by // the configured VUs. func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := si.config.GetVUs(si.executionState.ExecutionTuple) - iterations := si.config.GetIterations(si.executionState.ExecutionTuple) + iterations := si.et.ScaleInt64(si.config.Iterations.Int64) duration := time.Duration(si.config.MaxDuration.Duration) gracefulStop := si.config.GetGracefulStop() @@ -186,7 +204,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta return float64(currentDoneIters) / float64(totalIters), right } si.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, si, progresFn) + go trackProgress(ctx, maxDurationCtx, regDurationCtx, &si, progresFn) // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} From e693715b13aa94fc922aaf57b6d1e2ee69139f6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 17 Apr 2020 12:14:27 +0200 Subject: [PATCH 233/350] Add support for non-default function execution See #1300 --- core/local/local.go | 35 ++++--- core/local/local_test.go | 84 +++++++++++++++- js/bundle.go | 130 +++++++++++++++---------- js/bundle_test.go | 29 +++--- js/initcontext_test.go | 4 +- js/runner.go | 26 ++++- lib/executor/constant_arrival_rate.go | 2 + lib/executor/constant_looping_vus.go | 2 + lib/executor/externally_controlled.go | 14 ++- lib/executor/per_vu_iterations.go | 2 + lib/executor/shared_iterations.go | 3 + lib/executor/variable_arrival_rate.go | 2 + lib/executor/variable_looping_vus.go | 4 +- lib/executor/vu_handle.go | 5 +- lib/runner.go | 18 +++- lib/testutils/minirunner/minirunner.go | 6 ++ 16 files changed, 265 insertions(+), 101 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index 7b1ab217664..7f916939db8 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -224,6 +224,30 @@ func (e *ExecutionScheduler) initVUsConcurrently( func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.SampleContainer) error { logger := e.logger.WithField("phase", "local-execution-scheduler-init") + // Initialize each executor and do some basic validation. + e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) + logger.Debugf("Start initializing executors...") + errMsg := "error while initializing executor %s: %s" + exports := e.runner.GetExports() + for _, exec := range e.executors { + executorConfig := exec.GetConfig() + execFn := executorConfig.GetExec().ValueOrZero() + execName := executorConfig.GetName() + + if execFn == "" { + execFn = "default" + } + if _, ok := exports[execFn]; !ok { + return fmt.Errorf(errMsg, execName, + fmt.Sprintf("function '%s' not found in exports", execFn)) + } + if err := exec.Init(ctx); err != nil { + return fmt.Errorf(errMsg, executorConfig.GetName(), err) + } + logger.Debugf("Initialized executor %s", executorConfig.GetName()) + } + + logger.Debugf("Finished initializing executors, start initializing VUs...") vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) logger.WithFields(logrus.Fields{ "neededVUs": vusToInitialize, @@ -265,17 +289,6 @@ func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.S return e.initVU(samplesOut, logger) }) - e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) - logger.Debugf("Finished initializing needed VUs, start initializing executors...") - for _, exec := range e.executors { - executorConfig := exec.GetConfig() - - if err := exec.Init(ctx); err != nil { - return fmt.Errorf("error while initializing executor %s: %s", executorConfig.GetName(), err) - } - logger.Debugf("Initialized executor %s", executorConfig.GetName()) - } - e.state.SetExecutionStatus(lib.ExecutionStatusInitDone) logger.Debugf("Initialization completed") return nil diff --git a/core/local/local_test.go b/core/local/local_test.go index c0ab0e24f57..4bc6c32a5ee 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -30,6 +30,12 @@ import ( "testing" "time" + "github.com/sirupsen/logrus" + logtest "github.com/sirupsen/logrus/hooks/test" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/executor" @@ -40,11 +46,6 @@ import ( "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" - logtest "github.com/sirupsen/logrus/hooks/test" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" ) func newTestExecutionScheduler( @@ -96,6 +97,79 @@ func TestExecutionSchedulerRun(t *testing.T) { assert.NoError(t, <-err) } +func TestExecutionSchedulerRunNonDefault(t *testing.T) { + t.Parallel() + + testCases := []struct { + name, script, expErr string + }{ + {"defaultOK", `export default function () {}`, ""}, + {"nonDefaultOK", ` + export let options = { + execution: { + per_vu_iters: { + type: "per-vu-iterations", + vus: 1, + iterations: 1, + exec: "nonDefault", + }, + } + } + export function nonDefault() {}`, ""}, + {"nonDefaultErr", ` + export let options = { + execution: { + per_vu_iters: { + type: "per-vu-iterations", + vus: 1, + iterations: 1, + exec: "nonDefaultErr", + }, + } + } + export function nonDefault() {}`, + "error while initializing executor per_vu_iters: function 'nonDefaultErr' not found in exports"}, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + runner, err := js.New(&loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, Data: []byte(tc.script)}, + nil, lib.RuntimeOptions{}) + require.NoError(t, err) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + done := make(chan struct{}) + samples := make(chan stats.SampleContainer) + go func() { + err := execScheduler.Init(ctx, samples) + if tc.expErr != "" { + assert.EqualError(t, err, tc.expErr) + } else { + assert.NoError(t, err) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) + } + close(done) + }() + for { + select { + case <-samples: + case <-done: + return + } + } + }) + } +} + func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Parallel() t.Run("Normal", func(t *testing.T) { diff --git a/js/bundle.go b/js/bundle.go index 4d1b33ec30c..104e1a56fdb 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -29,13 +29,14 @@ import ( "github.com/loadimpact/k6/lib/consts" "github.com/dop251/goja" + "github.com/pkg/errors" + "github.com/spf13/afero" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/js/compiler" jslib "github.com/loadimpact/k6/js/lib" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" - "github.com/pkg/errors" - "github.com/spf13/afero" ) // A Bundle is a self-contained bundle of scripts and resources. @@ -44,7 +45,9 @@ type Bundle struct { Filename *url.URL Source string Program *goja.Program - Options lib.Options + // exported functions, for validation only + Exports map[string]struct{} + Options lib.Options BaseInitContext *InitContext @@ -56,7 +59,8 @@ type Bundle struct { type BundleInstance struct { Runtime *goja.Runtime Context *context.Context - Default goja.Callable + // exported functions, ready for execution + Exports map[string]goja.Callable } // NewBundle creates a new bundle from a source file and a filesystem. @@ -79,6 +83,7 @@ func NewBundle(src *loader.SourceData, filesystems map[string]afero.Fs, rtOpts l Filename: src.URL, Source: code, Program: pgm, + Exports: make(map[string]struct{}), BaseInitContext: NewInitContext(rt, c, compatMode, new(context.Context), filesystems, loader.Dir(src.URL)), Env: rtOpts.Env, @@ -88,44 +93,9 @@ func NewBundle(src *loader.SourceData, filesystems map[string]afero.Fs, rtOpts l return nil, err } - // Grab exports. - exportsV := rt.Get("exports") - if goja.IsNull(exportsV) || goja.IsUndefined(exportsV) { - return nil, errors.New("exports must be an object") - } - exports := exportsV.ToObject(rt) - - // Validate the default function. - def := exports.Get("default") - if def == nil || goja.IsNull(def) || goja.IsUndefined(def) { - return nil, errors.New("script must export a default function") - } - if _, ok := goja.AssertFunction(def); !ok { - return nil, errors.New("default export must be a function") - } - - // Extract/validate other exports. - for _, k := range exports.Keys() { - v := exports.Get(k) - switch k { - case "default": // Already checked above. - case "options": - data, err := json.Marshal(v.Export()) - if err != nil { - return nil, err - } - if err := json.Unmarshal(data, &bundle.Options); err != nil { - return nil, err - } - case "setup": - if _, ok := goja.AssertFunction(v); !ok { - return nil, errors.New("exported 'setup' must be a function") - } - case "teardown": - if _, ok := goja.AssertFunction(v); !ok { - return nil, errors.New("exported 'teardown' must be a function") - } - } + err = bundle.getExports(rt) + if err != nil { + return nil, err } return &bundle, nil @@ -153,8 +123,8 @@ func NewBundleFromArchive(arc *lib.Archive, rtOpts lib.RuntimeOptions) (*Bundle, if err != nil { return nil, err } - - initctx := NewInitContext(goja.New(), c, compatMode, + rt := goja.New() + initctx := NewInitContext(rt, c, compatMode, new(context.Context), arc.Filesystems, arc.PwdURL) env := arc.Env @@ -170,14 +140,22 @@ func NewBundleFromArchive(arc *lib.Archive, rtOpts lib.RuntimeOptions) (*Bundle, Filename: arc.FilenameURL, Source: string(arc.Data), Program: pgm, + Exports: make(map[string]struct{}), Options: arc.Options, BaseInitContext: initctx, Env: env, CompatibilityMode: compatMode, } - if err := bundle.instantiate(bundle.BaseInitContext.runtime, bundle.BaseInitContext); err != nil { + + if err = bundle.instantiate(rt, bundle.BaseInitContext); err != nil { + return nil, err + } + + err = bundle.getExports(rt) + if err != nil { return nil, err } + return bundle, nil } @@ -202,6 +180,47 @@ func (b *Bundle) makeArchive() *lib.Archive { return arc } +// getExports validates and extracts exported objects +func (b *Bundle) getExports(rt *goja.Runtime) error { + exportsV := rt.Get("exports") + if goja.IsNull(exportsV) || goja.IsUndefined(exportsV) { + return errors.New("exports must be an object") + } + exports := exportsV.ToObject(rt) + + for _, k := range exports.Keys() { + v := exports.Get(k) + switch k { + case "options": + data, err := json.Marshal(v.Export()) + if err != nil { + return err + } + if err := json.Unmarshal(data, &b.Options); err != nil { + return err + } + case "setup": + if _, ok := goja.AssertFunction(v); !ok { + return errors.New("exported 'setup' must be a function") + } + case "teardown": + if _, ok := goja.AssertFunction(v); !ok { + return errors.New("exported 'teardown' must be a function") + } + default: + if _, ok := goja.AssertFunction(v); ok { + b.Exports[k] = struct{}{} + } + } + } + + if len(b.Exports) == 0 { + return errors.New("no exported functions in script") + } + + return nil +} + // Instantiate creates a new runtime from this bundle. func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { // TODO: actually use a real context here, so that the instantiation can be killed @@ -216,11 +235,18 @@ func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { return nil, err } - // Grab the default function; type is already checked in NewBundle(). + bi = &BundleInstance{ + Runtime: rt, + Context: ctxPtr, + Exports: make(map[string]goja.Callable), + } + + // Grab any exported functions that could be executed. These were + // already pre-validated in NewBundle(), just get them here. exports := rt.Get("exports").ToObject(rt) - def, ok := goja.AssertFunction(exports.Get("default")) - if !ok || def == nil { - panic("exported default is not a function") + for k := range b.Exports { + fn, _ := goja.AssertFunction(exports.Get(k)) + bi.Exports[k] = fn } jsOptions := rt.Get("options") @@ -237,11 +263,7 @@ func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { } }) - return &BundleInstance{ - Runtime: rt, - Context: ctxPtr, - Default: def, - }, instErr + return bi, instErr } // Instantiates the bundle into an existing runtime. Not public because it also messes with a bunch diff --git a/js/bundle_test.go b/js/bundle_test.go index 84f76c5f116..cff448b394d 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -33,15 +33,16 @@ import ( "time" "github.com/dop251/goja" + "github.com/spf13/afero" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/fsext" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" - "github.com/spf13/afero" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" ) const isWindows = runtime.GOOS == "windows" @@ -72,7 +73,7 @@ func getSimpleBundle(filename, data string, opts ...interface{}) (*Bundle, error func TestNewBundle(t *testing.T) { t.Run("Blank", func(t *testing.T) { _, err := getSimpleBundle("/script.js", "") - assert.EqualError(t, err, "script must export a default function") + assert.EqualError(t, err, "no exported functions in script") }) t.Run("Invalid", func(t *testing.T) { _, err := getSimpleBundle("/script.js", "\x00") @@ -89,15 +90,15 @@ func TestNewBundle(t *testing.T) { }) t.Run("DefaultUndefined", func(t *testing.T) { _, err := getSimpleBundle("/script.js", `export default undefined;`) - assert.EqualError(t, err, "script must export a default function") + assert.EqualError(t, err, "no exported functions in script") }) t.Run("DefaultNull", func(t *testing.T) { _, err := getSimpleBundle("/script.js", `export default null;`) - assert.EqualError(t, err, "script must export a default function") + assert.EqualError(t, err, "no exported functions in script") }) t.Run("DefaultWrongType", func(t *testing.T) { _, err := getSimpleBundle("/script.js", `export default 12345;`) - assert.EqualError(t, err, "default export must be a function") + assert.EqualError(t, err, "no exported functions in script") }) t.Run("Minimal", func(t *testing.T) { _, err := getSimpleBundle("/script.js", `export default function() {};`) @@ -435,7 +436,7 @@ func TestNewBundleFromArchive(t *testing.T) { assert.Equal(t, lib.Options{VUs: null.IntFrom(12345)}, b.Options) bi, err := b.Instantiate() require.NoError(t, err) - val, err := bi.Default(goja.Undefined()) + val, err := bi.Exports["default"](goja.Undefined()) require.NoError(t, err) assert.Equal(t, "hi!", val.Export()) } @@ -647,7 +648,7 @@ func TestOpen(t *testing.T) { t.Run(source, func(t *testing.T) { bi, err := b.Instantiate() require.NoError(t, err) - v, err := bi.Default(goja.Undefined()) + v, err := bi.Exports["default"](goja.Undefined()) require.NoError(t, err) assert.Equal(t, "hi", v.Export()) }) @@ -685,7 +686,7 @@ func TestBundleInstantiate(t *testing.T) { } t.Run("Run", func(t *testing.T) { - v, err := bi.Default(goja.Undefined()) + v, err := bi.Exports["default"](goja.Undefined()) if assert.NoError(t, err) { assert.Equal(t, true, v.Export()) } @@ -693,7 +694,7 @@ func TestBundleInstantiate(t *testing.T) { t.Run("SetAndRun", func(t *testing.T) { bi.Runtime.Set("val", false) - v, err := bi.Default(goja.Undefined()) + v, err := bi.Exports["default"](goja.Undefined()) if assert.NoError(t, err) { assert.Equal(t, false, v.Export()) } @@ -748,7 +749,7 @@ func TestBundleEnv(t *testing.T) { bi, err := b.Instantiate() if assert.NoError(t, err) { - _, err := bi.Default(goja.Undefined()) + _, err := bi.Exports["default"](goja.Undefined()) assert.NoError(t, err) } }) @@ -789,7 +790,7 @@ func TestBundleNotSharable(t *testing.T) { require.NoError(t, err) for j := 0; j < iters; j++ { bi.Runtime.Set("__ITER", j) - _, err := bi.Default(goja.Undefined()) + _, err := bi.Exports["default"](goja.Undefined()) assert.NoError(t, err) } } diff --git a/js/initcontext_test.go b/js/initcontext_test.go index a2b805c2454..16d6dc3a6c2 100644 --- a/js/initcontext_test.go +++ b/js/initcontext_test.go @@ -229,7 +229,7 @@ func TestInitContextRequire(t *testing.T) { if !assert.NoError(t, err) { return } - _, err = bi.Default(goja.Undefined()) + _, err = bi.Exports["default"](goja.Undefined()) assert.NoError(t, err) }) }) @@ -401,7 +401,7 @@ func TestRequestWithBinaryFile(t *testing.T) { ctx = common.WithRuntime(ctx, bi.Runtime) *bi.Context = ctx - v, err := bi.Default(goja.Undefined()) + v, err := bi.Exports["default"](goja.Undefined()) assert.NoError(t, err) assert.NotNil(t, v) assert.Equal(t, true, v.Export()) diff --git a/js/runner.go b/js/runner.go index bfa8c6f9da6..6f02f790bc1 100644 --- a/js/runner.go +++ b/js/runner.go @@ -24,6 +24,7 @@ import ( "context" "crypto/tls" "encoding/json" + "fmt" "net" "net/http" "net/http/cookiejar" @@ -269,6 +270,12 @@ func (r *Runner) GetDefaultGroup() *lib.Group { return r.defaultGroup } +// GetExports returns the names of exported functions in the script +// (excluding setup() and teardown()) that can be used for execution. +func (r *Runner) GetExports() map[string]struct{} { + return r.Bundle.Exports +} + func (r *Runner) GetOptions() lib.Options { return r.Bundle.Options } @@ -385,7 +392,10 @@ type ActiveVU struct { // Activate the VU so it will be able to run code. func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() - // u.Env = params.Env + + if params.Exec == "" { + params.Exec = "default" + } avu := &ActiveVU{ VU: u, @@ -410,7 +420,7 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { return avu } -// RunOnce runs the default function once. +// RunOnce runs the configured Exec function once. func (u *ActiveVU) RunOnce() error { select { case <-u.RunContext.Done(): @@ -436,8 +446,16 @@ func (u *ActiveVU) RunOnce() error { } } - // Call the default function. - _, isFullIteration, totalTime, err := u.runFn(u.RunContext, u.Runner.defaultGroup, true, u.Default, u.setupData) + fn, ok := u.Exports[u.Exec] + if !ok { + // Shouldn't happen; this is validated in ExecutionScheduler.Init() + panic(fmt.Sprintf("function '%s' not found in exports", u.Exec)) + } + + // Call the exported function. + _, isFullIteration, totalTime, err := u.runFn( + u.RunContext, u.Runner.defaultGroup, true, fn, u.setupData, + ) // If MinIterationDuration is specified and the iteration wasn't cancelled // and was less than it, sleep for the remainder diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 8f441316e45..1e5ea3f2d50 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -232,10 +232,12 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } }() + execFn := car.GetConfig().GetExec().ValueOrZero() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, + Exec: execFn, DeactivateCallback: func() { car.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 76c31eeb916..87ef89da4c8 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -176,12 +176,14 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executionState, clv.logger) + execFn := clv.GetConfig().GetExec().ValueOrZero() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, + Exec: execFn, DeactivateCallback: func() { clv.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index e87191f8275..a62a02594a3 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -341,7 +341,8 @@ type manualVUHandle struct { func newManualVUHandle( parentCtx context.Context, state *lib.ExecutionState, - localActiveVUsCount *int64, initVU lib.InitializedVU, logger *logrus.Entry, + localActiveVUsCount *int64, initVU lib.InitializedVU, exec string, + logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} getVU := func() (lib.InitializedVU, error) { @@ -357,7 +358,7 @@ func newManualVUHandle( } ctx, cancel := context.WithCancel(parentCtx) return &manualVUHandle{ - vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, logger), + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, logger), initVU: initVU, wg: &wg, cancelVU: cancel, @@ -376,6 +377,7 @@ type externallyControlledRunState struct { maxVUs *int64 // the current number of initialized VUs vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused + exec string runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration } @@ -391,7 +393,8 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { return vuGetErr } vuHandle := newManualVUHandle( - rs.ctx, rs.executor.executionState, rs.activeVUsCount, initVU, rs.executor.logger.WithField("vuNum", i), + rs.ctx, rs.executor.executionState, rs.activeVUsCount, + initVU, rs.exec, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle @@ -449,7 +452,8 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern return vuInitErr } vuHandle := newManualVUHandle( - rs.ctx, executionState, rs.activeVUsCount, initVU, rs.executor.logger.WithField("vuNum", i), + rs.ctx, executionState, rs.activeVUsCount, initVU, rs.exec, + rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) @@ -511,10 +515,12 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats logrus.Fields{"type": externallyControlledType, "duration": duration}, ).Debug("Starting executor run...") + execFn := mex.GetConfig().GetExec().ValueOrZero() startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.config.MaxVUs.Int64) runState := &externallyControlledRunState{ ctx: ctx, executor: mex, + exec: execFn, startMaxVUs: startMaxVUs, duration: duration, vuHandles: make([]*manualVUHandle, startMaxVUs), diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 92236d7d6ed..87cdb9a39c4 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -196,12 +196,14 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executionState, pvi.logger) + execFn := pvi.GetConfig().GetExec().ValueOrZero() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, + Exec: execFn, DeactivateCallback: func() { pvi.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index e17f6efe0ce..d2b23ec1b88 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -215,12 +215,15 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta runIteration := getIterationRunner(si.executionState, si.logger) attemptedIters := new(uint64) + + execFn := si.GetConfig().GetExec().ValueOrZero() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, + Exec: execFn, DeactivateCallback: func() { si.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 4689e9666c2..db3cb675839 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -315,10 +315,12 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample } }() + execFn := varr.GetConfig().GetExec().ValueOrZero() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, + Exec: execFn, DeactivateCallback: func() { varr.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index bedc95c3fcf..535fe909e60 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -595,9 +595,11 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo activeVUs.Done() } + execFn := vlv.GetConfig().GetExec().ValueOrZero() vuHandles := make([]*vuHandle, maxVUs) for i := uint64(0); i < maxVUs; i++ { - vuHandle := newStoppedVUHandle(maxDurationCtx, getVU, returnVU, vlv.logger.WithField("vuNum", i)) + vuHandle := newStoppedVUHandle( + maxDurationCtx, getVU, returnVU, execFn, vlv.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(runIteration) vuHandles[i] = vuHandle } diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 3f94a50753a..44a37d80948 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -39,6 +39,7 @@ type vuHandle struct { parentCtx context.Context getVU func() (lib.InitializedVU, error) returnVU func(lib.InitializedVU) + exec string canStartIter chan struct{} @@ -49,7 +50,7 @@ type vuHandle struct { func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), - returnVU func(lib.InitializedVU), logger *logrus.Entry, + returnVU func(lib.InitializedVU), exec string, logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) @@ -58,6 +59,7 @@ func newStoppedVUHandle( parentCtx: parentCtx, getVU: getVU, returnVU: returnVU, + exec: exec, canStartIter: make(chan struct{}), @@ -158,6 +160,7 @@ mainLoop: vh.returnVU(initVU) } vu = initVU.Activate(&lib.VUActivationParams{ + Exec: vh.exec, RunContext: ctx, DeactivateCallback: deactivateVU, }) diff --git a/lib/runner.go b/lib/runner.go index ff5e389e281..2312bf5253c 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -28,8 +28,9 @@ import ( // ActiveVU represents an actively running virtual user. type ActiveVU interface { - // Runs the VU once. The only way to interrupt the execution is to cancel - // the context given to InitializedVU.Activate() + // Run the configured exported function in the VU once. The only + // way to interrupt the execution is to cancel the context given + // to InitializedVU.Activate() RunOnce() error } @@ -47,9 +48,9 @@ type InitializedVU interface { type VUActivationParams struct { RunContext context.Context DeactivateCallback func() - // Env map[string]string - // Tags map[string]string - // Exec null.String + Env map[string]string + Tags map[string]string + Exec string } // A Runner is a factory for VUs. It should precompute as much as possible upon @@ -90,4 +91,11 @@ type Runner interface { // values and write it back to the runner. GetOptions() Options SetOptions(opts Options) error + + // GetExports returns the names of exported functions in the script + // (excluding setup() and teardown()) that can be used for execution. + // This is a bit janky, but it's needed for validation during + // ExecutionScheduler.Init(). The empty struct is to avoid a + // circular dep or make lib depend on goja :-/ + GetExports() map[string]struct{} } diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index 3dc88b9a462..0c4979f50bd 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -95,6 +95,12 @@ func (r MiniRunner) GetDefaultGroup() *lib.Group { return r.Group } +// GetExports satisfies lib.Runner, but is mocked for MiniRunner since +// it doesn't deal with JS. +func (r MiniRunner) GetExports() map[string]struct{} { + return map[string]struct{}{"default": {}} +} + // GetOptions returns the supplied options struct. func (r MiniRunner) GetOptions() lib.Options { return r.Options From 4913ad02e47741c8db3960fa4e317de4481bb979 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Fri, 17 Apr 2020 15:54:52 +0200 Subject: [PATCH 234/350] Add support for executor-specific env vars See #1300 --- core/local/local_test.go | 107 ++++++++++++++++++++++++++ js/runner.go | 9 +++ lib/executor/constant_arrival_rate.go | 2 + lib/executor/constant_looping_vus.go | 2 + lib/executor/externally_controlled.go | 13 ++-- lib/executor/per_vu_iterations.go | 2 + lib/executor/shared_iterations.go | 2 + lib/executor/variable_arrival_rate.go | 2 + lib/executor/variable_looping_vus.go | 4 +- lib/executor/vu_handle.go | 6 +- 10 files changed, 142 insertions(+), 7 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 4bc6c32a5ee..27ed36d9c83 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -23,6 +23,7 @@ package local import ( "context" "errors" + "fmt" "net" "net/url" "runtime" @@ -170,6 +171,112 @@ func TestExecutionSchedulerRunNonDefault(t *testing.T) { } } +func TestExecutionSchedulerRunEnv(t *testing.T) { + t.Parallel() + + scriptTemplate := ` + import { Counter } from "k6/metrics"; + + let errors = new Counter("errors"); + + export let options = { + execution: { + executor: { + type: "%[1]s", + gracefulStop: "0.5s", + %[2]s + } + } + } + + export default function () { + if (__ENV.TESTVAR !== "%[3]s") { + console.error('Wrong env var value. Expected: %[3]s, actual: ', __ENV.TESTVAR); + errors.add(1); + } + }` + + executorConfigs := map[string]string{ + "constant-arrival-rate": ` + rate: 1, + timeUnit: "0.5s", + duration: "0.5s", + preAllocatedVUs: 1, + maxVUs: 2,`, + "constant-looping-vus": ` + vus: 1, + duration: "0.5s",`, + "externally-controlled": ` + vus: 1, + duration: "0.5s",`, + "per-vu-iterations": ` + vus: 1, + iterations: 1,`, + "shared-iterations": ` + vus: 1, + iterations: 1,`, + "variable-arrival-rate": ` + startRate: 1, + timeUnit: "0.5s", + preAllocatedVUs: 1, + maxVUs: 2, + stages: [ { target: 1, duration: "0.5s" } ],`, + "variable-looping-vus": ` + startVUs: 1, + stages: [ { target: 1, duration: "0.5s" } ],`, + } + + testCases := []struct{ name, script string }{} + + // Generate tests using global env and with env override + for ename, econf := range executorConfigs { + testCases = append(testCases, struct{ name, script string }{ + "global/" + ename, fmt.Sprintf(scriptTemplate, ename, econf, "global")}) + configWithEnvOverride := econf + "env: { TESTVAR: 'overridden' }" + testCases = append(testCases, struct{ name, script string }{ + "override/" + ename, fmt.Sprintf(scriptTemplate, ename, configWithEnvOverride, "overridden")}) + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + runner, err := js.New(&loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, + Data: []byte(tc.script)}, + nil, lib.RuntimeOptions{Env: map[string]string{"TESTVAR": "global"}}) + require.NoError(t, err) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + done := make(chan struct{}) + samples := make(chan stats.SampleContainer) + go func() { + assert.NoError(t, execScheduler.Init(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) + close(done) + }() + for { + select { + case sample := <-samples: + // TODO: Implement a more robust way of reporting + // errors in these high-level functional tests. + if _, ok := sample.(stats.Sample); ok { + assert.FailNow(t, "received error sample from test") + } + case <-done: + return + } + } + }) + } +} + func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Parallel() t.Run("Normal", func(t *testing.T) { diff --git a/js/runner.go b/js/runner.go index 6f02f790bc1..958c94c96d1 100644 --- a/js/runner.go +++ b/js/runner.go @@ -393,6 +393,15 @@ type ActiveVU struct { func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() + // Override the preset global env with any custom env vars + if len(params.Env) > 0 { + env := u.Runtime.Get("__ENV").Export().(map[string]string) + for key, value := range params.Env { + env[key] = value + } + u.Runtime.Set("__ENV", env) + } + if params.Exec == "" { params.Exec = "default" } diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 1e5ea3f2d50..154880ca586 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -233,11 +233,13 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC }() execFn := car.GetConfig().GetExec().ValueOrZero() + env := car.GetConfig().GetEnv() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, Exec: execFn, + Env: env, DeactivateCallback: func() { car.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 87ef89da4c8..ccc1610c5d7 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -177,6 +177,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo runIteration := getIterationRunner(clv.executionState, clv.logger) execFn := clv.GetConfig().GetExec().ValueOrZero() + env := clv.GetConfig().GetEnv() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -184,6 +185,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, Exec: execFn, + Env: env, DeactivateCallback: func() { clv.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index a62a02594a3..a0d88e4cd7b 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -341,8 +341,8 @@ type manualVUHandle struct { func newManualVUHandle( parentCtx context.Context, state *lib.ExecutionState, - localActiveVUsCount *int64, initVU lib.InitializedVU, exec string, - logger *logrus.Entry, + localActiveVUsCount *int64, initVU lib.InitializedVU, + exec string, env map[string]string, logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} getVU := func() (lib.InitializedVU, error) { @@ -358,7 +358,7 @@ func newManualVUHandle( } ctx, cancel := context.WithCancel(parentCtx) return &manualVUHandle{ - vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, logger), + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, env, logger), initVU: initVU, wg: &wg, cancelVU: cancel, @@ -378,6 +378,7 @@ type externallyControlledRunState struct { vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused exec string + env map[string]string runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration } @@ -394,7 +395,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { } vuHandle := newManualVUHandle( rs.ctx, rs.executor.executionState, rs.activeVUsCount, - initVU, rs.exec, rs.executor.logger.WithField("vuNum", i), + initVU, rs.exec, rs.env, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle @@ -453,7 +454,7 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern } vuHandle := newManualVUHandle( rs.ctx, executionState, rs.activeVUsCount, initVU, rs.exec, - rs.executor.logger.WithField("vuNum", i), + rs.env, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) @@ -516,11 +517,13 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats ).Debug("Starting executor run...") execFn := mex.GetConfig().GetExec().ValueOrZero() + env := mex.GetConfig().GetEnv() startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.config.MaxVUs.Int64) runState := &externallyControlledRunState{ ctx: ctx, executor: mex, exec: execFn, + env: env, startMaxVUs: startMaxVUs, duration: duration, vuHandles: make([]*manualVUHandle, startMaxVUs), diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 87cdb9a39c4..48274812bd5 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -197,6 +197,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta runIteration := getIterationRunner(pvi.executionState, pvi.logger) execFn := pvi.GetConfig().GetExec().ValueOrZero() + env := pvi.GetConfig().GetEnv() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -204,6 +205,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, Exec: execFn, + Env: env, DeactivateCallback: func() { pvi.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index d2b23ec1b88..c8f1c6dcccf 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -217,6 +217,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) execFn := si.GetConfig().GetExec().ValueOrZero() + env := si.GetConfig().GetEnv() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -224,6 +225,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta vu := initVU.Activate(&lib.VUActivationParams{ RunContext: ctx, Exec: execFn, + Env: env, DeactivateCallback: func() { si.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index db3cb675839..0e1f2d3cfac 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -316,11 +316,13 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample }() execFn := varr.GetConfig().GetExec().ValueOrZero() + env := varr.GetConfig().GetEnv() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, Exec: execFn, + Env: env, DeactivateCallback: func() { varr.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 535fe909e60..6aed1e1e03a 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -596,10 +596,12 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo } execFn := vlv.GetConfig().GetExec().ValueOrZero() + env := vlv.GetConfig().GetEnv() vuHandles := make([]*vuHandle, maxVUs) for i := uint64(0); i < maxVUs; i++ { vuHandle := newStoppedVUHandle( - maxDurationCtx, getVU, returnVU, execFn, vlv.logger.WithField("vuNum", i)) + maxDurationCtx, getVU, returnVU, execFn, env, + vlv.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(runIteration) vuHandles[i] = vuHandle } diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 44a37d80948..4ff3c06fd28 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -40,6 +40,7 @@ type vuHandle struct { getVU func() (lib.InitializedVU, error) returnVU func(lib.InitializedVU) exec string + env map[string]string canStartIter chan struct{} @@ -50,7 +51,8 @@ type vuHandle struct { func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), - returnVU func(lib.InitializedVU), exec string, logger *logrus.Entry, + returnVU func(lib.InitializedVU), exec string, env map[string]string, + logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) @@ -60,6 +62,7 @@ func newStoppedVUHandle( getVU: getVU, returnVU: returnVU, exec: exec, + env: env, canStartIter: make(chan struct{}), @@ -162,6 +165,7 @@ mainLoop: vu = initVU.Activate(&lib.VUActivationParams{ Exec: vh.exec, RunContext: ctx, + Env: vh.env, DeactivateCallback: deactivateVU, }) } From 71a718010c409fc699261b958ae4ddc94e29c7f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 21 Apr 2020 12:22:36 +0200 Subject: [PATCH 235/350] Add support for executor-specific tags See #1300 --- core/local/local_test.go | 109 ++++++++++++++++++++++++++ js/runner.go | 8 ++ lib/executor/base_config.go | 11 ++- lib/executor/constant_arrival_rate.go | 7 +- lib/executor/constant_looping_vus.go | 7 +- lib/executor/externally_controlled.go | 18 +++-- lib/executor/per_vu_iterations.go | 7 +- lib/executor/shared_iterations.go | 7 +- lib/executor/variable_arrival_rate.go | 7 +- lib/executor/variable_looping_vus.go | 8 +- lib/executor/vu_handle.go | 5 +- lib/executors.go | 1 + lib/runner.go | 3 +- 13 files changed, 173 insertions(+), 25 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 27ed36d9c83..1463839b6b0 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -42,7 +42,9 @@ import ( "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/netext" + "github.com/loadimpact/k6/lib/netext/httpext" "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" @@ -277,6 +279,113 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { } } +func TestExecutionSchedulerRunCustomTags(t *testing.T) { + t.Parallel() + tb := httpmultibin.NewHTTPMultiBin(t) + defer tb.Cleanup() + sr := tb.Replacer.Replace + + scriptTemplate := sr(` + import http from "k6/http"; + + export let options = { + execution: { + executor: { + type: "%s", + gracefulStop: "0.5s", + %s + } + } + } + + export default function () { + http.get("HTTPBIN_IP_URL/"); + }`) + + executorConfigs := map[string]string{ + "constant-arrival-rate": ` + rate: 1, + timeUnit: "0.5s", + duration: "0.5s", + preAllocatedVUs: 1, + maxVUs: 2,`, + "constant-looping-vus": ` + vus: 1, + duration: "0.5s",`, + "externally-controlled": ` + vus: 1, + duration: "0.5s",`, + "per-vu-iterations": ` + vus: 1, + iterations: 1,`, + "shared-iterations": ` + vus: 1, + iterations: 1,`, + "variable-arrival-rate": ` + startRate: 5, + timeUnit: "0.5s", + preAllocatedVUs: 1, + maxVUs: 2, + stages: [ { target: 10, duration: "1s" } ],`, + "variable-looping-vus": ` + startVUs: 1, + stages: [ { target: 1, duration: "0.5s" } ],`, + } + + testCases := []struct{ name, script string }{} + + // Generate tests using custom tags + for ename, econf := range executorConfigs { + configWithCustomTag := econf + "tags: { customTag: 'value' }" + testCases = append(testCases, struct{ name, script string }{ + ename, fmt.Sprintf(scriptTemplate, ename, configWithCustomTag)}) + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + runner, err := js.New(&loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, + Data: []byte(tc.script)}, + nil, lib.RuntimeOptions{}) + require.NoError(t, err) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + done := make(chan struct{}) + samples := make(chan stats.SampleContainer) + go func() { + assert.NoError(t, execScheduler.Init(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) + close(done) + }() + var gotTag bool + for { + select { + case sample := <-samples: + if trail, ok := sample.(*httpext.Trail); ok && !gotTag { + tags := trail.Tags.CloneTags() + if v, ok := tags["customTag"]; ok && v == "value" { + gotTag = true + } + } + case <-done: + if !gotTag { + assert.FailNow(t, "sample with tag wasn't received") + } + return + } + } + }) + } +} + func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Parallel() t.Run("Normal", func(t *testing.T) { diff --git a/js/runner.go b/js/runner.go index 958c94c96d1..bdd39324292 100644 --- a/js/runner.go +++ b/js/runner.go @@ -406,6 +406,14 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { params.Exec = "default" } + if len(params.Tags) > 0 { + tags := u.Runner.Bundle.Options.RunTags.CloneTags() + for k, v := range params.Tags { + tags[k] = v + } + u.Runner.Bundle.Options.RunTags = stats.IntoSampleTags(&tags) + } + avu := &ActiveVU{ VU: u, VUActivationParams: params, diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index 45823625d06..d0bfc1611e9 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -26,8 +26,9 @@ import ( "strings" "time" - "github.com/loadimpact/k6/lib/types" null "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) // DefaultGracefulStopValue is the graceful top value for all executors, unless @@ -46,8 +47,9 @@ type BaseConfig struct { GracefulStop types.NullDuration `json:"gracefulStop"` Env map[string]string `json:"env"` Exec null.String `json:"exec"` // function name, externally validated + Tags map[string]string `json:"tags"` - // TODO: future extensions like tags, distribution, others? + // TODO: future extensions like distribution, others? } // NewBaseConfig returns a default base config with the default values @@ -122,6 +124,11 @@ func (bc BaseConfig) GetExec() null.String { return bc.Exec } +// GetTags returns any custom tags configured for the executor. +func (bc BaseConfig) GetTags() map[string]string { + return bc.Tags +} + // IsDistributable returns true since by default all executors could be run in // a distributed manner. func (bc BaseConfig) IsDistributable() bool { diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 154880ca586..4a1ca5acb64 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -232,14 +232,17 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } }() - execFn := car.GetConfig().GetExec().ValueOrZero() - env := car.GetConfig().GetEnv() + conf := car.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, Exec: execFn, Env: env, + Tags: tags, DeactivateCallback: func() { car.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index ccc1610c5d7..b1f0d9a8d32 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -176,8 +176,10 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executionState, clv.logger) - execFn := clv.GetConfig().GetExec().ValueOrZero() - env := clv.GetConfig().GetEnv() + conf := clv.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -186,6 +188,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo RunContext: ctx, Exec: execFn, Env: env, + Tags: tags, DeactivateCallback: func() { clv.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index a0d88e4cd7b..1562b3eae2e 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -342,7 +342,8 @@ type manualVUHandle struct { func newManualVUHandle( parentCtx context.Context, state *lib.ExecutionState, localActiveVUsCount *int64, initVU lib.InitializedVU, - exec string, env map[string]string, logger *logrus.Entry, + exec string, env map[string]string, tags map[string]string, + logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} getVU := func() (lib.InitializedVU, error) { @@ -358,7 +359,7 @@ func newManualVUHandle( } ctx, cancel := context.WithCancel(parentCtx) return &manualVUHandle{ - vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, env, logger), + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, env, tags, logger), initVU: initVU, wg: &wg, cancelVU: cancel, @@ -378,7 +379,7 @@ type externallyControlledRunState struct { vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused exec string - env map[string]string + env, tags map[string]string runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration } @@ -395,7 +396,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { } vuHandle := newManualVUHandle( rs.ctx, rs.executor.executionState, rs.activeVUsCount, - initVU, rs.exec, rs.env, rs.executor.logger.WithField("vuNum", i), + initVU, rs.exec, rs.env, rs.tags, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle @@ -454,7 +455,7 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern } vuHandle := newManualVUHandle( rs.ctx, executionState, rs.activeVUsCount, initVU, rs.exec, - rs.env, rs.executor.logger.WithField("vuNum", i), + rs.env, rs.tags, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) @@ -516,14 +517,17 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats logrus.Fields{"type": externallyControlledType, "duration": duration}, ).Debug("Starting executor run...") - execFn := mex.GetConfig().GetExec().ValueOrZero() - env := mex.GetConfig().GetEnv() + conf := mex.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.config.MaxVUs.Int64) runState := &externallyControlledRunState{ ctx: ctx, executor: mex, exec: execFn, env: env, + tags: tags, startMaxVUs: startMaxVUs, duration: duration, vuHandles: make([]*manualVUHandle, startMaxVUs), diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 48274812bd5..794d7ec323f 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -196,8 +196,10 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executionState, pvi.logger) - execFn := pvi.GetConfig().GetExec().ValueOrZero() - env := pvi.GetConfig().GetEnv() + conf := pvi.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -206,6 +208,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta RunContext: ctx, Exec: execFn, Env: env, + Tags: tags, DeactivateCallback: func() { pvi.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index c8f1c6dcccf..6f645bb7fc4 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -216,8 +216,10 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) - execFn := si.GetConfig().GetExec().ValueOrZero() - env := si.GetConfig().GetEnv() + conf := si.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() @@ -226,6 +228,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta RunContext: ctx, Exec: execFn, Env: env, + Tags: tags, DeactivateCallback: func() { si.executionState.ReturnVU(initVU, true) activeVUs.Done() diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 0e1f2d3cfac..799a701b8f3 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -315,14 +315,17 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample } }() - execFn := varr.GetConfig().GetExec().ValueOrZero() - env := varr.GetConfig().GetEnv() + conf := varr.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) activeVU := initVU.Activate(&lib.VUActivationParams{ RunContext: maxDurationCtx, Exec: execFn, Env: env, + Tags: tags, DeactivateCallback: func() { varr.executionState.ReturnVU(initVU, true) activeVUsWg.Done() diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 6aed1e1e03a..f7d3e3e3a23 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -595,13 +595,15 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo activeVUs.Done() } - execFn := vlv.GetConfig().GetExec().ValueOrZero() - env := vlv.GetConfig().GetEnv() + conf := vlv.GetConfig() + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() vuHandles := make([]*vuHandle, maxVUs) for i := uint64(0); i < maxVUs; i++ { vuHandle := newStoppedVUHandle( maxDurationCtx, getVU, returnVU, execFn, env, - vlv.logger.WithField("vuNum", i)) + tags, vlv.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(runIteration) vuHandles[i] = vuHandle } diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 4ff3c06fd28..0ca751a99dd 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -41,6 +41,7 @@ type vuHandle struct { returnVU func(lib.InitializedVU) exec string env map[string]string + tags map[string]string canStartIter chan struct{} @@ -52,7 +53,7 @@ type vuHandle struct { func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), returnVU func(lib.InitializedVU), exec string, env map[string]string, - logger *logrus.Entry, + tags map[string]string, logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) @@ -63,6 +64,7 @@ func newStoppedVUHandle( returnVU: returnVU, exec: exec, env: env, + tags: tags, canStartIter: make(chan struct{}), @@ -166,6 +168,7 @@ mainLoop: Exec: vh.exec, RunContext: ctx, Env: vh.env, + Tags: vh.tags, DeactivateCallback: deactivateVU, }) } diff --git a/lib/executors.go b/lib/executors.go index 7e65b74d221..e25fe4d2446 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -94,6 +94,7 @@ type ExecutorConfig interface { // // TODO: use interface{} so plain http requests can be specified? GetExec() null.String + GetTags() map[string]string // Calculates the VU requirements in different stages of the executor's // execution, including any extensions caused by waiting for iterations to diff --git a/lib/runner.go b/lib/runner.go index 2312bf5253c..b49173147d2 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -48,8 +48,7 @@ type InitializedVU interface { type VUActivationParams struct { RunContext context.Context DeactivateCallback func() - Env map[string]string - Tags map[string]string + Env, Tags map[string]string Exec string } From 702f80ddde2393c5f7b0a771d48f6e1142ca6888 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 30 Apr 2020 15:37:43 +0200 Subject: [PATCH 236/350] Ignore funlen linter for PerVUIterations.Run --- lib/executor/per_vu_iterations.go | 1 + 1 file changed, 1 insertion(+) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 794d7ec323f..5391b5a5ad0 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -150,6 +150,7 @@ type PerVUIterations struct { var _ lib.Executor = &PerVUIterations{} // Run executes a specific number of iterations with each configured VU. +// nolint:funlen func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := pvi.config.GetVUs(pvi.executionState.ExecutionTuple) iterations := pvi.config.GetIterations() From f7bc0fa7a64f410894b3d91581ae4db82cf24f19 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 4 May 2020 12:33:31 +0200 Subject: [PATCH 237/350] Move script function names to lib/consts package --- cmd/run.go | 4 ++-- js/bundle.go | 6 +++--- js/runner.go | 19 ++++++++----------- lib/consts/js.go | 29 +++++++++++++++++++++++++++++ lib/timeout_error.go | 12 +++--------- lib/timeout_error_test.go | 10 ++++++---- 6 files changed, 51 insertions(+), 29 deletions(-) create mode 100644 lib/consts/js.go diff --git a/cmd/run.go b/cmd/run.go index e27a1cc5496..25437e99c88 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -382,9 +382,9 @@ func getExitCodeFromEngine(err error) ExitCode { switch e := errors.Cause(err).(type) { case lib.TimeoutError: switch e.Place() { - case "setup": + case consts.SetupFn: return ExitCode{error: err, Code: setupTimeoutErrorCode, Hint: e.Hint()} - case "teardown": + case consts.TeardownFn: return ExitCode{error: err, Code: teardownTimeoutErrorCode, Hint: e.Hint()} default: return ExitCode{error: err, Code: genericTimeoutErrorCode} diff --git a/js/bundle.go b/js/bundle.go index 104e1a56fdb..e5d4026f0de 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -191,7 +191,7 @@ func (b *Bundle) getExports(rt *goja.Runtime) error { for _, k := range exports.Keys() { v := exports.Get(k) switch k { - case "options": + case consts.Options: data, err := json.Marshal(v.Export()) if err != nil { return err @@ -199,11 +199,11 @@ func (b *Bundle) getExports(rt *goja.Runtime) error { if err := json.Unmarshal(data, &b.Options); err != nil { return err } - case "setup": + case consts.SetupFn: if _, ok := goja.AssertFunction(v); !ok { return errors.New("exported 'setup' must be a function") } - case "teardown": + case consts.TeardownFn: if _, ok := goja.AssertFunction(v); !ok { return errors.New("exported 'teardown' must be a function") } diff --git a/js/runner.go b/js/runner.go index bdd39324292..fde5504d653 100644 --- a/js/runner.go +++ b/js/runner.go @@ -42,17 +42,14 @@ import ( "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/netext" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" ) //nolint:gochecknoglobals -var ( - errInterrupt = errors.New("context cancelled") - stageSetup = "setup" - stageTeardown = "teardown" -) +var errInterrupt = errors.New("context cancelled") // Ensure Runner implements the lib.Runner interface var _ lib.Runner = &Runner{} @@ -219,7 +216,7 @@ func (r *Runner) Setup(ctx context.Context, out chan<- stats.SampleContainer) er ) defer setupCancel() - v, err := r.runPart(setupCtx, out, stageSetup, nil) + v, err := r.runPart(setupCtx, out, consts.SetupFn, nil) if err != nil { return err } @@ -231,7 +228,7 @@ func (r *Runner) Setup(ctx context.Context, out chan<- stats.SampleContainer) er r.setupData, err = json.Marshal(v.Export()) if err != nil { - return errors.Wrap(err, stageSetup) + return errors.Wrap(err, consts.SetupFn) } var tmp interface{} return json.Unmarshal(r.setupData, &tmp) @@ -257,12 +254,12 @@ func (r *Runner) Teardown(ctx context.Context, out chan<- stats.SampleContainer) var data interface{} if r.setupData != nil { if err := json.Unmarshal(r.setupData, &data); err != nil { - return errors.Wrap(err, stageTeardown) + return errors.Wrap(err, consts.TeardownFn) } } else { data = goja.Undefined() } - _, err := r.runPart(teardownCtx, out, stageTeardown, data) + _, err := r.runPart(teardownCtx, out, consts.TeardownFn, data) return err } @@ -349,9 +346,9 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, func (r *Runner) timeoutErrorDuration(stage string) time.Duration { d := time.Duration(0) switch stage { - case stageSetup: + case consts.SetupFn: return time.Duration(r.Bundle.Options.SetupTimeout.Duration) - case stageTeardown: + case consts.TeardownFn: return time.Duration(r.Bundle.Options.TeardownTimeout.Duration) } return d diff --git a/lib/consts/js.go b/lib/consts/js.go new file mode 100644 index 00000000000..55603629cf8 --- /dev/null +++ b/lib/consts/js.go @@ -0,0 +1,29 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2020 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package consts + +// JS constants +const ( + DefaultFn = "default" + Options = "options" + SetupFn = "setup" + TeardownFn = "teardown" +) diff --git a/lib/timeout_error.go b/lib/timeout_error.go index 13da407c4eb..5d50e3801c2 100644 --- a/lib/timeout_error.go +++ b/lib/timeout_error.go @@ -3,14 +3,8 @@ package lib import ( "fmt" "time" -) -//nolint:gochecknoglobals -// Keep stages in sync with js/runner.go -// We set it here to prevent import cycle. -var ( - stageSetup = "setup" - stageTeardown = "teardown" + "github.com/loadimpact/k6/lib/consts" ) // TimeoutError is used when somethings timeouts @@ -45,9 +39,9 @@ func (t TimeoutError) Hint() string { hint := "" switch t.place { - case stageSetup: + case consts.SetupFn: hint = "You can increase the time limit via the setupTimeout option" - case stageTeardown: + case consts.TeardownFn: hint = "You can increase the time limit via the teardownTimeout option" } return hint diff --git a/lib/timeout_error_test.go b/lib/timeout_error_test.go index 86232fd1dd8..5a3ce9b61f3 100644 --- a/lib/timeout_error_test.go +++ b/lib/timeout_error_test.go @@ -4,6 +4,8 @@ import ( "strings" "testing" "time" + + "github.com/loadimpact/k6/lib/consts" ) func TestTimeoutError(t *testing.T) { @@ -11,8 +13,8 @@ func TestTimeoutError(t *testing.T) { stage, expectedStrContain string d time.Duration }{ - {"setup", "1 seconds", time.Second}, - {"teardown", "2 seconds", time.Second * 2}, + {consts.SetupFn, "1 seconds", time.Second}, + {consts.TeardownFn, "2 seconds", time.Second * 2}, {"", "0 seconds", time.Duration(0)}, } @@ -29,8 +31,8 @@ func TestTimeoutErrorHint(t *testing.T) { stage string empty bool }{ - {"setup", false}, - {"teardown", false}, + {consts.SetupFn, false}, + {consts.TeardownFn, false}, {"not handle", true}, } From a1859244b6a10f6bca659a7e3588cc1e02062d1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 4 May 2020 12:50:59 +0200 Subject: [PATCH 238/350] Validate executor config earlier This leaves executor init before VU init, but moves the config validation to a more suitable place before the engine/ExecutionScheduler is initialized. It now also properly reports errors from more than one executor. Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r418140357 --- cmd/archive.go | 5 ++-- cmd/cloud.go | 2 +- cmd/config.go | 26 +++++++++++++++++--- cmd/config_test.go | 53 +++++++++++++++++++++++++++++++++++++++- cmd/run.go | 2 +- core/local/local.go | 13 +--------- core/local/local_test.go | 13 ---------- 7 files changed, 81 insertions(+), 33 deletions(-) diff --git a/cmd/archive.go b/cmd/archive.go index 7c0466f7ce8..c6ea9f4d51b 100644 --- a/cmd/archive.go +++ b/cmd/archive.go @@ -23,10 +23,11 @@ package cmd import ( "os" - "github.com/loadimpact/k6/loader" "github.com/spf13/afero" "github.com/spf13/cobra" "github.com/spf13/pflag" + + "github.com/loadimpact/k6/loader" ) var archiveOut = "archive.tar" @@ -77,7 +78,7 @@ An archive is a fully self-contained test run, and can be executed identically e return err } - if _, cerr := deriveAndValidateConfig(conf); cerr != nil { + if _, cerr := deriveAndValidateConfig(conf, r.GetExports()); cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/cmd/cloud.go b/cmd/cloud.go index a3a60333784..05d3a8cde3e 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -105,7 +105,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - derivedConf, cerr := deriveAndValidateConfig(conf) + derivedConf, cerr := deriveAndValidateConfig(conf, r.GetExports()) if cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/cmd/config.go b/cmd/config.go index f2ad066185b..2a3b2854f3a 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -34,6 +34,7 @@ import ( null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/cloud" @@ -262,17 +263,24 @@ func applyDefault(conf Config) Config { return conf } -func deriveAndValidateConfig(conf Config) (result Config, err error) { +func deriveAndValidateConfig(conf Config, exports map[string]struct{}) (result Config, err error) { result = conf result.Options, err = executor.DeriveExecutionFromShortcuts(conf.Options) if err != nil { return result, err } - return result, validateConfig(result) + return result, validateConfig(result, exports) } -func validateConfig(conf Config) error { +func validateConfig(conf Config, exports map[string]struct{}) error { errList := conf.Validate() + + for _, ec := range conf.Execution { + if err := validateExecutorConfig(ec, exports); err != nil { + errList = append(errList, err) + } + } + if len(errList) == 0 { return nil } @@ -284,3 +292,15 @@ func validateConfig(conf Config) error { return errors.New(strings.Join(errMsgParts, "\n")) } + +func validateExecutorConfig(conf lib.ExecutorConfig, exports map[string]struct{}) error { + execFn := conf.GetExec().ValueOrZero() + if execFn == "" { + execFn = consts.DefaultFn + } + if _, ok := exports[execFn]; !ok { + return fmt.Errorf("executor %s: %s", conf.GetName(), + fmt.Sprintf("function '%s' not found in exports", execFn)) + } + return nil +} diff --git a/cmd/config_test.go b/cmd/config_test.go index 75244027ce1..a0cc4251e00 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -23,11 +23,16 @@ package cmd import ( "fmt" "testing" + "time" "github.com/kelseyhightower/envconfig" - "github.com/loadimpact/k6/lib/testutils" "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/executor" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/types" ) type testCmdData struct { @@ -134,3 +139,49 @@ func TestConfigApply(t *testing.T) { assert.Equal(t, []string{"influxdb", "json"}, conf.Out) }) } + +func TestDeriveAndValidateConfig(t *testing.T) { + t.Parallel() + + testCases := []struct { + name string + conf Config + exports map[string]struct{} + err string + }{ + {"defaultOK", Config{}, map[string]struct{}{"default": {}}, ""}, + {"defaultErr", Config{}, map[string]struct{}{}, + "executor default: function 'default' not found in exports"}, + {"nonDefaultOK", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ + "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ + Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefault")}, + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), + MaxDuration: types.NullDurationFrom(time.Second), + }}}}, + map[string]struct{}{"nonDefault": {}}, "", + }, + {"nonDefaultErr", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ + "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ + Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefaultErr")}, + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), + MaxDuration: types.NullDurationFrom(time.Second), + }}}}, + map[string]struct{}{"nonDefault": {}}, + "executor per_vu_iters: function 'nonDefaultErr' not found in exports", + }, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + _, err := deriveAndValidateConfig(tc.conf, tc.exports) + if tc.err != "" { + assert.Contains(t, err.Error(), tc.err) + } else { + assert.NoError(t, err) + } + }) + } +} diff --git a/cmd/run.go b/cmd/run.go index 25437e99c88..bba66f2957e 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -134,7 +134,7 @@ a commandline interface for interacting with it.`, return err } - conf, cerr := deriveAndValidateConfig(conf) + conf, cerr := deriveAndValidateConfig(conf, r.GetExports()) if cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/core/local/local.go b/core/local/local.go index 7f916939db8..b3439faded4 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -224,23 +224,12 @@ func (e *ExecutionScheduler) initVUsConcurrently( func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.SampleContainer) error { logger := e.logger.WithField("phase", "local-execution-scheduler-init") - // Initialize each executor and do some basic validation. + // Initialize each executor e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) logger.Debugf("Start initializing executors...") errMsg := "error while initializing executor %s: %s" - exports := e.runner.GetExports() for _, exec := range e.executors { executorConfig := exec.GetConfig() - execFn := executorConfig.GetExec().ValueOrZero() - execName := executorConfig.GetName() - - if execFn == "" { - execFn = "default" - } - if _, ok := exports[execFn]; !ok { - return fmt.Errorf(errMsg, execName, - fmt.Sprintf("function '%s' not found in exports", execFn)) - } if err := exec.Init(ctx); err != nil { return fmt.Errorf(errMsg, executorConfig.GetName(), err) } diff --git a/core/local/local_test.go b/core/local/local_test.go index 1463839b6b0..ee5f00d458b 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -119,19 +119,6 @@ func TestExecutionSchedulerRunNonDefault(t *testing.T) { } } export function nonDefault() {}`, ""}, - {"nonDefaultErr", ` - export let options = { - execution: { - per_vu_iters: { - type: "per-vu-iterations", - vus: 1, - iterations: 1, - exec: "nonDefaultErr", - }, - } - } - export function nonDefault() {}`, - "error while initializing executor per_vu_iters: function 'nonDefaultErr' not found in exports"}, } for _, tc := range testCases { From f7e0214cc0238c42d1d33c570e673b75cd016813 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 4 May 2020 16:53:27 +0200 Subject: [PATCH 239/350] Use helper function for VUActivationParams, simplify by passing BaseConfig Resolves: - https://github.com/loadimpact/k6/pull/1428#discussion_r418039599 - https://github.com/loadimpact/k6/pull/1428#discussion_r418130098 --- js/runner.go | 2 +- js/runner_test.go | 2 +- lib/executor/constant_arrival_rate.go | 20 ++++++------------- lib/executor/constant_looping_vus.go | 25 ++++++++++-------------- lib/executor/externally_controlled.go | 23 +++++++--------------- lib/executor/helpers.go | 20 +++++++++++++++++++ lib/executor/per_vu_iterations.go | 25 ++++++++++-------------- lib/executor/shared_iterations.go | 25 ++++++++++-------------- lib/executor/variable_arrival_rate.go | 20 ++++++------------- lib/executor/variable_looping_vus.go | 8 ++------ lib/executor/vu_handle.go | 27 ++++++++------------------ lib/runner.go | 2 +- lib/testutils/minirunner/minirunner.go | 2 +- 13 files changed, 83 insertions(+), 118 deletions(-) diff --git a/js/runner.go b/js/runner.go index fde5504d653..81472a292d1 100644 --- a/js/runner.go +++ b/js/runner.go @@ -427,7 +427,7 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { avu.busy <- struct{}{} if params.DeactivateCallback != nil { - params.DeactivateCallback() + params.DeactivateCallback(u) } }() diff --git a/js/runner_test.go b/js/runner_test.go index ad88b95e941..319252f9da5 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -612,7 +612,7 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { newCtx, newCancel := context.WithCancel(ctx) vu := initVU.Activate(&lib.VUActivationParams{ RunContext: newCtx, - DeactivateCallback: func() { wg.Done() }, + DeactivateCallback: func(_ lib.InitializedVU) { wg.Done() }, }) ch := make(chan struct{}) go func() { diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 4a1ca5acb64..21ad366a270 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -232,22 +232,14 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } }() - conf := car.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() + activationParams := getVUActivationParams(maxDurationCtx, car.config.BaseConfig, + func(u lib.InitializedVU) { + car.executionState.ReturnVU(u, true) + activeVUsWg.Done() + }) activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) - activeVU := initVU.Activate(&lib.VUActivationParams{ - RunContext: maxDurationCtx, - Exec: execFn, - Env: env, - Tags: tags, - DeactivateCallback: func() { - car.executionState.ReturnVU(initVU, true) - activeVUsWg.Done() - }, - }) + activeVU := initVU.Activate(activationParams) car.executionState.ModCurrentlyActiveVUsCount(+1) atomic.AddUint64(&activeVUsCount, 1) return activeVU diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index b1f0d9a8d32..af4850b2539 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -176,24 +176,19 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executionState, clv.logger) - conf := clv.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() + activationParams := getVUActivationParams(maxDurationCtx, clv.config.BaseConfig, + func(u lib.InitializedVU) { + clv.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - vu := initVU.Activate(&lib.VUActivationParams{ - RunContext: ctx, - Exec: execFn, - Env: env, - Tags: tags, - DeactivateCallback: func() { - clv.executionState.ReturnVU(initVU, true) - activeVUs.Done() - }, - }) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for { select { @@ -202,7 +197,7 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo default: // continue looping } - runIteration(maxDurationCtx, vu) + runIteration(maxDurationCtx, activeVU) } } diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 1562b3eae2e..0c352d80042 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -342,8 +342,7 @@ type manualVUHandle struct { func newManualVUHandle( parentCtx context.Context, state *lib.ExecutionState, localActiveVUsCount *int64, initVU lib.InitializedVU, - exec string, env map[string]string, tags map[string]string, - logger *logrus.Entry, + config *BaseConfig, logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} getVU := func() (lib.InitializedVU, error) { @@ -359,7 +358,7 @@ func newManualVUHandle( } ctx, cancel := context.WithCancel(parentCtx) return &manualVUHandle{ - vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, exec, env, tags, logger), + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, config, logger), initVU: initVU, wg: &wg, cancelVU: cancel, @@ -378,8 +377,6 @@ type externallyControlledRunState struct { maxVUs *int64 // the current number of initialized VUs vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused - exec string - env, tags map[string]string runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration } @@ -395,8 +392,8 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { return vuGetErr } vuHandle := newManualVUHandle( - rs.ctx, rs.executor.executionState, rs.activeVUsCount, - initVU, rs.exec, rs.env, rs.tags, rs.executor.logger.WithField("vuNum", i), + rs.ctx, rs.executor.executionState, rs.activeVUsCount, initVU, + &rs.executor.config.BaseConfig, rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle @@ -454,8 +451,9 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern return vuInitErr } vuHandle := newManualVUHandle( - rs.ctx, executionState, rs.activeVUsCount, initVU, rs.exec, - rs.env, rs.tags, rs.executor.logger.WithField("vuNum", i), + rs.ctx, executionState, rs.activeVUsCount, initVU, + &rs.executor.config.BaseConfig, + rs.executor.logger.WithField("vuNum", i), ) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) @@ -517,17 +515,10 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats logrus.Fields{"type": externallyControlledType, "duration": duration}, ).Debug("Starting executor run...") - conf := mex.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() startMaxVUs := mex.executionState.Options.ExecutionSegment.Scale(mex.config.MaxVUs.Int64) runState := &externallyControlledRunState{ ctx: ctx, executor: mex, - exec: execFn, - env: env, - tags: tags, startMaxVUs: startMaxVUs, duration: duration, vuHandles: make([]*manualVUHandle, startMaxVUs), diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 9db491fc245..e46bb7f7a6e 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -31,6 +31,7 @@ import ( "github.com/sirupsen/logrus" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/types" ) @@ -215,3 +216,22 @@ func getArrivalRatePerSec(scaledArrivalRate *big.Rat) *big.Rat { perSecRate := big.NewRat(int64(time.Second), 1) return perSecRate.Mul(perSecRate, scaledArrivalRate) } + +func getActivationParams( + ctx context.Context, conf BaseConfig, deactivateCallback func(lib.InitializedVU), +) *lib.VUActivationParams { + execFn := conf.GetExec().ValueOrZero() + env := conf.GetEnv() + tags := conf.GetTags() + if execFn == "" { + execFn = consts.DefaultFn + } + + return &lib.VUActivationParams{ + RunContext: ctx, + Exec: execFn, + Env: env, + Tags: tags, + DeactivateCallback: deactivateCallback, + } +} diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 5391b5a5ad0..c6fe8691014 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -197,24 +197,19 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executionState, pvi.logger) - conf := pvi.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() + activationParams := getVUActivationParams(maxDurationCtx, pvi.config.BaseConfig, + func(u lib.InitializedVU) { + pvi.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - vu := initVU.Activate(&lib.VUActivationParams{ - RunContext: ctx, - Exec: execFn, - Env: env, - Tags: tags, - DeactivateCallback: func() { - pvi.executionState.ReturnVU(initVU, true) - activeVUs.Done() - }, - }) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for i := int64(0); i < iterations; i++ { select { @@ -223,7 +218,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta default: // continue looping } - runIteration(maxDurationCtx, vu) + runIteration(maxDurationCtx, activeVU) atomic.AddUint64(doneIters, 1) } } diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 6f645bb7fc4..5dc4663cedd 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -216,24 +216,19 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) - conf := si.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() + activationParams := getVUActivationParams(maxDurationCtx, si.config.BaseConfig, + func(u lib.InitializedVU) { + si.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - vu := initVU.Activate(&lib.VUActivationParams{ - RunContext: ctx, - Exec: execFn, - Env: env, - Tags: tags, - DeactivateCallback: func() { - si.executionState.ReturnVU(initVU, true) - activeVUs.Done() - }, - }) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for { select { @@ -248,7 +243,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta return } - runIteration(maxDurationCtx, vu) + runIteration(maxDurationCtx, activeVU) atomic.AddUint64(doneIters, 1) } } diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 799a701b8f3..2e46428154a 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -315,22 +315,14 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample } }() - conf := varr.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() + activationParams := getVUActivationParams(maxDurationCtx, varr.config.BaseConfig, + func(u lib.InitializedVU) { + varr.executionState.ReturnVU(u, true) + activeVUsWg.Done() + }) activateVU := func(initVU lib.InitializedVU) lib.ActiveVU { activeVUsWg.Add(1) - activeVU := initVU.Activate(&lib.VUActivationParams{ - RunContext: maxDurationCtx, - Exec: execFn, - Env: env, - Tags: tags, - DeactivateCallback: func() { - varr.executionState.ReturnVU(initVU, true) - activeVUsWg.Done() - }, - }) + activeVU := initVU.Activate(activationParams) varr.executionState.ModCurrentlyActiveVUsCount(+1) atomic.AddUint64(&activeVUsCount, 1) return activeVU diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index f7d3e3e3a23..b8ae969fe32 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -595,15 +595,11 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo activeVUs.Done() } - conf := vlv.GetConfig() - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() vuHandles := make([]*vuHandle, maxVUs) for i := uint64(0); i < maxVUs; i++ { vuHandle := newStoppedVUHandle( - maxDurationCtx, getVU, returnVU, execFn, env, - tags, vlv.logger.WithField("vuNum", i)) + maxDurationCtx, getVU, returnVU, &vlv.config.BaseConfig, + vlv.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(runIteration) vuHandles[i] = vuHandle } diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 0ca751a99dd..42affe53842 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -39,9 +39,7 @@ type vuHandle struct { parentCtx context.Context getVU func() (lib.InitializedVU, error) returnVU func(lib.InitializedVU) - exec string - env map[string]string - tags map[string]string + config *BaseConfig canStartIter chan struct{} @@ -52,8 +50,7 @@ type vuHandle struct { func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), - returnVU func(lib.InitializedVU), exec string, env map[string]string, - tags map[string]string, logger *logrus.Entry, + returnVU func(lib.InitializedVU), config *BaseConfig, logger *logrus.Entry, ) *vuHandle { lock := &sync.RWMutex{} ctx, cancel := context.WithCancel(parentCtx) @@ -62,9 +59,7 @@ func newStoppedVUHandle( parentCtx: parentCtx, getVU: getVU, returnVU: returnVU, - exec: exec, - env: env, - tags: tags, + config: config, canStartIter: make(chan struct{}), @@ -113,7 +108,6 @@ func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.ActiveV executorDone := vh.parentCtx.Done() var vu lib.ActiveVU - var deactivateVU func() mainLoop: for { @@ -161,16 +155,11 @@ mainLoop: if err != nil { return } - deactivateVU = func() { - vh.returnVU(initVU) - } - vu = initVU.Activate(&lib.VUActivationParams{ - Exec: vh.exec, - RunContext: ctx, - Env: vh.env, - Tags: vh.tags, - DeactivateCallback: deactivateVU, - }) + activationParams := getVUActivationParams(ctx, *vh.config, + func(u lib.InitializedVU) { + vh.returnVU(u) + }) + vu = initVU.Activate(activationParams) } runIter(ctx, vu) diff --git a/lib/runner.go b/lib/runner.go index b49173147d2..0292d954329 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -47,7 +47,7 @@ type InitializedVU interface { // the buffer pool and activates it for use. type VUActivationParams struct { RunContext context.Context - DeactivateCallback func() + DeactivateCallback func(InitializedVU) Env, Tags map[string]string Exec string } diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index 0c4979f50bd..e62137f88a3 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -143,7 +143,7 @@ func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { avu.busy <- struct{}{} if params.DeactivateCallback != nil { - params.DeactivateCallback() + params.DeactivateCallback(vu) } }() From 1eafd1141caf4c6f4d22db65d4cc7d60b9f9ed13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 6 May 2020 17:59:25 +0200 Subject: [PATCH 240/350] Refactor exec fallback Not sure if this is an improvement honestly, but I didn't find a way to set a default value for null.String. --- cmd/config.go | 6 +----- js/runner.go | 8 ++++---- lib/executor/base_config.go | 9 +++++++-- lib/executor/helpers.go | 16 ++++------------ lib/executors.go | 3 +-- 5 files changed, 17 insertions(+), 25 deletions(-) diff --git a/cmd/config.go b/cmd/config.go index 2a3b2854f3a..94eed747820 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -34,7 +34,6 @@ import ( null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/executor" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/cloud" @@ -294,10 +293,7 @@ func validateConfig(conf Config, exports map[string]struct{}) error { } func validateExecutorConfig(conf lib.ExecutorConfig, exports map[string]struct{}) error { - execFn := conf.GetExec().ValueOrZero() - if execFn == "" { - execFn = consts.DefaultFn - } + execFn := conf.GetExec() if _, ok := exports[execFn]; !ok { return fmt.Errorf("executor %s: %s", conf.GetName(), fmt.Sprintf("function '%s' not found in exports", execFn)) diff --git a/js/runner.go b/js/runner.go index 81472a292d1..6409a826255 100644 --- a/js/runner.go +++ b/js/runner.go @@ -390,6 +390,10 @@ type ActiveVU struct { func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() + if params.Exec == "" { + params.Exec = consts.DefaultFn + } + // Override the preset global env with any custom env vars if len(params.Env) > 0 { env := u.Runtime.Get("__ENV").Export().(map[string]string) @@ -399,10 +403,6 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.Set("__ENV", env) } - if params.Exec == "" { - params.Exec = "default" - } - if len(params.Tags) > 0 { tags := u.Runner.Bundle.Options.RunTags.CloneTags() for k, v := range params.Tags { diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index d0bfc1611e9..86ecc92e1d4 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -28,6 +28,7 @@ import ( null "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/types" ) @@ -120,8 +121,12 @@ func (bc BaseConfig) GetEnv() map[string]string { } // GetExec returns the configured custom exec value, if any. -func (bc BaseConfig) GetExec() null.String { - return bc.Exec +func (bc BaseConfig) GetExec() string { + exec := bc.Exec.ValueOrZero() + if exec == "" { + exec = consts.DefaultFn + } + return exec } // GetTags returns any custom tags configured for the executor. diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index e46bb7f7a6e..a1b53580f1b 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -31,7 +31,6 @@ import ( "github.com/sirupsen/logrus" "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/types" ) @@ -217,21 +216,14 @@ func getArrivalRatePerSec(scaledArrivalRate *big.Rat) *big.Rat { return perSecRate.Mul(perSecRate, scaledArrivalRate) } -func getActivationParams( +func getVUActivationParams( ctx context.Context, conf BaseConfig, deactivateCallback func(lib.InitializedVU), ) *lib.VUActivationParams { - execFn := conf.GetExec().ValueOrZero() - env := conf.GetEnv() - tags := conf.GetTags() - if execFn == "" { - execFn = consts.DefaultFn - } - return &lib.VUActivationParams{ RunContext: ctx, - Exec: execFn, - Env: env, - Tags: tags, + Exec: conf.GetExec(), + Env: conf.GetEnv(), + Tags: conf.GetTags(), DeactivateCallback: deactivateCallback, } } diff --git a/lib/executors.go b/lib/executors.go index e25fe4d2446..8a5f6e4dd99 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -30,7 +30,6 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" @@ -93,7 +92,7 @@ type ExecutorConfig interface { // has been specified. // // TODO: use interface{} so plain http requests can be specified? - GetExec() null.String + GetExec() string GetTags() map[string]string // Calculates the VU requirements in different stages of the executor's From d90078de3a8b88d8fd29b5fb788ce964dcf0beb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 7 May 2020 12:09:56 +0200 Subject: [PATCH 241/350] Check exports with lambda instead of passing object This now also grabs all exported functions, including setup/teardown. Partially resolves: - https://github.com/loadimpact/k6/pull/1428#discussion_r421334198 - https://github.com/loadimpact/k6/pull/1428#discussion_r418036576 --- cmd/archive.go | 2 +- cmd/cloud.go | 2 +- cmd/config.go | 12 ++++---- cmd/config_test.go | 21 +++++++------- cmd/run.go | 2 +- js/bundle.go | 38 ++++++++++++-------------- js/bundle_test.go | 12 ++++---- js/initcontext_test.go | 4 +-- js/runner.go | 15 +++++----- lib/runner.go | 9 ++---- lib/testutils/minirunner/minirunner.go | 6 ++-- 11 files changed, 58 insertions(+), 65 deletions(-) diff --git a/cmd/archive.go b/cmd/archive.go index c6ea9f4d51b..9657e054bd8 100644 --- a/cmd/archive.go +++ b/cmd/archive.go @@ -78,7 +78,7 @@ An archive is a fully self-contained test run, and can be executed identically e return err } - if _, cerr := deriveAndValidateConfig(conf, r.GetExports()); cerr != nil { + if _, cerr := deriveAndValidateConfig(conf, r.IsExecutable); cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/cmd/cloud.go b/cmd/cloud.go index 05d3a8cde3e..c4dfc5dc3c4 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -105,7 +105,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - derivedConf, cerr := deriveAndValidateConfig(conf, r.GetExports()) + derivedConf, cerr := deriveAndValidateConfig(conf, r.IsExecutable) if cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/cmd/config.go b/cmd/config.go index 94eed747820..725d72d13cf 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -262,20 +262,20 @@ func applyDefault(conf Config) Config { return conf } -func deriveAndValidateConfig(conf Config, exports map[string]struct{}) (result Config, err error) { +func deriveAndValidateConfig(conf Config, isExecutable func(string) bool) (result Config, err error) { result = conf result.Options, err = executor.DeriveExecutionFromShortcuts(conf.Options) if err != nil { return result, err } - return result, validateConfig(result, exports) + return result, validateConfig(result, isExecutable) } -func validateConfig(conf Config, exports map[string]struct{}) error { +func validateConfig(conf Config, isExecutable func(string) bool) error { errList := conf.Validate() for _, ec := range conf.Execution { - if err := validateExecutorConfig(ec, exports); err != nil { + if err := validateExecutorConfig(ec, isExecutable); err != nil { errList = append(errList, err) } } @@ -292,9 +292,9 @@ func validateConfig(conf Config, exports map[string]struct{}) error { return errors.New(strings.Join(errMsgParts, "\n")) } -func validateExecutorConfig(conf lib.ExecutorConfig, exports map[string]struct{}) error { +func validateExecutorConfig(conf lib.ExecutorConfig, isExecutable func(string) bool) error { execFn := conf.GetExec() - if _, ok := exports[execFn]; !ok { + if !isExecutable(execFn) { return fmt.Errorf("executor %s: %s", conf.GetName(), fmt.Sprintf("function '%s' not found in exports", execFn)) } diff --git a/cmd/config_test.go b/cmd/config_test.go index a0cc4251e00..35cf32f0cd6 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -144,13 +144,13 @@ func TestDeriveAndValidateConfig(t *testing.T) { t.Parallel() testCases := []struct { - name string - conf Config - exports map[string]struct{} - err string + name string + conf Config + isExec bool + err string }{ - {"defaultOK", Config{}, map[string]struct{}{"default": {}}, ""}, - {"defaultErr", Config{}, map[string]struct{}{}, + {"defaultOK", Config{}, true, ""}, + {"defaultErr", Config{}, false, "executor default: function 'default' not found in exports"}, {"nonDefaultOK", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ @@ -158,8 +158,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { VUs: null.IntFrom(1), Iterations: null.IntFrom(1), MaxDuration: types.NullDurationFrom(time.Second), - }}}}, - map[string]struct{}{"nonDefault": {}}, "", + }}}}, true, "", }, {"nonDefaultErr", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ @@ -167,8 +166,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { VUs: null.IntFrom(1), Iterations: null.IntFrom(1), MaxDuration: types.NullDurationFrom(time.Second), - }}}}, - map[string]struct{}{"nonDefault": {}}, + }}}}, false, "executor per_vu_iters: function 'nonDefaultErr' not found in exports", }, } @@ -176,7 +174,8 @@ func TestDeriveAndValidateConfig(t *testing.T) { for _, tc := range testCases { tc := tc t.Run(tc.name, func(t *testing.T) { - _, err := deriveAndValidateConfig(tc.conf, tc.exports) + _, err := deriveAndValidateConfig(tc.conf, + func(_ string) bool { return tc.isExec }) if tc.err != "" { assert.Contains(t, err.Error(), tc.err) } else { diff --git a/cmd/run.go b/cmd/run.go index bba66f2957e..33900a572a7 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -134,7 +134,7 @@ a commandline interface for interacting with it.`, return err } - conf, cerr := deriveAndValidateConfig(conf, r.GetExports()) + conf, cerr := deriveAndValidateConfig(conf, r.IsExecutable) if cerr != nil { return ExitCode{error: cerr, Code: invalidConfigErrorCode} } diff --git a/js/bundle.go b/js/bundle.go index e5d4026f0de..f6e2686957b 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -45,22 +45,22 @@ type Bundle struct { Filename *url.URL Source string Program *goja.Program - // exported functions, for validation only - Exports map[string]struct{} - Options lib.Options + Options lib.Options BaseInitContext *InitContext Env map[string]string CompatibilityMode lib.CompatibilityMode + + exports map[string]goja.Callable } // A BundleInstance is a self-contained instance of a Bundle. type BundleInstance struct { Runtime *goja.Runtime Context *context.Context - // exported functions, ready for execution - Exports map[string]goja.Callable + + exports map[string]goja.Callable } // NewBundle creates a new bundle from a source file and a filesystem. @@ -83,11 +83,11 @@ func NewBundle(src *loader.SourceData, filesystems map[string]afero.Fs, rtOpts l Filename: src.URL, Source: code, Program: pgm, - Exports: make(map[string]struct{}), BaseInitContext: NewInitContext(rt, c, compatMode, new(context.Context), filesystems, loader.Dir(src.URL)), Env: rtOpts.Env, CompatibilityMode: compatMode, + exports: make(map[string]goja.Callable), } if err := bundle.instantiate(rt, bundle.BaseInitContext); err != nil { return nil, err @@ -140,11 +140,11 @@ func NewBundleFromArchive(arc *lib.Archive, rtOpts lib.RuntimeOptions) (*Bundle, Filename: arc.FilenameURL, Source: string(arc.Data), Program: pgm, - Exports: make(map[string]struct{}), Options: arc.Options, BaseInitContext: initctx, Env: env, CompatibilityMode: compatMode, + exports: make(map[string]goja.Callable), } if err = bundle.instantiate(rt, bundle.BaseInitContext); err != nil { @@ -190,6 +190,10 @@ func (b *Bundle) getExports(rt *goja.Runtime) error { for _, k := range exports.Keys() { v := exports.Get(k) + if fn, ok := goja.AssertFunction(v); ok && k != consts.Options { + b.exports[k] = fn + continue + } switch k { case consts.Options: data, err := json.Marshal(v.Export()) @@ -200,21 +204,13 @@ func (b *Bundle) getExports(rt *goja.Runtime) error { return err } case consts.SetupFn: - if _, ok := goja.AssertFunction(v); !ok { - return errors.New("exported 'setup' must be a function") - } + return errors.New("exported 'setup' must be a function") case consts.TeardownFn: - if _, ok := goja.AssertFunction(v); !ok { - return errors.New("exported 'teardown' must be a function") - } - default: - if _, ok := goja.AssertFunction(v); ok { - b.Exports[k] = struct{}{} - } + return errors.New("exported 'teardown' must be a function") } } - if len(b.Exports) == 0 { + if len(b.exports) == 0 { return errors.New("no exported functions in script") } @@ -238,15 +234,15 @@ func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { bi = &BundleInstance{ Runtime: rt, Context: ctxPtr, - Exports: make(map[string]goja.Callable), + exports: make(map[string]goja.Callable), } // Grab any exported functions that could be executed. These were // already pre-validated in NewBundle(), just get them here. exports := rt.Get("exports").ToObject(rt) - for k := range b.Exports { + for k := range b.exports { fn, _ := goja.AssertFunction(exports.Get(k)) - bi.Exports[k] = fn + bi.exports[k] = fn } jsOptions := rt.Get("options") diff --git a/js/bundle_test.go b/js/bundle_test.go index cff448b394d..fa8ea1800f0 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -436,7 +436,7 @@ func TestNewBundleFromArchive(t *testing.T) { assert.Equal(t, lib.Options{VUs: null.IntFrom(12345)}, b.Options) bi, err := b.Instantiate() require.NoError(t, err) - val, err := bi.Exports["default"](goja.Undefined()) + val, err := bi.exports["default"](goja.Undefined()) require.NoError(t, err) assert.Equal(t, "hi!", val.Export()) } @@ -648,7 +648,7 @@ func TestOpen(t *testing.T) { t.Run(source, func(t *testing.T) { bi, err := b.Instantiate() require.NoError(t, err) - v, err := bi.Exports["default"](goja.Undefined()) + v, err := bi.exports["default"](goja.Undefined()) require.NoError(t, err) assert.Equal(t, "hi", v.Export()) }) @@ -686,7 +686,7 @@ func TestBundleInstantiate(t *testing.T) { } t.Run("Run", func(t *testing.T) { - v, err := bi.Exports["default"](goja.Undefined()) + v, err := bi.exports["default"](goja.Undefined()) if assert.NoError(t, err) { assert.Equal(t, true, v.Export()) } @@ -694,7 +694,7 @@ func TestBundleInstantiate(t *testing.T) { t.Run("SetAndRun", func(t *testing.T) { bi.Runtime.Set("val", false) - v, err := bi.Exports["default"](goja.Undefined()) + v, err := bi.exports["default"](goja.Undefined()) if assert.NoError(t, err) { assert.Equal(t, false, v.Export()) } @@ -749,7 +749,7 @@ func TestBundleEnv(t *testing.T) { bi, err := b.Instantiate() if assert.NoError(t, err) { - _, err := bi.Exports["default"](goja.Undefined()) + _, err := bi.exports["default"](goja.Undefined()) assert.NoError(t, err) } }) @@ -790,7 +790,7 @@ func TestBundleNotSharable(t *testing.T) { require.NoError(t, err) for j := 0; j < iters; j++ { bi.Runtime.Set("__ITER", j) - _, err := bi.Exports["default"](goja.Undefined()) + _, err := bi.exports["default"](goja.Undefined()) assert.NoError(t, err) } } diff --git a/js/initcontext_test.go b/js/initcontext_test.go index 16d6dc3a6c2..216f6e66c73 100644 --- a/js/initcontext_test.go +++ b/js/initcontext_test.go @@ -229,7 +229,7 @@ func TestInitContextRequire(t *testing.T) { if !assert.NoError(t, err) { return } - _, err = bi.Exports["default"](goja.Undefined()) + _, err = bi.exports["default"](goja.Undefined()) assert.NoError(t, err) }) }) @@ -401,7 +401,7 @@ func TestRequestWithBinaryFile(t *testing.T) { ctx = common.WithRuntime(ctx, bi.Runtime) *bi.Context = ctx - v, err := bi.Exports["default"](goja.Undefined()) + v, err := bi.exports["default"](goja.Undefined()) assert.NoError(t, err) assert.NotNil(t, v) assert.Equal(t, true, v.Export()) diff --git a/js/runner.go b/js/runner.go index 6409a826255..8170a61c55f 100644 --- a/js/runner.go +++ b/js/runner.go @@ -267,16 +267,17 @@ func (r *Runner) GetDefaultGroup() *lib.Group { return r.defaultGroup } -// GetExports returns the names of exported functions in the script -// (excluding setup() and teardown()) that can be used for execution. -func (r *Runner) GetExports() map[string]struct{} { - return r.Bundle.Exports -} - func (r *Runner) GetOptions() lib.Options { return r.Bundle.Options } +// IsExecutable returns whether the given name is an exported and +// executable function in the script. +func (r *Runner) IsExecutable(name string) bool { + _, exists := r.Bundle.exports[name] + return exists +} + func (r *Runner) SetOptions(opts lib.Options) error { r.Bundle.Options = opts @@ -460,7 +461,7 @@ func (u *ActiveVU) RunOnce() error { } } - fn, ok := u.Exports[u.Exec] + fn, ok := u.exports[u.Exec] if !ok { // Shouldn't happen; this is validated in ExecutionScheduler.Init() panic(fmt.Sprintf("function '%s' not found in exports", u.Exec)) diff --git a/lib/runner.go b/lib/runner.go index 0292d954329..2bc44522828 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -91,10 +91,7 @@ type Runner interface { GetOptions() Options SetOptions(opts Options) error - // GetExports returns the names of exported functions in the script - // (excluding setup() and teardown()) that can be used for execution. - // This is a bit janky, but it's needed for validation during - // ExecutionScheduler.Init(). The empty struct is to avoid a - // circular dep or make lib depend on goja :-/ - GetExports() map[string]struct{} + // Returns whether the given name is an exported and executable + // function in the script. + IsExecutable(string) bool } diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index e62137f88a3..63dcce0fbed 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -95,10 +95,10 @@ func (r MiniRunner) GetDefaultGroup() *lib.Group { return r.Group } -// GetExports satisfies lib.Runner, but is mocked for MiniRunner since +// IsExecutable satisfies lib.Runner, but is mocked for MiniRunner since // it doesn't deal with JS. -func (r MiniRunner) GetExports() map[string]struct{} { - return map[string]struct{}{"default": {}} +func (r MiniRunner) IsExecutable(name string) bool { + return true } // GetOptions returns the supplied options struct. From ae5ef34fabb03b83f8d99b874e537bc6e2f164f1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 11:18:43 +0200 Subject: [PATCH 242/350] Fix custom exec tags not leaking between executors Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r422189193 --- core/local/local_test.go | 144 +++++++++++++++++++++++++++++++--- js/runner.go | 19 ++--- lib/netext/httpext/request.go | 3 + lib/state.go | 1 + 4 files changed, 147 insertions(+), 20 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index ee5f00d458b..f5363b144c8 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -26,6 +26,7 @@ import ( "fmt" "net" "net/url" + "reflect" "runtime" "sync/atomic" "testing" @@ -253,9 +254,7 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { for { select { case sample := <-samples: - // TODO: Implement a more robust way of reporting - // errors in these high-level functional tests. - if _, ok := sample.(stats.Sample); ok { + if s, ok := sample.(stats.Sample); ok && s.Metric.Name == "errors" { assert.FailNow(t, "received error sample from test") } case <-done: @@ -342,7 +341,7 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { execScheduler, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) defer cancel() done := make(chan struct{}) @@ -352,19 +351,25 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) close(done) }() - var gotTag bool + var gotTrailTag, gotNetTrailTag bool for { select { case sample := <-samples: - if trail, ok := sample.(*httpext.Trail); ok && !gotTag { + if trail, ok := sample.(*httpext.Trail); ok && !gotTrailTag { tags := trail.Tags.CloneTags() if v, ok := tags["customTag"]; ok && v == "value" { - gotTag = true + gotTrailTag = true + } + } + if netTrail, ok := sample.(*netext.NetTrail); ok && !gotNetTrailTag { + tags := netTrail.Tags.CloneTags() + if v, ok := tags["customTag"]; ok && v == "value" { + gotNetTrailTag = true } } case <-done: - if !gotTag { - assert.FailNow(t, "sample with tag wasn't received") + if !gotTrailTag || !gotNetTrailTag { + assert.FailNow(t, "a sample with expected tag wasn't received") } return } @@ -373,6 +378,127 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { } } +// Ensure that custom executor settings are unique per executor and +// that there's no "crossover"/"pollution" between executors. +func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { + t.Parallel() + tb := httpmultibin.NewHTTPMultiBin(t) + defer tb.Cleanup() + sr := tb.Replacer.Replace + + script := sr(` + import http from "k6/http"; + import { Counter } from 'k6/metrics'; + + let errors = new Counter('errors'); + + export let options = { + execution: { + scenario1: { + type: 'per-vu-iterations', + vus: 1, + iterations: 1, + gracefulStop: '0.5s', + exec: 's1func', + env: { TESTVAR1: 'scenario1' }, + tags: { testtag1: 'scenario1' }, + }, + scenario2: { + type: 'shared-iterations', + vus: 1, + iterations: 1, + gracefulStop: '0.5s', + exec: 's2func', + env: { TESTVAR2: 'scenario2' }, + tags: { testtag2: 'scenario2' }, + }, + } + } + + function checkVar(name, expected) { + if (__ENV[name] !== expected) { + console.error('Wrong ' + name + " env var value. Expected: '" + + expected + "', actual: '" + __ENV[name] + "'"); + errors.add(1); + } + } + + export function s1func() { + checkVar('TESTVAR1', 'scenario1'); + checkVar('TESTVAR2', undefined); + checkVar('TESTGLOBALVAR', 'global'); + + http.get('HTTPBIN_IP_URL/', { tags: { reqtag: 'scenario1' }}); + } + + export function s2func() { + checkVar('TESTVAR1', undefined); + checkVar('TESTVAR2', 'scenario2'); + checkVar('TESTGLOBALVAR', 'global'); + + http.get('HTTPBIN_IP_URL/', { tags: { reqtag: 'scenario2' }}); + }`) + + runner, err := js.New(&loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, + Data: []byte(script)}, + nil, lib.RuntimeOptions{Env: map[string]string{"TESTGLOBALVAR": "global"}}) + require.NoError(t, err) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + + done := make(chan struct{}) + samples := make(chan stats.SampleContainer) + go func() { + assert.NoError(t, execScheduler.Init(ctx, samples)) + assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) + close(done) + }() + + expectedTrailTags := []map[string]string{ + {"testtag1": "scenario1", "reqtag": "scenario1"}, + {"testtag2": "scenario2", "reqtag": "scenario2"}, + } + expectedNetTrailTags := []map[string]string{ + {"testtag1": "scenario1"}, + {"testtag2": "scenario2"}, + } + var gotSampleTags int + for { + select { + case sample := <-samples: + if s, ok := sample.(stats.Sample); ok && s.Metric.Name == "errors" { + assert.FailNow(t, "received error sample from test") + } + if trail, ok := sample.(*httpext.Trail); ok { + tags := trail.Tags.CloneTags() + for _, expTags := range expectedTrailTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ + } + } + } + if netTrail, ok := sample.(*netext.NetTrail); ok { + tags := netTrail.Tags.CloneTags() + for _, expTags := range expectedNetTrailTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ + } + } + } + case <-done: + require.Equal(t, 4, gotSampleTags, "received wrong amount of samples with expected tags") + return + } + } +} + func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { t.Parallel() t.Run("Normal", func(t *testing.T) { diff --git a/js/runner.go b/js/runner.go index 8170a61c55f..b1267376ad9 100644 --- a/js/runner.go +++ b/js/runner.go @@ -328,7 +328,7 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, return goja.Undefined(), err } - v, _, _, err := vu.runFn(ctx, group, false, fn, vu.Runtime.ToValue(arg)) + v, _, _, err := vu.runFn(ctx, group, false, nil, fn, vu.Runtime.ToValue(arg)) // deadline is reached so we have timeouted but this might've not been registered correctly if deadline, ok := ctx.Deadline(); ok && time.Now().After(deadline) { @@ -404,14 +404,6 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.Set("__ENV", env) } - if len(params.Tags) > 0 { - tags := u.Runner.Bundle.Options.RunTags.CloneTags() - for k, v := range params.Tags { - tags[k] = v - } - u.Runner.Bundle.Options.RunTags = stats.IntoSampleTags(&tags) - } - avu := &ActiveVU{ VU: u, VUActivationParams: params, @@ -469,7 +461,7 @@ func (u *ActiveVU) RunOnce() error { // Call the exported function. _, isFullIteration, totalTime, err := u.runFn( - u.RunContext, u.Runner.defaultGroup, true, fn, u.setupData, + u.RunContext, u.Runner.defaultGroup, true, u.Tags, fn, u.setupData, ) // If MinIterationDuration is specified and the iteration wasn't cancelled @@ -485,7 +477,8 @@ func (u *ActiveVU) RunOnce() error { } func (u *VU) runFn( - ctx context.Context, group *lib.Group, isDefault bool, fn goja.Callable, args ...goja.Value, + ctx context.Context, group *lib.Group, isDefault bool, customTags map[string]string, + fn goja.Callable, args ...goja.Value, ) (goja.Value, bool, time.Duration, error) { cookieJar := u.CookieJar if !u.Runner.Bundle.Options.NoCookiesReset.ValueOrZero() { @@ -509,6 +502,7 @@ func (u *VU) runFn( Vu: u.ID, Samples: u.Samples, Iteration: u.Iteration, + Tags: customTags, } newctx := common.WithRuntime(ctx, u.Runtime) @@ -541,6 +535,9 @@ func (u *VU) runFn( if state.Options.SystemTags.Has(stats.TagGroup) { tags["group"] = group.Path } + for k, v := range customTags { + tags[k] = v + } if u.Runner.Bundle.Options.NoVUConnectionReuse.Bool { u.Transport.CloseIdleConnections() diff --git a/lib/netext/httpext/request.go b/lib/netext/httpext/request.go index 72acd545022..7ea374cea65 100644 --- a/lib/netext/httpext/request.go +++ b/lib/netext/httpext/request.go @@ -232,6 +232,9 @@ func MakeRequest(ctx context.Context, preq *ParsedHTTPRequest) (*Response, error } tags := state.Options.RunTags.CloneTags() + for k, v := range state.Tags { + tags[k] = v + } for k, v := range preq.Tags { tags[k] = v } diff --git a/lib/state.go b/lib/state.go index 08df1ee05f9..1ffa2807c3a 100644 --- a/lib/state.go +++ b/lib/state.go @@ -67,4 +67,5 @@ type State struct { BPool *bpool.BufferPool Vu, Iteration int64 + Tags map[string]string } From 60c3c099bba633e207c0c85559cafad74bb2a57d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 13:14:06 +0200 Subject: [PATCH 243/350] Revert moving executor init before VU init This was done in e693715b to ensure exec validation would fail before VUs are initialized, but now that this validation is done much earlier, this is no longer needed. Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r422157781 --- core/local/local.go | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index b3439faded4..7b1ab217664 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -224,19 +224,6 @@ func (e *ExecutionScheduler) initVUsConcurrently( func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.SampleContainer) error { logger := e.logger.WithField("phase", "local-execution-scheduler-init") - // Initialize each executor - e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) - logger.Debugf("Start initializing executors...") - errMsg := "error while initializing executor %s: %s" - for _, exec := range e.executors { - executorConfig := exec.GetConfig() - if err := exec.Init(ctx); err != nil { - return fmt.Errorf(errMsg, executorConfig.GetName(), err) - } - logger.Debugf("Initialized executor %s", executorConfig.GetName()) - } - - logger.Debugf("Finished initializing executors, start initializing VUs...") vusToInitialize := lib.GetMaxPlannedVUs(e.executionPlan) logger.WithFields(logrus.Fields{ "neededVUs": vusToInitialize, @@ -278,6 +265,17 @@ func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.S return e.initVU(samplesOut, logger) }) + e.state.SetExecutionStatus(lib.ExecutionStatusInitExecutors) + logger.Debugf("Finished initializing needed VUs, start initializing executors...") + for _, exec := range e.executors { + executorConfig := exec.GetConfig() + + if err := exec.Init(ctx); err != nil { + return fmt.Errorf("error while initializing executor %s: %s", executorConfig.GetName(), err) + } + logger.Debugf("Initialized executor %s", executorConfig.GetName()) + } + e.state.SetExecutionStatus(lib.ExecutionStatusInitDone) logger.Debugf("Initialization completed") return nil From 25a26ab65076e80007eecfb859485600c9827ffb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 13:21:45 +0200 Subject: [PATCH 244/350] Get new VUActivationParams instance per VU in some executors The minor optimization wasn't worth the messiness. Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r422187179 --- lib/executor/constant_looping_vus.go | 15 ++++++--------- lib/executor/per_vu_iterations.go | 15 ++++++--------- lib/executor/shared_iterations.go | 15 ++++++--------- 3 files changed, 18 insertions(+), 27 deletions(-) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index af4850b2539..70a83e1a1d4 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -176,19 +176,16 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executionState, clv.logger) - activationParams := getVUActivationParams(maxDurationCtx, clv.config.BaseConfig, - func(u lib.InitializedVU) { - clv.executionState.ReturnVU(u, true) - activeVUs.Done() - }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - newParams := *activationParams - newParams.RunContext = ctx - - activeVU := initVU.Activate(&newParams) + activationParams := getVUActivationParams(ctx, clv.config.BaseConfig, + func(u lib.InitializedVU) { + clv.executionState.ReturnVU(u, true) + activeVUs.Done() + }) + activeVU := initVU.Activate(activationParams) for { select { diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index c6fe8691014..8f3ada21058 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -197,19 +197,16 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executionState, pvi.logger) - activationParams := getVUActivationParams(maxDurationCtx, pvi.config.BaseConfig, - func(u lib.InitializedVU) { - pvi.executionState.ReturnVU(u, true) - activeVUs.Done() - }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - newParams := *activationParams - newParams.RunContext = ctx - - activeVU := initVU.Activate(&newParams) + activationParams := getVUActivationParams(ctx, pvi.config.BaseConfig, + func(u lib.InitializedVU) { + pvi.executionState.ReturnVU(u, true) + activeVUs.Done() + }) + activeVU := initVU.Activate(activationParams) for i := int64(0); i < iterations; i++ { select { diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 5dc4663cedd..64012114e89 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -216,19 +216,16 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) - activationParams := getVUActivationParams(maxDurationCtx, si.config.BaseConfig, - func(u lib.InitializedVU) { - si.executionState.ReturnVU(u, true) - activeVUs.Done() - }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - newParams := *activationParams - newParams.RunContext = ctx - - activeVU := initVU.Activate(&newParams) + activationParams := getVUActivationParams(ctx, si.config.BaseConfig, + func(u lib.InitializedVU) { + si.executionState.ReturnVU(u, true) + activeVUs.Done() + }) + activeVU := initVU.Activate(activationParams) for { select { From 80bff4d3c72906a6f75b4880cdeb9ed4e1e83285 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 14:33:47 +0200 Subject: [PATCH 245/350] Add custom executor tags to WebSocket metrics --- core/local/local_test.go | 55 ++++++++++++++++++++++++++++++++++++++-- js/modules/k6/ws/ws.go | 4 +++ 2 files changed, 57 insertions(+), 2 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index f5363b144c8..ade561251e7 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -388,6 +388,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { script := sr(` import http from "k6/http"; + import ws from 'k6/ws'; import { Counter } from 'k6/metrics'; let errors = new Counter('errors'); @@ -412,6 +413,15 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { env: { TESTVAR2: 'scenario2' }, tags: { testtag2: 'scenario2' }, }, + scenario3: { + type: 'per-vu-iterations', + vus: 1, + iterations: 1, + gracefulStop: '0.5s', + exec: 's3funcWS', + env: { TESTVAR3: 'scenario3' }, + tags: { testtag3: 'scenario3' }, + }, } } @@ -426,6 +436,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { export function s1func() { checkVar('TESTVAR1', 'scenario1'); checkVar('TESTVAR2', undefined); + checkVar('TESTVAR3', undefined); checkVar('TESTGLOBALVAR', 'global'); http.get('HTTPBIN_IP_URL/', { tags: { reqtag: 'scenario1' }}); @@ -434,10 +445,39 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { export function s2func() { checkVar('TESTVAR1', undefined); checkVar('TESTVAR2', 'scenario2'); + checkVar('TESTVAR3', undefined); checkVar('TESTGLOBALVAR', 'global'); http.get('HTTPBIN_IP_URL/', { tags: { reqtag: 'scenario2' }}); - }`) + } + + export function s3funcWS() { + checkVar('TESTVAR1', undefined); + checkVar('TESTVAR2', undefined); + checkVar('TESTVAR3', 'scenario3'); + checkVar('TESTGLOBALVAR', 'global'); + + const customTags = { wstag: 'scenario3' }; + const response = ws.connect('WSBIN_URL/ws-echo', { tags: customTags }, + function (socket) { + socket.on('open', function() { + socket.send('hello'); + }); + socket.on('message', function(msg) { + if (msg != 'hello') { + console.error("Expected to receive 'hello' but got '" + msg + "' instead!"); + errors.add(1); + } + socket.close() + }); + socket.on('error', function (e) { + console.log('ws error: ' + e.error()); + errors.add(1); + }); + } + ); + } +`) runner, err := js.New(&loader.SourceData{ URL: &url.URL{Path: "/script.js"}, @@ -469,6 +509,9 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { {"testtag1": "scenario1"}, {"testtag2": "scenario2"}, } + expectedConnSampleTags := map[string]string{ + "testtag3": "scenario3", "wstag": "scenario3", + } var gotSampleTags int for { select { @@ -492,8 +535,16 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { } } } + if cs, ok := sample.(stats.ConnectedSamples); ok { + for _, s := range cs.Samples { + tags := s.Tags.CloneTags() + if reflect.DeepEqual(expectedConnSampleTags, tags) { + gotSampleTags++ + } + } + } case <-done: - require.Equal(t, 4, gotSampleTags, "received wrong amount of samples with expected tags") + require.Equal(t, 6, gotSampleTags, "received wrong amount of samples with expected tags") return } } diff --git a/js/modules/k6/ws/ws.go b/js/modules/k6/ws/ws.go index 1302126cd98..3bb7aa45c05 100644 --- a/js/modules/k6/ws/ws.go +++ b/js/modules/k6/ws/ws.go @@ -34,6 +34,7 @@ import ( "github.com/dop251/goja" "github.com/gorilla/websocket" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" @@ -104,6 +105,9 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP var header http.Header tags := state.Options.RunTags.CloneTags() + for k, v := range state.Tags { + tags[k] = v + } // Parse the optional second argument (params) if !goja.IsUndefined(paramsV) && !goja.IsNull(paramsV) { From 06aa32d0a8ec67033ae2170beaf12627df5922db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 14:53:41 +0200 Subject: [PATCH 246/350] Add custom executor tags to checks and group metrics --- core/local/local_test.go | 54 ++++++++++++++++++++++++++-------------- js/modules/k6/k6.go | 9 ++++++- 2 files changed, 44 insertions(+), 19 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index ade561251e7..2331c88bf4d 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -380,6 +380,7 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { // Ensure that custom executor settings are unique per executor and // that there's no "crossover"/"pollution" between executors. +// Also test that custom tags are properly set on checks and groups metrics. func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { t.Parallel() tb := httpmultibin.NewHTTPMultiBin(t) @@ -390,6 +391,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { import http from "k6/http"; import ws from 'k6/ws'; import { Counter } from 'k6/metrics'; + import { check, group } from 'k6'; let errors = new Counter('errors'); @@ -458,24 +460,27 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { checkVar('TESTGLOBALVAR', 'global'); const customTags = { wstag: 'scenario3' }; - const response = ws.connect('WSBIN_URL/ws-echo', { tags: customTags }, - function (socket) { - socket.on('open', function() { - socket.send('hello'); - }); - socket.on('message', function(msg) { - if (msg != 'hello') { - console.error("Expected to receive 'hello' but got '" + msg + "' instead!"); + group('wsgroup', function() { + const response = ws.connect('WSBIN_URL/ws-echo', { tags: customTags }, + function (socket) { + socket.on('open', function() { + socket.send('hello'); + }); + socket.on('message', function(msg) { + if (msg != 'hello') { + console.error("Expected to receive 'hello' but got '" + msg + "' instead!"); + errors.add(1); + } + socket.close() + }); + socket.on('error', function (e) { + console.log('ws error: ' + e.error()); errors.add(1); - } - socket.close() - }); - socket.on('error', function (e) { - console.log('ws error: ' + e.error()); - errors.add(1); - }); - } - ); + }); + } + ); + check(response, { 'status is 101': (r) => r && r.status === 101 }, customTags); + }); } `) @@ -512,6 +517,10 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { expectedConnSampleTags := map[string]string{ "testtag3": "scenario3", "wstag": "scenario3", } + expectedPlainSampleTags := []map[string]string{ + {"testtag3": "scenario3"}, + {"testtag3": "scenario3", "wstag": "scenario3"}, + } var gotSampleTags int for { select { @@ -543,8 +552,17 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { } } } + if s, ok := sample.(stats.Sample); ok && + (s.Metric.Name == "checks" || s.Metric.Name == "group_duration") { + tags := s.Tags.CloneTags() + for _, expTags := range expectedPlainSampleTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ + } + } + } case <-done: - require.Equal(t, 6, gotSampleTags, "received wrong amount of samples with expected tags") + require.Equal(t, 8, gotSampleTags, "received wrong amount of samples with expected tags") return } } diff --git a/js/modules/k6/k6.go b/js/modules/k6/k6.go index 084ece55915..a97a9d254df 100644 --- a/js/modules/k6/k6.go +++ b/js/modules/k6/k6.go @@ -28,11 +28,12 @@ import ( "time" "github.com/dop251/goja" + "github.com/pkg/errors" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/stats" - "github.com/pkg/errors" ) type K6 struct{} @@ -91,6 +92,9 @@ func (*K6) Group(ctx context.Context, name string, fn goja.Callable) (goja.Value t := time.Now() tags := state.Options.RunTags.CloneTags() + for k, v := range state.Tags { + tags[k] = v + } if state.Options.SystemTags.Has(stats.TagGroup) { tags["group"] = g.Path } @@ -121,6 +125,9 @@ func (*K6) Check(ctx context.Context, arg0, checks goja.Value, extras ...goja.Va // Prepare tags, make sure the `group` tag can't be overwritten. commonTags := state.Options.RunTags.CloneTags() + for k, v := range state.Tags { + commonTags[k] = v + } if state.Options.SystemTags.Has(stats.TagGroup) { commonTags["group"] = state.Group.Path } From d25c71a24c337029f132e82d5d978cac70af56f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 14:59:44 +0200 Subject: [PATCH 247/350] Remove superfluous use of fmt.Sprintf Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r422140465 --- cmd/config.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cmd/config.go b/cmd/config.go index 725d72d13cf..51b9a07ad88 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -295,8 +295,7 @@ func validateConfig(conf Config, isExecutable func(string) bool) error { func validateExecutorConfig(conf lib.ExecutorConfig, isExecutable func(string) bool) error { execFn := conf.GetExec() if !isExecutable(execFn) { - return fmt.Errorf("executor %s: %s", conf.GetName(), - fmt.Sprintf("function '%s' not found in exports", execFn)) + return fmt.Errorf("executor %s: function '%s' not found in exports", conf.GetName(), execFn) } return nil } From 8c5d8df3e1c7d62a516b97d12b0a6bff628118d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 17:57:57 +0200 Subject: [PATCH 248/350] Define httpbin.local host in NoCrossover test --- core/local/local_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/local/local_test.go b/core/local/local_test.go index 2331c88bf4d..09bfa369165 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -396,6 +396,8 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { let errors = new Counter('errors'); export let options = { + // Required for WS tests + hosts: { 'httpbin.local': '127.0.0.1' }, execution: { scenario1: { type: 'per-vu-iterations', From 28032b695334a771f99a782d895f7090e11578bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 11 May 2020 18:44:52 +0200 Subject: [PATCH 249/350] Cleanup NoCrossover test a bit --- core/local/local_test.go | 41 ++++++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 21 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 09bfa369165..e28accb9a0e 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -527,42 +527,41 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { for { select { case sample := <-samples: - if s, ok := sample.(stats.Sample); ok && s.Metric.Name == "errors" { - assert.FailNow(t, "received error sample from test") - } - if trail, ok := sample.(*httpext.Trail); ok { - tags := trail.Tags.CloneTags() + switch s := sample.(type) { + case stats.Sample: + if s.Metric.Name == "errors" { + assert.FailNow(t, "received error sample from test") + } + if s.Metric.Name == "checks" || s.Metric.Name == "group_duration" { + tags := s.Tags.CloneTags() + for _, expTags := range expectedPlainSampleTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ + } + } + } + case *httpext.Trail: + tags := s.Tags.CloneTags() for _, expTags := range expectedTrailTags { if reflect.DeepEqual(expTags, tags) { gotSampleTags++ } } - } - if netTrail, ok := sample.(*netext.NetTrail); ok { - tags := netTrail.Tags.CloneTags() + case *netext.NetTrail: + tags := s.Tags.CloneTags() for _, expTags := range expectedNetTrailTags { if reflect.DeepEqual(expTags, tags) { gotSampleTags++ } } - } - if cs, ok := sample.(stats.ConnectedSamples); ok { - for _, s := range cs.Samples { - tags := s.Tags.CloneTags() + case stats.ConnectedSamples: + for _, sm := range s.Samples { + tags := sm.Tags.CloneTags() if reflect.DeepEqual(expectedConnSampleTags, tags) { gotSampleTags++ } } } - if s, ok := sample.(stats.Sample); ok && - (s.Metric.Name == "checks" || s.Metric.Name == "group_duration") { - tags := s.Tags.CloneTags() - for _, expTags := range expectedPlainSampleTags { - if reflect.DeepEqual(expTags, tags) { - gotSampleTags++ - } - } - } case <-done: require.Equal(t, 8, gotSampleTags, "received wrong amount of samples with expected tags") return From ba076f6ee5c3554e41ac5f9cafcbfa6747c02214 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 12 May 2020 12:06:19 +0200 Subject: [PATCH 250/350] Revert "Get new VUActivationParams instance per VU in some executors" This reverts commit 25a26ab65076e80007eecfb859485600c9827ffb. Concurrent calls to getVUActivationParams would cause race conditions when reading from BaseConfig and several tests would fail, so this needs to be done only once per executor, and we need this RunContext overriding after all. --- lib/executor/constant_looping_vus.go | 15 +++++++++------ lib/executor/per_vu_iterations.go | 15 +++++++++------ lib/executor/shared_iterations.go | 15 +++++++++------ 3 files changed, 27 insertions(+), 18 deletions(-) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 70a83e1a1d4..af4850b2539 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -176,16 +176,19 @@ func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(clv.executionState, clv.logger) + activationParams := getVUActivationParams(maxDurationCtx, clv.config.BaseConfig, + func(u lib.InitializedVU) { + clv.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - activationParams := getVUActivationParams(ctx, clv.config.BaseConfig, - func(u lib.InitializedVU) { - clv.executionState.ReturnVU(u, true) - activeVUs.Done() - }) - activeVU := initVU.Activate(activationParams) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for { select { diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 8f3ada21058..c6fe8691014 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -197,16 +197,19 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(pvi.executionState, pvi.logger) + activationParams := getVUActivationParams(maxDurationCtx, pvi.config.BaseConfig, + func(u lib.InitializedVU) { + pvi.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - activationParams := getVUActivationParams(ctx, pvi.config.BaseConfig, - func(u lib.InitializedVU) { - pvi.executionState.ReturnVU(u, true) - activeVUs.Done() - }) - activeVU := initVU.Activate(activationParams) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for i := int64(0); i < iterations; i++ { select { diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 64012114e89..5dc4663cedd 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -216,16 +216,19 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta attemptedIters := new(uint64) + activationParams := getVUActivationParams(maxDurationCtx, si.config.BaseConfig, + func(u lib.InitializedVU) { + si.executionState.ReturnVU(u, true) + activeVUs.Done() + }) handleVU := func(initVU lib.InitializedVU) { ctx, cancel := context.WithCancel(maxDurationCtx) defer cancel() - activationParams := getVUActivationParams(ctx, si.config.BaseConfig, - func(u lib.InitializedVU) { - si.executionState.ReturnVU(u, true) - activeVUs.Done() - }) - activeVU := initVU.Activate(activationParams) + newParams := *activationParams + newParams.RunContext = ctx + + activeVU := initVU.Activate(&newParams) for { select { From 4cf716c64c2a0a670ebfa9d3fa819f648aefc6a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 12 May 2020 10:25:01 +0200 Subject: [PATCH 251/350] Merge RunTags into state.Tags and reuse that everywhere This avoids having to read state.Options in some cases, as that should be done only once in runFn. Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r423027859 --- js/modules/k6/k6.go | 20 ++------------------ js/modules/k6/metrics/metrics.go | 6 +++++- js/modules/k6/ws/ws.go | 2 +- js/runner.go | 32 ++++++++++++++++---------------- 4 files changed, 24 insertions(+), 36 deletions(-) diff --git a/js/modules/k6/k6.go b/js/modules/k6/k6.go index a97a9d254df..23de61d8229 100644 --- a/js/modules/k6/k6.go +++ b/js/modules/k6/k6.go @@ -23,7 +23,6 @@ package k6 import ( "context" "math/rand" - "strconv" "sync/atomic" "time" @@ -91,19 +90,10 @@ func (*K6) Group(ctx context.Context, name string, fn goja.Callable) (goja.Value ret, err := fn(goja.Undefined()) t := time.Now() - tags := state.Options.RunTags.CloneTags() + tags := map[string]string{} for k, v := range state.Tags { tags[k] = v } - if state.Options.SystemTags.Has(stats.TagGroup) { - tags["group"] = g.Path - } - if state.Options.SystemTags.Has(stats.TagVU) { - tags["vu"] = strconv.FormatInt(state.Vu, 10) - } - if state.Options.SystemTags.Has(stats.TagIter) { - tags["iter"] = strconv.FormatInt(state.Iteration, 10) - } stats.PushIfNotDone(ctx, state.Samples, stats.Sample{ Time: t, @@ -124,7 +114,7 @@ func (*K6) Check(ctx context.Context, arg0, checks goja.Value, extras ...goja.Va t := time.Now() // Prepare tags, make sure the `group` tag can't be overwritten. - commonTags := state.Options.RunTags.CloneTags() + commonTags := map[string]string{} for k, v := range state.Tags { commonTags[k] = v } @@ -137,12 +127,6 @@ func (*K6) Check(ctx context.Context, arg0, checks goja.Value, extras ...goja.Va commonTags[k] = obj.Get(k).String() } } - if state.Options.SystemTags.Has(stats.TagVU) { - commonTags["vu"] = strconv.FormatInt(state.Vu, 10) - } - if state.Options.SystemTags.Has(stats.TagIter) { - commonTags["iter"] = strconv.FormatInt(state.Iteration, 10) - } succ := true var exc error diff --git a/js/modules/k6/metrics/metrics.go b/js/modules/k6/metrics/metrics.go index 4de9651f5e8..8153673f53b 100644 --- a/js/modules/k6/metrics/metrics.go +++ b/js/modules/k6/metrics/metrics.go @@ -28,6 +28,7 @@ import ( "time" "github.com/dop251/goja" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" @@ -73,7 +74,10 @@ func (m Metric) Add(ctx context.Context, v goja.Value, addTags ...map[string]str return false, ErrMetricsAddInInitContext } - tags := state.Options.RunTags.CloneTags() + tags := map[string]string{} + for k, v := range state.Tags { + tags[k] = v + } if state.Options.SystemTags.Has(stats.TagGroup) { tags["group"] = state.Group.Path } diff --git a/js/modules/k6/ws/ws.go b/js/modules/k6/ws/ws.go index 3bb7aa45c05..dfa1ffe439c 100644 --- a/js/modules/k6/ws/ws.go +++ b/js/modules/k6/ws/ws.go @@ -104,7 +104,7 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP // Leave header to nil by default so we can pass it directly to the Dialer var header http.Header - tags := state.Options.RunTags.CloneTags() + tags := map[string]string{} for k, v := range state.Tags { tags[k] = v } diff --git a/js/runner.go b/js/runner.go index b1267376ad9..cc326994abb 100644 --- a/js/runner.go +++ b/js/runner.go @@ -489,6 +489,21 @@ func (u *VU) runFn( } } + opts := &u.Runner.Bundle.Options + tags := opts.RunTags.CloneTags() + for k, v := range customTags { + tags[k] = v + } + if opts.SystemTags.Has(stats.TagVU) { + tags["vu"] = strconv.FormatInt(u.ID, 10) + } + if opts.SystemTags.Has(stats.TagIter) { + tags["iter"] = strconv.FormatInt(u.Iteration, 10) + } + if opts.SystemTags.Has(stats.TagGroup) { + tags["group"] = group.Path + } + state := &lib.State{ Logger: u.Runner.Logger, Options: u.Runner.Bundle.Options, @@ -502,7 +517,7 @@ func (u *VU) runFn( Vu: u.ID, Samples: u.Samples, Iteration: u.Iteration, - Tags: customTags, + Tags: tags, } newctx := common.WithRuntime(ctx, u.Runtime) @@ -510,7 +525,6 @@ func (u *VU) runFn( *u.Context = newctx u.Runtime.Set("__ITER", u.Iteration) - iter := u.Iteration u.Iteration++ startTime := time.Now() @@ -525,20 +539,6 @@ func (u *VU) runFn( isFullIteration = true } - tags := state.Options.RunTags.CloneTags() - if state.Options.SystemTags.Has(stats.TagVU) { - tags["vu"] = strconv.FormatInt(u.ID, 10) - } - if state.Options.SystemTags.Has(stats.TagIter) { - tags["iter"] = strconv.FormatInt(iter, 10) - } - if state.Options.SystemTags.Has(stats.TagGroup) { - tags["group"] = group.Path - } - for k, v := range customTags { - tags[k] = v - } - if u.Runner.Bundle.Options.NoVUConnectionReuse.Bool { u.Transport.CloseIdleConnections() } From 438b422e8111eda2ed5cce45cea4654f8f310e64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 12 May 2020 15:58:16 +0200 Subject: [PATCH 252/350] Move newManualVUHandle function to EC RunState Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r423596022 --- lib/executor/externally_controlled.go | 26 +++++++++----------------- 1 file changed, 9 insertions(+), 17 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 0c352d80042..311031bae6b 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -339,26 +339,25 @@ type manualVUHandle struct { cancelVU func() } -func newManualVUHandle( - parentCtx context.Context, state *lib.ExecutionState, - localActiveVUsCount *int64, initVU lib.InitializedVU, - config *BaseConfig, logger *logrus.Entry, +func (rs *externallyControlledRunState) newManualVUHandle( + initVU lib.InitializedVU, logger *logrus.Entry, ) *manualVUHandle { wg := sync.WaitGroup{} + state := rs.executor.executionState getVU := func() (lib.InitializedVU, error) { wg.Add(1) state.ModCurrentlyActiveVUsCount(+1) - atomic.AddInt64(localActiveVUsCount, +1) + atomic.AddInt64(rs.activeVUsCount, +1) return initVU, nil } returnVU := func(_ lib.InitializedVU) { state.ModCurrentlyActiveVUsCount(-1) - atomic.AddInt64(localActiveVUsCount, -1) + atomic.AddInt64(rs.activeVUsCount, -1) wg.Done() } - ctx, cancel := context.WithCancel(parentCtx) + ctx, cancel := context.WithCancel(rs.ctx) return &manualVUHandle{ - vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, config, logger), + vuHandle: newStoppedVUHandle(ctx, getVU, returnVU, &rs.executor.config.BaseConfig, logger), initVU: initVU, wg: &wg, cancelVU: cancel, @@ -391,10 +390,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { if vuGetErr != nil { return vuGetErr } - vuHandle := newManualVUHandle( - rs.ctx, rs.executor.executionState, rs.activeVUsCount, initVU, - &rs.executor.config.BaseConfig, rs.executor.logger.WithField("vuNum", i), - ) + vuHandle := rs.newManualVUHandle(initVU, rs.executor.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles[i] = vuHandle } @@ -450,11 +446,7 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern if vuInitErr != nil { return vuInitErr } - vuHandle := newManualVUHandle( - rs.ctx, executionState, rs.activeVUsCount, initVU, - &rs.executor.config.BaseConfig, - rs.executor.logger.WithField("vuNum", i), - ) + vuHandle := rs.newManualVUHandle(initVU, rs.executor.logger.WithField("vuNum", i)) go vuHandle.runLoopsIfPossible(rs.runIteration) rs.vuHandles = append(rs.vuHandles, vuHandle) } From 56c0a89106144342a812ef00334ed6ff8ffd5813 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 12 May 2020 16:41:46 +0200 Subject: [PATCH 253/350] Minor simplification of VU activation in vuHandle Resolves https://github.com/loadimpact/k6/pull/1428#discussion_r423598061 --- lib/executor/vu_handle.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 42affe53842..1dbf6410f0b 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -155,10 +155,7 @@ mainLoop: if err != nil { return } - activationParams := getVUActivationParams(ctx, *vh.config, - func(u lib.InitializedVU) { - vh.returnVU(u) - }) + activationParams := getVUActivationParams(ctx, *vh.config, vh.returnVU) vu = initVU.Activate(activationParams) } From 60b2ffe9fb2f07383a33d5d8fcf74ffc14efed5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 12 May 2020 16:35:09 +0200 Subject: [PATCH 254/350] Change defer order in VLV Run This applies the same fixes as eb13303f and d01e2c19 for the VLV executor, to ensure activeVUsWG.Wait() is called last, after the context has been cancelled. It resolves some race conditions exhibited by the TestExecutionSchedulerRun* tests (e.g. [1]). [1]: https://circleci.com/gh/loadimpact/k6/10139 --- lib/executor/variable_looping_vus.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index b8ae969fe32..3253734d3a6 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -548,6 +548,9 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo maxVUs := lib.GetMaxPlannedVUs(gracefulExecutionSteps) gracefulStop := maxDuration - regularDuration + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, regularDuration, gracefulStop) defer cancel() @@ -575,9 +578,6 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo // Actually schedule the VUs and iterations, likely the most complicated // executor among all of them... - activeVUs := &sync.WaitGroup{} - defer activeVUs.Wait() - runIteration := getIterationRunner(vlv.executionState, vlv.logger) getVU := func() (lib.InitializedVU, error) { initVU, err := vlv.executionState.GetPlannedVU(vlv.logger, false) From 62b9c0b1de7b839e3bba925c1b7c84dce5653589 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 13 May 2020 12:14:08 +0300 Subject: [PATCH 255/350] Limit parallel VU initialization by GOMAXPROCS not CPU count --- 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 7b1ab217664..0d2543a06dc 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -234,7 +234,7 @@ func (e *ExecutionScheduler) Init(ctx context.Context, samplesOut chan<- stats.S defer cancel() e.state.SetExecutionStatus(lib.ExecutionStatusInitVUs) - doneInits := e.initVUsConcurrently(subctx, samplesOut, vusToInitialize, runtime.NumCPU(), logger) + doneInits := e.initVUsConcurrently(subctx, samplesOut, vusToInitialize, runtime.GOMAXPROCS(0), logger) initializedVUs := new(uint64) vusFmt := pb.GetFixedLengthIntFormat(int64(vusToInitialize)) From 0f665e1d4100b4a17401c2e2fcc3e8e7e0dd58c0 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 12 May 2020 16:13:45 +0300 Subject: [PATCH 256/350] Refactor ExecutionSegment sequences, wrappers and tuples This cleans up the interfaces and makes it possible to re-use the heavy calculations for a segment sequence. --- lib/execution_segment.go | 457 +++++++++++---------- lib/execution_segment_test.go | 50 ++- lib/executor/constant_arrival_rate.go | 12 +- lib/executor/constant_arrival_rate_test.go | 38 +- lib/executor/constant_looping_vus.go | 2 +- lib/executor/externally_controlled.go | 16 +- lib/executor/per_vu_iterations.go | 4 +- lib/executor/shared_iterations.go | 18 +- lib/executor/variable_arrival_rate.go | 30 +- lib/executor/variable_arrival_rate_test.go | 53 +-- lib/executor/variable_looping_vus.go | 11 +- lib/executors.go | 8 +- lib/options.go | 6 +- 13 files changed, 386 insertions(+), 319 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 7440d4b7f85..6f320192a9e 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -26,7 +26,6 @@ import ( "math/big" "sort" "strings" - "sync" ) // ExecutionSegment represents a (start, end] partition of the total execution @@ -401,10 +400,10 @@ func (ess ExecutionSegmentSequence) String() string { return strings.Join(result, ",") } -// lowest common denominator +// LCD calculates the lowest common denominator of the sequence. // https://en.wikipedia.org/wiki/Least_common_multiple#Using_the_greatest_common_divisor -func (ess ExecutionSegmentSequence) lcd() int64 { - var acc = ess[0].length.Denom().Int64() +func (ess ExecutionSegmentSequence) LCD() int64 { + acc := ess[0].length.Denom().Int64() var n int64 for _, seg := range ess[1:] { n = seg.length.Denom().Int64() @@ -430,199 +429,177 @@ func gcd(a, b int64) int64 { return a } -type sortInterfaceWrapper struct { // TODO: rename ? delete ? and replace ? - slice []struct { // TODO better name ? maybe a type of it's own ? - numerator int64 - originalIndex int - } - lcd int64 +// IsFull returns whether the sequences is full, that is, whether it starts at 0 +// and ends at 1. Use GetFilledExecutionSegmentSequence() to get a full sequence. +func (ess ExecutionSegmentSequence) IsFull() bool { + return ess != nil && len(ess) != 0 && ess[0].from.Cmp(zeroRat) == 0 && ess[len(ess)-1].to.Cmp(oneRat) == 0 } -func newWrapper(ess ExecutionSegmentSequence) sortInterfaceWrapper { - var result = sortInterfaceWrapper{ - slice: make([]struct { - numerator int64 - originalIndex int - }, len(ess)), - lcd: ess.lcd(), +// FindSegmentPosition returns the index of the supplied execution segment in +// the sequence, or an error if the segment isn't present. This shouldn't be +// used on a nil or empty sequence, it's best to use this method on the result +// of GetFilledExecutionSegmentSequence(). +func (ess ExecutionSegmentSequence) FindSegmentPosition(segment *ExecutionSegment) (int, error) { + from := zeroRat + if segment != nil { + from = segment.from } + index := sort.Search(len(ess), func(i int) bool { + return ess[i].from.Cmp(from) >= 0 + }) - for i := range ess { - result.slice[i].numerator = ess[i].length.Num().Int64() * (result.lcd / ess[i].length.Denom().Int64()) - result.slice[i].originalIndex = i + if index < 0 || index >= len(ess) || !ess[index].Equal(segment) { + return -1, fmt.Errorf("couldn't find segment %s in sequence %s", segment, ess) } - - sort.SliceStable(result.slice, func(i, j int) bool { - return result.slice[i].numerator > result.slice[j].numerator - }) - return result + return index, nil } -// Imagine you have a number of rational numbers which all add up to 1 (or less) and call them -// segments. -// If you want each to get proportional amount of anything you need to give them their numerator -// count of elements for each denominator amount from the original elements. So for 1/3 you give 1 -// element for each 3 elements. For 3/5 - 3 elements for each 5. -// If you have for example a sequence of with element with length 3/5 and 1/3 in order to know how -// to distribute it accurately you need to get the LCD(lowest common denominitor) in this case -// between 3 and 5 this is 15 and then to transform the numbers to have the same, LCD equal, -// denominator. So 3/5 becomes 9/15 and 1/3 becomes 5/15. So now for each 15 elements 9 need to go -// to the 3/5, and 5 need to go to 1/3. -// -// We use the below algorithm to split elements between ExecutionSegments by using their length as -// the rational number. As we would like to get non sequential elements we try to get the maximum -// distance between them. That is the number of elements divided by the number of elements for any -// given segment, which concidently is the length of the segment reversed. -// The algorithm below does the following: -// 1. Goes through the elements from 0 to the lcd-1 -// 2. For each of element goes through the segments and looks if the amount of already taken -// elements by the given segment multiplied by that segment length inverted is equal to or less to -// the current element index. if it is give that element to that segment if not continue with the -// next element. -// -// The code below specifically avoids using big.Rat which complicates the code somewhat. -// As additional note the sorting of the segments from biggest to smallest helps with the fact that -// the biggest elements will need to take the most elements and for them it will be the hardest to -// not get sequential elements. -func (e sortInterfaceWrapper) stripingAlgorithm(saveIndex func(iteration int64, index int, numerator int64) bool) { - var chosenCounts = make([]int64, len(e.slice)) - -outer: - for i := int64(0); i < e.lcd; i++ { - for index, chosenCount := range chosenCounts { - num := chosenCount * e.lcd - denom := e.slice[index].numerator - if i > num/denom || (i == num/denom && num%denom == 0) { - chosenCounts[index]++ - if saveIndex(i, e.slice[index].originalIndex, denom) { - break outer - } - break - } +// GetFilledExecutionSegmentSequence makes sure we don't have any gaps in the +// given execution segment sequence, or a nil one. It makes sure that the whole +// 0-1 range is filled. +func GetFilledExecutionSegmentSequence( + sequence *ExecutionSegmentSequence, fallback *ExecutionSegment, +) (result ExecutionSegmentSequence) { + if sequence == nil || len(*sequence) == 0 { + if fallback == nil || fallback.length.Cmp(oneRat) == 0 { + // There is no sequence or a segment, so it means the whole test run + // is being planned/executed. So we make sure not to have a nil + // sequence, returning a full; "0,1" sequence instead, otherwise we + // will need to check for nil everywhere... + return ExecutionSegmentSequence{newExecutionSegment(zeroRat, oneRat)} } + // We don't have a sequence, but we have a defined segment, so we + // fill around it with the missing pieces for a full sequence. + result = ExecutionSegmentSequence{fallback} + } else { + result = *sequence } -} -// ExecutionTuple is here to represent the combination of ExecutionSegmentSequence and -// ExecutionSegment and to give easy access to a couple of algorithms based on them in a way that is -// somewhat perfomant for which it generally needs to cache the results -type ExecutionTuple struct { // TODO rename - ES *ExecutionSegment // TODO unexport this as well? + if result[0].from.Cmp(zeroRat) != 0 { + es := newExecutionSegment(zeroRat, result[0].from) + result = append(ExecutionSegmentSequence{es}, result...) + } - esIndex int - sequence ExecutionSegmentSequence - offsetsCache [][]int64 - lcd int64 - // TODO discuss if we just don't want to fillCache in the constructor and not need to use pointer receivers everywhere - once *sync.Once + if result[len(result)-1].to.Cmp(oneRat) != 0 { + es := newExecutionSegment(result[len(result)-1].to, oneRat) + result = append(result, es) + } + return result } -func (et *ExecutionTuple) String() string { - return fmt.Sprintf("%s in %s", et.ES, et.sequence) +// ExecutionSegmentSequenceWrapper is a caching layer on top of the execution +// segment sequence that allows us to make fast and useful calculations, after +// a somewhat slow initialization. +type ExecutionSegmentSequenceWrapper struct { + ExecutionSegmentSequence // a filled-out segment sequence + lcd int64 // pre-calculated least common denominator + + // The striped offsets, i.e. the repeating indexes that "belong" to each + // execution segment in the sequence. + offsets [][]int64 } -func fillSequence(sequence ExecutionSegmentSequence) ExecutionSegmentSequence { - if sequence[0].from.Cmp(zeroRat) != 0 { - es := newExecutionSegment(zeroRat, sequence[0].from) - sequence = append(ExecutionSegmentSequence{es}, sequence...) +// NewExecutionSegmentSequenceWrapper expects a filled-out execution segment +// sequence. It pre-calculates the initial caches of and returns a new +// ExecutionSegmentSequenceWrapper, but doesn't calculate the striped offsets. +func NewExecutionSegmentSequenceWrapper(ess ExecutionSegmentSequence) *ExecutionSegmentSequenceWrapper { + if !ess.IsFull() { + panic(fmt.Sprintf("Cannot wrap around a non-full execution segment sequence '%s'", ess)) } - if sequence[len(sequence)-1].to.Cmp(oneRat) != 0 { - es := newExecutionSegment(sequence[len(sequence)-1].to, oneRat) - sequence = append(sequence, es) - } - return sequence -} + sequenceLength := len(ess) + offsets := make([][]int64, sequenceLength) + lcd := ess.LCD() -// NewExecutionTuple returns a new ExecutionTuple for the provided segment and sequence -func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequence) (*ExecutionTuple, error) { - et := ExecutionTuple{ - once: new(sync.Once), - ES: segment, - } - if sequence == nil || len(*sequence) == 0 { - if segment == nil || segment.length.Cmp(oneRat) == 0 { - // here we replace it with a not nil as we otherwise will need to check it everywhere - et.sequence = ExecutionSegmentSequence{newExecutionSegment(zeroRat, oneRat)} - } else { - et.sequence = fillSequence(ExecutionSegmentSequence{segment}) - } - } else { - et.sequence = fillSequence(*sequence) - } + // This will contain the normalized numerator values (i.e. what they would have + // been if all denominators were equal to the LCD), sorted in descending + // order (i.e. biggest segments are first), with references to their actual + // indexes in the execution segment sequence (i.e. `seq` above). + sortedNormalizedIndexes := make([]struct { + normNumerator int64 + originalIndex int + }, sequenceLength) - et.esIndex = et.find(segment) - if et.esIndex == -1 { - return nil, fmt.Errorf("couldn't find segment %s in sequence %s", segment, sequence) + for i := range ess { + normalizedNumerator := ess[i].length.Num().Int64() * (lcd / ess[i].length.Denom().Int64()) + sortedNormalizedIndexes[i].normNumerator = normalizedNumerator + sortedNormalizedIndexes[i].originalIndex = i + offsets[i] = make([]int64, 0, normalizedNumerator+1) } - return &et, nil -} -func (et *ExecutionTuple) find(segment *ExecutionSegment) int { - if segment == nil { - if len(et.sequence) == 1 { - return 0 - } - return -1 - } - index := sort.Search(len(et.sequence), func(i int) bool { - return et.sequence[i].from.Cmp(segment.from) >= 0 + sort.SliceStable(sortedNormalizedIndexes, func(i, j int) bool { + return sortedNormalizedIndexes[i].normNumerator > sortedNormalizedIndexes[j].normNumerator }) - if index < 0 || index >= len(et.sequence) || !et.sequence[index].Equal(segment) { - return -1 - } - return index -} - -// ScaleInt64 scales the provided value based on the ExecutionTuple -func (et *ExecutionTuple) ScaleInt64(value int64) int64 { - if et.esIndex == -1 { - return 0 + // This is the striping algorithm. Imagine you have a number of rational + // numbers which all add up to 1 (or less), and call them segments. If you + // want each to get proportional amount of anything, you need to give them + // their numerator count of elements for each denominator amount from the + // original elements. So, for 1/3, you give 1 element for each 3 elements. + // For 3/5 - 3 elements for each 5. If you have, for example, a sequence + // with elements with length 3/5 and 1/3, in order to know how to distribute + // it accurately, you need to get the LCD(lowest common denominitor). In + // this case, between 3 and 5, the LCD is 15. Then to transform the numbers + // to have the same, LCD equal, denominator. So 3/5 becomes 9/15 and 1/3 + // becomes 5/15. So now for each 15 elements 9 need to go to the 3/5, and 5 + // need to go to 1/3. This is what we did above in sortedNormalizedIndexes. + // + // We use the algorithm below to split elements between ExecutionSegments by + // using their length as the rational number. As we would like to get + // non-sequential elements, we try to get the maximum distance between them. + // That is the number of elements divided by the number of elements for any + // given segment, which concidently is the length of the segment reversed. + // The algorithm below does the following: + // 1. Goes through the elements from 0 to the lcd-1 + // 2. For each of element, it goes through the segments and looks if the + // amount of already taken elements by the given segment, multiplied by + // that segment's length inverted, is equal to or less to the current + // element index. If it is, give that element to that segment. If not, + // continue with the next element. + // The code below specifically avoids using big.Rat, for performance + // reasons, which complicates the code somewhat. As additional note, the + // sorting of the segments from biggest to smallest helps with the fact that + // the biggest elements will need to take the most elements, and for them it + // will be the hardest to not get sequential elements. + prev := make([]int64, sequenceLength) + chosenCounts := make([]int64, sequenceLength) + saveIndex := func(iteration int64, index int, numerator int64) { + offsets[index] = append(offsets[index], iteration-prev[index]) + prev[index] = iteration + if int64(len(offsets[index])) == numerator { + offsets[index] = append(offsets[index], offsets[index][0]+lcd-iteration) + } } - if len(et.sequence) == 1 { - return value + for i := int64(0); i < lcd; i++ { + for sortedIndex, chosenCount := range chosenCounts { + num := chosenCount * lcd + denom := sortedNormalizedIndexes[sortedIndex].normNumerator + if i > num/denom || (i == num/denom && num%denom == 0) { + chosenCounts[sortedIndex]++ + saveIndex(i, sortedNormalizedIndexes[sortedIndex].originalIndex, denom) + break + } + } } - et.once.Do(et.fillCache) - offsets := et.offsetsCache[et.esIndex] - return scaleInt64(value, offsets[0], offsets[1:], et.lcd) -} -// scaleInt64With scales the provided value based on the ExecutionTuples' -// sequence and the segment provided -func (et *ExecutionTuple) scaleInt64With(value int64, es *ExecutionSegment) int64 { //nolint:unused - start, offsets, lcd := et.GetStripedOffsets(es) - return scaleInt64(value, start, offsets, lcd) + return &ExecutionSegmentSequenceWrapper{ExecutionSegmentSequence: ess, lcd: lcd, offsets: offsets} } -func scaleInt64(value, start int64, offsets []int64, lcd int64) int64 { - endValue := (value / lcd) * int64(len(offsets)) - for gi, i := 0, start; i < value%lcd; gi, i = gi+1, i+offsets[gi] { - endValue++ - } - return endValue +// LCD returns the (cached) least common denominator of the sequence - no need +// to calculate it again, since we did it in the constructor. +func (essw *ExecutionSegmentSequenceWrapper) LCD() int64 { + return essw.lcd } -func (et *ExecutionTuple) fillCache() { - var wrapper = newWrapper(et.sequence) - - et.offsetsCache = make([][]int64, len(et.sequence)) - for i := range et.offsetsCache { - et.offsetsCache[i] = make([]int64, 0, wrapper.slice[i].numerator+1) - } - - var prev = make([]int64, len(et.sequence)) - var saveIndex = func(iteration int64, index int, numerator int64) bool { - et.offsetsCache[index] = append(et.offsetsCache[index], iteration-prev[index]) - prev[index] = iteration - if int64(len(et.offsetsCache[index])) == numerator { - et.offsetsCache[index] = append(et.offsetsCache[index], et.offsetsCache[index][0]+wrapper.lcd-iteration) - } - return false +// ScaleInt64 scales the provided value for the given segment. +func (essw *ExecutionSegmentSequenceWrapper) ScaleInt64(segmentIndex int, value int64) int64 { + start := essw.offsets[segmentIndex][0] + offsets := essw.offsets[segmentIndex][1:] + result := (value / essw.lcd) * int64(len(offsets)) + for gi, i := 0, start; i < value%essw.lcd; gi, i = gi+1, i+offsets[gi] { + result++ } - - wrapper.stripingAlgorithm(saveIndex) - et.lcd = wrapper.lcd + return result } // GetStripedOffsets returns the stripped offsets for the given segment @@ -633,52 +610,120 @@ func (et *ExecutionTuple) fillCache() { // into lcd sized chunks // - lcd: the LCD of the lengths of all segments in the sequence. This is also the number of // elements after which the algorithm starts to loop and give the same values -func (et *ExecutionTuple) GetStripedOffsets(segment *ExecutionSegment) (int64, []int64, int64) { - et.once.Do(et.fillCache) - index := et.find(segment) - if index == -1 { - return -1, nil, et.lcd - } - offsets := et.offsetsCache[index] - return offsets[0], offsets[1:], et.lcd -} - -// GetNewExecutionTupleBasedOnValue uses the value provided, splits it using the striping offsets -// between all the segments in the sequence and returns a new ExecutionTuple with a new sequence and -// segments, such that each new segment in the new sequence has length `Scale(value)/value` while -// keeping the order. The main segment in the new ExecutionTuple is the correspoding one from the -// original, if that segmetn would've been with length 0 then it is nil, and obviously isn't part of -// the sequence. -func (et *ExecutionTuple) GetNewExecutionTupleBasedOnValue(value int64) *ExecutionTuple { - et.once.Do(et.fillCache) - if value != 0 && value%et.lcd == 0 { // the value is perfectly divisible so we will get the same tuple - return et - } - - var ( - newESS = make(ExecutionSegmentSequence, 0, len(et.sequence)) // this can be smaller - newES *ExecutionSegment - esIndex = -1 - prev int64 - ) - for i := range et.sequence { - offsets := et.offsetsCache[i] - newValue := scaleInt64(value, offsets[0], offsets[1:], et.lcd) +func (essw *ExecutionSegmentSequenceWrapper) GetStripedOffsets(segmentIndex int) (int64, []int64, int64) { + offsets := essw.offsets[segmentIndex] + return offsets[0], offsets[1:], essw.lcd +} + +// GetTuple returns an ExecutionTuple for the specified segment index. +func (essw *ExecutionSegmentSequenceWrapper) GetTuple(segmentIndex int) *ExecutionTuple { + return &ExecutionTuple{ + Sequence: essw, + Segment: essw.ExecutionSegmentSequence[segmentIndex], + SegmentIndex: segmentIndex, + } +} + +// GetNewExecutionSegmentSequenceFromValue uses the value provided, splits it +// between all the segments, using the striping offsets in the sequence, +// generating a new segment sequence. It then returns a new +// ExecutionSegmentSequenceWrapper, with the new sequence and segments, such +// that each new segment in the new sequence has length `Scale(value)/value` +// while keeping the order. +// +// Additionally, the position of a given segment index can be tracked (since +// empty segments are removed), so that you can reconstruct an ExecutionTuple, +// if required. If the segment with the trackedIndex is not part of the new +// sequence, or if a new sequence cannot be generated (for example, for 0 +// values), an error will be returned. +func (essw *ExecutionSegmentSequenceWrapper) GetNewExecutionSegmentSequenceFromValue(value int64, trackedIndex int) ( + newSequence *ExecutionSegmentSequenceWrapper, newIndex int, err error, +) { + if value < 1 { + return nil, -1, fmt.Errorf("cannot generate new sequence for value %d", value) + } + + if value%essw.lcd == 0 { // the value is perfectly divisible so we will get the same tuple + return essw, trackedIndex, nil + } + + newIndex = -1 + newESS := make(ExecutionSegmentSequence, 0, len(essw.ExecutionSegmentSequence)) // this can be smaller + + prev := int64(0) + for i := range essw.ExecutionSegmentSequence { + newValue := essw.ScaleInt64(i, value) if newValue == 0 { continue } - var currentES = newExecutionSegment(big.NewRat(prev, value), big.NewRat(prev+newValue, value)) + currentES := newExecutionSegment(big.NewRat(prev, value), big.NewRat(prev+newValue, value)) prev += newValue - if i == et.esIndex { - newES = currentES - esIndex = len(newESS) + if i == trackedIndex { + newIndex = len(newESS) } newESS = append(newESS, currentES) } - return &ExecutionTuple{ - ES: newES, - sequence: newESS, - esIndex: esIndex, - once: new(sync.Once), + + if newIndex == -1 { + return nil, -1, fmt.Errorf( + "segment %d (%s) isn't present in the new sequence", + trackedIndex, essw.ExecutionSegmentSequence[trackedIndex], + ) + } + + return NewExecutionSegmentSequenceWrapper(newESS), newIndex, nil +} + +// ExecutionTuple is the combination of an ExecutionSegmentSequence(Wrapper) and +// a specific ExecutionSegment from it. It gives easy access to the efficient +// scaling and striping algorithms for that specific segment, since the results +// are cached in the sequence wrapper. +type ExecutionTuple struct { // TODO rename? make fields private and have getter methods? + Sequence *ExecutionSegmentSequenceWrapper + Segment *ExecutionSegment + SegmentIndex int +} + +func (et *ExecutionTuple) String() string { + return fmt.Sprintf("%s in %s", et.Segment, et.Sequence) +} + +// NewExecutionTuple returns a new ExecutionTuple for the provided segment and +// sequence. +// +// TODO: don't return a pointer? +func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequence) (*ExecutionTuple, error) { + filledSeq := GetFilledExecutionSegmentSequence(sequence, segment) + wrapper := NewExecutionSegmentSequenceWrapper(filledSeq) + index, err := wrapper.FindSegmentPosition(segment) + if err != nil { + return nil, err } + return &ExecutionTuple{Sequence: wrapper, Segment: segment, SegmentIndex: index}, nil +} + +// ScaleInt64 scales the provided value for our execution segment. +func (et *ExecutionTuple) ScaleInt64(value int64) int64 { + return et.Sequence.ScaleInt64(et.SegmentIndex, value) +} + +// GetStripedOffsets returns the striped offsets for our execution segment. +func (et *ExecutionTuple) GetStripedOffsets() (int64, []int64, int64) { + return et.Sequence.GetStripedOffsets(et.SegmentIndex) +} + +// GetNewExecutionTupleFromValue re-segments the sequence, based on the given +// value (see GetNewExecutionSegmentSequenceFromValue() above), and either +// returns the new tuple, or an error if the current segment isn't present in +// the new sequence. +func (et *ExecutionTuple) GetNewExecutionTupleFromValue(value int64) (*ExecutionTuple, error) { + newSequenceWrapper, newIndex, err := et.Sequence.GetNewExecutionSegmentSequenceFromValue(value, et.SegmentIndex) + if err != nil { + return nil, err + } + return &ExecutionTuple{ + Sequence: newSequenceWrapper, + Segment: newSequenceWrapper.ExecutionSegmentSequence[newIndex], + SegmentIndex: newIndex, + }, nil } diff --git a/lib/execution_segment_test.go b/lib/execution_segment_test.go index 7f430db8a62..202e34cc87e 100644 --- a/lib/execution_segment_test.go +++ b/lib/execution_segment_test.go @@ -494,8 +494,9 @@ func TestExecutionTupleScaleConsistency(t *testing.T) { require.NoError(t, err) t.Run(fmt.Sprintf("%d_%s", scale, seq), func(t *testing.T) { var total int64 - for _, segment := range seq { - total += et.scaleInt64With(int64(scale), segment) + for i, segment := range seq { + assert.True(t, segment.Equal(et.Sequence.ExecutionSegmentSequence[i])) + total += et.Sequence.ScaleInt64(i, int64(scale)) } assert.Equal(t, int64(scale), total) }) @@ -585,7 +586,7 @@ func TestGetStripedOffsets(t *testing.T) { et, err := NewExecutionTuple(segment, &ess) require.NoError(t, err) - start, offsets, lcd := et.GetStripedOffsets(segment) + start, offsets, lcd := et.GetStripedOffsets() assert.Equal(t, tc.start, start) assert.Equal(t, tc.offsets, offsets) @@ -615,7 +616,7 @@ func TestSequenceLCD(t *testing.T) { t.Run(fmt.Sprintf("seq:%s", tc.seq), func(t *testing.T) { ess, err := NewExecutionSegmentSequenceFromString(tc.seq) require.NoError(t, err) - require.Equal(t, tc.lcd, ess.lcd()) + require.Equal(t, tc.lcd, ess.LCD()) }) } } @@ -634,7 +635,7 @@ func BenchmarkGetStripedOffsets(b *testing.B) { segment := sequence[int(r.Int63())%len(sequence)] et, err := NewExecutionTuple(segment, &sequence) require.NoError(b, err) - _, _, _ = et.GetStripedOffsets(segment) + _, _, _ = et.GetStripedOffsets() } }) } @@ -670,7 +671,7 @@ func BenchmarkGetStripedOffsetsEven(b *testing.B) { segment := sequence[111233%len(sequence)] et, err := NewExecutionTuple(segment, &sequence) require.NoError(b, err) - _, _, _ = et.GetStripedOffsets(segment) + _, _, _ = et.GetStripedOffsets() } }) } @@ -700,8 +701,9 @@ func TestGetNewExecutionTupleBesedOnValue(t *testing.T) { et, err := NewExecutionTuple(segment, &ess) require.NoError(t, err) - newET := et.GetNewExecutionTupleBasedOnValue(tc.value) - require.Equal(t, tc.expected, newET.sequence.String()) + newET, err := et.GetNewExecutionTupleFromValue(tc.value) + require.NoError(t, err) + require.Equal(t, tc.expected, newET.Sequence.String()) }) } } @@ -727,7 +729,7 @@ func TestNewExecutionTuple(t *testing.T) { seg *ExecutionSegment seq *ExecutionSegmentSequence scaleTests map[int64]int64 - newScaleTests map[int64]map[int64]int64 // this is for after calling GetNewExecutionTupleBasedOnValue + newScaleTests map[int64]map[int64]int64 // this is for after calling GetNewExecutionSegmentSequenceFromValue }{ { // both segment and sequence are nil @@ -739,7 +741,7 @@ func TestNewExecutionTuple(t *testing.T) { newScaleTests: map[int64]map[int64]int64{ 50: {50: 50, 1: 1, 0: 0}, 1: {50: 50, 1: 1, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 0: nil, }, }, { @@ -753,7 +755,7 @@ func TestNewExecutionTuple(t *testing.T) { newScaleTests: map[int64]map[int64]int64{ 50: {50: 50, 1: 1, 0: 0}, 1: {50: 50, 1: 1, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 0: nil, }, }, { @@ -767,7 +769,7 @@ func TestNewExecutionTuple(t *testing.T) { newScaleTests: map[int64]map[int64]int64{ 50: {50: 50, 1: 1, 0: 0}, 1: {50: 50, 1: 1, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 0: nil, }, }, { @@ -781,7 +783,7 @@ func TestNewExecutionTuple(t *testing.T) { newScaleTests: map[int64]map[int64]int64{ 50: {50: 50, 1: 1, 0: 0}, 1: {50: 50, 1: 1, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 0: nil, }, }, { @@ -800,7 +802,7 @@ func TestNewExecutionTuple(t *testing.T) { 3: {50: 17, 1: 1, 0: 0}, 2: {50: 25, 1: 1, 0: 0}, 1: {50: 50, 1: 1, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 0: nil, }, }, { @@ -818,8 +820,8 @@ func TestNewExecutionTuple(t *testing.T) { 20: {50: 17, 1: 0, 0: 0}, 3: {50: 17, 1: 0, 0: 0}, 2: {50: 25, 1: 0, 0: 0}, - 1: {50: 0, 1: 0, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 1: nil, + 0: nil, }, }, { @@ -836,9 +838,9 @@ func TestNewExecutionTuple(t *testing.T) { 50: {50: 16, 1: 0, 0: 0}, 20: {50: 15, 1: 0, 0: 0}, 3: {50: 16, 1: 0, 0: 0}, - 2: {50: 0, 1: 0, 0: 0}, - 1: {50: 0, 1: 0, 0: 0}, - 0: {50: 0, 1: 0, 0: 0}, + 2: nil, + 1: nil, + 0: nil, }, }, } @@ -848,15 +850,21 @@ func TestNewExecutionTuple(t *testing.T) { t.Run(fmt.Sprintf("seg:'%s',seq:'%s'", testCase.seg, testCase.seq), func(t *testing.T) { et, err := NewExecutionTuple(testCase.seg, testCase.seq) require.NoError(t, err) + for scaleValue, result := range testCase.scaleTests { require.Equal(t, result, et.ScaleInt64(scaleValue), "%d->%d", scaleValue, result) } for value, newResult := range testCase.newScaleTests { - newET := et.GetNewExecutionTupleBasedOnValue(value) + newET, err := et.GetNewExecutionTupleFromValue(value) + if newResult == nil { + require.Error(t, err) + continue + } + require.NoError(t, err) for scaleValue, result := range newResult { require.Equal(t, result, newET.ScaleInt64(scaleValue), - "getNewExecutionTupleBasedOnValue(%d)%d->%d", value, scaleValue, result) + "GetNewExecutionTupleFromValue(%d)%d->%d", value, scaleValue, result) } } }) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 8f441316e45..ec635546d42 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -189,9 +189,11 @@ type ConstantArrivalRate struct { var _ lib.Executor = &ConstantArrivalRate{} // Init values needed for the execution -func (car *ConstantArrivalRate) Init(ctx context.Context) error { - car.et = car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(car.config.MaxVUs.Int64) - return nil +func (car *ConstantArrivalRate) Init(ctx context.Context) (err error) { + // err should always be nil, because Init() won't be called for executors + // with no work, as determined by their config's HasWork() method. + car.et, err = car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(car.config.MaxVUs.Int64) + return } // Run executes a constant number of iterations per second. @@ -203,7 +205,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC preAllocatedVUs := car.config.GetPreAllocatedVUs(car.executionState.ExecutionTuple) maxVUs := car.config.GetMaxVUs(car.executionState.ExecutionTuple) // TODO: refactor and simplify - arrivalRate := getScaledArrivalRate(car.et.ES, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) + arrivalRate := getScaledArrivalRate(car.et.Segment, car.config.Rate.Int64, time.Duration(car.config.TimeUnit.Duration)) tickerPeriod := time.Duration(getTickerPeriod(arrivalRate).Duration) arrivalRatePerSec, _ := getArrivalRatePerSec(arrivalRate).Float64() @@ -287,7 +289,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs - start, offsets, _ := car.et.GetStripedOffsets(car.et.ES) + start, offsets, _ := car.et.GetStripedOffsets() startTime = time.Now() timer := time.NewTimer(time.Hour * 24) // here the we need the not scaled one diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 5b300a94917..355e2bdbf94 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -48,7 +48,6 @@ func newExecutionSegmentFromString(str string) *lib.ExecutionSegment { func newExecutionSegmentSequenceFromString(str string) *lib.ExecutionSegmentSequence { r, err := lib.NewExecutionSegmentSequenceFromString(str) - if err != nil { panic(err) } @@ -71,7 +70,7 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, logHook = setupExecutor( + ctx, cancel, executor, logHook := setupExecutor( t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) @@ -79,7 +78,7 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { }), ) defer cancel() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) require.NoError(t, err) entries := logHook.Drain() @@ -98,7 +97,7 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, logHook = setupExecutor( + ctx, cancel, executor, logHook := setupExecutor( t, getTestConstantArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) @@ -119,7 +118,7 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { require.InDelta(t, 50, currentCount, 1) } }() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) @@ -127,7 +126,7 @@ func TestConstantArrivalRateRunCorrectRate(t *testing.T) { } func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { - var tests = []struct { + tests := []struct { segment *lib.ExecutionSegment sequence *lib.ExecutionSegmentSequence start time.Duration @@ -191,17 +190,18 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { ExecutionSegmentSequence: test.sequence, }, et, 10, 50) var count int64 - var config = getTestConstantArrivalRateConfig() - newET := es.ExecutionTuple.GetNewExecutionTupleBasedOnValue(config.MaxVUs.Int64) + config := getTestConstantArrivalRateConfig() + newET, err := es.ExecutionTuple.GetNewExecutionTupleFromValue(config.MaxVUs.Int64) + require.NoError(t, err) rateScaled := newET.ScaleInt64(config.Rate.Int64) - var startTime = time.Now() - var expectedTimeInt64 = int64(test.start) - var ctx, cancel, executor, logHook = setupExecutor( + startTime := time.Now() + expectedTimeInt64 := int64(test.start) + ctx, cancel, executor, logHook := setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { current := atomic.AddInt64(&count, 1) - var expectedTime = test.start + expectedTime := test.start if current != 1 { expectedTime = time.Duration(atomic.AddInt64(&expectedTimeInt64, int64(time.Millisecond)*test.steps[(current-2)%int64(len(test.steps))])) @@ -232,7 +232,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { } }() startTime = time.Now() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) @@ -244,7 +244,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { func TestArrivalRateCancel(t *testing.T) { t.Parallel() - var testCases = map[string]lib.ExecutorConfig{ + testCases := map[string]lib.ExecutorConfig{ "constant": getTestConstantArrivalRateConfig(), "variable": getTestVariableArrivalRateConfig(), } @@ -252,13 +252,13 @@ func TestArrivalRateCancel(t *testing.T) { config := config t.Run(name, func(t *testing.T) { t.Parallel() - var ch = make(chan struct{}) - var errCh = make(chan error, 1) - var weAreDoneCh = make(chan struct{}) + ch := make(chan struct{}) + errCh := make(chan error, 1) + weAreDoneCh := make(chan struct{}) et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, logHook = setupExecutor( + ctx, cancel, executor, logHook := setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { select { case <-ch: @@ -273,7 +273,7 @@ func TestArrivalRateCancel(t *testing.T) { go func() { defer wg.Done() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) errCh <- executor.Run(ctx, engineOut) close(weAreDoneCh) }() diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 76c31eeb916..e2e8ea5508d 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -72,7 +72,7 @@ var _ lib.ExecutorConfig = &ConstantLoopingVUsConfig{} // GetVUs returns the scaled VUs for the executor. func (clvc ConstantLoopingVUsConfig) GetVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(clvc.VUs.Int64) + return et.Segment.Scale(clvc.VUs.Int64) } // GetDescription returns a human-readable description of the executor options diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index e87191f8275..a8aec50fd6a 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -143,8 +143,8 @@ func (mec ExternallyControlledConfig) Validate() []error { func (mec ExternallyControlledConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { startVUs := lib.ExecutionStep{ TimeOffset: 0, - PlannedVUs: uint64(et.ES.Scale(mec.MaxVUs.Int64)), // user-configured, VUs to be pre-initialized - MaxUnplannedVUs: 0, // intentional, see function comment + PlannedVUs: uint64(et.Segment.Scale(mec.MaxVUs.Int64)), // user-configured, VUs to be pre-initialized + MaxUnplannedVUs: 0, // intentional, see function comment } maxDuration := time.Duration(mec.Duration.Duration) @@ -208,9 +208,11 @@ type ExternallyControlled struct { } // Make sure we implement all the interfaces -var _ lib.Executor = &ExternallyControlled{} -var _ lib.PausableExecutor = &ExternallyControlled{} -var _ lib.LiveUpdatableExecutor = &ExternallyControlled{} +var ( + _ lib.Executor = &ExternallyControlled{} + _ lib.PausableExecutor = &ExternallyControlled{} + _ lib.LiveUpdatableExecutor = &ExternallyControlled{} +) // GetCurrentConfig just returns the executor's current configuration. func (mex *ExternallyControlled) GetCurrentConfig() ExternallyControlledConfig { @@ -313,7 +315,7 @@ func (mex *ExternallyControlled) stopWhenDurationIsReached(ctx context.Context, checkInterval.Stop() return - //TODO: something more optimized that sleeps for pauses? + // TODO: something more optimized that sleeps for pauses? case <-checkInterval.C: if mex.executionState.GetCurrentTestRunDuration() >= duration { cancel() @@ -400,7 +402,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { } func (rs *externallyControlledRunState) progresFn() (float64, []string) { - //TODO: simulate spinner for the other case or cycle 0-100? + // TODO: simulate spinner for the other case or cycle 0-100? currentActiveVUs := atomic.LoadInt64(rs.activeVUsCount) currentMaxVUs := atomic.LoadInt64(rs.maxVUs) vusFmt := pb.GetFixedLengthIntFormat(currentMaxVUs) diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 92236d7d6ed..cb7b64632ea 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -60,7 +60,7 @@ func NewPerVUIterationsConfig(name string) PerVUIterationsConfig { BaseConfig: NewBaseConfig(name, perVUIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), - MaxDuration: types.NewNullDuration(10*time.Minute, false), //TODO: shorten? + MaxDuration: types.NewNullDuration(10*time.Minute, false), // TODO: shorten? } } @@ -69,7 +69,7 @@ var _ lib.ExecutorConfig = &PerVUIterationsConfig{} // GetVUs returns the scaled VUs for the executor. func (pvic PerVUIterationsConfig) GetVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(pvic.VUs.Int64) + return et.Segment.Scale(pvic.VUs.Int64) } // GetIterations returns the UNSCALED iteration count for the executor. It's diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index e17f6efe0ce..0315b640d6d 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -63,7 +63,7 @@ func NewSharedIterationsConfig(name string) SharedIterationsConfig { BaseConfig: NewBaseConfig(name, sharedIterationsType), VUs: null.NewInt(1, false), Iterations: null.NewInt(1, false), - MaxDuration: types.NewNullDuration(10*time.Minute, false), //TODO: shorten? + MaxDuration: types.NewNullDuration(10*time.Minute, false), // TODO: shorten? } } @@ -77,8 +77,12 @@ func (sic SharedIterationsConfig) GetVUs(et *lib.ExecutionTuple) int64 { // GetIterations returns the scaled iteration count for the executor. func (sic SharedIterationsConfig) GetIterations(et *lib.ExecutionTuple) int64 { - // Optimize this by probably changing the whole Config API - return et.GetNewExecutionTupleBasedOnValue(sic.VUs.Int64).ScaleInt64(sic.Iterations.Int64) + // TODO: Optimize this by probably changing the whole Config API + newTuple, err := et.GetNewExecutionTupleFromValue(sic.VUs.Int64) + if err != nil { + return 0 + } + return newTuple.ScaleInt64(sic.Iterations.Int64) } // GetDescription returns a human-readable description of the executor options @@ -166,9 +170,11 @@ func (sic SharedIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { } // Init values needed for the execution -func (si *SharedIterations) Init(ctx context.Context) error { - si.et = si.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleBasedOnValue(si.config.VUs.Int64) - return nil +func (si *SharedIterations) Init(ctx context.Context) (err error) { + // err should always be nil, because Init() won't be called for executors + // with no work, as determined by their config's HasWork() method. + si.et, err = si.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(si.config.VUs.Int64) + return } // Run executes a specific total number of iterations, which are all shared by diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 4689e9666c2..122ec025abf 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -77,24 +77,24 @@ var _ lib.ExecutorConfig = &VariableArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(varc.PreAllocatedVUs.Int64) + return et.Segment.Scale(varc.PreAllocatedVUs.Int64) } // GetMaxVUs is just a helper method that returns the scaled max VUs. func (varc VariableArrivalRateConfig) GetMaxVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(varc.MaxVUs.Int64) + return et.Segment.Scale(varc.MaxVUs.Int64) } // GetDescription returns a human-readable description of the executor options func (varc VariableArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) string { - //TODO: something better? always show iterations per second? - maxVUsRange := fmt.Sprintf("maxVUs: %d", et.ES.Scale(varc.PreAllocatedVUs.Int64)) + // TODO: something better? always show iterations per second? + maxVUsRange := fmt.Sprintf("maxVUs: %d", et.Segment.Scale(varc.PreAllocatedVUs.Int64)) if varc.MaxVUs.Int64 > varc.PreAllocatedVUs.Int64 { - maxVUsRange += fmt.Sprintf("-%d", et.ES.Scale(varc.MaxVUs.Int64)) + maxVUsRange += fmt.Sprintf("-%d", et.Segment.Scale(varc.MaxVUs.Int64)) } maxUnscaledRate := getStagesUnscaledMaxTarget(varc.StartRate.Int64, varc.Stages) maxArrRatePerSec, _ := getArrivalRatePerSec( - getScaledArrivalRate(et.ES, maxUnscaledRate, time.Duration(varc.TimeUnit.Duration)), + getScaledArrivalRate(et.Segment, maxUnscaledRate, time.Duration(varc.TimeUnit.Duration)), ).Float64() return fmt.Sprintf("Up to %.2f iterations/s for %s over %d stages%s", @@ -140,8 +140,8 @@ func (varc VariableArrivalRateConfig) GetExecutionRequirements(et *lib.Execution return []lib.ExecutionStep{ { TimeOffset: 0, - PlannedVUs: uint64(et.ES.Scale(varc.PreAllocatedVUs.Int64)), - MaxUnplannedVUs: uint64(et.ES.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), + PlannedVUs: uint64(et.Segment.Scale(varc.PreAllocatedVUs.Int64)), + MaxUnplannedVUs: uint64(et.Segment.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), }, { TimeOffset: sumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), @@ -168,7 +168,7 @@ func (varc VariableArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { // VariableArrivalRate tries to execute a specific number of iterations for a // specific period. -//TODO: combine with the ConstantArrivalRate? +// TODO: combine with the ConstantArrivalRate? type VariableArrivalRate struct { *BaseExecutor config VariableArrivalRateConfig @@ -236,7 +236,7 @@ var _ lib.Executor = &VariableArrivalRate{} // the striping algorithm from the lib.ExecutionTuple for additional speed up but this could // possibly be refactored if need for this arises. func (varc VariableArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time.Duration) { - start, offsets, _ := et.GetStripedOffsets(et.ES) + start, offsets, _ := et.GetStripedOffsets() li := -1 // TODO: move this to a utility function, or directly what GetStripedOffsets uses once we see everywhere we will use it next := func() int64 { @@ -279,7 +279,7 @@ func (varc VariableArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time // Run executes a variable number of iterations per second. func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen - segment := varr.executionState.ExecutionTuple.ES + segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) preAllocatedVUs := varr.config.GetPreAllocatedVUs(varr.executionState.ExecutionTuple) @@ -382,9 +382,9 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample remainingUnplannedVUs := maxVUs - preAllocatedVUs - var timer = time.NewTimer(time.Hour) - var start = time.Now() - var ch = make(chan time.Duration, 10) // buffer 10 iteration times ahead + timer := time.NewTimer(time.Hour) + start := time.Now() + ch := make(chan time.Duration, 10) // buffer 10 iteration times ahead var prevTime time.Duration go varr.config.cal(varr.executionState.ExecutionTuple, ch) for nextTime := range ch { @@ -411,7 +411,7 @@ func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.Sample // ideally, we get the VU from the buffer without any issues default: if remainingUnplannedVUs == 0 { - //TODO: emit an error metric? + // TODO: emit an error metric? varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) continue } diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 8eea969adce..c90622f60cf 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -68,7 +68,7 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, logHook = setupExecutor( + ctx, cancel, executor, logHook := setupExecutor( t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) @@ -76,7 +76,7 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { }), ) defer cancel() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) require.NoError(t, err) entries := logHook.Drain() @@ -95,7 +95,7 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var ctx, cancel, executor, logHook = setupExecutor( + ctx, cancel, executor, logHook := setupExecutor( t, getTestVariableArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) @@ -122,7 +122,7 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { currentCount = atomic.SwapInt64(&count, 0) assert.InDelta(t, 50, currentCount, 2) }() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) wg.Wait() require.NoError(t, err) @@ -132,13 +132,14 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { func TestVariableArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { t.Parallel() var count int64 - var now = time.Now() + now := time.Now() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - var expectedTimes = []time.Duration{ - time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6} - var ctx, cancel, executor, logHook = setupExecutor( + expectedTimes := []time.Duration{ + time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6, + } + ctx, cancel, executor, logHook := setupExecutor( t, VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Stages: []Stage{ @@ -175,7 +176,7 @@ func TestVariableArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { }), ) defer cancel() - var engineOut = make(chan stats.SampleContainer, 1000) + engineOut := make(chan stats.SampleContainer, 1000) err = executor.Run(ctx, engineOut) require.NoError(t, err) require.Equal(t, int64(len(expectedTimes)), count) @@ -252,9 +253,11 @@ func TestVariableArrivalRateCal(t *testing.T) { et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), }, { - expectedTimes: []time.Duration{time.Millisecond * 1825, time.Millisecond * 2581, time.Millisecond * 3162, time.Millisecond * 3651, time.Millisecond * 4082, time.Millisecond * 4472, + expectedTimes: []time.Duration{ + time.Millisecond * 1825, time.Millisecond * 2581, time.Millisecond * 3162, time.Millisecond * 3651, time.Millisecond * 4082, time.Millisecond * 4472, time.Millisecond * 4830, time.Millisecond * 5166, time.Millisecond * 5499, time.Millisecond * 5833, time.Millisecond * 6169, time.Millisecond * 6527, - time.Millisecond * 6917, time.Millisecond * 7348, time.Millisecond * 7837, time.Millisecond * 8418, time.Millisecond * 9174, time.Millisecond * 10999}, + time.Millisecond * 6917, time.Millisecond * 7348, time.Millisecond * 7837, time.Millisecond * 8418, time.Millisecond * 9174, time.Millisecond * 10999, + }, et: mustNewExecutionTuple(nil, nil), timeUnit: time.Second / 3, // three times as fast }, @@ -270,9 +273,9 @@ func TestVariableArrivalRateCal(t *testing.T) { } t.Run(fmt.Sprintf("%s timeunit %s", et, config.TimeUnit), func(t *testing.T) { - var ch = make(chan time.Duration) + ch := make(chan time.Duration) go config.cal(et, ch) - var changes = make([]time.Duration, 0, len(expectedTimes)) + changes := make([]time.Duration, 0, len(expectedTimes)) for c := range ch { changes = append(changes, c) } @@ -292,7 +295,7 @@ func BenchmarkCal(b *testing.B) { } { t := t b.Run(t.String(), func(b *testing.B) { - var config = VariableArrivalRateConfig{ + config := VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), Stages: []Stage{ @@ -311,7 +314,7 @@ func BenchmarkCal(b *testing.B) { b.ResetTimer() b.RunParallel(func(pb *testing.PB) { for pb.Next() { - var ch = make(chan time.Duration, 20) + ch := make(chan time.Duration, 20) go config.cal(et, ch) for c := range ch { _ = c @@ -328,7 +331,7 @@ func BenchmarkCalRat(b *testing.B) { } { t := t b.Run(t.String(), func(b *testing.B) { - var config = VariableArrivalRateConfig{ + config := VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), Stages: []Stage{ @@ -347,7 +350,7 @@ func BenchmarkCalRat(b *testing.B) { b.ResetTimer() b.RunParallel(func(pb *testing.PB) { for pb.Next() { - var ch = make(chan time.Duration, 20) + ch := make(chan time.Duration, 20) go config.calRat(et, ch) for c := range ch { _ = c @@ -364,7 +367,7 @@ func TestCompareCalImplementation(t *testing.T) { // in my experiment the difference is 1(nanosecond) in 7 case for the whole test // the duration is 1 second for each stage as calRat takes way longer - a longer better test can // be done when/if it's performance is improved - var config = VariableArrivalRateConfig{ + config := VariableArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(0), Stages: []Stage{ @@ -408,8 +411,8 @@ func TestCompareCalImplementation(t *testing.T) { } et := mustNewExecutionTuple(nil, nil) - var chRat = make(chan time.Duration, 20) - var ch = make(chan time.Duration, 20) + chRat := make(chan time.Duration, 20) + ch := make(chan time.Duration, 20) go config.calRat(et, chRat) go config.cal(et, ch) count := 0 @@ -436,7 +439,7 @@ func sqrtRat(x *big.Rat) *big.Rat { var ns, ds big.Int ni, di := x.Num(), x.Denom() z.SetFrac(ns.Rsh(ni, uint(ni.BitLen())/2), ds.Rsh(di, uint(di.BitLen())/2)) - for i := 10; i > 0; i-- { //TODO: better termination + for i := 10; i > 0; i-- { // TODO: better termination a.Sub(a.Mul(&z, &z), x) f, _ := a.Float64() if f == 0 { @@ -452,7 +455,7 @@ func sqrtRat(x *big.Rat) *big.Rat { func (varc VariableArrivalRateConfig) calRat(et *lib.ExecutionTuple, ch chan<- time.Duration) { defer close(ch) - start, offsets, _ := et.GetStripedOffsets(et.ES) + start, offsets, _ := et.GetStripedOffsets() li := -1 next := func() int64 { li++ @@ -460,9 +463,9 @@ func (varc VariableArrivalRateConfig) calRat(et *lib.ExecutionTuple, ch chan<- t } iRat := big.NewRat(start+1, 1) - var carry = big.NewRat(0, 1) - var doneSoFar = big.NewRat(0, 1) - var endCount = big.NewRat(0, 1) + carry := big.NewRat(0, 1) + doneSoFar := big.NewRat(0, 1) + endCount := big.NewRat(0, 1) curr := varc.StartRate.ValueOrZero() var base time.Duration for _, stage := range varc.Stages { diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index bedc95c3fcf..9166132af8d 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -53,7 +53,7 @@ func init() { type Stage struct { Duration types.NullDuration `json:"duration"` Target null.Int `json:"target"` // TODO: maybe rename this to endVUs? something else? - //TODO: add a progression function? + // TODO: add a progression function? } // VariableLoopingVUsConfig stores the configuration for the stages executor @@ -78,7 +78,7 @@ var _ lib.ExecutorConfig = &VariableLoopingVUsConfig{} // GetStartVUs is just a helper method that returns the scaled starting VUs. func (vlvc VariableLoopingVUsConfig) GetStartVUs(et *lib.ExecutionTuple) int64 { - return et.ES.Scale(vlvc.StartVUs.Int64) + return et.Segment.Scale(vlvc.StartVUs.Int64) } // GetGracefulRampDown is just a helper method that returns the graceful @@ -89,7 +89,7 @@ func (vlvc VariableLoopingVUsConfig) GetGracefulRampDown() time.Duration { // GetDescription returns a human-readable description of the executor options func (vlvc VariableLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { - maxVUs := et.ES.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) + maxVUs := et.Segment.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) @@ -185,7 +185,7 @@ func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple var ( timeTillEnd time.Duration fromVUs = vlvc.StartVUs.Int64 - start, offsets, lcd = et.GetStripedOffsets(et.ES) + start, offsets, lcd = et.GetStripedOffsets() steps = make([]lib.ExecutionStep, 0, vlvc.precalculateTheRequiredSteps(et, zeroEnd)) index = segmentedIndex{start: start, lcd: lcd, offsets: offsets} ) @@ -554,7 +554,8 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo // Make sure the log and the progress bar have accurate information vlv.logger.WithFields(logrus.Fields{ "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(vlv.executionState.ExecutionTuple), "maxVUs": maxVUs, - "duration": regularDuration, "numStages": len(vlv.config.Stages)}, + "duration": regularDuration, "numStages": len(vlv.config.Stages), + }, ).Debug("Starting executor run...") activeVUsCount := new(int64) diff --git a/lib/executors.go b/lib/executors.go index 7e65b74d221..54a6ef329a0 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -36,7 +36,7 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -//TODO: remove globals and use some type of explicit dependency injection? +// TODO: remove globals and use some type of explicit dependency injection? //nolint:gochecknoglobals var ( executorConfigTypesMutex sync.RWMutex @@ -72,7 +72,7 @@ type ExecutionStep struct { MaxUnplannedVUs uint64 } -//TODO: make []ExecutionStep or []ExecutorConfig their own type? +// TODO: make []ExecutionStep or []ExecutorConfig their own type? // ExecutorConfig is an interface that should be implemented by all executor config types type ExecutorConfig interface { @@ -119,7 +119,7 @@ type Executor interface { GetProgress() *pb.ProgressBar GetLogger() *logrus.Entry - Init(ctx context.Context) error //TODO: remove, since it's currently unused? + Init(ctx context.Context) error Run(ctx context.Context, engineOut chan<- stats.SampleContainer) error } @@ -173,7 +173,7 @@ func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { return nil } - //TODO: use a more sophisticated combination of dec.Token() and dec.More(), + // TODO: use a more sophisticated combination of dec.Token() and dec.More(), // which would allow us to support both arrays and maps for this config? var protoConfigs map[string]protoExecutorConfig if err := StrictJSONUnmarshal(data, &protoConfigs); err != nil { diff --git a/lib/options.go b/lib/options.go index 5ce54e9a651..1d92964236c 100644 --- a/lib/options.go +++ b/lib/options.go @@ -315,7 +315,7 @@ func (o Options) Apply(opts Options) Options { // config tier, they will be preserved, so the validation after we've consolidated // all of the options can return an error. if opts.Duration.Valid || opts.Iterations.Valid || opts.Stages != nil || opts.Execution != nil { - //TODO: emit a warning or a notice log message if overwrite lower tier config options? + // TODO: emit a warning or a notice log message if overwrite lower tier config options? o.Duration = types.NewNullDuration(0, false) o.Iterations = null.NewInt(0, false) o.Stages = nil @@ -449,8 +449,8 @@ func (o Options) Apply(opts Options) Options { // Validate checks if all of the specified options make sense func (o Options) Validate() []error { - //TODO: validate all of the other options... that we should have already been validating... - //TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation + // TODO: validate all of the other options... that we should have already been validating... + // TODO: maybe integrate an external validation lib: https://github.com/avelino/awesome-go#validation var errors []error if o.ExecutionSegmentSequence != nil { var segmentFound bool From e09562d4a686167fa1181923ea964d1b2b5c40ca Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 13 May 2020 13:11:34 +0300 Subject: [PATCH 257/350] Resolve PR comments --- lib/execution_segment.go | 3 +++ lib/executor/constant_arrival_rate.go | 7 ++++--- lib/executor/shared_iterations.go | 7 ++++--- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/lib/execution_segment.go b/lib/execution_segment.go index 6f320192a9e..081ef985bfb 100644 --- a/lib/execution_segment.go +++ b/lib/execution_segment.go @@ -704,6 +704,9 @@ func NewExecutionTuple(segment *ExecutionSegment, sequence *ExecutionSegmentSequ // ScaleInt64 scales the provided value for our execution segment. func (et *ExecutionTuple) ScaleInt64(value int64) int64 { + if len(et.Sequence.ExecutionSegmentSequence) == 1 { + return value // if we don't have any segmentation, just return the original value + } return et.Sequence.ScaleInt64(et.SegmentIndex, value) } diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index ec635546d42..e2d03524727 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -189,11 +189,12 @@ type ConstantArrivalRate struct { var _ lib.Executor = &ConstantArrivalRate{} // Init values needed for the execution -func (car *ConstantArrivalRate) Init(ctx context.Context) (err error) { +func (car *ConstantArrivalRate) Init(ctx context.Context) error { // err should always be nil, because Init() won't be called for executors // with no work, as determined by their config's HasWork() method. - car.et, err = car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(car.config.MaxVUs.Int64) - return + et, err := car.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(car.config.MaxVUs.Int64) + car.et = et + return err } // Run executes a constant number of iterations per second. diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 0315b640d6d..e743c4d71f9 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -170,11 +170,12 @@ func (sic SharedIterationsConfig) HasWork(et *lib.ExecutionTuple) bool { } // Init values needed for the execution -func (si *SharedIterations) Init(ctx context.Context) (err error) { +func (si *SharedIterations) Init(ctx context.Context) error { // err should always be nil, because Init() won't be called for executors // with no work, as determined by their config's HasWork() method. - si.et, err = si.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(si.config.VUs.Int64) - return + et, err := si.BaseExecutor.executionState.ExecutionTuple.GetNewExecutionTupleFromValue(si.config.VUs.Int64) + si.et = et + return err } // Run executes a specific total number of iterations, which are all shared by From a44509da63ca05a49897cea1d5ec5cadb48144e9 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 18 May 2020 11:56:52 +0300 Subject: [PATCH 258/350] Fix a bunch of minor issues and repetitions in #1428 --- core/local/local_test.go | 3 +- js/modules/k6/http/request_test.go | 75 ++--------------------- js/modules/k6/http/response_test.go | 14 +++-- js/modules/k6/http/tls_go_1_11_test.go | 25 -------- js/modules/k6/http/tls_go_1_12_test.go | 25 -------- js/modules/k6/k6.go | 28 ++++----- js/modules/k6/k6_test.go | 1 + js/modules/k6/metrics/metrics.go | 9 +-- js/modules/k6/metrics/metrics_test.go | 3 + js/modules/k6/ws/ws.go | 8 +-- js/runner_test.go | 83 ++++++++++++++++++++++++++ lib/netext/httpext/request.go | 15 +---- lib/state.go | 9 +++ stats/stats.go | 11 ++-- 14 files changed, 133 insertions(+), 176 deletions(-) delete mode 100644 js/modules/k6/http/tls_go_1_11_test.go delete mode 100644 js/modules/k6/http/tls_go_1_12_test.go diff --git a/core/local/local_test.go b/core/local/local_test.go index e28accb9a0e..5061dab098a 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -385,9 +385,8 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { t.Parallel() tb := httpmultibin.NewHTTPMultiBin(t) defer tb.Cleanup() - sr := tb.Replacer.Replace - script := sr(` + script := tb.Replacer.Replace(` import http from "k6/http"; import ws from 'k6/ws'; import { Counter } from 'k6/metrics'; diff --git a/js/modules/k6/http/request_test.go b/js/modules/k6/http/request_test.go index 87564f1ee45..aabd415a5be 100644 --- a/js/modules/k6/http/request_test.go +++ b/js/modules/k6/http/request_test.go @@ -137,6 +137,7 @@ func newRuntime( Transport: tb.HTTPTransport, BPool: bpool.NewBufferPool(1), Samples: samples, + Tags: map[string]string{"group": root.Path}, } ctx := new(context.Context) @@ -924,9 +925,9 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("tags-precedence", func(t *testing.T) { - oldOpts := state.Options - defer func() { state.Options = oldOpts }() - state.Options.RunTags = stats.IntoSampleTags(&map[string]string{"runtag1": "val1", "runtag2": "val2"}) + oldTags := state.Tags + defer func() { state.Tags = oldTags }() + state.Tags = map[string]string{"runtag1": "val1", "runtag2": "val2"} _, err := common.RunString(rt, sr(` let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: { method: "test", name: "myName", runtag1: "fromreq" } }); @@ -1268,74 +1269,6 @@ func TestRequestAndBatch(t *testing.T) { }) }) } -func TestSystemTags(t *testing.T) { - t.Parallel() - tb, state, samples, rt, _ := newRuntime(t) - defer tb.Cleanup() - - // Handple paths with custom logic - tb.Mux.HandleFunc("/wrong-redirect", func(w http.ResponseWriter, r *http.Request) { - w.Header().Add("Location", "%") - w.WriteHeader(http.StatusTemporaryRedirect) - }) - - httpGet := fmt.Sprintf(`http.get("%s");`, tb.ServerHTTP.URL) - httpsGet := fmt.Sprintf(`http.get("%s");`, tb.ServerHTTPS.URL) - - httpURL, err := url.Parse(tb.ServerHTTP.URL) - require.NoError(t, err) - - testedSystemTags := []struct{ tag, code, expVal string }{ - {"proto", httpGet, "HTTP/1.1"}, - {"status", httpGet, "200"}, - {"method", httpGet, "GET"}, - {"url", httpGet, tb.ServerHTTP.URL}, - {"url", httpsGet, tb.ServerHTTPS.URL}, - {"ip", httpGet, httpURL.Hostname()}, - {"name", httpGet, tb.ServerHTTP.URL}, - {"group", httpGet, ""}, - {"vu", httpGet, "0"}, - {"iter", httpGet, "0"}, - {"tls_version", httpsGet, expectedTLSVersion}, - {"ocsp_status", httpsGet, "unknown"}, - { - "error", - tb.Replacer.Replace(`http.get("http://127.0.0.1:1");`), - `dial: connection refused`, - }, - { - "error_code", - tb.Replacer.Replace(`http.get("http://127.0.0.1:1");`), - "1212", - }, - } - - state.Options.Throw = null.BoolFrom(false) - state.Options.Apply(lib.Options{TLSVersion: &lib.TLSVersions{Max: lib.TLSVersion13}}) - - for num, tc := range testedSystemTags { - tc := tc - t.Run(fmt.Sprintf("TC %d with only %s", num, tc.tag), func(t *testing.T) { - state.Options.SystemTags = stats.ToSystemTagSet([]string{tc.tag}) - - _, err := common.RunString(rt, tc.code) - assert.NoError(t, err) - - 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() { - assert.Equal(t, tc.tag, emittedTag) - assert.Equal(t, tc.expVal, emittedVal) - } - } - } - }) - } -} func TestRequestCompression(t *testing.T) { t.Parallel() diff --git a/js/modules/k6/http/response_test.go b/js/modules/k6/http/response_test.go index 4c77562df81..5753cb9416b 100644 --- a/js/modules/k6/http/response_test.go +++ b/js/modules/k6/http/response_test.go @@ -80,7 +80,7 @@ const jsonData = `{"glossary": { "GlossSeeAlso": ["GML","XML"]}, "GlossSee": "markup"}}}}}` -const invalidJSONData = `{ +const invalidJSONData = `{ "a":"apple", "t":testing" }` @@ -168,7 +168,11 @@ func TestResponse(t *testing.T) { if assert.NoError(t, err) { old := state.Group state.Group = g - defer func() { state.Group = old }() + state.Tags["group"] = g.Path + defer func() { + state.Group = old + state.Tags["group"] = old.Path + }() } _, err = common.RunString(rt, sr(` @@ -217,7 +221,7 @@ func TestResponse(t *testing.T) { if (value != undefined) { throw new Error("Expected undefined, but got: " + value); } - value = res.json("glossary.null") + value = res.json("glossary.null") if (value != null) { throw new Error("Expected null, but got: " + value); } @@ -233,8 +237,8 @@ func TestResponse(t *testing.T) { if (value != true) { throw new Error("Expected boolean true, but got: " + value); } - value = res.json("glossary.GlossDiv.GlossList.GlossEntry.GlossDef.title") - if (value != "example glossary") + value = res.json("glossary.GlossDiv.GlossList.GlossEntry.GlossDef.title") + if (value != "example glossary") { throw new Error("Expected 'example glossary'', but got: " + value); } value = res.json("glossary.friends.#.first")[0] diff --git a/js/modules/k6/http/tls_go_1_11_test.go b/js/modules/k6/http/tls_go_1_11_test.go deleted file mode 100644 index 7c34dfd356b..00000000000 --- a/js/modules/k6/http/tls_go_1_11_test.go +++ /dev/null @@ -1,25 +0,0 @@ -// +build !go1.12 - -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2019 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package http - -const expectedTLSVersion = "tls1.2" diff --git a/js/modules/k6/http/tls_go_1_12_test.go b/js/modules/k6/http/tls_go_1_12_test.go deleted file mode 100644 index e248f84e96f..00000000000 --- a/js/modules/k6/http/tls_go_1_12_test.go +++ /dev/null @@ -1,25 +0,0 @@ -// +build go1.12 - -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2019 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package http - -const expectedTLSVersion = "tls1.3" diff --git a/js/modules/k6/k6.go b/js/modules/k6/k6.go index 23de61d8229..7bbfddd5e94 100644 --- a/js/modules/k6/k6.go +++ b/js/modules/k6/k6.go @@ -84,17 +84,23 @@ func (*K6) Group(ctx context.Context, name string, fn goja.Callable) (goja.Value old := state.Group state.Group = g - defer func() { state.Group = old }() + + shouldUpdateTag := state.Options.SystemTags.Has(stats.TagGroup) + if shouldUpdateTag { + state.Tags["group"] = g.Path + } + defer func() { + state.Group = old + if shouldUpdateTag { + state.Tags["group"] = old.Path + } + }() startTime := time.Now() ret, err := fn(goja.Undefined()) t := time.Now() - tags := map[string]string{} - for k, v := range state.Tags { - tags[k] = v - } - + tags := state.CloneTags() stats.PushIfNotDone(ctx, state.Samples, stats.Sample{ Time: t, Metric: metrics.GroupDuration, @@ -113,14 +119,8 @@ func (*K6) Check(ctx context.Context, arg0, checks goja.Value, extras ...goja.Va rt := common.GetRuntime(ctx) t := time.Now() - // Prepare tags, make sure the `group` tag can't be overwritten. - commonTags := map[string]string{} - for k, v := range state.Tags { - commonTags[k] = v - } - if state.Options.SystemTags.Has(stats.TagGroup) { - commonTags["group"] = state.Group.Path - } + // Prepare the metric tags + commonTags := state.CloneTags() if len(extras) > 0 { obj := extras[0].ToObject(rt) for _, k := range obj.Keys() { diff --git a/js/modules/k6/k6_test.go b/js/modules/k6/k6_test.go index 6dbe4c1ef95..b3649d92903 100644 --- a/js/modules/k6/k6_test.go +++ b/js/modules/k6/k6_test.go @@ -155,6 +155,7 @@ func TestCheck(t *testing.T) { SystemTags: &stats.DefaultSystemTagSet, }, Samples: samples, + Tags: map[string]string{"group": root.Path}, }, samples } t.Run("Object", func(t *testing.T) { diff --git a/js/modules/k6/metrics/metrics.go b/js/modules/k6/metrics/metrics.go index 8153673f53b..437b9bc0189 100644 --- a/js/modules/k6/metrics/metrics.go +++ b/js/modules/k6/metrics/metrics.go @@ -74,14 +74,7 @@ func (m Metric) Add(ctx context.Context, v goja.Value, addTags ...map[string]str return false, ErrMetricsAddInInitContext } - tags := map[string]string{} - for k, v := range state.Tags { - tags[k] = v - } - if state.Options.SystemTags.Has(stats.TagGroup) { - tags["group"] = state.Group.Path - } - + tags := state.CloneTags() for _, ts := range addTags { for k, v := range ts { tags[k] = v diff --git a/js/modules/k6/metrics/metrics_test.go b/js/modules/k6/metrics/metrics_test.go index e99b104cbf7..58acf585c03 100644 --- a/js/modules/k6/metrics/metrics_test.go +++ b/js/modules/k6/metrics/metrics_test.go @@ -72,6 +72,7 @@ func TestMetrics(t *testing.T) { Options: lib.Options{SystemTags: stats.NewSystemTagSet(stats.TagGroup)}, Group: root, Samples: samples, + Tags: map[string]string{"group": root.Path}, } isTimeString := "" @@ -96,8 +97,10 @@ func TestMetrics(t *testing.T) { "Child": child, } for name, g := range groups { + name, g := name, g t.Run(name, func(t *testing.T) { state.Group = g + state.Tags["group"] = g.Path for name, val := range values { t.Run(name, func(t *testing.T) { t.Run("Simple", func(t *testing.T) { diff --git a/js/modules/k6/ws/ws.go b/js/modules/k6/ws/ws.go index dfa1ffe439c..078b976ac94 100644 --- a/js/modules/k6/ws/ws.go +++ b/js/modules/k6/ws/ws.go @@ -104,10 +104,7 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP // Leave header to nil by default so we can pass it directly to the Dialer var header http.Header - tags := map[string]string{} - for k, v := range state.Tags { - tags[k] = v - } + tags := state.CloneTags() // Parse the optional second argument (params) if !goja.IsUndefined(paramsV) && !goja.IsNull(paramsV) { @@ -147,9 +144,6 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP if state.Options.SystemTags.Has(stats.TagURL) { tags["url"] = url } - if state.Options.SystemTags.Has(stats.TagGroup) { - tags["group"] = state.Group.Path - } // Pass a custom net.Dial function to websocket.Dialer that will substitute // the underlying net.Conn with our own tracked netext.Conn diff --git a/js/runner_test.go b/js/runner_test.go index 319252f9da5..d0e6f967a3a 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -31,6 +31,7 @@ import ( stdlog "log" "net" "net/http" + "net/url" "os" "sync" "testing" @@ -1555,3 +1556,85 @@ func TestStuffNotPanicking(t *testing.T) { require.NoError(t, err) } } + +func TestSystemTags(t *testing.T) { + t.Parallel() + tb := httpmultibin.NewHTTPMultiBin(t) + defer tb.Cleanup() + + // Handle paths with custom logic + tb.Mux.HandleFunc("/wrong-redirect", func(w http.ResponseWriter, r *http.Request) { + w.Header().Add("Location", "%") + w.WriteHeader(http.StatusTemporaryRedirect) + }) + + r, err := getSimpleRunnerWithOptions("/script.js", tb.Replacer.Replace(` + var http = require("k6/http"); + + exports.http_get = function() { + http.get("HTTPBIN_IP_URL"); + }; + exports.https_get = function() { + http.get("HTTPSBIN_IP_URL"); + }; + exports.bad_url_get = function() { + http.get("http://127.0.0.1:1"); + }; + exports.noop = function() {}; + `), lib.RuntimeOptions{CompatibilityMode: null.StringFrom("base")}) + require.NoError(t, err) + + httpURL, err := url.Parse(tb.ServerHTTP.URL) + require.NoError(t, err) + + testedSystemTags := []struct{ tag, exec, expVal string }{ + {"proto", "http_get", "HTTP/1.1"}, + {"status", "http_get", "200"}, + {"method", "http_get", "GET"}, + {"url", "http_get", tb.ServerHTTP.URL}, + {"url", "https_get", tb.ServerHTTPS.URL}, + {"ip", "http_get", httpURL.Hostname()}, + {"name", "http_get", tb.ServerHTTP.URL}, + {"group", "http_get", ""}, + {"vu", "http_get", "8"}, + {"vu", "noop", "9"}, + {"iter", "http_get", "0"}, + {"iter", "noop", "0"}, + {"tls_version", "https_get", "tls1.3"}, + {"ocsp_status", "https_get", "unknown"}, + {"error", "bad_url_get", `dial: connection refused`}, + {"error_code", "bad_url_get", "1212"}, + //TODO: add more tests + } + + samples := make(chan stats.SampleContainer, 100) + for num, tc := range testedSystemTags { + num, tc := num, tc + t.Run(fmt.Sprintf("TC %d with only %s", num, tc.tag), func(t *testing.T) { + require.NoError(t, r.SetOptions(r.GetOptions().Apply(lib.Options{ + Throw: null.BoolFrom(false), + TLSVersion: &lib.TLSVersions{Max: lib.TLSVersion13}, + SystemTags: stats.ToSystemTagSet([]string{tc.tag}), + InsecureSkipTLSVerify: null.BoolFrom(true), + }))) + + vu, err := r.NewVU(int64(num), samples) + require.NoError(t, err) + activeVU := vu.Activate(&lib.VUActivationParams{ + RunContext: context.Background(), + Exec: tc.exec, + }) + require.NoError(t, activeVU.RunOnce()) + + bufSamples := stats.GetBufferedSamples(samples) + assert.NotEmpty(t, bufSamples) + for _, sample := range bufSamples[0].GetSamples() { + assert.NotEmpty(t, sample.Tags) + for emittedTag, emittedVal := range sample.Tags.CloneTags() { + assert.Equal(t, tc.tag, emittedTag) + assert.Equal(t, tc.expVal, emittedVal) + } + } + }) + } +} diff --git a/lib/netext/httpext/request.go b/lib/netext/httpext/request.go index 7ea374cea65..9a9af70ae34 100644 --- a/lib/netext/httpext/request.go +++ b/lib/netext/httpext/request.go @@ -231,14 +231,10 @@ func MakeRequest(ctx context.Context, preq *ParsedHTTPRequest) (*Response, error } } - tags := state.Options.RunTags.CloneTags() - for k, v := range state.Tags { - tags[k] = v - } + tags := state.CloneTags() for k, v := range preq.Tags { tags[k] = v } - if state.Options.SystemTags.Has(stats.TagMethod) { tags["method"] = preq.Req.Method } @@ -250,15 +246,6 @@ func MakeRequest(ctx context.Context, preq *ParsedHTTPRequest) (*Response, error if _, ok := tags["name"]; !ok && state.Options.SystemTags.Has(stats.TagName) { tags["name"] = preq.URL.Name } - if state.Options.SystemTags.Has(stats.TagGroup) { - tags["group"] = state.Group.Path - } - if state.Options.SystemTags.Has(stats.TagVU) { - tags["vu"] = strconv.FormatInt(state.Vu, 10) - } - if state.Options.SystemTags.Has(stats.TagIter) { - tags["iter"] = strconv.FormatInt(state.Iteration, 10) - } // Check rate limit *after* we've prepared a request; no need to wait with that part. if rpsLimit := state.RPSLimit; rpsLimit != nil { diff --git a/lib/state.go b/lib/state.go index 1ffa2807c3a..464604d3253 100644 --- a/lib/state.go +++ b/lib/state.go @@ -69,3 +69,12 @@ type State struct { Vu, Iteration int64 Tags map[string]string } + +// CloneTags makes a copy of the tags map and returns it. +func (s *State) CloneTags() map[string]string { + tags := make(map[string]string, len(s.Tags)) + for k, v := range s.Tags { + tags[k] = v + } + return tags +} diff --git a/stats/stats.go b/stats/stats.go index 7bfe04782eb..d1fdc6731b5 100644 --- a/stats/stats.go +++ b/stats/stats.go @@ -252,11 +252,12 @@ func (st *SampleTags) UnmarshalJSON(data []byte) error { // CloneTags copies the underlying set of a sample tags and // returns it. If the receiver is nil, it returns an empty non-nil map. func (st *SampleTags) CloneTags() map[string]string { - res := map[string]string{} - if st != nil { - for k, v := range st.tags { - res[k] = v - } + if st == nil { + return map[string]string{} + } + res := make(map[string]string, len(st.tags)) + for k, v := range st.tags { + res[k] = v } return res } From e1a5c0af001977b6a10945791f5b16288937d391 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 18 May 2020 12:23:45 +0300 Subject: [PATCH 259/350] Fix __ENV polution --- js/bundle.go | 4 ++++ js/runner.go | 13 +++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/js/bundle.go b/js/bundle.go index f6e2686957b..b9d69810246 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -60,6 +60,9 @@ type BundleInstance struct { Runtime *goja.Runtime Context *context.Context + //TODO: maybe just have a reference to the Bundle? or save and pass rtOpts? + env map[string]string + exports map[string]goja.Callable } @@ -235,6 +238,7 @@ func (b *Bundle) Instantiate() (bi *BundleInstance, instErr error) { Runtime: rt, Context: ctxPtr, exports: make(map[string]goja.Callable), + env: b.Env, } // Grab any exported functions that could be executed. These were diff --git a/js/runner.go b/js/runner.go index cc326994abb..d0f95f8ee74 100644 --- a/js/runner.go +++ b/js/runner.go @@ -396,13 +396,14 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { } // Override the preset global env with any custom env vars - if len(params.Env) > 0 { - env := u.Runtime.Get("__ENV").Export().(map[string]string) - for key, value := range params.Env { - env[key] = value - } - u.Runtime.Set("__ENV", env) + env := make(map[string]string, len(u.env)+len(params.Env)) + for key, value := range u.env { + env[key] = value + } + for key, value := range params.Env { + env[key] = value } + u.Runtime.Set("__ENV", env) avu := &ActiveVU{ VU: u, From b2b86482ef97982969d024b54cf5730b2eba6dce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 18 May 2020 12:46:53 +0200 Subject: [PATCH 260/350] Tweak NoCrossover test to better cover VU reuse scenarios This would previously fail without the fixes in #1453. --- core/local/local_test.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 5061dab098a..77f76ee3ed1 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -402,7 +402,8 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { type: 'per-vu-iterations', vus: 1, iterations: 1, - gracefulStop: '0.5s', + gracefulStop: '0s', + maxDuration: '0.5s', exec: 's1func', env: { TESTVAR1: 'scenario1' }, tags: { testtag1: 'scenario1' }, @@ -412,6 +413,8 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { vus: 1, iterations: 1, gracefulStop: '0.5s', + startTime: '0.5s', + maxDuration: '2s', exec: 's2func', env: { TESTVAR2: 'scenario2' }, tags: { testtag2: 'scenario2' }, @@ -442,6 +445,9 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { checkVar('TESTVAR3', undefined); checkVar('TESTGLOBALVAR', 'global'); + // Intentionally try to pollute the env + __ENV.TESTVAR2 = 'overridden'; + http.get('HTTPBIN_IP_URL/', { tags: { reqtag: 'scenario1' }}); } @@ -496,7 +502,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { execScheduler, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() done := make(chan struct{}) From d1aeda5b5d49154c578a0162084e36c4766a0871 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 20 May 2020 14:36:11 +0300 Subject: [PATCH 261/350] Prevent panics from multiple calls to stop the engine --- core/engine.go | 5 ++++- core/engine_test.go | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/engine.go b/core/engine.go index 40b9df85500..973d0d73b8c 100644 --- a/core/engine.go +++ b/core/engine.go @@ -60,6 +60,7 @@ type Engine struct { SummaryExport bool logger *logrus.Entry + stopOnce sync.Once stopChan chan struct{} Metrics map[string]*stats.Metric @@ -288,7 +289,9 @@ func (e *Engine) IsTainted() bool { // Stop closes a signal channel, forcing a running Engine to return func (e *Engine) Stop() { - close(e.stopChan) + e.stopOnce.Do(func() { + close(e.stopChan) + }) } // IsStopped returns a bool indicating whether the Engine has been stopped diff --git a/core/engine_test.go b/core/engine_test.go index 38e421f5248..4944c174bff 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -193,6 +193,7 @@ func TestEngineStopped(t *testing.T) { assert.Equal(t, false, e.IsStopped(), "engine should be running") e.Stop() assert.Equal(t, true, e.IsStopped(), "engine should be stopped") + e.Stop() // test that a second stop doesn't panic } func TestEngineCollector(t *testing.T) { From 2180b1900bd5fef81d7dee6c0aa1dbe1f6cd13b5 Mon Sep 17 00:00:00 2001 From: thinkerou Date: Wed, 20 May 2020 00:02:26 +0800 Subject: [PATCH 262/350] doc: add copyright --- cmd/ui_test.go | 20 ++++++++++++++++++++ js/common/init_error.go | 20 ++++++++++++++++++++ js/http_bench_test.go | 20 ++++++++++++++++++++ js/modules/k6/html/element.go | 20 ++++++++++++++++++++ js/modules/k6/html/elements.go | 20 ++++++++++++++++++++ js/modules/k6/html/elements_gen_test.go | 20 ++++++++++++++++++++ js/modules/k6/html/gen/gen_elements.go | 20 ++++++++++++++++++++ js/modules/k6/html/util.go | 20 ++++++++++++++++++++ lib/consts/consts.go | 20 ++++++++++++++++++++ lib/context.go | 20 ++++++++++++++++++++ lib/context_test.go | 20 ++++++++++++++++++++ lib/fsext/cacheonread.go | 20 ++++++++++++++++++++ lib/fsext/changepathfs.go | 20 ++++++++++++++++++++ lib/fsext/changepathfs_test.go | 20 ++++++++++++++++++++ lib/fsext/trimpathseparator_test.go | 20 ++++++++++++++++++++ lib/fsext/walk.go | 20 ++++++++++++++++++++ lib/netext/httpext/compression.go | 20 ++++++++++++++++++++ lib/netext/httpext/request_test.go | 20 ++++++++++++++++++++ lib/netext/tls.go | 20 ++++++++++++++++++++ lib/old_archive_test.go | 20 ++++++++++++++++++++ lib/testutils/env.go | 20 ++++++++++++++++++++ lib/timeout_error.go | 20 ++++++++++++++++++++ lib/timeout_error_test.go | 20 ++++++++++++++++++++ loader/filesystems.go | 20 ++++++++++++++++++++ loader/readsource.go | 20 ++++++++++++++++++++ loader/readsource_test.go | 20 ++++++++++++++++++++ stats/cloud/bench_test.go | 20 ++++++++++++++++++++ stats/datadog/collector_test.go | 20 ++++++++++++++++++++ stats/influxdb/bench_test.go | 20 ++++++++++++++++++++ stats/influxdb/collector_test.go | 20 ++++++++++++++++++++ stats/statsd/collector_test.go | 20 ++++++++++++++++++++ stats/statsd/common/collector_test.go | 20 ++++++++++++++++++++ stats/statsd/common/testutil/test_helper.go | 20 ++++++++++++++++++++ stats/system_tag.go | 20 ++++++++++++++++++++ stats/system_tag_test.go | 20 ++++++++++++++++++++ 35 files changed, 700 insertions(+) diff --git a/cmd/ui_test.go b/cmd/ui_test.go index 9f03df39bae..c0971547074 100644 --- a/cmd/ui_test.go +++ b/cmd/ui_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package cmd import ( diff --git a/js/common/init_error.go b/js/common/init_error.go index 8cfd53b644e..59237c90dee 100644 --- a/js/common/init_error.go +++ b/js/common/init_error.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package common // InitContextError is an error that happened during the a test init context diff --git a/js/http_bench_test.go b/js/http_bench_test.go index 63e92dfc836..aeb5cf50613 100644 --- a/js/http_bench_test.go +++ b/js/http_bench_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package js import ( diff --git a/js/modules/k6/html/element.go b/js/modules/k6/html/element.go index 96770512a7c..65c9f8c916d 100644 --- a/js/modules/k6/html/element.go +++ b/js/modules/k6/html/element.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package html import ( diff --git a/js/modules/k6/html/elements.go b/js/modules/k6/html/elements.go index b6426ed7d10..a1b1f344caa 100644 --- a/js/modules/k6/html/elements.go +++ b/js/modules/k6/html/elements.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package html import ( diff --git a/js/modules/k6/html/elements_gen_test.go b/js/modules/k6/html/elements_gen_test.go index 5bc77255a61..0d8525c7557 100644 --- a/js/modules/k6/html/elements_gen_test.go +++ b/js/modules/k6/html/elements_gen_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package html import ( diff --git a/js/modules/k6/html/gen/gen_elements.go b/js/modules/k6/html/gen/gen_elements.go index 5b9098c2aa2..45b8b4b06ae 100644 --- a/js/modules/k6/html/gen/gen_elements.go +++ b/js/modules/k6/html/gen/gen_elements.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package main import ( diff --git a/js/modules/k6/html/util.go b/js/modules/k6/html/util.go index a60dec7a48c..f0cf35f23dc 100644 --- a/js/modules/k6/html/util.go +++ b/js/modules/k6/html/util.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package html import ( diff --git a/lib/consts/consts.go b/lib/consts/consts.go index d78269ea459..e6100f5a8bd 100644 --- a/lib/consts/consts.go +++ b/lib/consts/consts.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package consts import ( diff --git a/lib/context.go b/lib/context.go index 9c626072c9a..a7544e4de4b 100644 --- a/lib/context.go +++ b/lib/context.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package lib import "context" diff --git a/lib/context_test.go b/lib/context_test.go index 8cd12b7a9ce..35dbc4ea6c6 100644 --- a/lib/context_test.go +++ b/lib/context_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package lib import ( diff --git a/lib/fsext/cacheonread.go b/lib/fsext/cacheonread.go index 9a534f3f664..7f57ddcdc22 100644 --- a/lib/fsext/cacheonread.go +++ b/lib/fsext/cacheonread.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package fsext import ( diff --git a/lib/fsext/changepathfs.go b/lib/fsext/changepathfs.go index 7f7a81d6f2f..473a92d2874 100644 --- a/lib/fsext/changepathfs.go +++ b/lib/fsext/changepathfs.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package fsext import ( diff --git a/lib/fsext/changepathfs_test.go b/lib/fsext/changepathfs_test.go index 5e91ea77f97..006da3f4cc6 100644 --- a/lib/fsext/changepathfs_test.go +++ b/lib/fsext/changepathfs_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package fsext import ( diff --git a/lib/fsext/trimpathseparator_test.go b/lib/fsext/trimpathseparator_test.go index 678b0c1b758..d5a2865e733 100644 --- a/lib/fsext/trimpathseparator_test.go +++ b/lib/fsext/trimpathseparator_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package fsext import ( diff --git a/lib/fsext/walk.go b/lib/fsext/walk.go index 2869f022c82..345561418e8 100644 --- a/lib/fsext/walk.go +++ b/lib/fsext/walk.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package fsext import ( diff --git a/lib/netext/httpext/compression.go b/lib/netext/httpext/compression.go index 2a7299602ca..d52f07c8a73 100644 --- a/lib/netext/httpext/compression.go +++ b/lib/netext/httpext/compression.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package httpext import ( diff --git a/lib/netext/httpext/request_test.go b/lib/netext/httpext/request_test.go index 059bc2fa9cf..2ee74249bac 100644 --- a/lib/netext/httpext/request_test.go +++ b/lib/netext/httpext/request_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package httpext import ( diff --git a/lib/netext/tls.go b/lib/netext/tls.go index e4bd24eb3c1..223421ba747 100644 --- a/lib/netext/tls.go +++ b/lib/netext/tls.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package netext import ( diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index d87ef2dde1c..af2cdc88aa9 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package lib import ( diff --git a/lib/testutils/env.go b/lib/testutils/env.go index d191e880f2d..6ad78d9f848 100644 --- a/lib/testutils/env.go +++ b/lib/testutils/env.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package testutils import ( diff --git a/lib/timeout_error.go b/lib/timeout_error.go index 5d50e3801c2..16cccded9e9 100644 --- a/lib/timeout_error.go +++ b/lib/timeout_error.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package lib import ( diff --git a/lib/timeout_error_test.go b/lib/timeout_error_test.go index 5a3ce9b61f3..99ae2d7b063 100644 --- a/lib/timeout_error_test.go +++ b/lib/timeout_error_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package lib import ( diff --git a/loader/filesystems.go b/loader/filesystems.go index 2bfde5c1205..6baabc852f5 100644 --- a/loader/filesystems.go +++ b/loader/filesystems.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package loader import ( diff --git a/loader/readsource.go b/loader/readsource.go index 3efd4ce85a5..89f01ffabe2 100644 --- a/loader/readsource.go +++ b/loader/readsource.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package loader import ( diff --git a/loader/readsource_test.go b/loader/readsource_test.go index e962f304f87..f3a5fad83d5 100644 --- a/loader/readsource_test.go +++ b/loader/readsource_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package loader import ( diff --git a/stats/cloud/bench_test.go b/stats/cloud/bench_test.go index afd871401b0..3d42c86edbe 100644 --- a/stats/cloud/bench_test.go +++ b/stats/cloud/bench_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package cloud import ( diff --git a/stats/datadog/collector_test.go b/stats/datadog/collector_test.go index 205e7279a5a..83bcce5a4ea 100644 --- a/stats/datadog/collector_test.go +++ b/stats/datadog/collector_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package datadog import ( diff --git a/stats/influxdb/bench_test.go b/stats/influxdb/bench_test.go index e3f3b046fa8..4fc01093bd2 100644 --- a/stats/influxdb/bench_test.go +++ b/stats/influxdb/bench_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package influxdb import ( diff --git a/stats/influxdb/collector_test.go b/stats/influxdb/collector_test.go index 61e0b0edc98..92692426bd3 100644 --- a/stats/influxdb/collector_test.go +++ b/stats/influxdb/collector_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package influxdb import ( diff --git a/stats/statsd/collector_test.go b/stats/statsd/collector_test.go index e5947861a09..33665176923 100644 --- a/stats/statsd/collector_test.go +++ b/stats/statsd/collector_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package statsd import ( diff --git a/stats/statsd/common/collector_test.go b/stats/statsd/common/collector_test.go index 5804063343a..58c2f787a7c 100644 --- a/stats/statsd/common/collector_test.go +++ b/stats/statsd/common/collector_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package common import ( diff --git a/stats/statsd/common/testutil/test_helper.go b/stats/statsd/common/testutil/test_helper.go index b7c49e53a1b..b5bb55c8545 100644 --- a/stats/statsd/common/testutil/test_helper.go +++ b/stats/statsd/common/testutil/test_helper.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package testutil import ( diff --git a/stats/system_tag.go b/stats/system_tag.go index fac1d0dda4d..fde913d34cd 100644 --- a/stats/system_tag.go +++ b/stats/system_tag.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package stats import ( diff --git a/stats/system_tag_test.go b/stats/system_tag_test.go index 64de86433af..27c98e3fd17 100644 --- a/stats/system_tag_test.go +++ b/stats/system_tag_test.go @@ -1,3 +1,23 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2019 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + package stats import ( From 6aadc03e862bd50e14b47c48a847062fcfb04304 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Apr 2020 15:39:02 +0300 Subject: [PATCH 263/350] dry getSimpleRunner the same way as getSimpleBundle --- js/console_test.go | 42 +++++++++++++++++++-------------------- js/module_loading_test.go | 16 +++++++-------- js/runner_test.go | 10 +++++----- 3 files changed, 34 insertions(+), 34 deletions(-) diff --git a/js/console_test.go b/js/console_test.go index f25a575b850..aa8b938ab9f 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -70,29 +70,29 @@ func TestConsoleContext(t *testing.T) { assert.Equal(t, "b", entry.Message) } } -func getSimpleRunner(path, data string) (*Runner, error) { - return getSimpleRunnerWithFileFs(path, data, afero.NewMemMapFs()) -} - -func getSimpleRunnerWithOptions(path, data string, options lib.RuntimeOptions) (*Runner, error) { - return New(&loader.SourceData{ - URL: &url.URL{Path: path, Scheme: "file"}, - Data: []byte(data), - }, map[string]afero.Fs{ - "file": afero.NewMemMapFs(), - "https": afero.NewMemMapFs()}, - options) +func getSimpleRunner(filename, data string, opts ...interface{}) (*Runner, error) { + var ( + fs = afero.NewMemMapFs() + rtOpts = lib.RuntimeOptions{} + ) + for _, o := range opts { + switch opt := o.(type) { + case afero.Fs: + fs = opt + case lib.RuntimeOptions: + rtOpts = opt + } + } + return New( + &loader.SourceData{ + URL: &url.URL{Path: filename, Scheme: "file"}, + Data: []byte(data), + }, + map[string]afero.Fs{"file": fs, "https": afero.NewMemMapFs()}, + rtOpts, + ) } -func getSimpleRunnerWithFileFs(path, data string, fileFs afero.Fs) (*Runner, error) { - return New(&loader.SourceData{ - URL: &url.URL{Path: path, Scheme: "file"}, - Data: []byte(data), - }, map[string]afero.Fs{ - "file": fileFs, - "https": afero.NewMemMapFs()}, - lib.RuntimeOptions{}) -} func TestConsole(t *testing.T) { levels := map[string]logrus.Level{ "log": logrus.InfoLevel, diff --git a/js/module_loading_test.go b/js/module_loading_test.go index daee1431330..6e64c62dc2c 100644 --- a/js/module_loading_test.go +++ b/js/module_loading_test.go @@ -87,7 +87,7 @@ func TestLoadOnceGlobalVars(t *testing.T) { return c.C(); } `), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", ` + r1, err := getSimpleRunner("/script.js", ` import { A } from "./A.js"; import { B } from "./B.js"; @@ -136,7 +136,7 @@ func TestLoadExportsIsUsableInModule(t *testing.T) { return exports.A() + "B"; } `), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", ` + r1, err := getSimpleRunner("/script.js", ` import { A, B } from "./A.js"; export default function(data) { @@ -185,7 +185,7 @@ func TestLoadDoesntBreakHTTPGet(t *testing.T) { return http.get("HTTPBIN_URL/get"); } `)), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", ` + r1, err := getSimpleRunner("/script.js", ` import { A } from "./A.js"; export default function(data) { @@ -228,7 +228,7 @@ func TestLoadGlobalVarsAreNotSharedBetweenVUs(t *testing.T) { return globalVar; } `), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", ` + r1, err := getSimpleRunner("/script.js", ` import { A } from "./A.js"; export default function(data) { @@ -302,7 +302,7 @@ func TestLoadCycle(t *testing.T) { `), os.ModePerm)) data, err := afero.ReadFile(fs, "/main.js") require.NoError(t, err) - r1, err := getSimpleRunnerWithFileFs("/main.js", string(data), fs) + r1, err := getSimpleRunner("/main.js", string(data), fs) require.NoError(t, err) arc := r1.MakeArchive() @@ -351,7 +351,7 @@ func TestLoadCycleBinding(t *testing.T) { } `), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/main.js", ` + r1, err := getSimpleRunner("/main.js", ` import {foo} from './a.js'; import {bar} from './b.js'; export default function() { @@ -410,7 +410,7 @@ func TestBrowserified(t *testing.T) { }); `), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", ` + r1, err := getSimpleRunner("/script.js", ` import {alpha, bravo } from "./browserified.js"; export default function(data) { @@ -465,7 +465,7 @@ func TestLoadingUnexistingModuleDoesntPanic(t *testing.T) { } }` require.NoError(t, afero.WriteFile(fs, "/script.js", []byte(data), 0644)) - r1, err := getSimpleRunnerWithFileFs("/script.js", data, fs) + r1, err := getSimpleRunner("/script.js", data, fs) require.NoError(t, err) arc := r1.MakeArchive() diff --git a/js/runner_test.go b/js/runner_test.go index d0e6f967a3a..61e9b11bc5e 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -155,7 +155,7 @@ func TestOptionsSettingToScript(t *testing.T) { throw new Error("expected teardownTimeout to be " + __ENV.expectedTeardownTimeout + " but it was " + options.teardownTimeout); } };` - r, err := getSimpleRunnerWithOptions("/script.js", data, + r, err := getSimpleRunner("/script.js", data, lib.RuntimeOptions{Env: map[string]string{"expectedTeardownTimeout": "4s"}}) require.NoError(t, err) @@ -193,7 +193,7 @@ func TestOptionsPropagationToScript(t *testing.T) { };` expScriptOptions := lib.Options{SetupTimeout: types.NullDurationFrom(1 * time.Second)} - r1, err := getSimpleRunnerWithOptions("/script.js", data, + r1, err := getSimpleRunner("/script.js", data, lib.RuntimeOptions{Env: map[string]string{"expectedSetupTimeout": "1s"}}) require.NoError(t, err) require.Equal(t, expScriptOptions, r1.GetOptions()) @@ -466,7 +466,7 @@ func TestRunnerIntegrationImports(t *testing.T) { for name, data := range testdata { name, data := name, data t.Run(name, func(t *testing.T) { - r1, err := getSimpleRunnerWithFileFs(data.filename, fmt.Sprintf(` + r1, err := getSimpleRunner(data.filename, fmt.Sprintf(` import hi from "%s"; export default function() { if (hi != "hi!") { throw new Error("incorrect value"); } @@ -1442,7 +1442,7 @@ func TestArchiveRunningIntegrity(t *testing.T) { `) require.NoError(t, afero.WriteFile(fs, "/home/somebody/test.json", []byte(`42`), os.ModePerm)) require.NoError(t, afero.WriteFile(fs, "/script.js", []byte(data), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", data, fs) + r1, err := getSimpleRunner("/script.js", data, fs) require.NoError(t, err) buf := bytes.NewBuffer(nil) @@ -1477,7 +1477,7 @@ func TestArchiveNotPanicking(t *testing.T) { fs := afero.NewMemMapFs() require.NoError(t, afero.WriteFile(fs, "/non/existent", []byte(`42`), os.ModePerm)) - r1, err := getSimpleRunnerWithFileFs("/script.js", tb.Replacer.Replace(` + r1, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` let fput = open("/non/existent"); export default function(data) { } From 252319aa69e21366e7c1f1ebb17dc416f4dacf29 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Apr 2020 16:43:54 +0300 Subject: [PATCH 264/350] optimize js package test by using compatibility mode base by default --- js/console_test.go | 6 +- js/module_loading_test.go | 17 +-- js/runner_test.go | 253 +++++++++++++++++++------------------- 3 files changed, 141 insertions(+), 135 deletions(-) diff --git a/js/console_test.go b/js/console_test.go index aa8b938ab9f..2d65af9cbdf 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -73,7 +73,7 @@ func TestConsoleContext(t *testing.T) { func getSimpleRunner(filename, data string, opts ...interface{}) (*Runner, error) { var ( fs = afero.NewMemMapFs() - rtOpts = lib.RuntimeOptions{} + rtOpts = lib.RuntimeOptions{CompatibilityMode: null.NewString("base", true)} ) for _, o := range opts { switch opt := o.(type) { @@ -117,7 +117,7 @@ func TestConsole(t *testing.T) { args, result := args, result t.Run(args, func(t *testing.T) { r, err := getSimpleRunner("/script.js", fmt.Sprintf( - `export default function() { console.%s(%s); }`, + `exports.default = function() { console.%s(%s); }`, name, args, )) assert.NoError(t, err) @@ -209,7 +209,7 @@ func TestFileConsole(t *testing.T) { } r, err := getSimpleRunner("/script", fmt.Sprintf( - `export default function() { console.%s(%s); }`, + `exports.default = function() { console.%s(%s); }`, name, args, )) assert.NoError(t, err) diff --git a/js/module_loading_test.go b/js/module_loading_test.go index 6e64c62dc2c..57b0f5728c3 100644 --- a/js/module_loading_test.go +++ b/js/module_loading_test.go @@ -28,6 +28,7 @@ import ( "github.com/spf13/afero" "github.com/stretchr/testify/require" + null "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/httpmultibin" @@ -99,7 +100,7 @@ func TestLoadOnceGlobalVars(t *testing.T) { throw new Error("A() != B() (" + A() + ") != (" + B() + ")"); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -148,7 +149,7 @@ func TestLoadExportsIsUsableInModule(t *testing.T) { throw new Error("wrong value of B() " + B()); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -194,7 +195,7 @@ func TestLoadDoesntBreakHTTPGet(t *testing.T) { throw new Error("wrong status "+ resp.status); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) require.NoError(t, r1.SetOptions(lib.Options{Hosts: tb.Dialer.Hosts})) @@ -239,7 +240,7 @@ func TestLoadGlobalVarsAreNotSharedBetweenVUs(t *testing.T) { throw new Error("wrong value of a " + a); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -302,7 +303,7 @@ func TestLoadCycle(t *testing.T) { `), os.ModePerm)) data, err := afero.ReadFile(fs, "/main.js") require.NoError(t, err) - r1, err := getSimpleRunner("/main.js", string(data), fs) + r1, err := getSimpleRunner("/main.js", string(data), fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -364,7 +365,7 @@ func TestLoadCycleBinding(t *testing.T) { throw new Error("Wrong value of bar() "+ barMessage); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -428,7 +429,7 @@ func TestBrowserified(t *testing.T) { throw new Error("bravo.B() != 'b' (" + bravo.B() + ") != 'b'"); } } - `, fs) + `, fs, lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) require.NoError(t, err) arc := r1.MakeArchive() @@ -459,7 +460,7 @@ func TestLoadingUnexistingModuleDoesntPanic(t *testing.T) { } catch (err) { b = "correct"; } - export default function() { + exports.default = function() { if (b != "correct") { throw new Error("wrong b "+ JSON.stringify(b)); } diff --git a/js/runner_test.go b/js/runner_test.go index 61e9b11bc5e..3251abe0d00 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -64,8 +64,8 @@ import ( func TestRunnerNew(t *testing.T) { t.Run("Valid", func(t *testing.T) { r, err := getSimpleRunner("/script.js", ` - let counter = 0; - export default function() { counter++; } + var counter = 0; + exports.default = function() { counter++; } `) assert.NoError(t, err) @@ -94,7 +94,7 @@ func TestRunnerNew(t *testing.T) { } func TestRunnerGetDefaultGroup(t *testing.T) { - r1, err := getSimpleRunner("/script.js", `export default function() {};`) + r1, err := getSimpleRunner("/script.js", `exports.default = function() {};`) if assert.NoError(t, err) { assert.NotNil(t, r1.GetDefaultGroup()) } @@ -106,7 +106,7 @@ func TestRunnerGetDefaultGroup(t *testing.T) { } func TestRunnerOptions(t *testing.T) { - r1, err := getSimpleRunner("/script.js", `export default function() {};`) + r1, err := getSimpleRunner("/script.js", `exports.default = function() {};`) if !assert.NoError(t, err) { return } @@ -136,10 +136,10 @@ func TestOptionsSettingToScript(t *testing.T) { optionVariants := []string{ "", - "let options = null;", - "let options = undefined;", - "let options = {};", - "let options = {teardownTimeout: '1s'};", + "var options = null;", + "var options = undefined;", + "var options = {};", + "var options = {teardownTimeout: '1s'};", } for i, variant := range optionVariants { @@ -147,7 +147,7 @@ func TestOptionsSettingToScript(t *testing.T) { t.Run(fmt.Sprintf("Variant#%d", i), func(t *testing.T) { t.Parallel() data := variant + ` - export default function() { + exports.default = function() { if (!options) { throw new Error("Expected options to be defined!"); } @@ -179,8 +179,9 @@ func TestOptionsSettingToScript(t *testing.T) { func TestOptionsPropagationToScript(t *testing.T) { t.Parallel() data := ` - export let options = { setupTimeout: "1s", myOption: "test" }; - export default function() { + var options = { setupTimeout: "1s", myOption: "test" }; + exports.options = options; + exports.default = function() { if (options.external) { throw new Error("Unexpected property external!"); } @@ -204,7 +205,7 @@ func TestOptionsPropagationToScript(t *testing.T) { require.Equal(t, expScriptOptions, r2.GetOptions()) newOptions := lib.Options{SetupTimeout: types.NullDurationFrom(3 * time.Second)} - r2.SetOptions(newOptions) + require.NoError(t, r2.SetOptions(newOptions)) require.Equal(t, newOptions, r2.GetOptions()) testdata := map[string]*Runner{"Source": r1, "Archive": r2} @@ -230,11 +231,11 @@ func TestMetricName(t *testing.T) { defer tb.Cleanup() script := tb.Replacer.Replace(` - import { Counter } from "k6/metrics"; + var Counter = require("k6/metrics").Counter; - let myCounter = new Counter("not ok name @"); + var myCounter = new Counter("not ok name @"); - export default function(data) { + exports.default = function(data) { myCounter.add(1); } `) @@ -248,9 +249,9 @@ func TestSetupDataIsolation(t *testing.T) { defer tb.Cleanup() script := tb.Replacer.Replace(` - import { Counter } from "k6/metrics"; + var Counter = require("k6/metrics").Counter; - export let options = { + exports.options = { execution: { shared_iters: { type: "shared-iterations", @@ -261,13 +262,13 @@ func TestSetupDataIsolation(t *testing.T) { teardownTimeout: "5s", setupTimeout: "5s", }; - let myCounter = new Counter("mycounter"); + var myCounter = new Counter("mycounter"); - export function setup() { + exports.setup = function() { return { v: 0 }; } - export default function(data) { + exports.default = function(data) { if (data.v !== __ITER) { throw new Error("default: wrong data for iter " + __ITER + ": " + JSON.stringify(data)); } @@ -275,7 +276,7 @@ func TestSetupDataIsolation(t *testing.T) { myCounter.add(1); } - export function teardown(data) { + exports.teardown = function(data) { if (data.v !== 0) { throw new Error("teardown: wrong data: " + data.v); } @@ -356,17 +357,17 @@ func testSetupDataHelper(t *testing.T, data string) { func TestSetupDataReturnValue(t *testing.T) { testSetupDataHelper(t, ` - export let options = { setupTimeout: "1s", teardownTimeout: "1s" }; - export function setup() { + exports.options = { setupTimeout: "1s", teardownTimeout: "1s" }; + exports.setup = function() { return 42; } - export default function(data) { + exports.default = function(data) { if (data != 42) { throw new Error("default: wrong data: " + JSON.stringify(data)) } }; - export function teardown(data) { + exports.teardown = function(data) { if (data != 42) { throw new Error("teardown: wrong data: " + JSON.stringify(data)) } @@ -375,14 +376,14 @@ func TestSetupDataReturnValue(t *testing.T) { func TestSetupDataNoSetup(t *testing.T) { testSetupDataHelper(t, ` - export let options = { setupTimeout: "1s", teardownTimeout: "1s" }; - export default function(data) { + exports.options = { setupTimeout: "1s", teardownTimeout: "1s" }; + exports.default = function(data) { if (data !== undefined) { throw new Error("default: wrong data: " + JSON.stringify(data)) } }; - export function teardown(data) { + exports.teardown = function(data) { if (data !== undefined) { console.log(data); throw new Error("teardown: wrong data: " + JSON.stringify(data)) @@ -393,7 +394,7 @@ func TestSetupDataNoSetup(t *testing.T) { func TestConsoleInInitContext(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` console.log("1"); - export default function(data) { + exports.default = function(data) { }; `) require.NoError(t, err) @@ -417,15 +418,15 @@ func TestConsoleInInitContext(t *testing.T) { func TestSetupDataNoReturn(t *testing.T) { testSetupDataHelper(t, ` - export let options = { setupTimeout: "1s", teardownTimeout: "1s" }; - export function setup() { } - export default function(data) { + exports.options = { setupTimeout: "1s", teardownTimeout: "1s" }; + exports.setup = function() { } + exports.default = function(data) { if (data !== undefined) { throw new Error("default: wrong data: " + JSON.stringify(data)) } }; - export function teardown(data) { + exports.teardown = function(data) { if (data !== undefined) { throw new Error("teardown: wrong data: " + JSON.stringify(data)) } @@ -440,11 +441,12 @@ func TestRunnerIntegrationImports(t *testing.T) { "k6/metrics", "k6/html", } + rtOpts := lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")} for _, mod := range modules { mod := mod t.Run(mod, func(t *testing.T) { t.Run("Source", func(t *testing.T) { - _, err := getSimpleRunner("/script.js", fmt.Sprintf(`import "%s"; export default function() {}`, mod)) + _, err := getSimpleRunner("/script.js", fmt.Sprintf(`import "%s"; exports.default = function() {}`, mod), rtOpts) assert.NoError(t, err) }) }) @@ -454,7 +456,7 @@ func TestRunnerIntegrationImports(t *testing.T) { t.Run("Files", func(t *testing.T) { fs := afero.NewMemMapFs() require.NoError(t, fs.MkdirAll("/path/to", 0755)) - require.NoError(t, afero.WriteFile(fs, "/path/to/lib.js", []byte(`export default "hi!";`), 0644)) + require.NoError(t, afero.WriteFile(fs, "/path/to/lib.js", []byte(`exports.default = "hi!";`), 0644)) testdata := map[string]struct{ filename, path string }{ "Absolute": {"/path/script.js", "/path/to/lib.js"}, @@ -467,8 +469,8 @@ func TestRunnerIntegrationImports(t *testing.T) { name, data := name, data t.Run(name, func(t *testing.T) { r1, err := getSimpleRunner(data.filename, fmt.Sprintf(` - import hi from "%s"; - export default function() { + var hi = require("%s").default; + exports.default = function() { if (hi != "hi!") { throw new Error("incorrect value"); } }`, data.path), fs) require.NoError(t, err) @@ -496,8 +498,8 @@ func TestRunnerIntegrationImports(t *testing.T) { func TestVURunContext(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - export let options = { vus: 10 }; - export default function() { fn(); } + exports.options = { vus: 10 }; + exports.default = function() { fn(); } `) require.NoError(t, err) r1.SetOptions(r1.GetOptions().Apply(lib.Options{Throw: null.BoolFrom(true)})) @@ -548,7 +550,7 @@ func TestVURunInterrupt(t *testing.T) { } r1, err := getSimpleRunner("/script.js", ` - export default function() { while(true) {} } + exports.default = function() { while(true) {} } `) require.NoError(t, err) require.NoError(t, r1.SetOptions(lib.Options{Throw: null.BoolFrom(true)})) @@ -586,7 +588,7 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { } r1, err := getSimpleRunner("/script.js", ` - export default function() { while(true) {} } + exports.default = function() { while(true) {} } `) require.NoError(t, err) require.NoError(t, r1.SetOptions(lib.Options{Throw: null.BoolFrom(true)})) @@ -633,8 +635,8 @@ func TestVURunInterruptDoesntPanic(t *testing.T) { func TestVUIntegrationGroups(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import { group } from "k6"; - export default function() { + var group = require("k6").group; + exports.default = function() { fnOuter(); group("my group", function() { fnInner(); @@ -692,10 +694,10 @@ func TestVUIntegrationGroups(t *testing.T) { func TestVUIntegrationMetrics(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import { group } from "k6"; - import { Trend } from "k6/metrics"; - let myMetric = new Trend("my_metric"); - export default function() { myMetric.add(5); } + var group = require("k6").group; + var Trend = require("k6/metrics").Trend; + var myMetric = new Trend("my_metric"); + exports.default = function() { myMetric.add(5); } `) require.NoError(t, err) @@ -767,8 +769,8 @@ func TestVUIntegrationInsecureRequests(t *testing.T) { data := data t.Run(name, func(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import http from "k6/http"; - export default function() { http.get("https://expired.badssl.com/"); } + var http = require("k6/http");; + exports.default = function() { http.get("https://expired.badssl.com/"); } `) require.NoError(t, err) require.NoError(t, r1.SetOptions(lib.Options{Throw: null.BoolFrom(true)}.Apply(data.opts))) @@ -804,8 +806,8 @@ func TestVUIntegrationInsecureRequests(t *testing.T) { func TestVUIntegrationBlacklistOption(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import http from "k6/http"; - export default function() { http.get("http://10.1.2.3/"); } + var http = require("k6/http");; + exports.default = function() { http.get("http://10.1.2.3/"); } `) require.NoError(t, err) @@ -844,14 +846,14 @@ func TestVUIntegrationBlacklistOption(t *testing.T) { func TestVUIntegrationBlacklistScript(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import http from "k6/http"; + var http = require("k6/http");; - export let options = { + exports.options = { throw: true, blacklistIPs: ["10.0.0.0/8"], }; - export default function() { http.get("http://10.1.2.3/"); } + exports.default = function() { http.get("http://10.1.2.3/"); } `) if !assert.NoError(t, err) { return @@ -887,12 +889,14 @@ func TestVUIntegrationHosts(t *testing.T) { r1, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - import { check, fail } from "k6"; - import http from "k6/http"; - export default function() { - let res = http.get("http://test.loadimpact.com:HTTPBIN_PORT/"); + var k6 = require("k6"); + var check = k6.check; + var fail = k6.fail; + var http = require("k6/http");; + exports.default = function() { + var res = http.get("http://test.loadimpact.com:HTTPBIN_PORT/"); check(res, { - "is correct IP": (r) => r.remote_ip === "127.0.0.1" + "is correct IP": function(r) { return r.remote_ip === "127.0.0.1" } }) || fail("failed to override dns"); } `)) @@ -973,8 +977,8 @@ func TestVUIntegrationTLSConfig(t *testing.T) { data := data t.Run(name, func(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - import http from "k6/http"; - export default function() { http.get("https://sha256.badssl.com/"); } + var http = require("k6/http");; + exports.default = function() { http.get("https://sha256.badssl.com/"); } `) if !assert.NoError(t, err) { return @@ -1014,7 +1018,7 @@ func TestVUIntegrationTLSConfig(t *testing.T) { func TestVUIntegrationOpenFunctionError(t *testing.T) { r, err := getSimpleRunner("/script.js", ` - export default function() { open("/tmp/foo") } + exports.default = function() { open("/tmp/foo") } `) assert.NoError(t, err) @@ -1033,16 +1037,16 @@ func TestVUIntegrationCookiesReset(t *testing.T) { defer tb.Cleanup() r1, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - import http from "k6/http"; - export default function() { - let url = "HTTPBIN_URL"; - let preRes = http.get(url + "/cookies"); + var http = require("k6/http");; + exports.default = function() { + var url = "HTTPBIN_URL"; + var preRes = http.get(url + "/cookies"); if (preRes.status != 200) { throw new Error("wrong status (pre): " + preRes.status); } if (preRes.json().k1 || preRes.json().k2) { throw new Error("cookies persisted: " + preRes.body); } - let res = http.get(url + "/cookies/set?k2=v2&k1=v1"); + var res = http.get(url + "/cookies/set?k2=v2&k1=v1"); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.json().k1 != "v1" || res.json().k2 != "v2") { throw new Error("wrong cookies: " + res.body); @@ -1087,11 +1091,11 @@ func TestVUIntegrationCookiesNoReset(t *testing.T) { defer tb.Cleanup() r1, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - import http from "k6/http"; - export default function() { - let url = "HTTPBIN_URL"; + var http = require("k6/http");; + exports.default = function() { + var url = "HTTPBIN_URL"; if (__ITER == 0) { - let res = http.get(url + "/cookies/set?k2=v2&k1=v1"); + var res = http.get(url + "/cookies/set?k2=v2&k1=v1"); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.json().k1 != "v1" || res.json().k2 != "v2") { throw new Error("wrong cookies: " + res.body); @@ -1099,7 +1103,7 @@ func TestVUIntegrationCookiesNoReset(t *testing.T) { } if (__ITER == 1) { - let res = http.get(url + "/cookies"); + var res = http.get(url + "/cookies"); if (res.status != 200) { throw new Error("wrong status (pre): " + res.status); } if (res.json().k1 != "v1" || res.json().k2 != "v2") { throw new Error("wrong cookies: " + res.body); @@ -1145,7 +1149,7 @@ func TestVUIntegrationCookiesNoReset(t *testing.T) { func TestVUIntegrationVUID(t *testing.T) { r1, err := getSimpleRunner("/script.js", ` - export default function() { + exports.default = function() { if (__VU != 1234) { throw new Error("wrong __VU: " + __VU); } }`, ) @@ -1242,8 +1246,8 @@ func TestVUIntegrationClientCerts(t *testing.T) { go func() { _ = srv.Serve(listener) }() r1, err := getSimpleRunner("/script.js", fmt.Sprintf(` - import http from "k6/http"; - export default function() { http.get("https://%s")} + var http = require("k6/http");; + exports.default = function() { http.get("https://%s")} `, listener.Addr().String())) if !assert.NoError(t, err) { return @@ -1331,10 +1335,12 @@ func TestHTTPRequestInInitContext(t *testing.T) { defer tb.Cleanup() _, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - import { check, fail } from "k6"; - import http from "k6/http"; - let res = http.get("HTTPBIN_URL/"); - export default function() { + var k6 = require("k6"); + var check = k6.check; + var fail = k6.fail; + var http = require("k6/http");; + var res = http.get("HTTPBIN_URL/"); + exports.default = function() { console.log(test); } `)) @@ -1350,42 +1356,42 @@ func TestInitContextForbidden(t *testing.T) { table := [...][3]string{ { "http.request", - `import http from "k6/http"; - let res = http.get("HTTPBIN_URL"); - export default function() { console.log("p"); }`, + `var http = require("k6/http");; + var res = http.get("HTTPBIN_URL"); + exports.default = function() { console.log("p"); }`, k6http.ErrHTTPForbiddenInInitContext.Error(), }, { "http.batch", - `import http from "k6/http"; - let res = http.batch("HTTPBIN_URL/something", "HTTPBIN_URL/else"); - export default function() { console.log("p"); }`, + `var http = require("k6/http");; + var res = http.batch("HTTPBIN_URL/something", "HTTPBIN_URL/else"); + exports.default = function() { console.log("p"); }`, k6http.ErrBatchForbiddenInInitContext.Error(), }, { "http.cookieJar", - `import http from "k6/http"; - let jar = http.cookieJar(); - export default function() { console.log("p"); }`, + `var http = require("k6/http");; + var jar = http.cookieJar(); + exports.default = function() { console.log("p"); }`, k6http.ErrJarForbiddenInInitContext.Error(), }, { "check", - `import { check } from "k6"; - check("test", {'is test': (test) => test == "test"}) - export default function() { console.log("p"); }`, + `var check = require("k6").check; + check("test", {'is test': function(test) { return test == "test"}}) + exports.default = function() { console.log("p"); }`, k6.ErrCheckInInitContext.Error(), }, { "group", - `import { group } from "k6"; + `var group = require("k6").group; group("group1", function () { console.log("group1");}) - export default function() { console.log("p"); }`, + exports.default = function() { console.log("p"); }`, k6.ErrGroupInInitContext.Error(), }, { "ws", - `import ws from "k6/ws"; + `var ws = require("k6/ws"); var url = "ws://echo.websocket.org"; var params = { "tags": { "my_tag": "hello" } }; var response = ws.connect(url, params, function (socket) { @@ -1394,15 +1400,15 @@ func TestInitContextForbidden(t *testing.T) { }) }); - export default function() { console.log("p"); }`, + exports.default = function() { console.log("p"); }`, ws.ErrWSInInitContext.Error(), }, { "metric", - `import { Counter } from "k6/metrics"; - let counter = Counter("myCounter"); + `var Counter = require("k6/metrics").Counter; + var counter = Counter("myCounter"); counter.add(1); - export default function() { console.log("p"); }`, + exports.default = function() { console.log("p"); }`, k6metrics.ErrMetricsAddInInitContext.Error(), }, } @@ -1429,12 +1435,12 @@ func TestArchiveRunningIntegrity(t *testing.T) { fs := afero.NewMemMapFs() data := tb.Replacer.Replace(` - let fput = open("/home/somebody/test.json"); - export let options = { setupTimeout: "10s", teardownTimeout: "10s" }; - export function setup() { + var fput = open("/home/somebody/test.json"); + exports.options = { setupTimeout: "10s", teardownTimeout: "10s" }; + exports.setup = function () { return JSON.parse(fput); } - export default function(data) { + exports.default = function(data) { if (data != 42) { throw new Error("incorrect answer " + data); } @@ -1478,9 +1484,8 @@ func TestArchiveNotPanicking(t *testing.T) { fs := afero.NewMemMapFs() require.NoError(t, afero.WriteFile(fs, "/non/existent", []byte(`42`), os.ModePerm)) r1, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - let fput = open("/non/existent"); - export default function(data) { - } + var fput = open("/non/existent"); + exports.default = function(data) {} `), fs) require.NoError(t, err) @@ -1498,25 +1503,25 @@ func TestStuffNotPanicking(t *testing.T) { defer tb.Cleanup() r, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` - import http from "k6/http"; - import ws from "k6/ws"; - import { group } from "k6"; - import { parseHTML } from "k6/html"; - - export let options = { iterations: 1, vus: 1, vusMax: 1 }; - - export default function() { - const doc = parseHTML(http.get("HTTPBIN_URL/html").body); - - let testCases = [ - () => group(), - () => group("test"), - () => group("test", "wat"), - () => doc.find('p').each(), - () => doc.find('p').each("wat"), - () => doc.find('p').map(), - () => doc.find('p').map("wat"), - () => ws.connect("WSBIN_URL/ws-echo"), + var http = require("k6/http"); + var ws = require("k6/ws"); + var group = require("k6").group; + var parseHTML = require("k6/html").parseHTML; + + exports.options = { iterations: 1, vus: 1, vusMax: 1 }; + + exports.default = function() { + var doc = parseHTML(http.get("HTTPBIN_URL/html").body); + + var testCases = [ + function() { return group()}, + function() { return group("test")}, + function() { return group("test", "wat")}, + function() { return doc.find('p').each()}, + function() { return doc.find('p').each("wat")}, + function() { return doc.find('p').map()}, + function() { return doc.find('p').map("wat")}, + function() { return ws.connect("WSBIN_URL/ws-echo")}, ]; testCases.forEach(function(fn, idx) { From 923f886716b278c858f543aebe1fdb43fad657ff Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 30 Apr 2020 19:26:14 +0300 Subject: [PATCH 265/350] Introduce RunES6String and replace RunString to not support ES6 this should again lower the amount of time test take for close to zero additional scripting. --- js/common/bridge_test.go | 12 +- js/common/util.go | 9 +- js/common/util_test.go | 4 +- js/modules/k6/crypto/crypto_test.go | 128 ++++++------ js/modules/k6/crypto/x509/x509_test.go | 213 ++++++++++---------- js/modules/k6/encoding/encoding_test.go | 48 ++--- js/modules/k6/html/element_test.go | 2 +- js/modules/k6/html/elements_gen_test.go | 4 +- js/modules/k6/html/elements_test.go | 4 +- js/modules/k6/html/html_test.go | 2 +- js/modules/k6/html/serialize_test.go | 2 +- js/modules/k6/http/http_test.go | 2 +- js/modules/k6/http/request_test.go | 250 ++++++++++++------------ js/modules/k6/http/response_test.go | 40 ++-- js/modules/k6/http/tls_test.go | 2 +- js/modules/k6/k6_test.go | 10 +- js/modules/k6/metrics/metrics_test.go | 2 +- js/modules/k6/ws/ws_test.go | 56 +++--- 18 files changed, 395 insertions(+), 395 deletions(-) diff --git a/js/common/bridge_test.go b/js/common/bridge_test.go index be2fef15ff4..4ce64637370 100644 --- a/js/common/bridge_test.go +++ b/js/common/bridge_test.go @@ -324,7 +324,7 @@ func TestBind(t *testing.T) { {"Methods", bridgeTestMethodsType{}, func(t *testing.T, obj interface{}, rt *goja.Runtime) { t.Run("unexportedFn", func(t *testing.T) { _, err := RunString(rt, `obj.unexportedFn()`) - assert.EqualError(t, err, "TypeError: Object has no member 'unexportedFn' at :1:30(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'unexportedFn' at :1:17(3)") }) t.Run("ExportedFn", func(t *testing.T) { _, err := RunString(rt, `obj.exportedFn()`) @@ -332,7 +332,7 @@ func TestBind(t *testing.T) { }) t.Run("unexportedPtrFn", func(t *testing.T) { _, err := RunString(rt, `obj.unexportedPtrFn()`) - assert.EqualError(t, err, "TypeError: Object has no member 'unexportedPtrFn' at :1:33(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'unexportedPtrFn' at :1:20(3)") }) t.Run("ExportedPtrFn", func(t *testing.T) { _, err := RunString(rt, `obj.exportedPtrFn()`) @@ -340,7 +340,7 @@ func TestBind(t *testing.T) { case *bridgeTestMethodsType: assert.NoError(t, err) case bridgeTestMethodsType: - assert.EqualError(t, err, "TypeError: Object has no member 'exportedPtrFn' at :1:31(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'exportedPtrFn' at :1:18(3)") default: assert.Fail(t, "INVALID TYPE") } @@ -527,7 +527,7 @@ func TestBind(t *testing.T) { _, err := RunString(rt, `obj.contextInject()`) switch impl := obj.(type) { case bridgeTestContextInjectType: - assert.EqualError(t, err, "TypeError: Object has no member 'contextInject' at :1:31(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'contextInject' at :1:18(3)") case *bridgeTestContextInjectType: assert.EqualError(t, err, "GoError: contextInject() can only be called from within default()") assert.Equal(t, nil, impl.ctx) @@ -546,7 +546,7 @@ func TestBind(t *testing.T) { _, err := RunString(rt, `obj.contextInjectPtr()`) switch impl := obj.(type) { case bridgeTestContextInjectPtrType: - assert.EqualError(t, err, "TypeError: Object has no member 'contextInjectPtr' at :1:34(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'contextInjectPtr' at :1:21(3)") case *bridgeTestContextInjectPtrType: assert.NoError(t, err) assert.Equal(t, ctxPtr, impl.ctxPtr) @@ -566,7 +566,7 @@ func TestBind(t *testing.T) { } case bridgeTestCounterType: _, err := RunString(rt, `obj.count()`) - assert.EqualError(t, err, "TypeError: Object has no member 'count' at :1:23(3)") + assert.EqualError(t, err, "TypeError: Object has no member 'count' at :1:10(3)") default: assert.Fail(t, "UNKNOWN TYPE") } diff --git a/js/common/util.go b/js/common/util.go index 397633bed45..aa986c0f7e0 100644 --- a/js/common/util.go +++ b/js/common/util.go @@ -25,15 +25,20 @@ import ( "github.com/loadimpact/k6/js/compiler" ) -// Runs an ES6 string in the given runtime. Use this rather than writing ES5 in tests. +// RunString Runs an string in the given runtime. Use this if writing ES5 in tests isn't a problem. func RunString(rt *goja.Runtime, src string) (goja.Value, error) { + return rt.RunString(src) +} + +// RunES6String Runs an ES6 string in the given runtime. Use this rather than writing ES5 in tests. +func RunES6String(rt *goja.Runtime, src string) (goja.Value, error) { var err error c := compiler.New() src, _, err = c.Transform(src, "__string__") if err != nil { return goja.Undefined(), err } - return rt.RunString(src) + return RunString(rt, src) } // Throws a JS error; avoids re-wrapping GoErrors. diff --git a/js/common/util_test.go b/js/common/util_test.go index f99c242ef15..3d35d418f0c 100644 --- a/js/common/util_test.go +++ b/js/common/util_test.go @@ -30,11 +30,11 @@ import ( func TestRunString(t *testing.T) { t.Run("Valid", func(t *testing.T) { - _, err := RunString(goja.New(), `let a = 1;`) + _, err := RunES6String(goja.New(), `let a = 1;`) assert.NoError(t, err) }) t.Run("Invalid", func(t *testing.T) { - _, err := RunString(goja.New(), `let a = #;`) + _, err := RunES6String(goja.New(), `let a = #;`) assert.NotNil(t, err) assert.Contains(t, err.Error(), "SyntaxError: __string__: Unexpected character '#' (1:8)\n> 1 | let a = #;\n") }) diff --git a/js/modules/k6/crypto/crypto_test.go b/js/modules/k6/crypto/crypto_test.go index c6098aeef1f..384f8987dd2 100644 --- a/js/modules/k6/crypto/crypto_test.go +++ b/js/modules/k6/crypto/crypto_test.go @@ -51,7 +51,7 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("RandomBytesSuccess", func(t *testing.T) { _, err := common.RunString(rt, ` - let bytes = crypto.randomBytes(5); + var bytes = crypto.randomBytes(5); if (bytes.length !== 5) { throw new Error("Incorrect size: " + bytes.length); }`) @@ -78,8 +78,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("MD4", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "aa010fbc1d14c795d86ef98c95479d17"; - let hash = crypto.md4("hello world", "hex"); + var correct = "aa010fbc1d14c795d86ef98c95479d17"; + var hash = crypto.md4("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -88,8 +88,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("MD5", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "5eb63bbbe01eeed093cb22bb8f5acdc3"; - let hash = crypto.md5("hello world", "hex"); + var correct = "5eb63bbbe01eeed093cb22bb8f5acdc3"; + var hash = crypto.md5("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -99,8 +99,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA1", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "2aae6c35c94fcfb415dbe95f408b9ce91ee846ed"; - let hash = crypto.sha1("hello world", "hex"); + var correct = "2aae6c35c94fcfb415dbe95f408b9ce91ee846ed"; + var hash = crypto.sha1("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -110,8 +110,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA256", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "b94d27b9934d3e08a52e52d7da7dabfac484efe37a5380ee9088f7ace2efcde9"; - let hash = crypto.sha256("hello world", "hex"); + var correct = "b94d27b9934d3e08a52e52d7da7dabfac484efe37a5380ee9088f7ace2efcde9"; + var hash = crypto.sha256("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -121,8 +121,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA384", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "fdbd8e75a67f29f701a4e040385e2e23986303ea10239211af907fcbb83578b3e417cb71ce646efd0819dd8c088de1bd"; - let hash = crypto.sha384("hello world", "hex"); + var correct = "fdbd8e75a67f29f701a4e040385e2e23986303ea10239211af907fcbb83578b3e417cb71ce646efd0819dd8c088de1bd"; + var hash = crypto.sha384("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -132,8 +132,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA512", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "309ecc489c12d6eb4cc40f50c902f2b4d0ed77ee511a7c7a9bcd3ca86d4cd86f989dd35bc5ff499670da34255b45b0cfd830e81f605dcf7dc5542e93ae9cd76f"; - let hash = crypto.sha512("hello world", "hex"); + var correct = "309ecc489c12d6eb4cc40f50c902f2b4d0ed77ee511a7c7a9bcd3ca86d4cd86f989dd35bc5ff499670da34255b45b0cfd830e81f605dcf7dc5542e93ae9cd76f"; + var hash = crypto.sha512("hello world", "hex"); if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -143,8 +143,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA512_224", func(t *testing.T) { _, err := common.RunString(rt, ` - let hash = crypto.sha512_224("hello world", "hex"); - const correct = "22e0d52336f64a998085078b05a6e37b26f8120f43bf4db4c43a64ee"; + var hash = crypto.sha512_224("hello world", "hex"); + var correct = "22e0d52336f64a998085078b05a6e37b26f8120f43bf4db4c43a64ee"; if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -154,8 +154,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("SHA512_256", func(t *testing.T) { _, err := common.RunString(rt, ` - let hash = crypto.sha512_256("hello world", "hex"); - const correct = "0ac561fac838104e3f2e4ad107b4bee3e938bf15f2b15f009ccccd61a913f017"; + var hash = crypto.sha512_256("hello world", "hex"); + var correct = "0ac561fac838104e3f2e4ad107b4bee3e938bf15f2b15f009ccccd61a913f017"; if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -165,8 +165,8 @@ func TestCryptoAlgorithms(t *testing.T) { t.Run("RIPEMD160", func(t *testing.T) { _, err := common.RunString(rt, ` - let hash = crypto.ripemd160("hello world", "hex"); - const correct = "98c615784ccb5fe5936fbc0cbe9dfdb408d92f0f"; + var hash = crypto.ripemd160("hello world", "hex"); + var correct = "98c615784ccb5fe5936fbc0cbe9dfdb408d92f0f"; if (hash !== correct) { throw new Error("Hash mismatch: " + hash); }`) @@ -195,11 +195,11 @@ func TestStreamingApi(t *testing.T) { // Empty strings are still hashable t.Run("Empty", func(t *testing.T) { _, err := common.RunString(rt, ` - const correctHex = "d41d8cd98f00b204e9800998ecf8427e"; + var correctHex = "d41d8cd98f00b204e9800998ecf8427e"; - let hasher = crypto.createHash("md5"); + var hasher = crypto.createHash("md5"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -209,12 +209,12 @@ func TestStreamingApi(t *testing.T) { t.Run("UpdateOnce", func(t *testing.T) { _, err := common.RunString(rt, ` - const correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; + var correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; - let hasher = crypto.createHash("md5"); + var hasher = crypto.createHash("md5"); hasher.update("hello world"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -224,14 +224,14 @@ func TestStreamingApi(t *testing.T) { t.Run("UpdateMultiple", func(t *testing.T) { _, err := common.RunString(rt, ` - const correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; + var correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; - let hasher = crypto.createHash("md5"); + var hasher = crypto.createHash("md5"); hasher.update("hello"); hasher.update(" "); hasher.update("world"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -259,31 +259,31 @@ func TestOutputEncoding(t *testing.T) { t.Run("Valid", func(t *testing.T) { _, err := common.RunString(rt, ` - const correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; - const correctBase64 = "XrY7u+Ae7tCTyyK7j1rNww=="; - const correctBase64URL = "XrY7u-Ae7tCTyyK7j1rNww==" - const correctBase64RawURL = "XrY7u-Ae7tCTyyK7j1rNww"; - const correctBinary = [94,182,59,187,224,30,238,208,147,203,34,187,143,90,205,195]; + var correctHex = "5eb63bbbe01eeed093cb22bb8f5acdc3"; + var correctBase64 = "XrY7u+Ae7tCTyyK7j1rNww=="; + var correctBase64URL = "XrY7u-Ae7tCTyyK7j1rNww==" + var correctBase64RawURL = "XrY7u-Ae7tCTyyK7j1rNww"; + var correctBinary = [94,182,59,187,224,30,238,208,147,203,34,187,143,90,205,195]; - let hasher = crypto.createHash("md5"); + var hasher = crypto.createHash("md5"); hasher.update("hello world"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); } - const resultBase64 = hasher.digest("base64"); + var resultBase64 = hasher.digest("base64"); if (resultBase64 !== correctBase64) { throw new Error("Base64 encoding mismatch: " + resultBase64); } - const resultBase64URL = hasher.digest("base64url"); + var resultBase64URL = hasher.digest("base64url"); if (resultBase64URL !== correctBase64URL) { throw new Error("Base64 URL encoding mismatch: " + resultBase64URL); } - const resultBase64RawURL = hasher.digest("base64rawurl"); + var resultBase64RawURL = hasher.digest("base64rawurl"); if (resultBase64RawURL !== correctBase64RawURL) { throw new Error("Base64 raw URL encoding mismatch: " + resultBase64RawURL); } @@ -300,7 +300,7 @@ func TestOutputEncoding(t *testing.T) { return true; } - const resultBinary = hasher.digest("binary"); + var resultBinary = hasher.digest("binary"); if (!arraysEqual(resultBinary, correctBinary)) { throw new Error("Binary encoding mismatch: " + JSON.stringify(resultBinary)); } @@ -311,7 +311,7 @@ func TestOutputEncoding(t *testing.T) { t.Run("Invalid", func(t *testing.T) { _, err := common.RunString(rt, ` - let hasher = crypto.createHash("md5"); + var hasher = crypto.createHash("md5"); hasher.update("hello world"); hasher.digest("someInvalidEncoding"); `) @@ -352,10 +352,10 @@ func TestHMac(t *testing.T) { rt.Set("algorithm", rt.ToValue(algorithm)) t.Run(algorithm+" hasher: valid", func(t *testing.T) { _, err := common.RunString(rt, ` - let hasher = crypto.createHMAC(algorithm, "a secret"); + var hasher = crypto.createHMAC(algorithm, "a secret"); hasher.update("some data to hash"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -365,7 +365,7 @@ func TestHMac(t *testing.T) { t.Run(algorithm+" wrapper: valid", func(t *testing.T) { _, err := common.RunString(rt, ` - let resultHex = crypto.hmac(algorithm, "a secret", "some data to hash", "hex"); + var resultHex = crypto.hmac(algorithm, "a secret", "some data to hash", "hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -385,10 +385,10 @@ func TestHMac(t *testing.T) { rt.Set("algorithm", rt.ToValue(algorithm)) t.Run(algorithm+" hasher: invalid", func(t *testing.T) { _, err := common.RunString(rt, ` - let hasher = crypto.createHMAC(algorithm, "a secret"); + var hasher = crypto.createHMAC(algorithm, "a secret"); hasher.update("some data to hash"); - const resultHex = hasher.digest("hex"); + var resultHex = hasher.digest("hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -398,7 +398,7 @@ func TestHMac(t *testing.T) { t.Run(algorithm+" wrapper: invalid", func(t *testing.T) { _, err := common.RunString(rt, ` - let resultHex = crypto.hmac(algorithm, "a secret", "some data to hash", "hex"); + var resultHex = crypto.hmac(algorithm, "a secret", "some data to hash", "hex"); if (resultHex !== correctHex) { throw new Error("Hex encoding mismatch: " + resultHex); }`) @@ -417,40 +417,40 @@ func TestAWSv4(t *testing.T) { rt.Set("crypto", common.Bind(rt, New(), &ctx)) _, err := common.RunString(rt, ` - let HexEncode = crypto.hexEncode; - let HmacSHA256 = function(data, key) { + var HexEncode = crypto.hexEncode; + var HmacSHA256 = function(data, key) { return crypto.hmac("sha256",key, data, "binary"); }; - let expectedKDate = '969fbb94feb542b71ede6f87fe4d5fa29c789342b0f407474670f0c2489e0a0d' - let expectedKRegion = '69daa0209cd9c5ff5c8ced464a696fd4252e981430b10e3d3fd8e2f197d7a70c' - let expectedKService = 'f72cfd46f26bc4643f06a11eabb6c0ba18780c19a8da0c31ace671265e3c87fa' - let expectedKSigning = 'f4780e2d9f65fa895f9c67b32ce1baf0b0d8a43505a000a1a9e090d414db404d' + var expectedKDate = '969fbb94feb542b71ede6f87fe4d5fa29c789342b0f407474670f0c2489e0a0d' + var expectedKRegion = '69daa0209cd9c5ff5c8ced464a696fd4252e981430b10e3d3fd8e2f197d7a70c' + var expectedKService = 'f72cfd46f26bc4643f06a11eabb6c0ba18780c19a8da0c31ace671265e3c87fa' + var expectedKSigning = 'f4780e2d9f65fa895f9c67b32ce1baf0b0d8a43505a000a1a9e090d414db404d' - let key = 'wJalrXUtnFEMI/K7MDENG+bPxRfiCYEXAMPLEKEY'; - let dateStamp = '20120215'; - let regionName = 'us-east-1'; - let serviceName = 'iam'; + var key = 'wJalrXUtnFEMI/K7MDENG+bPxRfiCYEXAMPLEKEY'; + var dateStamp = '20120215'; + var regionName = 'us-east-1'; + var serviceName = 'iam'; - let kDate = HmacSHA256(dateStamp, "AWS4" + key); - let kRegion = HmacSHA256(regionName, kDate); - let kService = HmacSHA256(serviceName, kRegion); - let kSigning = HmacSHA256("aws4_request", kService); + var kDate = HmacSHA256(dateStamp, "AWS4" + key); + var kRegion = HmacSHA256(regionName, kDate); + var kService = HmacSHA256(serviceName, kRegion); + var kSigning = HmacSHA256("aws4_request", kService); - let hexKDate = HexEncode(kDate); + var hexKDate = HexEncode(kDate); if (expectedKDate != hexKDate) { throw new Error("Wrong kDate: expected '" + expectedKDate + "' got '" + hexKDate + "'"); } - let hexKRegion = HexEncode(kRegion); + var hexKRegion = HexEncode(kRegion); if (expectedKRegion != hexKRegion) { throw new Error("Wrong kRegion: expected '" + expectedKRegion + "' got '" + hexKRegion + "'"); } - let hexKService = HexEncode(kService); + var hexKService = HexEncode(kService); if (expectedKService != hexKService) { throw new Error("Wrong kService: expected '" + expectedKService + "' got '" + hexKService + "'"); } - let hexKSigning = HexEncode(kSigning); + var hexKSigning = HexEncode(kSigning); if (expectedKSigning != hexKSigning) { throw new Error("Wrong kSigning: expected '" + expectedKSigning + "' got '" + hexKSigning + "'"); } diff --git a/js/modules/k6/crypto/x509/x509_test.go b/js/modules/k6/crypto/x509/x509_test.go index d35912f4bf4..18d8712bb31 100644 --- a/js/modules/k6/crypto/x509/x509_test.go +++ b/js/modules/k6/crypto/x509/x509_test.go @@ -24,7 +24,6 @@ import ( "context" gox509 "crypto/x509" "fmt" - "strings" "testing" "github.com/dop251/goja" @@ -49,7 +48,7 @@ type Material struct { } var material = Material{ //nolint:gochecknoglobals - dsaCertificate: template(`-----BEGIN CERTIFICATE----- + dsaCertificate: `-----BEGIN CERTIFICATE----- MIIFnzCCBUSgAwIBAgIJAPOE4rArGHVcMAsGCWCGSAFlAwQDAjCBsTELMAkGA1UE BhMCWloxGTAXBgNVBAgMEEtvcHVuY2V6aXMgS3JhaXMxETAPBgNVBAcMCEFzaHRp bm9rMRwwGgYDVQQKDBNFeHVtYnJhbiBDb252ZW50aW9uMRkwFwYDVQQLDBBFeHVt @@ -81,8 +80,8 @@ JqGGJU+MCQZEoTAfBgNVHSMEGDAWgBSSb364iDHRI6/2JqGGJU+MCQZEoTAPBgNV HRMBAf8EBTADAQH/MAsGCWCGSAFlAwQDAgNIADBFAiEA1nr63IX9aaGUPeOUC0Bh w3Y7mpv5+sVgtoIi8ljxVSICIFCpEl70YjRVIUKL8N/lJwKxisrJ4+Xxg/DIeGP8 L8GA ------END CERTIFICATE-----`), - ecdsaCertificate: template(`-----BEGIN CERTIFICATE----- +-----END CERTIFICATE-----`, + ecdsaCertificate: `-----BEGIN CERTIFICATE----- MIIDXjCCAwWgAwIBAgICBNIwCgYIKoZIzj0EAwIwgdsxCzAJBgNVBAYTAlpaMRkw FwYDVQQIExBLb3B1bmNlemlzIEtyYWlzMREwDwYDVQQHEwhBc2h0aW5vazEaMBgG A1UECRMRMjIxQiBCYWtlciBTdHJlZXQxDjAMBgNVBBETBTk5OTk5MRwwGgYDVQQK @@ -102,8 +101,8 @@ Ly9wcmVzcy5leGNvdW5jaWwuenqGJ2h0dHA6Ly9sZWFybmluZy5leGNvdW5jaWwu enovaW5kZXguaHRtbDAKBggqhkjOPQQDAgNHADBEAiA/X4Y+Zaw4ziqL4grkY+rm srWfS/JGxLvN49r68cczSwIgWEXFIHMwE+OhKC6z01mIPe2G2CguYHukWyL+BHtT +20= ------END CERTIFICATE-----`), - rsaCertificate: template(`-----BEGIN CERTIFICATE----- +-----END CERTIFICATE-----`, + rsaCertificate: `-----BEGIN CERTIFICATE----- MIIE6zCCA9OgAwIBAgICBNIwDQYJKoZIhvcNAQELBQAwgdsxCzAJBgNVBAYTAlpa MRkwFwYDVQQIExBLb3B1bmNlemlzIEtyYWlzMREwDwYDVQQHEwhBc2h0aW5vazEa MBgGA1UECRMRMjIxQiBCYWtlciBTdHJlZXQxDjAMBgNVBBETBTk5OTk5MRwwGgYD @@ -131,17 +130,13 @@ gzg3dNaCY65aH0cJE/dVwiS/F2XTr1zvr+uBPExgrA21+FSIlHM0Dot+VGKdCLEO 6HugOCDBdzKF2hsHeI5LvgXUX5zQ0gnsd93+QuxUmiN7QZZs8tDMD/+efo4OWvp/ xytSVXVn+cECQLg9hVn+Zx3XO2FA0eOzaWEONnUGghT/Ivw06lUxis5tkAoAU93d ddBqJe0XUeAX8Zr6EJ82 ------END CERTIFICATE-----`), - publicKey: template(`-----BEGIN PUBLIC KEY----- +-----END CERTIFICATE-----`, + publicKey: `-----BEGIN PUBLIC KEY----- MIGfMA0GCSqGSIb3DQEBAQUAA4GNADCBiQKBgQDXMLr/Y/vUtIFY75jj0YXfp6lQ 7iEIbps3BvRE4isTpxs8fXLnLM8LAuJScxiKyrGnj8EMb7LIHkSMBlz6iVj9atY6 EUEm/VHUnElNquzGyBA50TCfpv6NHPaTvOoB45yQbZ/YB4LO+CsT9eIMDZ4tcU9Z +xD10ifJhhIwpZUFIQIDAQAB ------END PUBLIC KEY-----`), -} - -func template(value string) string { - return fmt.Sprintf("`%s`", value) +-----END PUBLIC KEY-----`, } func TestParse(t *testing.T) { @@ -159,21 +154,21 @@ func TestParse(t *testing.T) { t.Run("ParseFailure", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; + var pem = %q; x509.parse(pem);`, material.publicKey)) if assert.Error(t, err) { - assert.True(t, strings.HasPrefix( + assert.Contains(t, err.Error(), "GoError: failed to parse certificate", - )) + ) } }) t.Run("SignatureAlgorithm", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.signatureAlgorithm; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.signatureAlgorithm; if (value !== "SHA256-RSA") { throw new Error("Bad signature algorithm: " + value); }`, material.rsaCertificate)) @@ -182,8 +177,8 @@ func TestParse(t *testing.T) { t.Run("Subject", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); + var pem = %q; + var cert = x509.parse(pem); if (typeof cert.subject !== "object") { throw new Error("Bad subject: " + typeof cert.subject); }`, material.rsaCertificate)) @@ -192,9 +187,9 @@ func TestParse(t *testing.T) { t.Run("SubjectCommonName", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.commonName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.commonName : null; if (value !== "excouncil.zz") { throw new Error("Bad subject common name: " + value); }`, material.rsaCertificate)) @@ -203,9 +198,9 @@ func TestParse(t *testing.T) { t.Run("SubjectCountry", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.country : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.country : null; if (value !== "ZZ") { throw new Error("Bad subject country: " + value); }`, material.rsaCertificate)) @@ -214,9 +209,9 @@ func TestParse(t *testing.T) { t.Run("SubjectPostalCode", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.postalCode : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.postalCode : null; if (value !== "99999") { throw new Error("Bad subject postal code: " + value); }`, material.rsaCertificate)) @@ -225,9 +220,9 @@ func TestParse(t *testing.T) { t.Run("SubjectProvince", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.stateOrProvinceName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.stateOrProvinceName : null; if (value !== "Kopuncezis Krais") { throw new Error("Bad subject province: " + value); }`, material.rsaCertificate)) @@ -236,9 +231,9 @@ func TestParse(t *testing.T) { t.Run("SubjectLocality", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.localityName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.localityName : null; if (value !== "Ashtinok") { throw new Error("Bad subject locality: " + value); }`, material.rsaCertificate)) @@ -247,9 +242,9 @@ func TestParse(t *testing.T) { t.Run("SubjectStreetAddress", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.streetAddress : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.streetAddress : null; if (value !== "221B Baker Street") { throw new Error("Bad subject street address: " + value); }`, material.rsaCertificate)) @@ -258,9 +253,9 @@ func TestParse(t *testing.T) { t.Run("SubjectOrganization", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.subject ? cert.subject.organizationName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.subject ? cert.subject.organizationName : null; if (value !== "Exumbran Convention") { throw new Error("Bad subject organization: " + value); }`, material.rsaCertificate)) @@ -269,9 +264,9 @@ func TestParse(t *testing.T) { t.Run("SubjectOrganizationalUnit", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const values = + var pem = %q; + var cert = x509.parse(pem); + var values = cert.subject ? cert.subject.organizationalUnitName : null; if (!( values.length === 2 && @@ -287,11 +282,11 @@ func TestParse(t *testing.T) { t.Run("SubjectNames", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const values = cert.subject ? cert.subject.names : null; - const strings = values - ? values.map(entry => entry.type + ": " + entry.value) + var pem = %q; + var cert = x509.parse(pem); + var values = cert.subject ? cert.subject.names : null; + var strings = values + ? values.map(function(entry) { return entry.type + ": " + entry.value}) : null; Array.prototype.includes = function (value) { return this.indexOf(value) !== -1 } @@ -316,8 +311,8 @@ func TestParse(t *testing.T) { t.Run("Issuer", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); + var pem = %q; + var cert = x509.parse(pem); if (typeof cert.issuer !== "object") { throw new Error("Bad issuer: " + typeof cert.issuer); }`, material.rsaCertificate)) @@ -326,9 +321,9 @@ func TestParse(t *testing.T) { t.Run("IssuerCommonName", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.issuer ? cert.issuer.commonName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.issuer ? cert.issuer.commonName : null; if (value !== "excouncil.zz") { throw new Error("Bad issuer common name: " + value); }`, material.rsaCertificate)) @@ -337,9 +332,9 @@ func TestParse(t *testing.T) { t.Run("IssuerCountry", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.issuer ? cert.issuer.country : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.issuer ? cert.issuer.country : null; if (value !== "ZZ") { throw new Error("Bad issuer country: " + value); }`, material.rsaCertificate)) @@ -348,9 +343,9 @@ func TestParse(t *testing.T) { t.Run("IssuerProvince", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.issuer ? cert.issuer.stateOrProvinceName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.issuer ? cert.issuer.stateOrProvinceName : null; if (value !== "Kopuncezis Krais") { throw new Error("Bad issuer province: " + value); }`, material.rsaCertificate)) @@ -359,9 +354,9 @@ func TestParse(t *testing.T) { t.Run("IssuerLocality", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.issuer ? cert.issuer.localityName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.issuer ? cert.issuer.localityName : null; if (value !== "Ashtinok") { throw new Error("Bad issuer locality: " + value); }`, material.rsaCertificate)) @@ -370,9 +365,9 @@ func TestParse(t *testing.T) { t.Run("IssuerOrganization", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.issuer ? cert.issuer.organizationName : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.issuer ? cert.issuer.organizationName : null; if (value !== "Exumbran Convention") { throw new Error("Bad issuer organization: " + value); }`, material.rsaCertificate)) @@ -381,11 +376,11 @@ func TestParse(t *testing.T) { t.Run("IssuerNames", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const values = cert.issuer ? cert.issuer.names : null; - const strings = values - ? values.map(entry => entry.type + ": " + entry.value) + var pem = %q; + var cert = x509.parse(pem); + var values = cert.issuer ? cert.issuer.names : null; + var strings = values + ? values.map(function(entry) { return entry.type + ": " + entry.value}) : null; Array.prototype.includes = function (value) { return this.indexOf(value) !== -1 } @@ -410,9 +405,9 @@ func TestParse(t *testing.T) { t.Run("NotBefore", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.notBefore; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.notBefore; if (value !== "2019-01-01T00:00:00Z") { throw new Error("Bad lower bound: " + value) }`, material.rsaCertificate)) @@ -421,9 +416,9 @@ func TestParse(t *testing.T) { t.Run("NotAfter", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.notAfter; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.notAfter; if (value !== "2020-01-01T00:00:00Z") { throw new Error("Bad upper bound: " + value); }`, material.rsaCertificate)) @@ -432,9 +427,9 @@ func TestParse(t *testing.T) { t.Run("AltNames", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const values = cert.altNames; + var pem = %q; + var cert = x509.parse(pem); + var values = cert.altNames; if (!( values.length === 8 && values[0] === "council.exumbran.zz" && @@ -453,10 +448,10 @@ func TestParse(t *testing.T) { t.Run("FingerPrint", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.fingerPrint; - const expected = [ + var pem = %q; + var cert = x509.parse(pem); + var value = cert.fingerPrint; + var expected = [ 85, 119, 3, 199, 150, 144, 202, 145, 178, 46, 205, 132, 37, 235, 251, 208, 139, 161, 143, 14 ] @@ -468,8 +463,8 @@ func TestParse(t *testing.T) { t.Run("PublicKey", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); + var pem = %q; + var cert = x509.parse(pem); if (typeof cert.publicKey !== "object") { throw new Error("Bad public key: " + typeof cert.publicKey); }`, material.rsaCertificate)) @@ -478,9 +473,9 @@ func TestParse(t *testing.T) { t.Run("RSAPublicKey", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.publicKey; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.publicKey; if (!( value && typeof value === "object" && @@ -496,9 +491,9 @@ func TestParse(t *testing.T) { t.Run("RSAPublicKeyExponent", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.publicKey ? cert.publicKey.key.e : null; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.publicKey ? cert.publicKey.key.e : null; if (value !== 65537) { throw new Error("Bad RSA public key exponent: " + value); }`, material.rsaCertificate)) @@ -507,10 +502,10 @@ func TestParse(t *testing.T) { t.Run("RSAPublicKeyModulus", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.publicKey ? cert.publicKey.key.n.bytes() : null; - const expected = [ + var pem = %q; + var cert = x509.parse(pem); + var value = cert.publicKey ? cert.publicKey.key.n.bytes() : null; + var expected = [ 223, 249, 234, 71, 180, 36, 28, 62, 84, 141, 177, 118, 53, 2, 175, 45, 167, 89, 155, 216, 103, 86, 32, 216, 42, 92, 84, 125, 183, 102, 217, 40, 255, 129, 38, 203, 175, 98, 209, 147, 151, 106, 250, 12, @@ -538,9 +533,9 @@ func TestParse(t *testing.T) { t.Run("DSAPublicKey", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.publicKey; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.publicKey; if (!( value && typeof value === "object" && @@ -555,9 +550,9 @@ func TestParse(t *testing.T) { t.Run("ECDSAPublicKey", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const cert = x509.parse(pem); - const value = cert.publicKey; + var pem = %q; + var cert = x509.parse(pem); + var value = cert.publicKey; if (!( value && typeof value === "object" && @@ -586,8 +581,8 @@ func TestGetAltNames(t *testing.T) { t.Run("Success", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const altNames = x509.getAltNames(pem); + var pem = %q; + var altNames = x509.getAltNames(pem); if (!( Array.isArray(altNames) && altNames.length === 8 && @@ -620,8 +615,8 @@ func TestGetIssuer(t *testing.T) { t.Run("Success", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const issuer = x509.getIssuer(pem); + var pem = %q; + var issuer = x509.getIssuer(pem); if (!( typeof issuer === "object" && issuer.commonName === "excouncil.zz" && @@ -652,8 +647,8 @@ func TestGetSubject(t *testing.T) { t.Run("Success", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - const pem = %s; - const subject = x509.getSubject(pem); + var pem = %q; + var subject = x509.getSubject(pem); if (!( typeof subject === "object" && subject.commonName === "excouncil.zz" && diff --git a/js/modules/k6/encoding/encoding_test.go b/js/modules/k6/encoding/encoding_test.go index ac08edcdbf4..fd8b7de18d4 100644 --- a/js/modules/k6/encoding/encoding_test.go +++ b/js/modules/k6/encoding/encoding_test.go @@ -43,8 +43,8 @@ func TestEncodingAlgorithms(t *testing.T) { t.Run("Base64", func(t *testing.T) { t.Run("DefaultEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "aGVsbG8gd29ybGQ="; - let encoded = encoding.b64encode("hello world"); + var correct = "aGVsbG8gd29ybGQ="; + var encoded = encoding.b64encode("hello world"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -52,8 +52,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("DefaultDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "hello world"; - let decoded = encoding.b64decode("aGVsbG8gd29ybGQ="); + var correct = "hello world"; + var decoded = encoding.b64decode("aGVsbG8gd29ybGQ="); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) @@ -61,8 +61,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("DefaultUnicodeEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "44GT44KT44Gr44Gh44Gv5LiW55WM"; - let encoded = encoding.b64encode("こんにちは世界", "std"); + var correct = "44GT44KT44Gr44Gh44Gv5LiW55WM"; + var encoded = encoding.b64encode("こんにちは世界", "std"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -70,8 +70,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("DefaultUnicodeDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "こんにちは世界"; - let decoded = encoding.b64decode("44GT44KT44Gr44Gh44Gv5LiW55WM"); + var correct = "こんにちは世界"; + var decoded = encoding.b64decode("44GT44KT44Gr44Gh44Gv5LiW55WM"); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) @@ -79,8 +79,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("StdEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "aGVsbG8gd29ybGQ="; - let encoded = encoding.b64encode("hello world", "std"); + var correct = "aGVsbG8gd29ybGQ="; + var encoded = encoding.b64encode("hello world", "std"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -88,8 +88,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("StdDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "hello world"; - let decoded = encoding.b64decode("aGVsbG8gd29ybGQ=", "std"); + var correct = "hello world"; + var decoded = encoding.b64decode("aGVsbG8gd29ybGQ=", "std"); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) @@ -97,8 +97,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("RawStdEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "aGVsbG8gd29ybGQ"; - let encoded = encoding.b64encode("hello world", "rawstd"); + var correct = "aGVsbG8gd29ybGQ"; + var encoded = encoding.b64encode("hello world", "rawstd"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -106,8 +106,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("RawStdDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "hello world"; - let decoded = encoding.b64decode("aGVsbG8gd29ybGQ", "rawstd"); + var correct = "hello world"; + var decoded = encoding.b64decode("aGVsbG8gd29ybGQ", "rawstd"); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) @@ -115,8 +115,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("URLEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "5bCP6aO85by-Li4="; - let encoded = encoding.b64encode("小飼弾..", "url"); + var correct = "5bCP6aO85by-Li4="; + var encoded = encoding.b64encode("小飼弾..", "url"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -124,8 +124,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("URLDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "小飼弾.."; - let decoded = encoding.b64decode("5bCP6aO85by-Li4=", "url"); + var correct = "小飼弾.."; + var decoded = encoding.b64decode("5bCP6aO85by-Li4=", "url"); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) @@ -133,8 +133,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("RawURLEnc", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "5bCP6aO85by-Li4"; - let encoded = encoding.b64encode("小飼弾..", "rawurl"); + var correct = "5bCP6aO85by-Li4"; + var encoded = encoding.b64encode("小飼弾..", "rawurl"); if (encoded !== correct) { throw new Error("Encoding mismatch: " + encoded); }`) @@ -142,8 +142,8 @@ func TestEncodingAlgorithms(t *testing.T) { }) t.Run("RawURLDec", func(t *testing.T) { _, err := common.RunString(rt, ` - const correct = "小飼弾.."; - let decoded = encoding.b64decode("5bCP6aO85by-Li4", "rawurl"); + var correct = "小飼弾.."; + var decoded = encoding.b64decode("5bCP6aO85by-Li4", "rawurl"); if (decoded !== correct) { throw new Error("Decoding mismatch: " + decoded); }`) diff --git a/js/modules/k6/html/element_test.go b/js/modules/k6/html/element_test.go index 48120e3c581..415fc133666 100644 --- a/js/modules/k6/html/element_test.go +++ b/js/modules/k6/html/element_test.go @@ -62,7 +62,7 @@ func TestElement(t *testing.T) { rt.Set("html", common.Bind(rt, &HTML{}, &ctx)) // compileProtoElem() - _, err := common.RunString(rt, `let doc = html.parseHTML(src)`) + _, err := common.RunString(rt, `var doc = html.parseHTML(src)`) assert.NoError(t, err) assert.IsType(t, Selection{}, rt.Get("doc").Export()) diff --git a/js/modules/k6/html/elements_gen_test.go b/js/modules/k6/html/elements_gen_test.go index 0d8525c7557..a2a0e5534bc 100644 --- a/js/modules/k6/html/elements_gen_test.go +++ b/js/modules/k6/html/elements_gen_test.go @@ -318,7 +318,7 @@ var urlTests = []struct { } const testGenElems = ` - + @@ -410,7 +410,7 @@ func TestGenElements(t *testing.T) { rt.Set("src", testGenElems) rt.Set("html", common.Bind(rt, &HTML{}, &ctx)) - _, err := common.RunString(rt, "let doc = html.parseHTML(src)") + _, err := common.RunString(rt, "var doc = html.parseHTML(src)") assert.NoError(t, err) assert.IsType(t, Selection{}, rt.Get("doc").Export()) diff --git a/js/modules/k6/html/elements_test.go b/js/modules/k6/html/elements_test.go index 9c212b577b8..72302831fae 100644 --- a/js/modules/k6/html/elements_test.go +++ b/js/modules/k6/html/elements_test.go @@ -43,7 +43,7 @@ const testHTMLElems = ` 6 - +

@@ -92,7 +92,7 @@ func TestElements(t *testing.T) { rt.Set("src", testHTMLElems) rt.Set("html", common.Bind(rt, &HTML{}, &ctx)) - _, err := common.RunString(rt, `let doc = html.parseHTML(src)`) + _, err := common.RunString(rt, `var doc = html.parseHTML(src)`) assert.NoError(t, err) assert.IsType(t, Selection{}, rt.Get("doc").Export()) diff --git a/js/modules/k6/html/html_test.go b/js/modules/k6/html/html_test.go index 3be99a7b13f..6178230c753 100644 --- a/js/modules/k6/html/html_test.go +++ b/js/modules/k6/html/html_test.go @@ -72,7 +72,7 @@ func TestParseHTML(t *testing.T) { // TODO: I literally cannot think of a snippet that makes goquery error. // I'm not sure if it's even possible without like, an invalid reader or something, which would // be impossible to cause from the JS side. - _, err := common.RunString(rt, `let doc = html.parseHTML(src)`) + _, err := common.RunString(rt, `var doc = html.parseHTML(src)`) assert.NoError(t, err) assert.IsType(t, Selection{}, rt.Get("doc").Export()) diff --git a/js/modules/k6/html/serialize_test.go b/js/modules/k6/html/serialize_test.go index e8f1c77f7b1..db5deb05051 100644 --- a/js/modules/k6/html/serialize_test.go +++ b/js/modules/k6/html/serialize_test.go @@ -74,7 +74,7 @@ func TestSerialize(t *testing.T) { rt.Set("src", testSerializeHTML) rt.Set("html", common.Bind(rt, New(), &ctx)) - _, err := common.RunString(rt, `let doc = html.parseHTML(src)`) + _, err := common.RunString(rt, `var doc = html.parseHTML(src)`) assert.NoError(t, err) assert.IsType(t, Selection{}, rt.Get("doc").Export()) diff --git a/js/modules/k6/http/http_test.go b/js/modules/k6/http/http_test.go index f81c9ff5714..19f895a1fcf 100644 --- a/js/modules/k6/http/http_test.go +++ b/js/modules/k6/http/http_test.go @@ -47,7 +47,7 @@ func TestTagURL(t *testing.T) { t.Run("expr="+expr, func(t *testing.T) { tag, err := httpext.NewURL(data.u, data.n) require.NoError(t, err) - v, err := common.RunString(rt, "http.url`"+expr+"`") + v, err := common.RunES6String(rt, "http.url`"+expr+"`") if assert.NoError(t, err) { assert.Equal(t, tag, v.Export()) } diff --git a/js/modules/k6/http/request_test.go b/js/modules/k6/http/request_test.go index aabd415a5be..de912fbadac 100644 --- a/js/modules/k6/http/request_test.go +++ b/js/modules/k6/http/request_test.go @@ -165,7 +165,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Redirects", func(t *testing.T) { t.Run("tracing", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/redirect/9"); + var res = http.get("HTTPBIN_URL/redirect/9"); `)) assert.NoError(t, err) bufSamples := stats.GetBufferedSamples(samples) @@ -189,7 +189,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("11", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/redirect/11"); + var res = http.get("HTTPBIN_URL/redirect/11"); if (res.status != 302) { throw new Error("wrong status: " + res.status) } if (res.url != "HTTPBIN_URL/relative-redirect/1") { throw new Error("incorrect URL: " + res.url) } if (res.headers["Location"] != "/get") { throw new Error("incorrect Location header: " + res.headers["Location"]) } @@ -205,7 +205,7 @@ func TestRequestAndBatch(t *testing.T) { state.Options.MaxRedirects = null.NewInt(10, false) _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/redirect/11"); + var res = http.get("HTTPBIN_URL/redirect/11"); if (res.status != 302) { throw new Error("wrong status: " + res.status) } if (res.url != "HTTPBIN_URL/relative-redirect/1") { throw new Error("incorrect URL: " + res.url) } if (res.headers["Location"] != "/get") { throw new Error("incorrect Location header: " + res.headers["Location"]) } @@ -222,7 +222,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("requestScopeRedirects", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/redirect/1", {redirects: 3}); + var res = http.get("HTTPBIN_URL/redirect/1", {redirects: 3}); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.url != "HTTPBIN_URL/get") { throw new Error("incorrect URL: " + res.url) } `)) @@ -230,7 +230,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("requestScopeNoRedirects", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/redirect/1", {redirects: 0}); + var res = http.get("HTTPBIN_URL/redirect/1", {redirects: 0}); if (res.status != 302) { throw new Error("wrong status: " + res.status) } if (res.url != "HTTPBIN_URL/redirect/1") { throw new Error("incorrect URL: " + res.url) } if (res.headers["Location"] != "/get") { throw new Error("incorrect Location header: " + res.headers["Location"]) } @@ -246,7 +246,7 @@ func TestRequestAndBatch(t *testing.T) { http.Redirect(w, r, sr("HTTPBIN_URL/post"), http.StatusPermanentRedirect) })) _, err := common.RunString(rt, sr(` - let res = http.post("HTTPBIN_URL/post-redirect", "pesho", {redirects: 1}); + var res = http.post("HTTPBIN_URL/post-redirect", "pesho", {redirects: 1}); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.url != "HTTPBIN_URL/post") { throw new Error("incorrect URL: " + res.url) } @@ -290,7 +290,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("UserAgent", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/user-agent"); + var res = http.get("HTTPBIN_URL/user-agent"); if (res.json()['user-agent'] != "TestUserAgent") { throw new Error("incorrect user agent: " + res.json()['user-agent']) } @@ -299,7 +299,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Override", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/user-agent", { + var res = http.get("HTTPBIN_URL/user-agent", { headers: { "User-Agent": "OtherUserAgent" }, }); if (res.json()['user-agent'] != "OtherUserAgent") { @@ -312,7 +312,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Compression", func(t *testing.T) { t.Run("gzip", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPSBIN_IP_URL/gzip"); + var res = http.get("HTTPSBIN_IP_URL/gzip"); if (res.json()['gzipped'] != true) { throw new Error("unexpected body data: " + res.json()['gzipped']) } @@ -321,7 +321,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("deflate", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/deflate"); + var res = http.get("HTTPBIN_URL/deflate"); if (res.json()['deflated'] != true) { throw new Error("unexpected body data: " + res.json()['deflated']) } @@ -330,7 +330,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("zstd", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPSBIN_IP_URL/zstd"); + var res = http.get("HTTPSBIN_IP_URL/zstd"); if (res.json()['compression'] != 'zstd') { throw new Error("unexpected body data: " + res.json()['compression']) } @@ -339,7 +339,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("brotli", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPSBIN_IP_URL/brotli"); + var res = http.get("HTTPSBIN_IP_URL/brotli"); if (res.json()['compression'] != 'br') { throw new Error("unexpected body data: " + res.json()['compression']) } @@ -348,7 +348,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("zstd-br", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPSBIN_IP_URL/zstd-br"); + var res = http.get("HTTPSBIN_IP_URL/zstd-br"); if (res.json()['compression'] != 'zstd, br') { throw new Error("unexpected compression: " + res.json()['compression']) } @@ -364,7 +364,7 @@ func TestRequestAndBatch(t *testing.T) { })) _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/customcompression"); + var res = http.get("HTTPBIN_URL/customcompression"); if (res.json()["custom"] != true) { throw new Error("unexpected body data: " + res.body) } @@ -375,8 +375,8 @@ func TestRequestAndBatch(t *testing.T) { t.Run("CompressionWithAcceptEncodingHeader", func(t *testing.T) { t.Run("gzip", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let params = { headers: { "Accept-Encoding": "gzip" } }; - let res = http.get("HTTPBIN_URL/gzip", params); + var params = { headers: { "Accept-Encoding": "gzip" } }; + var res = http.get("HTTPBIN_URL/gzip", params); if (res.json()['gzipped'] != true) { throw new Error("unexpected body data: " + res.json()['gzipped']) } @@ -385,8 +385,8 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("deflate", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let params = { headers: { "Accept-Encoding": "deflate" } }; - let res = http.get("HTTPBIN_URL/deflate", params); + var params = { headers: { "Accept-Encoding": "deflate" } }; + var res = http.get("HTTPBIN_URL/deflate", params); if (res.json()['deflated'] != true) { throw new Error("unexpected body data: " + res.json()['deflated']) } @@ -411,7 +411,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("HTTP/2", func(t *testing.T) { stats.GetBufferedSamples(samples) // Clean up buffered samples from previous tests _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTP2BIN_URL/get"); + var res = http.request("GET", "HTTP2BIN_URL/get"); if (res.status != 200) { throw new Error("wrong status: " + res.status) } if (res.proto != "HTTP/2.0") { throw new Error("wrong proto: " + res.proto) } `)) @@ -443,7 +443,7 @@ func TestRequestAndBatch(t *testing.T) { for _, versionTest := range tlsVersionTests { t.Run(versionTest.Name, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.get("%s"); + var res = http.get("%s"); if (res.tls_version != %s) { throw new Error("wrong TLS version: " + res.tls_version); } `, versionTest.URL, versionTest.Version)) assert.NoError(t, err) @@ -459,7 +459,7 @@ func TestRequestAndBatch(t *testing.T) { for _, cipherSuiteTest := range tlsCipherSuiteTests { t.Run(cipherSuiteTest.Name, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.get("%s"); + var res = http.get("%s"); 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) @@ -468,7 +468,7 @@ func TestRequestAndBatch(t *testing.T) { } t.Run("ocsp_stapled_good", func(t *testing.T) { _, err := common.RunString(rt, ` - let res = http.request("GET", "https://www.microsoft.com/"); + var res = http.request("GET", "https://www.microsoft.com/"); if (res.ocsp.status != http.OCSP_STATUS_GOOD) { throw new Error("wrong ocsp stapled response status: " + res.ocsp.status); } `) assert.NoError(t, err) @@ -495,7 +495,7 @@ func TestRequestAndBatch(t *testing.T) { defer hook.Reset() _, err := common.RunString(rt, ` - let res = http.request("", "", { throw: false }); + var res = http.request("", "", { throw: false }); throw new Error(res.error); `) require.Error(t, err) @@ -518,7 +518,7 @@ func TestRequestAndBatch(t *testing.T) { for _, literal := range []string{`undefined`, `null`} { t.Run(literal, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, %s); + var 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) @@ -532,11 +532,11 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/cookies/set?key=value", null, { redirects: 0 }); + var 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); } - const props = ["name", "value", "domain", "path", "expires", "max_age", "secure", "http_only"]; - let cookie = res.cookies.key[0]; - for (let i = 0; i < props.length; i++) { + var props = ["name", "value", "domain", "path", "expires", "max_age", "secure", "http_only"]; + var cookie = res.cookies.key[0]; + for (var i = 0; i < props.length; i++) { if (cookie[props[i]] === undefined) { throw new Error("cookie property not found: " + props[i]); } @@ -554,12 +554,12 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); - let res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key2: "value2" } }); + var res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key2: "value2" } }); if (res.json().key != "value") { throw new Error("wrong cookie value: " + res.json().key); } if (res.json().key2 != "value2") { throw new Error("wrong cookie value: " + res.json().key2); } - let jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); + var jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); if (jarCookies.key[0] != "value") { throw new Error("wrong cookie value in jar"); } if (jarCookies.key2 != undefined) { throw new Error("unexpected cookie in jar"); } `)) @@ -572,10 +572,10 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key: "value" } }); + var 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); } - let jar = http.cookieJar(); - let jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); + var jar = http.cookieJar(); + var jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); if (jarCookies.key != undefined) { throw new Error("unexpected cookie in jar"); } `)) assert.NoError(t, err) @@ -587,11 +587,11 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); - let res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key: { value: "replaced", replace: true } } }); + var res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key: { value: "replaced", replace: true } } }); if (res.json().key != "replaced") { throw new Error("wrong cookie value: " + res.json().key); } - let jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); + var jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); if (jarCookies.key[0] != "value") { throw new Error("wrong cookie value in jar"); } `)) assert.NoError(t, err) @@ -616,7 +616,7 @@ func TestRequestAndBatch(t *testing.T) { require.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/redirect-to?url=HTTPSBIN_URL/set-cookie-without-redirect"); + var res = http.request("GET", "HTTPBIN_URL/redirect-to?url=HTTPSBIN_URL/set-cookie-without-redirect"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) require.NoError(t, err) @@ -642,7 +642,7 @@ func TestRequestAndBatch(t *testing.T) { require.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPSBIN_URL/cookies/set?key=value"); + var res = http.request("GET", "HTTPSBIN_URL/cookies/set?key=value"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) require.NoError(t, err) @@ -683,7 +683,7 @@ func TestRequestAndBatch(t *testing.T) { })) _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_IP_URL/redirect-to?url=HTTPSBIN_URL/set-cookie-and-redirect"); + var res = http.request("GET", "HTTPBIN_IP_URL/redirect-to?url=HTTPSBIN_URL/set-cookie-and-redirect"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `)) require.NoError(t, err) @@ -718,9 +718,9 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { domain: "HTTPBIN_DOMAIN" }); - let res = http.request("GET", "HTTPBIN_URL/cookies"); + var res = http.request("GET", "HTTPBIN_URL/cookies"); if (res.json().key != "value") { throw new Error("wrong cookie value 1: " + res.json().key); } @@ -742,9 +742,9 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { path: "/cookies" }); - let res = http.request("GET", "HTTPBIN_URL/cookies"); + var res = http.request("GET", "HTTPBIN_URL/cookies"); if (res.json().key != "value") { throw new Error("wrong cookie value: " + res.json().key); } @@ -766,9 +766,9 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value", { expires: "Sun, 24 Jul 1983 17:01:02 GMT" }); - let res = http.request("GET", "HTTPBIN_URL/cookies"); + var res = http.request("GET", "HTTPBIN_URL/cookies"); if (res.json().key != undefined) { throw new Error("cookie 'key' unexpectedly found"); } @@ -787,9 +787,9 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = http.cookieJar(); + var jar = http.cookieJar(); jar.set("HTTPSBIN_IP_URL/cookies", "key", "value", { secure: true }); - let res = http.request("GET", "HTTPSBIN_IP_URL/cookies"); + var res = http.request("GET", "HTTPSBIN_IP_URL/cookies"); if (res.json().key != "value") { throw new Error("wrong cookie value: " + res.json().key); } @@ -803,12 +803,12 @@ func TestRequestAndBatch(t *testing.T) { assert.NoError(t, err) state.CookieJar = cookieJar _, err = common.RunString(rt, sr(` - let jar = new http.CookieJar(); + var jar = new http.CookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); - let res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key2: "value2" }, jar: jar }); + var res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key2: "value2" }, jar: jar }); if (res.json().key != "value") { throw new Error("wrong cookie value: " + res.json().key); } if (res.json().key2 != "value2") { throw new Error("wrong cookie value: " + res.json().key2); } - let jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); + var jarCookies = jar.cookiesForURL("HTTPBIN_URL/cookies"); if (jarCookies.key[0] != "value") { throw new Error("wrong cookie value in jar: " + jarCookies.key[0]); } if (jarCookies.key2 != undefined) { throw new Error("unexpected cookie in jar"); } `)) @@ -823,7 +823,7 @@ func TestRequestAndBatch(t *testing.T) { urlExpected := sr("http://****:****@HTTPBIN_IP:HTTPBIN_PORT/basic-auth/bob/pass") _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.request("GET", "%s", null, {}); + var res = http.request("GET", "%s", null, {}); if (res.status != 200) { throw new Error("wrong status: " + res.status); } `, url)) assert.NoError(t, err) @@ -835,7 +835,7 @@ func TestRequestAndBatch(t *testing.T) { urlExpected := sr("http://****:****@HTTPBIN_IP:HTTPBIN_PORT/digest-auth/auth/bob/pass") _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.request("GET", "%s", null, { auth: "digest" }); + var res = http.request("GET", "%s", null, { auth: "digest" }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.error_code != 0) { throw new Error("wrong error code: " + res.error_code); } `, url)) @@ -851,7 +851,7 @@ func TestRequestAndBatch(t *testing.T) { url := sr("http://bob:pass@HTTPBIN_IP:HTTPBIN_PORT/digest-auth/failure") _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.request("GET", "%s", null, { auth: "digest", timeout: 1, throw: false }); + var res = http.request("GET", "%s", null, { auth: "digest", timeout: 1, throw: false }); `, url)) assert.NoError(t, err) }) @@ -862,7 +862,7 @@ func TestRequestAndBatch(t *testing.T) { for _, literal := range []string{`null`, `undefined`} { 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 }); + var 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) @@ -872,7 +872,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("object", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, { + var res = http.request("GET", "HTTPBIN_URL/headers", null, { headers: { "X-My-Header": "value" }, }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -884,7 +884,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Host", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, { + var res = http.request("GET", "HTTPBIN_URL/headers", null, { headers: { "Host": "HTTPBIN_DOMAIN" }, }); if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -899,7 +899,7 @@ func TestRequestAndBatch(t *testing.T) { for _, literal := range []string{`null`, `undefined`} { t.Run(literal, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: %s }); + var 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) @@ -909,7 +909,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("object", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: { tag: "value" } }); + var 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) @@ -930,7 +930,7 @@ func TestRequestAndBatch(t *testing.T) { state.Tags = map[string]string{"runtag1": "val1", "runtag2": "val2"} _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/headers", null, { tags: { method: "test", name: "myName", runtag1: "fromreq" } }); + var 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) @@ -962,7 +962,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("GET", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.get("HTTPBIN_URL/get?a=1&b=2"); + var res = http.get("HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().args.a != "1") { throw new Error("wrong ?a: " + res.json().args.a); } if (res.json().args.b != "2") { throw new Error("wrong ?b: " + res.json().args.b); } @@ -971,10 +971,10 @@ func TestRequestAndBatch(t *testing.T) { assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), "GET", sr("HTTPBIN_URL/get?a=1&b=2"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - _, err := common.RunString(rt, ` - let a = "1"; - let b = "2"; - let res = http.get(http.url`+"`"+sr(`HTTPBIN_URL/get?a=${a}&b=${b}`)+"`"+`); + _, err := common.RunES6String(rt, ` + var a = "1"; + var b = "2"; + var res = http.get(http.url`+"`"+sr(`HTTPBIN_URL/get?a=${a}&b=${b}`)+"`"+`); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().args.a != a) { throw new Error("wrong ?a: " + res.json().args.a); } if (res.json().args.b != b) { throw new Error("wrong ?b: " + res.json().args.b); } @@ -985,7 +985,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("HEAD", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.head("HTTPBIN_URL/get?a=1&b=2"); + var res = http.head("HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.body.length != 0) { throw new Error("HEAD responses shouldn't have a body"); } if (!res.headers["Content-Length"]) { throw new Error("Missing or invalid Content-Length header!"); } @@ -996,7 +996,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("OPTIONS", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.options("HTTPBIN_URL/?a=1&b=2"); + var 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!"); } `)) @@ -1010,7 +1010,7 @@ func TestRequestAndBatch(t *testing.T) { // https://tools.ietf.org/html/rfc7231#section-4.3.5 t.Run("DELETE", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.del("HTTPBIN_URL/delete?test=mest"); + var 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)); } `)) @@ -1026,7 +1026,7 @@ func TestRequestAndBatch(t *testing.T) { for method, fn := range postMethods { t.Run(method, func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(sr(` - let res = http.%s("HTTPBIN_URL/%s", "data"); + var res = http.%s("HTTPBIN_URL/%s", "data"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().data != "data") { throw new Error("wrong data: " + res.json().data); } if (res.json().headers["Content-Type"]) { throw new Error("content type set: " + res.json().headers["Content-Type"]); } @@ -1036,7 +1036,7 @@ func TestRequestAndBatch(t *testing.T) { t.Run("object", func(t *testing.T) { _, err := common.RunString(rt, fmt.Sprintf(sr(` - let res = http.%s("HTTPBIN_URL/%s", {a: "a", b: 2}); + var res = http.%s("HTTPBIN_URL/%s", {a: "a", b: 2}); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().form.a != "a") { throw new Error("wrong a=: " + res.json().form.a); } if (res.json().form.b != "2") { throw new Error("wrong b=: " + res.json().form.b); } @@ -1046,7 +1046,7 @@ func TestRequestAndBatch(t *testing.T) { assertRequestMetricsEmitted(t, stats.GetBufferedSamples(samples), method, sr("HTTPBIN_URL/")+strings.ToLower(method), "", 200, "") t.Run("Content-Type", func(t *testing.T) { _, 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"}}); + var 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); } if (res.json().form.a != "a") { throw new Error("wrong a=: " + res.json().form.a); } if (res.json().form.b != "2") { throw new Error("wrong b=: " + res.json().form.b); } @@ -1061,16 +1061,16 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Batch", func(t *testing.T) { t.Run("error", func(t *testing.T) { - _, err := common.RunString(rt, `let res = http.batch("https://somevalidurl.com");`) + _, err := common.RunString(rt, `var res = http.batch("https://somevalidurl.com");`) require.Error(t, err) }) t.Run("GET", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let reqs = [ + var reqs = [ ["GET", "HTTPBIN_URL/"], ["GET", "HTTPBIN_IP_URL/"], ]; - let res = http.batch(reqs); + var res = http.batch(reqs); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + res[key].status); } if (res[key].url != reqs[key][1]) { throw new Error("wrong url: " + res[key].url); } @@ -1081,7 +1081,7 @@ func TestRequestAndBatch(t *testing.T) { assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - _, err := common.RunString(rt, sr(` + _, err := common.RunES6String(rt, sr(` let fragment = "get"; let reqs = [ ["GET", http.url`+"`"+`HTTPBIN_URL/${fragment}`+"`"+`], @@ -1100,11 +1100,11 @@ func TestRequestAndBatch(t *testing.T) { t.Run("Shorthand", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let reqs = [ + var reqs = [ "HTTPBIN_URL/", "HTTPBIN_IP_URL/", ]; - let res = http.batch(reqs); + var res = http.batch(reqs); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].url != reqs[key]) { throw new Error("wrong url: " + key + ": " + res[key].url); } @@ -1115,7 +1115,7 @@ func TestRequestAndBatch(t *testing.T) { assertRequestMetricsEmitted(t, bufSamples, "GET", sr("HTTPBIN_IP_URL/"), "", 200, "") t.Run("Tagged", func(t *testing.T) { - _, err := common.RunString(rt, sr(` + _, err := common.RunES6String(rt, sr(` let fragment = "get"; let reqs = [ http.url`+"`"+`HTTPBIN_URL/${fragment}`+"`"+`, @@ -1135,11 +1135,11 @@ func TestRequestAndBatch(t *testing.T) { t.Run("ObjectForm", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let reqs = [ + var reqs = [ { method: "GET", url: "HTTPBIN_URL/" }, { url: "HTTPBIN_IP_URL/", method: "GET"}, ]; - let res = http.batch(reqs); + var res = http.batch(reqs); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].url != reqs[key].url) { throw new Error("wrong url: " + key + ": " + res[key].url + " != " + reqs[key].url); } @@ -1152,13 +1152,13 @@ func TestRequestAndBatch(t *testing.T) { t.Run("ObjectKeys", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let reqs = { + var reqs = { shorthand: "HTTPBIN_URL/get?r=shorthand", arr: ["GET", "HTTPBIN_URL/get?r=arr", null, {tags: {name: 'arr'}}], obj1: { method: "GET", url: "HTTPBIN_URL/get?r=obj1" }, obj2: { url: "HTTPBIN_URL/get?r=obj2", params: {tags: {name: 'obj2'}}, method: "GET"}, }; - let res = http.batch(reqs); + var res = http.batch(reqs); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].json().args.r != key) { throw new Error("wrong request id: " + key); } @@ -1177,7 +1177,7 @@ func TestRequestAndBatch(t *testing.T) { rt.Set("someBinFile", []byte(testStr)) _, err := common.RunString(rt, sr(` - let reqs = [ + var reqs = [ ["POST", "HTTPBIN_URL/post", "testbody"], ["POST", "HTTPBIN_URL/post", someStrFile], ["POST", "HTTPBIN_URL/post", someBinFile], @@ -1203,7 +1203,7 @@ func TestRequestAndBatch(t *testing.T) { }, }, ]; - let res = http.batch(reqs); + var res = http.batch(reqs); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].json().data != "testbody" && res[key].json().form.hello != "world!") { throw new Error("wrong response for " + key + ": " + res[key].body); } @@ -1216,7 +1216,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("POST", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.batch([ ["POST", "HTTPBIN_URL/post", { key: "value" }] ]); + var res = http.batch([ ["POST", "HTTPBIN_URL/post", { key: "value" }] ]); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].json().form.key != "value") { throw new Error("wrong form: " + key + ": " + JSON.stringify(res[key].json().form)); } @@ -1226,7 +1226,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("PUT", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.batch([ ["PUT", "HTTPBIN_URL/put", { key: "value" }] ]); + var res = http.batch([ ["PUT", "HTTPBIN_URL/put", { key: "value" }] ]); for (var key in res) { if (res[key].status != 200) { throw new Error("wrong status: " + key + ": " + res[key].status); } if (res[key].json().form.key != "value") { throw new Error("wrong form: " + key + ": " + JSON.stringify(res[key].json().form)); } @@ -1239,9 +1239,9 @@ func TestRequestAndBatch(t *testing.T) { t.Run("HTTPRequest", func(t *testing.T) { t.Run("EmptyBody", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let reqUrl = "HTTPBIN_URL/cookies" - let res = http.get(reqUrl); - let jar = new http.CookieJar(); + var reqUrl = "HTTPBIN_URL/cookies" + var res = http.get(reqUrl); + var jar = new http.CookieJar(); jar.set("HTTPBIN_URL/cookies", "key", "value"); res = http.request("GET", "HTTPBIN_URL/cookies", null, { cookies: { key2: "value2" }, jar: jar }); @@ -1261,7 +1261,7 @@ func TestRequestAndBatch(t *testing.T) { }) t.Run("NonEmptyBody", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.post("HTTPBIN_URL/post", {a: "a", b: 2}, {headers: {"Content-Type": "application/x-www-form-urlencoded; charset=utf-8"}}); + var res = http.post("HTTPBIN_URL/post", {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); } if (res.request["body"] != "a=a&b=2") { throw new Error("http request body was not set properly: " + JSON.stringify(res.request))} `)) @@ -1381,7 +1381,7 @@ func TestRequestCompression(t *testing.T) { } expectedEncoding = strings.Join(algos, ", ") actualEncoding = expectedEncoding - _, err := common.RunString(rt, tb.Replacer.Replace(` + _, err := common.RunES6String(rt, tb.Replacer.Replace(` http.post("HTTPBIN_URL/compressed-text", `+"`"+text+"`"+`, {"compression": "`+testCase.compression+`"}); `)) if testCase.expectedError == "" { @@ -1400,7 +1400,7 @@ func TestRequestCompression(t *testing.T) { logHook.Drain() t.Run("encoding", func(t *testing.T) { - _, err := common.RunString(rt, tb.Replacer.Replace(` + _, err := common.RunES6String(rt, tb.Replacer.Replace(` http.post("HTTPBIN_URL/compressed-text", `+"`"+text+"`"+`, {"compression": "`+actualEncoding+`", "headers": {"Content-Encoding": "`+expectedEncoding+`"} @@ -1412,7 +1412,7 @@ func TestRequestCompression(t *testing.T) { }) t.Run("encoding and length", func(t *testing.T) { - _, err := common.RunString(rt, tb.Replacer.Replace(` + _, err := common.RunES6String(rt, tb.Replacer.Replace(` http.post("HTTPBIN_URL/compressed-text", `+"`"+text+"`"+`, {"compression": "`+actualEncoding+`", "headers": {"Content-Encoding": "`+expectedEncoding+`", @@ -1426,7 +1426,7 @@ func TestRequestCompression(t *testing.T) { expectedEncoding = actualEncoding t.Run("correct encoding", func(t *testing.T) { - _, err := common.RunString(rt, tb.Replacer.Replace(` + _, err := common.RunES6String(rt, tb.Replacer.Replace(` http.post("HTTPBIN_URL/compressed-text", `+"`"+text+"`"+`, {"compression": "`+actualEncoding+`", "headers": {"Content-Encoding": "`+actualEncoding+`"} @@ -1448,7 +1448,7 @@ func TestRequestCompression(t *testing.T) { t.Run("content-length is set", func(t *testing.T) { _, err := common.RunString(rt, tb.Replacer.Replace(` - let resp = http.post("HTTPBIN_URL/post", "0123456789"); + var resp = http.post("HTTPBIN_URL/post", "0123456789"); if (resp.json().headers["Content-Length"][0] != "10") { throw new Error("content-length not set: " + JSON.stringify(resp.json().headers)); } @@ -1504,29 +1504,29 @@ func TestResponseTypes(t *testing.T) { } _, err := common.RunString(rt, replace(` - let expText = "EXP_TEXT"; - let expBinLength = EXP_BIN_LEN; + var expText = "EXP_TEXT"; + var expBinLength = EXP_BIN_LEN; // Check default behaviour with a unicode text - let respTextImplicit = http.get("HTTPBIN_URL/get-text").body; + var respTextImplicit = http.get("HTTPBIN_URL/get-text").body; if (respTextImplicit !== expText) { throw new Error("default response body should be '" + expText + "' but was '" + respTextImplicit + "'"); } http.post("HTTPBIN_URL/compare-text", respTextImplicit); // Check discarding of responses - let respNone = http.get("HTTPBIN_URL/get-text", { responseType: "none" }).body; + var respNone = http.get("HTTPBIN_URL/get-text", { responseType: "none" }).body; if (respNone != null) { throw new Error("none response body should be null but was " + respNone); } // Check binary transmission of the text response as well - let respTextInBin = http.get("HTTPBIN_URL/get-text", { responseType: "binary" }).body; + var respTextInBin = http.get("HTTPBIN_URL/get-text", { responseType: "binary" }).body; // Hack to convert a utf-8 array to a JS string - let strConv = ""; + var strConv = ""; function pad(n) { return n.length < 2 ? "0" + n : n; } - for( let i = 0; i < respTextInBin.length; i++ ) { + for( var i = 0; i < respTextInBin.length; i++ ) { strConv += ( "%" + pad(respTextInBin[i].toString(16))); } strConv = decodeURIComponent(strConv); @@ -1536,11 +1536,11 @@ func TestResponseTypes(t *testing.T) { http.post("HTTPBIN_URL/compare-text", respTextInBin); // Check binary response - let respBin = http.get("HTTPBIN_URL/get-bin", { responseType: "binary" }).body; + var respBin = http.get("HTTPBIN_URL/get-bin", { responseType: "binary" }).body; if (respBin.length !== expBinLength) { throw new Error("response body length should be '" + expBinLength + "' but was '" + respBin.length + "'"); } - for( let i = 0; i < respBin.length; i++ ) { + for( var i = 0; i < respBin.length; i++ ) { if ( respBin[i] !== i%256 ) { throw new Error("expected value " + (i%256) + " to be at position " + i + " but it was " + respBin[i]); } @@ -1553,16 +1553,16 @@ func TestResponseTypes(t *testing.T) { state.Options.DiscardResponseBodies = null.BoolFrom(true) _, err = common.RunString(rt, replace(` - let expText = "EXP_TEXT"; + var expText = "EXP_TEXT"; // Check default behaviour - let respDefault = http.get("HTTPBIN_URL/get-text").body; + var respDefault = http.get("HTTPBIN_URL/get-text").body; if (respDefault !== null) { throw new Error("default response body should be discarded and null but was " + respDefault); } // Check explicit text response - let respTextExplicit = http.get("HTTPBIN_URL/get-text", { responseType: "text" }).body; + var respTextExplicit = http.get("HTTPBIN_URL/get-text", { responseType: "text" }).body; if (respTextExplicit !== expText) { throw new Error("text response body should be '" + expText + "' but was '" + respTextExplicit + "'"); } @@ -1617,46 +1617,46 @@ func TestErrorCodes(t *testing.T) { name: "Unroutable", expectedErrorCode: 1101, expectedErrorMsg: "lookup: no such host", - script: `let res = http.request("GET", "http://sdafsgdhfjg/");`, + script: `var res = http.request("GET", "http://sdafsgdhfjg/");`, }, { name: "404", status: 404, expectedErrorCode: 1404, - script: `let res = http.request("GET", "HTTPBIN_URL/status/404");`, + script: `var res = http.request("GET", "HTTPBIN_URL/status/404");`, }, { name: "Unroutable redirect", expectedErrorCode: 1101, expectedErrorMsg: "lookup: no such host", moreSamples: 1, - script: `let res = http.request("GET", "HTTPBIN_URL/redirect-to?url=http://dafsgdhfjg/");`, + script: `var res = http.request("GET", "HTTPBIN_URL/redirect-to?url=http://dafsgdhfjg/");`, }, { name: "Non location redirect", expectedErrorCode: 1000, expectedErrorMsg: "302 response missing Location header", - script: `let res = http.request("GET", "HTTPBIN_URL/no-location-redirect");`, + script: `var res = http.request("GET", "HTTPBIN_URL/no-location-redirect");`, }, { name: "Bad location redirect", expectedErrorCode: 1000, expectedErrorMsg: "failed to parse Location header \"h\\t:/\": ", - script: `let res = http.request("GET", "HTTPBIN_URL/bad-location-redirect");`, + script: `var res = http.request("GET", "HTTPBIN_URL/bad-location-redirect");`, }, { name: "Missing protocol", expectedErrorCode: 1000, expectedErrorMsg: `unsupported protocol scheme ""`, - script: `let res = http.request("GET", "dafsgdhfjg/");`, + script: `var res = http.request("GET", "dafsgdhfjg/");`, }, { name: "Too many redirects", status: 302, moreSamples: 2, script: ` - let res = http.get("HTTPBIN_URL/relative-redirect/3", {redirects: 2}); + var res = http.get("HTTPBIN_URL/relative-redirect/3", {redirects: 2}); if (res.url != "HTTPBIN_URL/relative-redirect/1") { throw new Error("incorrect URL: " + res.url) }`, }, { @@ -1666,7 +1666,7 @@ func TestErrorCodes(t *testing.T) { expectedErrorMsg: `dial: connection refused`, expectedErrorCode: 1212, script: ` - let res = http.get("HTTPBIN_URL/redirect-to?url=http%3A%2F%2F127.0.0.1%3A1%2Fpesho"); + var res = http.get("HTTPBIN_URL/redirect-to?url=http%3A%2F%2F127.0.0.1%3A1%2Fpesho"); if (res.url != "http://127.0.0.1:1/pesho") { throw new Error("incorrect URL: " + res.url) }`, }, } @@ -1726,7 +1726,7 @@ func TestResponseWaitingAndReceivingTimings(t *testing.T) { })) _, err := common.RunString(rt, tb.Replacer.Replace(` - let resp = http.get("HTTPBIN_URL/slow-response"); + var resp = http.get("HTTPBIN_URL/slow-response"); if (resp.timings.waiting < 1000) { throw new Error("expected waiting time to be over 1000ms but was " + resp.timings.waiting); @@ -1752,7 +1752,7 @@ func TestResponseTimingsWhenTimeout(t *testing.T) { state.Options.Throw = null.BoolFrom(false) _, err := common.RunString(rt, tb.Replacer.Replace(` - let resp = http.get("HTTPBIN_URL/delay/10", { timeout: 2500 }); + var resp = http.get("HTTPBIN_URL/delay/10", { timeout: 2500 }); if (resp.timings.waiting < 2000) { throw new Error("expected waiting time to be over 2000ms but was " + resp.timings.waiting); @@ -1774,18 +1774,18 @@ func TestNoResponseBodyMangling(t *testing.T) { state.Options.Throw = null.BoolFrom(true) _, err := common.RunString(rt, tb.Replacer.Replace(` - const batchSize = 100; + var batchSize = 100; - let requests = []; + var requests = []; - for (let i = 0; i < batchSize; i++) { + for (var i = 0; i < batchSize; i++) { requests.push(["GET", "HTTPBIN_URL/get?req=" + i, null, { responseType: (i % 2 ? "binary" : "text") }]); } - let responses = http.batch(requests); + var responses = http.batch(requests); - for (let i = 0; i < batchSize; i++) { - let reqNumber = parseInt(responses[i].json().args.req[0], 10); + for (var i = 0; i < batchSize; i++) { + var reqNumber = parseInt(responses[i].json().args.req[0], 10); if (i !== reqNumber) { throw new Error("Response " + i + " has " + reqNumber + ", expected " + i) } @@ -1878,7 +1878,7 @@ func TestErrorsWithDecompression(t *testing.T) { _, err := common.RunString(rt, tb.Replacer.Replace(` function handleResponseEncodingError (encoding) { - let resp = http.get("HTTPBIN_URL/broken-archive?encoding=" + encoding); + var resp = http.get("HTTPBIN_URL/broken-archive?encoding=" + encoding); if (resp.error_code != 1701) { throw new Error("Expected error_code 1701 for '" + encoding +"', but got " + resp.error_code); } @@ -1910,7 +1910,7 @@ func TestDigestAuthWithBody(t *testing.T) { ) _, err := common.RunString(rt, fmt.Sprintf(` - let res = http.post(%q, "super secret body", { auth: "digest" }); + var res = http.post(%q, "super secret body", { auth: "digest" }); if (res.status !== 200) { throw new Error("wrong status: " + res.status); } if (res.error_code !== 0) { throw new Error("wrong error code: " + res.error_code); } `, urlWithCreds)) diff --git a/js/modules/k6/http/response_test.go b/js/modules/k6/http/response_test.go index 5753cb9416b..2d8fd286540 100644 --- a/js/modules/k6/http/response_test.go +++ b/js/modules/k6/http/response_test.go @@ -135,7 +135,7 @@ func TestResponse(t *testing.T) { t.Run("Html", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/html"); + var 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); } `)) @@ -176,7 +176,7 @@ func TestResponse(t *testing.T) { } _, err = common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/html"); + var 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); } `)) @@ -186,7 +186,7 @@ func TestResponse(t *testing.T) { }) t.Run("Json", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/get?a=1&b=2"); + var res = http.request("GET", "HTTPBIN_URL/get?a=1&b=2"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } if (res.json().args.a != "1") { throw new Error("wrong ?a: " + res.json().args.a); } if (res.json().args.b != "2") { throw new Error("wrong ?b: " + res.json().args.b); } @@ -208,7 +208,7 @@ func TestResponse(t *testing.T) { }) t.Run("JsonSelector", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/json"); + var res = http.request("GET", "HTTPBIN_URL/json"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } var value = res.json("glossary.friends.1") @@ -252,11 +252,11 @@ func TestResponse(t *testing.T) { t.Run("SubmitForm", func(t *testing.T) { t.Run("withoutArgs", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/forms/post"); + var res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.submitForm() if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let data = res.json().form + var data = res.json().form if (data.custname[0] !== "" || data.extradata !== undefined || data.comments[0] !== "" || @@ -271,11 +271,11 @@ func TestResponse(t *testing.T) { t.Run("withFields", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/forms/post"); + var res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.submitForm({ fields: { custname: "test", extradata: "test2" } }) if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let data = res.json().form + var data = res.json().form if (data.custname[0] !== "test" || data.extradata[0] !== "test2" || data.comments[0] !== "" || @@ -290,11 +290,11 @@ func TestResponse(t *testing.T) { t.Run("withRequestParams", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/forms/post"); + var res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.submitForm({ params: { headers: { "My-Fancy-Header": "SomeValue" } }}) if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let headers = res.json().headers + var headers = res.json().headers if (headers["My-Fancy-Header"][0] !== "SomeValue" ) { throw new Error("incorrect headers: " + JSON.stringify(headers)); } `)) assert.NoError(t, err) @@ -303,11 +303,11 @@ func TestResponse(t *testing.T) { t.Run("withFormSelector", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/forms/post"); + var res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.submitForm({ formSelector: 'form[method="post"]' }) if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let data = res.json().form + var data = res.json().form if (data.custname[0] !== "" || data.extradata !== undefined || data.comments[0] !== "" || @@ -322,7 +322,7 @@ func TestResponse(t *testing.T) { t.Run("withNonExistentForm", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/forms/post"); + var res = http.request("GET", "HTTPBIN_URL/forms/post"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res.submitForm({ formSelector: "#doesNotExist" }) `)) @@ -331,11 +331,11 @@ func TestResponse(t *testing.T) { t.Run("withGetMethod", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/myforms/get"); + var res = http.request("GET", "HTTPBIN_URL/myforms/get"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.submitForm() if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let data = res.json().query + var data = res.json().query if (data.input_with_value[0] !== "value" || data.input_without_value[0] !== "" || data.select_one[0] !== "yes this option" || @@ -351,7 +351,7 @@ func TestResponse(t *testing.T) { t.Run("ClickLink", func(t *testing.T) { t.Run("withoutArgs", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/links/10/0"); + var res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.clickLink() if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -362,7 +362,7 @@ func TestResponse(t *testing.T) { t.Run("withSelector", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/links/10/0"); + var res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.clickLink({ selector: 'a:nth-child(4)' }) if (res.status != 200) { throw new Error("wrong status: " + res.status); } @@ -373,7 +373,7 @@ func TestResponse(t *testing.T) { t.Run("withNonExistentLink", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL/links/10/0"); + var res = http.request("GET", "HTTPBIN_URL/links/10/0"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.clickLink({ selector: 'a#doesNotExist' }) `)) @@ -382,11 +382,11 @@ func TestResponse(t *testing.T) { t.Run("withRequestParams", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = http.request("GET", "HTTPBIN_URL"); + var res = http.request("GET", "HTTPBIN_URL"); if (res.status != 200) { throw new Error("wrong status: " + res.status); } res = res.clickLink({ selector: 'a[href="/get"]', params: { headers: { "My-Fancy-Header": "SomeValue" } } }) if (res.status != 200) { throw new Error("wrong status: " + res.status); } - let headers = res.json().headers + var headers = res.json().headers if (headers["My-Fancy-Header"][0] !== "SomeValue" ) { throw new Error("incorrect headers: " + JSON.stringify(headers)); } `)) assert.NoError(t, err) diff --git a/js/modules/k6/http/tls_test.go b/js/modules/k6/http/tls_test.go index a23ba69f741..e4416671289 100644 --- a/js/modules/k6/http/tls_test.go +++ b/js/modules/k6/http/tls_test.go @@ -47,7 +47,7 @@ func TestTLS13Support(t *testing.T) { state.Options.Apply(lib.Options{TLSVersion: &lib.TLSVersions{Max: lib.TLSVersion13}}) _, err := common.RunString(rt, tb.Replacer.Replace(` - let resp = http.get("HTTPSBIN_URL/tls-version"); + var resp = http.get("HTTPSBIN_URL/tls-version"); if (resp.body != "tls1.3") { throw new Error("unexpected tls version: " + resp.body); } diff --git a/js/modules/k6/k6_test.go b/js/modules/k6/k6_test.go index b3649d92903..3be8a357a03 100644 --- a/js/modules/k6/k6_test.go +++ b/js/modules/k6/k6_test.go @@ -94,14 +94,14 @@ func TestRandSeed(t *testing.T) { rand := 0.8487305991992138 _, err := common.RunString(rt, fmt.Sprintf(` - let rnd = Math.random(); + var rnd = Math.random(); if (rnd == %.16f) { throw new Error("wrong random: " + rnd); } `, rand)) assert.NoError(t, err) _, err = common.RunString(rt, fmt.Sprintf(` k6.randomSeed(12345) - let rnd = Math.random(); + var rnd = Math.random(); if (rnd != %.16f) { throw new Error("wrong random: " + rnd); } `, rand)) assert.NoError(t, err) @@ -256,7 +256,7 @@ func TestCheck(t *testing.T) { "b": function() { throw new Error("error B") }, }) `) - assert.EqualError(t, err, "Error: error A at a (:3:27(6))") + assert.EqualError(t, err, "Error: error A at :3:28(4)") bufSamples := stats.GetBufferedSamples(samples) if assert.Len(t, bufSamples, 1) { @@ -276,8 +276,8 @@ func TestCheck(t *testing.T) { t.Run("Types", func(t *testing.T) { templates := map[string]string{ "Literal": `k6.check(null,{"check": %s})`, - "Callable": `k6.check(null,{"check": ()=>%s})`, - "Callable/Arg": `k6.check(%s,{"check":(v)=>v})`, + "Callable": `k6.check(null,{"check": function() { return %s; }})`, + "Callable/Arg": `k6.check(%s,{"check": function(v) {return v; }})`, } testdata := map[string]bool{ `0`: false, diff --git a/js/modules/k6/metrics/metrics_test.go b/js/modules/k6/metrics/metrics_test.go index 58acf585c03..cb1d00f016c 100644 --- a/js/modules/k6/metrics/metrics_test.go +++ b/js/modules/k6/metrics/metrics_test.go @@ -80,7 +80,7 @@ func TestMetrics(t *testing.T) { isTimeString = `, true` } _, err := common.RunString(rt, - fmt.Sprintf(`let m = new metrics.%s("my_metric"%s)`, fn, isTimeString), + fmt.Sprintf(`var m = new metrics.%s("my_metric"%s)`, fn, isTimeString), ) if !assert.NoError(t, err) { return diff --git a/js/modules/k6/ws/ws_test.go b/js/modules/k6/ws/ws_test.go index f0090a831ba..04363f22dac 100644 --- a/js/modules/k6/ws/ws_test.go +++ b/js/modules/k6/ws/ws_test.go @@ -121,7 +121,7 @@ func TestSession(t *testing.T) { t.Run("connect_ws", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.close() }); if (res.status != 101) { throw new Error("connection failed with status: " + res.status); } @@ -132,7 +132,7 @@ func TestSession(t *testing.T) { t.Run("connect_wss", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = ws.connect("WSSBIN_URL/ws-echo", function(socket){ + var res = ws.connect("WSSBIN_URL/ws-echo", function(socket){ socket.close() }); if (res.status != 101) { throw new Error("TLS connection failed with status: " + res.status); } @@ -143,8 +143,8 @@ func TestSession(t *testing.T) { t.Run("open", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let opened = false; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var opened = false; + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function() { opened = true; socket.close() @@ -158,7 +158,7 @@ func TestSession(t *testing.T) { t.Run("send_receive", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function() { socket.send("test") }) @@ -180,8 +180,8 @@ func TestSession(t *testing.T) { t.Run("interval", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let counter = 0; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var counter = 0; + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.setInterval(function () { counter += 1; if (counter > 2) { socket.close(); } @@ -195,9 +195,9 @@ func TestSession(t *testing.T) { t.Run("timeout", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let start = new Date().getTime(); - let ellapsed = new Date().getTime() - start; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var start = new Date().getTime(); + var ellapsed = new Date().getTime() - start; + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.setTimeout(function () { ellapsed = new Date().getTime() - start; socket.close(); @@ -213,8 +213,8 @@ func TestSession(t *testing.T) { t.Run("ping", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let pongReceived = false; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var pongReceived = false; + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function(data) { socket.ping(); }); @@ -237,10 +237,10 @@ func TestSession(t *testing.T) { t.Run("multiple_handlers", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let pongReceived = false; - let otherPongReceived = false; + var pongReceived = false; + var otherPongReceived = false; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function(data) { socket.ping(); }); @@ -271,8 +271,8 @@ func TestSession(t *testing.T) { t.Run("client_close", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let closed = false; - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var closed = false; + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function() { socket.close() }) @@ -301,8 +301,8 @@ func TestSession(t *testing.T) { tc := tc t.Run(tc.name, func(t *testing.T) { _, err := common.RunString(rt, sr(fmt.Sprintf(` - let closed = false; - let res = ws.connect("WSBIN_URL%s", function(socket){ + var closed = false; + var res = ws.connect("WSBIN_URL%s", function(socket){ socket.on("open", function() { socket.send("test"); }) @@ -346,7 +346,7 @@ func TestErrors(t *testing.T) { t.Run("invalid_url", func(t *testing.T) { _, err := common.RunString(rt, ` - let res = ws.connect("INVALID", function(socket){ + var res = ws.connect("INVALID", function(socket){ socket.on("open", function() { socket.close(); }); @@ -358,7 +358,7 @@ func TestErrors(t *testing.T) { t.Run("invalid_url_message_panic", func(t *testing.T) { // Attempting to send a message to a non-existent socket shouldn't panic _, err := common.RunString(rt, ` - let res = ws.connect("INVALID", function(socket){ + var res = ws.connect("INVALID", function(socket){ socket.send("new message"); }); `) @@ -367,7 +367,7 @@ func TestErrors(t *testing.T) { t.Run("error_in_setup", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let res = ws.connect("WSBIN_URL/ws-echo-invalid", function(socket){ + var res = ws.connect("WSBIN_URL/ws-echo-invalid", function(socket){ throw new Error("error in setup"); }); `)) @@ -376,8 +376,8 @@ func TestErrors(t *testing.T) { t.Run("send_after_close", func(t *testing.T) { _, err := common.RunString(rt, sr(` - let hasError = false; - let res = ws.connect("WSBIN_URL/ws-echo-invalid", function(socket){ + var hasError = false; + var res = ws.connect("WSBIN_URL/ws-echo-invalid", function(socket){ socket.on("open", function() { socket.close(); socket.send("test"); @@ -398,7 +398,7 @@ func TestErrors(t *testing.T) { t.Run("error on close", func(t *testing.T) { _, err := common.RunString(rt, sr(` var closed = false; - let res = ws.connect("WSBIN_URL/ws-close", function(socket){ + var res = ws.connect("WSBIN_URL/ws-close", function(socket){ socket.on('open', function open() { socket.setInterval(function timeout() { socket.ping(); @@ -461,7 +461,7 @@ func TestSystemTags(t *testing.T) { t.Run("only "+expectedTag, func(t *testing.T) { state.Options.SystemTags = stats.ToSystemTagSet([]string{expectedTag}) _, err := common.RunString(rt, sr(` - let res = ws.connect("WSBIN_URL/ws-echo", function(socket){ + var res = ws.connect("WSBIN_URL/ws-echo", function(socket){ socket.on("open", function() { socket.send("test") }) @@ -525,7 +525,7 @@ func TestTLSConfig(t *testing.T) { } _, err := common.RunString(rt, sr(` - let res = ws.connect("WSSBIN_URL/ws-close", function(socket){ + var res = ws.connect("WSSBIN_URL/ws-close", function(socket){ socket.close() }); if (res.status != 101) { throw new Error("TLS connection failed with status: " + res.status); } @@ -538,7 +538,7 @@ func TestTLSConfig(t *testing.T) { state.TLSConfig = tb.TLSClientConfig _, err := common.RunString(rt, sr(` - let res = ws.connect("WSSBIN_URL/ws-close", function(socket){ + var res = ws.connect("WSSBIN_URL/ws-close", function(socket){ socket.close() }); if (res.status != 101) { From 6e4ea2036fa5e4f4b02775bd44e85ecbbc7202a5 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 21 May 2020 12:20:57 +0300 Subject: [PATCH 266/350] Fix a broken test --- js/runner_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/js/runner_test.go b/js/runner_test.go index 3251abe0d00..6f9f4a70857 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -1573,7 +1573,7 @@ func TestSystemTags(t *testing.T) { w.WriteHeader(http.StatusTemporaryRedirect) }) - r, err := getSimpleRunnerWithOptions("/script.js", tb.Replacer.Replace(` + r, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` var http = require("k6/http"); exports.http_get = function() { From 473f934f94233aa861d48db1276143e2799fb400 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 21 May 2020 12:21:14 +0300 Subject: [PATCH 267/350] Update the linter and fix linter issues --- .circleci/config.yml | 2 +- .golangci.yml | 1 + core/local/local_test.go | 2 +- lib/executor/executors_test.go | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 339216699c6..20990c1b6ff 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -26,7 +26,7 @@ jobs: - image: circleci/golang:1.14 environment: GOPATH: /home/circleci/.go_workspace - GOLANGCI_VERSION: v1.25.0 + GOLANGCI_VERSION: v1.27.0 working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 steps: # Workaround for custom env vars not available in cache keys diff --git a/.golangci.yml b/.golangci.yml index ea289deb24a..32b7895c413 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -58,4 +58,5 @@ linters: - testpackage - wsl - gomnd + - goerr113 # most of the errors here are meant for humans fast: false diff --git a/core/local/local_test.go b/core/local/local_test.go index 77f76ee3ed1..bd90f856775 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -53,7 +53,7 @@ import ( ) func newTestExecutionScheduler( - t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, //nolint: golint + t *testing.T, runner lib.Runner, logger *logrus.Logger, opts lib.Options, ) (ctx context.Context, cancel func(), execScheduler *ExecutionScheduler, samples chan stats.SampleContainer) { if runner == nil { runner = &minirunner.MiniRunner{} diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 9e8421fc9e0..3a6bfee3dcb 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -44,7 +44,7 @@ type configMapTestCase struct { expected exp } -//nolint:lll,gochecknoglobals +//nolint:gochecknoglobals var configMapTestCases = []configMapTestCase{ {"", exp{parseError: true}}, {"1234", exp{parseError: true}}, From e0ddf5d3d2440b17ef530ad6d494ab86077efb47 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 21 May 2020 12:56:12 +0300 Subject: [PATCH 268/350] Try to fix docker builds... --- .circleci/config.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 20990c1b6ff..7df85bd4d11 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -108,12 +108,10 @@ jobs: working_directory: /home/circleci/.go_workspace/src/github.com/loadimpact/k6 steps: - checkout - - setup_remote_docker: - version: 17.11.0-ce + - setup_remote_docker - run: name: Setup repo and docker command: | - git submodule update --init docker info echo "{\"https://index.docker.io/v1/\":{\"auth\":\"$DOCKER_AUTH\",\"email\":\"$DOCKER_EMAIL\"}}" >~/.dockercfg - run: From cb8c7313259e5648e5dd7f2888d6859e14188a75 Mon Sep 17 00:00:00 2001 From: thinkerou Date: Fri, 22 May 2020 14:26:02 +0800 Subject: [PATCH 269/350] Remove an unnecesary else statement (#1461) --- lib/executor/helpers.go | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index a1b53580f1b..b9e352ac988 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -54,21 +54,22 @@ func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 // A helper function to avoid code duplication func validateStages(stages []Stage) []error { var errors []error - if len(stages) == 0 { //nolint:nestif + if len(stages) == 0 { errors = append(errors, fmt.Errorf("at least one stage has to be specified")) - } else { - for i, s := range stages { - stageNum := i + 1 - if !s.Duration.Valid { - errors = append(errors, fmt.Errorf("stage %d doesn't have a duration", stageNum)) - } else if s.Duration.Duration < 0 { - errors = append(errors, fmt.Errorf("the duration for stage %d shouldn't be negative", stageNum)) - } - if !s.Target.Valid { - errors = append(errors, fmt.Errorf("stage %d doesn't have a target", stageNum)) - } else if s.Target.Int64 < 0 { - errors = append(errors, fmt.Errorf("the target for stage %d shouldn't be negative", stageNum)) - } + return errors + } + + for i, s := range stages { + stageNum := i + 1 + if !s.Duration.Valid { + errors = append(errors, fmt.Errorf("stage %d doesn't have a duration", stageNum)) + } else if s.Duration.Duration < 0 { + errors = append(errors, fmt.Errorf("the duration for stage %d shouldn't be negative", stageNum)) + } + if !s.Target.Valid { + errors = append(errors, fmt.Errorf("stage %d doesn't have a target", stageNum)) + } else if s.Target.Int64 < 0 { + errors = append(errors, fmt.Errorf("the target for stage %d shouldn't be negative", stageNum)) } } return errors From d3a1546ab6a4bf641972b0f9417d01e5a12e3fe3 Mon Sep 17 00:00:00 2001 From: na-- Date: Fri, 22 May 2020 11:31:10 +0300 Subject: [PATCH 270/350] Fix metrics processing after test run (#1460) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Process metrics and thresholds after test run finishes This is a quick and dirty fix for https://github.com/loadimpact/k6/issues/1434, though as I mention in that issue, the proper solution would be rewriting that code from scratch... Co-authored-by: Ivan Mirić --- cmd/run.go | 2 +- core/engine.go | 34 ++++++++++++++++++++++++++-------- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 33900a572a7..21a4bd2e275 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -315,7 +315,7 @@ a commandline interface for interacting with it.`, return getExitCodeFromEngine(err) } runCancel() - logger.Debug("Engine terminated cleanly") + logger.Debug("Engine run terminated cleanly") progressCancel() progressBarWG.Wait() diff --git a/core/engine.go b/core/engine.go index 973d0d73b8c..7db9fb2b5e7 100644 --- a/core/engine.go +++ b/core/engine.go @@ -132,6 +132,7 @@ func (e *Engine) Init(globalCtx, runCtx context.Context) (run func() error, wait runSubCtx, runSubCancel := context.WithCancel(runCtx) resultCh := make(chan error) + processMetricsAfterRun := make(chan struct{}) runFn := func() error { e.logger.Debug("Execution scheduler starting...") err := e.ExecutionScheduler.Run(globalCtx, runSubCtx, e.Samples) @@ -144,9 +145,14 @@ func (e *Engine) Init(globalCtx, runCtx context.Context) (run func() error, wait resultCh <- err // we finished normally, so send the result } + // Make the background jobs process the currently buffered metrics and + // run the thresholds, then wait for that to be done. + processMetricsAfterRun <- struct{}{} + <-processMetricsAfterRun + return err } - waitFn := e.startBackgroundProcesses(globalCtx, runCtx, resultCh, runSubCancel) + waitFn := e.startBackgroundProcesses(globalCtx, runCtx, resultCh, runSubCancel, processMetricsAfterRun) return runFn, waitFn, nil } @@ -155,7 +161,7 @@ func (e *Engine) Init(globalCtx, runCtx context.Context) (run func() error, wait // the provided context is called, to wait for the complete winding down of all // started goroutines. func (e *Engine) startBackgroundProcesses( //nolint:funlen - globalCtx, runCtx context.Context, runResult <-chan error, runSubCancel func(), + globalCtx, runCtx context.Context, runResult <-chan error, runSubCancel func(), processMetricsAfterRun chan struct{}, ) (wait func()) { processes := new(sync.WaitGroup) @@ -172,7 +178,7 @@ func (e *Engine) startBackgroundProcesses( //nolint:funlen processes.Add(1) go func() { defer processes.Done() - e.processMetrics(globalCtx) + e.processMetrics(globalCtx, processMetricsAfterRun) }() // Run VU metrics emission, only while the test is running. @@ -239,7 +245,7 @@ func (e *Engine) startBackgroundProcesses( //nolint:funlen return processes.Wait } -func (e *Engine) processMetrics(globalCtx context.Context) { +func (e *Engine) processMetrics(globalCtx context.Context, processMetricsAfterRun chan struct{}) { sampleContainers := []stats.SampleContainer{} defer func() { @@ -262,13 +268,25 @@ func (e *Engine) processMetrics(globalCtx context.Context) { defer ticker.Stop() e.logger.Debug("Metrics processing started...") + processSamples := func() { + if len(sampleContainers) > 0 { + e.processSamples(sampleContainers) + // Make the new container with the same size as the previous + // one, assuming that we produce roughly the same amount of + // metrics data between ticks... + sampleContainers = make([]stats.SampleContainer, 0, cap(sampleContainers)) + } + } for { select { case <-ticker.C: - if len(sampleContainers) > 0 { - e.processSamples(sampleContainers) - sampleContainers = []stats.SampleContainer{} // TODO: optimize? - } + processSamples() + case <-processMetricsAfterRun: + e.logger.Debug("Processing metrics and thresholds after the test run has ended...") + processSamples() + e.processThresholds() + processMetricsAfterRun <- struct{}{} + case sc := <-e.Samples: sampleContainers = append(sampleContainers, sc) case <-globalCtx.Done(): From 366a0085377df17d496e68fd3ecae7e43776d550 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 26 May 2020 13:01:30 +0300 Subject: [PATCH 271/350] Bump version in preparation for the v0.27.0 release --- lib/consts/consts.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/consts/consts.go b/lib/consts/consts.go index e6100f5a8bd..6437a97175a 100644 --- a/lib/consts/consts.go +++ b/lib/consts/consts.go @@ -28,7 +28,7 @@ import ( ) // Version contains the current semantic version of k6. -var Version = "0.26.2" //nolint:gochecknoglobals +var Version = "0.27.0-dev" //nolint:gochecknoglobals // VersionDetails can be set externally as part of the build process var VersionDetails = "" // nolint:gochecknoglobals From dd8885932a8c0598c29f9d442731c275b398044b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 26 May 2020 16:25:00 +0200 Subject: [PATCH 272/350] Avoid overwriting options when grabbing exports from archive Previously (since e693715b) running a TAR file would overwrite the options set in the metadata.json with the script's exported options. This is a hacky fix, but the alternatives weren't much cleaner. --- js/bundle.go | 11 ++++++++--- js/bundle_test.go | 21 +++++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/js/bundle.go b/js/bundle.go index b9d69810246..6a50b9833b6 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -96,7 +96,7 @@ func NewBundle(src *loader.SourceData, filesystems map[string]afero.Fs, rtOpts l return nil, err } - err = bundle.getExports(rt) + err = bundle.getExports(rt, true) if err != nil { return nil, err } @@ -154,7 +154,9 @@ func NewBundleFromArchive(arc *lib.Archive, rtOpts lib.RuntimeOptions) (*Bundle, return nil, err } - err = bundle.getExports(rt) + // Grab exported objects, but avoid overwriting options, which would + // be initialized from the metadata.json at this point. + err = bundle.getExports(rt, false) if err != nil { return nil, err } @@ -184,7 +186,7 @@ func (b *Bundle) makeArchive() *lib.Archive { } // getExports validates and extracts exported objects -func (b *Bundle) getExports(rt *goja.Runtime) error { +func (b *Bundle) getExports(rt *goja.Runtime, options bool) error { exportsV := rt.Get("exports") if goja.IsNull(exportsV) || goja.IsUndefined(exportsV) { return errors.New("exports must be an object") @@ -199,6 +201,9 @@ func (b *Bundle) getExports(rt *goja.Runtime) error { } switch k { case consts.Options: + if !options { + continue + } data, err := json.Marshal(v.Export()) if err != nil { return err diff --git a/js/bundle_test.go b/js/bundle_test.go index fa8ea1800f0..b25d879e6a6 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -512,6 +512,27 @@ func TestNewBundleFromArchive(t *testing.T) { checkArchive(t, arc, extCompatModeRtOpts, "") // works when I force the compat mode checkArchive(t, arc, baseCompatModeRtOpts, "Unexpected reserved word") // failes because of ES6 }) + + t.Run("script_options_dont_overwrite_metadata", func(t *testing.T) { + t.Parallel() + code := `export let options = { vus: 12345 }; export default function() { return options.vus; };` + arc := &lib.Archive{ + Type: "js", + FilenameURL: &url.URL{Scheme: "file", Path: "/script"}, + K6Version: consts.Version, + Data: []byte(code), + Options: lib.Options{VUs: null.IntFrom(999)}, + PwdURL: &url.URL{Scheme: "file", Path: "/"}, + Filesystems: nil, + } + b, err := NewBundleFromArchive(arc, lib.RuntimeOptions{}) + require.NoError(t, err) + bi, err := b.Instantiate() + require.NoError(t, err) + val, err := bi.exports[consts.DefaultFn](goja.Undefined()) + require.NoError(t, err) + assert.Equal(t, int64(999), val.Export()) + }) } func TestOpen(t *testing.T) { From 91b8023b5ba9dafe769836c6f97023a19e1dd990 Mon Sep 17 00:00:00 2001 From: thinkerou Date: Wed, 27 May 2020 21:26:06 +0800 Subject: [PATCH 273/350] Format import statements (#1464) --- api/v1/client/client.go | 5 ++--- api/v1/group.go | 3 ++- api/v1/group_routes.go | 3 ++- api/v1/group_routes_test.go | 9 +++++---- api/v1/group_test.go | 3 ++- api/v1/metric.go | 3 ++- api/v1/metric_routes.go | 3 ++- api/v1/metric_routes_test.go | 9 ++++----- api/v1/metric_test.go | 5 +++-- api/v1/routes_test.go | 3 ++- api/v1/setup_teardown_routes.go | 3 ++- api/v1/setup_teardown_routes_test.go | 8 ++++---- api/v1/status.go | 3 ++- api/v1/status_routes_test.go | 9 +++++---- cmd/cloud.go | 3 +-- cmd/collectors.go | 5 +++-- cmd/common.go | 5 +++-- cmd/config.go | 2 +- cmd/config_consolidation_test.go | 2 +- cmd/convert.go | 5 +++-- cmd/convert_test.go | 3 +-- cmd/inspect.go | 3 ++- cmd/login_cloud.go | 6 +++--- cmd/login_influxdb.go | 7 ++++--- cmd/options.go | 2 +- cmd/pause.go | 5 +++-- cmd/resume.go | 5 +++-- cmd/runtime_options_test.go | 6 +++--- cmd/scale.go | 5 +++-- cmd/stats.go | 3 ++- cmd/status.go | 3 ++- cmd/ui.go | 3 +-- cmd/version.go | 3 ++- converter/har/converter.go | 3 ++- converter/har/converter_test.go | 3 ++- core/engine.go | 5 +++-- core/engine_test.go | 4 ++-- core/local/local.go | 3 +-- core/local/local_test.go | 2 +- js/bundle.go | 3 +-- js/bundle_test.go | 2 +- js/common/util.go | 1 + js/compiler/compiler.go | 3 ++- js/compiler/compiler_test.go | 3 ++- js/console_test.go | 2 +- js/initcontext.go | 5 +++-- js/lib/lib.go | 2 +- js/module_loading_test.go | 2 +- js/modules/k6/crypto/crypto_test.go | 3 ++- js/modules/k6/crypto/x509/x509.go | 3 ++- js/modules/k6/crypto/x509/x509_test.go | 3 ++- js/modules/k6/encoding/encoding_test.go | 3 ++- js/modules/k6/html/element_test.go | 3 ++- js/modules/k6/html/elements_gen_test.go | 3 ++- js/modules/k6/html/elements_test.go | 3 ++- js/modules/k6/html/html.go | 3 ++- js/modules/k6/html/html_test.go | 3 ++- js/modules/k6/html/serialize_test.go | 3 ++- js/modules/k6/http/cookiejar.go | 3 ++- js/modules/k6/http/http_test.go | 5 +++-- js/modules/k6/http/http_url.go | 1 + js/modules/k6/http/request.go | 3 ++- js/modules/k6/http/request_test.go | 15 ++++++++------- js/modules/k6/http/response.go | 1 + js/modules/k6/http/response_test.go | 3 ++- js/modules/k6/http/tls_test.go | 5 +++-- js/modules/k6/k6_test.go | 8 ++++---- js/modules/k6/metrics/metrics_test.go | 5 +++-- js/modules/k6/ws/ws_test.go | 3 ++- js/runner_test.go | 2 +- lib/archive.go | 3 ++- lib/archive_test.go | 10 +++++----- lib/execution.go | 3 +-- lib/executor/base_config.go | 2 +- lib/executor/constant_arrival_rate.go | 2 +- lib/executor/constant_arrival_rate_test.go | 2 +- lib/executor/constant_looping_vus.go | 2 +- lib/executor/constant_looping_vus_test.go | 2 +- lib/executor/execution_config_shortcuts.go | 5 +++-- lib/executor/executors_test.go | 7 ++++--- lib/executor/externally_controlled.go | 2 +- lib/executor/externally_controlled_test.go | 3 +-- lib/executor/helpers.go | 3 +-- lib/executor/per_vu_iterations.go | 2 +- lib/executor/per_vu_iterations_test.go | 2 +- lib/executor/shared_iterations.go | 2 +- lib/executor/shared_iterations_test.go | 2 +- lib/executor/variable_arrival_rate.go | 2 +- lib/executor/variable_arrival_rate_test.go | 2 +- lib/executor/variable_looping_vus.go | 2 +- lib/executor/variable_looping_vus_test.go | 2 +- lib/models.go | 3 ++- lib/models_test.go | 3 ++- lib/netext/dialer.go | 4 ++-- lib/netext/httpext/error_codes.go | 3 ++- lib/netext/httpext/error_codes_test.go | 3 ++- lib/netext/httpext/request.go | 2 +- lib/netext/httpext/request_test.go | 5 +++-- lib/netext/httpext/response.go | 3 ++- lib/netext/httpext/tracer_test.go | 7 ++++--- lib/netext/tls.go | 3 ++- lib/old_archive_test.go | 3 ++- lib/options_test.go | 7 ++++--- lib/runtime_options.go | 2 +- lib/testutils/httpmultibin/httpmultibin.go | 5 +++-- lib/types/types.go | 2 +- lib/types/types_test.go | 2 +- loader/filesystems.go | 3 ++- loader/loader_test.go | 5 +++-- loader/readsource.go | 3 ++- loader/readsource_test.go | 3 ++- stats/cloud/api.go | 3 ++- stats/cloud/api_test.go | 3 ++- stats/cloud/bench_test.go | 8 ++++---- stats/cloud/collector_test.go | 4 +--- stats/cloud/config.go | 3 ++- stats/cloud/data_test.go | 5 +++-- stats/csv/collector.go | 5 +++-- stats/csv/collector_test.go | 5 ++--- stats/csv/config.go | 3 ++- stats/datadog/collector_test.go | 3 ++- stats/influxdb/collector.go | 3 ++- stats/influxdb/collector_test.go | 5 +++-- stats/influxdb/config.go | 5 +++-- stats/influxdb/config_test.go | 2 +- stats/influxdb/util.go | 2 +- stats/influxdb/util_test.go | 2 +- stats/json/wrapper_test.go | 3 ++- stats/kafka/collector_test.go | 3 ++- stats/kafka/config.go | 5 +++-- stats/kafka/config_test.go | 3 ++- stats/statsd/collector_test.go | 3 ++- stats/statsd/common/collector_test.go | 6 +++--- stats/statsd/common/config.go | 3 ++- stats/statsd/common/testutil/test_helper.go | 5 +++-- stats/thresholds_test.go | 3 ++- ui/pb/helpers_test.go | 3 ++- ui/summary.go | 5 +++-- 138 files changed, 291 insertions(+), 217 deletions(-) diff --git a/api/v1/client/client.go b/api/v1/client/client.go index 51c1998af7b..dbeef87c541 100644 --- a/api/v1/client/client.go +++ b/api/v1/client/client.go @@ -29,11 +29,10 @@ import ( "net/http" "net/url" - "github.com/sirupsen/logrus" - "github.com/manyminds/api2go/jsonapi" + "github.com/sirupsen/logrus" - v1 "github.com/loadimpact/k6/api/v1" + "github.com/loadimpact/k6/api/v1" ) // Client is a simple HTTP client for the REST API. diff --git a/api/v1/group.go b/api/v1/group.go index 429c5f76cdc..abc2c4dea25 100644 --- a/api/v1/group.go +++ b/api/v1/group.go @@ -21,9 +21,10 @@ package v1 import ( - "github.com/loadimpact/k6/lib" "github.com/manyminds/api2go/jsonapi" "github.com/pkg/errors" + + "github.com/loadimpact/k6/lib" ) type Check struct { diff --git a/api/v1/group_routes.go b/api/v1/group_routes.go index 24d65fda565..375ea2b8f34 100644 --- a/api/v1/group_routes.go +++ b/api/v1/group_routes.go @@ -24,8 +24,9 @@ import ( "net/http" "github.com/julienschmidt/httprouter" - "github.com/loadimpact/k6/api/common" "github.com/manyminds/api2go/jsonapi" + + "github.com/loadimpact/k6/api/common" ) func HandleGetGroups(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { diff --git a/api/v1/group_routes_test.go b/api/v1/group_routes_test.go index bb44adefceb..94894e863a3 100644 --- a/api/v1/group_routes_test.go +++ b/api/v1/group_routes_test.go @@ -26,14 +26,15 @@ import ( "net/http/httptest" "testing" - "github.com/loadimpact/k6/core" - "github.com/loadimpact/k6/core/local" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/core/local" + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils/minirunner" ) func TestGetGroups(t *testing.T) { diff --git a/api/v1/group_test.go b/api/v1/group_test.go index ed767e422fd..695fff47541 100644 --- a/api/v1/group_test.go +++ b/api/v1/group_test.go @@ -23,8 +23,9 @@ package v1 import ( "testing" - "github.com/loadimpact/k6/lib" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/lib" ) func TestNewCheck(t *testing.T) { diff --git a/api/v1/metric.go b/api/v1/metric.go index c71c7e8930b..c1950a898ce 100644 --- a/api/v1/metric.go +++ b/api/v1/metric.go @@ -25,8 +25,9 @@ import ( "encoding/json" "time" - "github.com/loadimpact/k6/stats" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" ) type NullMetricType struct { diff --git a/api/v1/metric_routes.go b/api/v1/metric_routes.go index 6178f69eb6f..f2f43eb8e46 100644 --- a/api/v1/metric_routes.go +++ b/api/v1/metric_routes.go @@ -25,8 +25,9 @@ import ( "time" "github.com/julienschmidt/httprouter" - "github.com/loadimpact/k6/api/common" "github.com/manyminds/api2go/jsonapi" + + "github.com/loadimpact/k6/api/common" ) func HandleGetMetrics(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { diff --git a/api/v1/metric_routes_test.go b/api/v1/metric_routes_test.go index 9b81eb90ef0..8fddda1491b 100644 --- a/api/v1/metric_routes_test.go +++ b/api/v1/metric_routes_test.go @@ -26,18 +26,17 @@ import ( "net/http/httptest" "testing" - "github.com/stretchr/testify/require" - + "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/loadimpact/k6/stats" - "github.com/manyminds/api2go/jsonapi" - "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" ) func TestGetMetrics(t *testing.T) { diff --git a/api/v1/metric_test.go b/api/v1/metric_test.go index 18fbfdeebc7..78f3614bf36 100644 --- a/api/v1/metric_test.go +++ b/api/v1/metric_test.go @@ -24,9 +24,10 @@ import ( "encoding/json" "testing" - "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" ) func TestNullMetricTypeJSON(t *testing.T) { diff --git a/api/v1/routes_test.go b/api/v1/routes_test.go index e289a9916d6..31317d90805 100644 --- a/api/v1/routes_test.go +++ b/api/v1/routes_test.go @@ -26,9 +26,10 @@ import ( "net/http/httptest" "testing" + "github.com/stretchr/testify/assert" + "github.com/loadimpact/k6/api/common" "github.com/loadimpact/k6/core" - "github.com/stretchr/testify/assert" ) func newRequestWithEngine(engine *core.Engine, method, target string, body io.Reader) *http.Request { diff --git a/api/v1/setup_teardown_routes.go b/api/v1/setup_teardown_routes.go index 513115f38ef..7a00bb84c7d 100644 --- a/api/v1/setup_teardown_routes.go +++ b/api/v1/setup_teardown_routes.go @@ -26,8 +26,9 @@ import ( "net/http" "github.com/julienschmidt/httprouter" - "github.com/loadimpact/k6/api/common" "github.com/manyminds/api2go/jsonapi" + + "github.com/loadimpact/k6/api/common" ) // NullSetupData is wrapper around null to satisfy jsonapi diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index d1bd974ac0b..20ba612e45c 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -30,7 +30,11 @@ import ( "testing" "time" + "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" @@ -38,10 +42,6 @@ import ( "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" - "github.com/manyminds/api2go/jsonapi" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" ) func TestSetupData(t *testing.T) { diff --git a/api/v1/status.go b/api/v1/status.go index 187d3e42193..e23934df888 100644 --- a/api/v1/status.go +++ b/api/v1/status.go @@ -21,9 +21,10 @@ package v1 import ( + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/lib" - "gopkg.in/guregu/null.v3" ) type Status struct { diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index f7bc36a5d05..321dba0e713 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -26,14 +26,15 @@ import ( "net/http/httptest" "testing" - "github.com/loadimpact/k6/core" - "github.com/loadimpact/k6/core/local" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/testutils/minirunner" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/core" + "github.com/loadimpact/k6/core/local" + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils/minirunner" ) func TestGetStatus(t *testing.T) { diff --git a/cmd/cloud.go b/cmd/cloud.go index c4dfc5dc3c4..d908799c138 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -32,6 +32,7 @@ import ( "github.com/kelseyhightower/envconfig" "github.com/pkg/errors" + "github.com/sirupsen/logrus" "github.com/spf13/afero" "github.com/spf13/cobra" "github.com/spf13/pflag" @@ -42,8 +43,6 @@ import ( "github.com/loadimpact/k6/stats/cloud" "github.com/loadimpact/k6/ui" "github.com/loadimpact/k6/ui/pb" - - "github.com/sirupsen/logrus" ) const ( diff --git a/cmd/collectors.go b/cmd/collectors.go index 27adce94929..6de6acb4972 100644 --- a/cmd/collectors.go +++ b/cmd/collectors.go @@ -27,6 +27,9 @@ import ( "gopkg.in/guregu/null.v3" "github.com/kelseyhightower/envconfig" + "github.com/pkg/errors" + "github.com/spf13/afero" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/loader" @@ -39,8 +42,6 @@ import ( "github.com/loadimpact/k6/stats/kafka" "github.com/loadimpact/k6/stats/statsd" "github.com/loadimpact/k6/stats/statsd/common" - "github.com/pkg/errors" - "github.com/spf13/afero" ) const ( diff --git a/cmd/common.go b/cmd/common.go index 3a38562f9f6..aa4fcce03f5 100644 --- a/cmd/common.go +++ b/cmd/common.go @@ -25,11 +25,12 @@ import ( "io" "os" - "github.com/loadimpact/k6/lib/types" "github.com/spf13/afero" "github.com/spf13/cobra" "github.com/spf13/pflag" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) // Use these when interacting with fs and writing to terminal, makes a command testable diff --git a/cmd/config.go b/cmd/config.go index 51b9a07ad88..1183031459d 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -31,7 +31,7 @@ import ( "github.com/kelseyhightower/envconfig" "github.com/spf13/afero" "github.com/spf13/pflag" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/executor" diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 70d3788e021..9a1995ba352 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -31,7 +31,7 @@ import ( "github.com/spf13/pflag" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/executor" diff --git a/cmd/convert.go b/cmd/convert.go index 5797109f8da..6c77fd758b3 100644 --- a/cmd/convert.go +++ b/cmd/convert.go @@ -26,10 +26,11 @@ import ( "io/ioutil" "path/filepath" + "github.com/spf13/cobra" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/converter/har" "github.com/loadimpact/k6/lib" - "github.com/spf13/cobra" - null "gopkg.in/guregu/null.v3" ) var ( diff --git a/cmd/convert_test.go b/cmd/convert_test.go index 98d54c401e8..d27c0bd8658 100644 --- a/cmd/convert_test.go +++ b/cmd/convert_test.go @@ -22,12 +22,11 @@ package cmd import ( "bytes" + "io/ioutil" "os" "regexp" "testing" - "io/ioutil" - "github.com/pmezard/go-difflib/difflib" "github.com/spf13/afero" "github.com/stretchr/testify/assert" diff --git a/cmd/inspect.go b/cmd/inspect.go index 8a53f1c5d88..6b0aac06d83 100644 --- a/cmd/inspect.go +++ b/cmd/inspect.go @@ -26,10 +26,11 @@ import ( "fmt" "os" + "github.com/spf13/cobra" + "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" - "github.com/spf13/cobra" ) // inspectCmd represents the resume command diff --git a/cmd/login_cloud.go b/cmd/login_cloud.go index c43545c7847..453743ad5a1 100644 --- a/cmd/login_cloud.go +++ b/cmd/login_cloud.go @@ -23,14 +23,14 @@ package cmd import ( "os" + "github.com/pkg/errors" + "github.com/spf13/afero" + "github.com/spf13/cobra" "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/stats/cloud" "github.com/loadimpact/k6/ui" - "github.com/pkg/errors" - "github.com/spf13/afero" - "github.com/spf13/cobra" ) // loginCloudCommand represents the 'login cloud' command diff --git a/cmd/login_influxdb.go b/cmd/login_influxdb.go index 134e471fdaf..44de72c141c 100644 --- a/cmd/login_influxdb.go +++ b/cmd/login_influxdb.go @@ -24,12 +24,13 @@ import ( "os" "time" - "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats/influxdb" - "github.com/loadimpact/k6/ui" "github.com/mitchellh/mapstructure" "github.com/spf13/afero" "github.com/spf13/cobra" + + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats/influxdb" + "github.com/loadimpact/k6/ui" ) // loginInfluxDBCommand represents the 'login influxdb' command diff --git a/cmd/options.go b/cmd/options.go index cecb90b4351..78480102526 100644 --- a/cmd/options.go +++ b/cmd/options.go @@ -27,7 +27,7 @@ import ( "github.com/pkg/errors" "github.com/spf13/pflag" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" diff --git a/cmd/pause.go b/cmd/pause.go index a15eda226b0..6ecc5066546 100644 --- a/cmd/pause.go +++ b/cmd/pause.go @@ -23,11 +23,12 @@ package cmd import ( "context" + "github.com/spf13/cobra" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/api/v1" "github.com/loadimpact/k6/api/v1/client" "github.com/loadimpact/k6/ui" - "github.com/spf13/cobra" - "gopkg.in/guregu/null.v3" ) // pauseCmd represents the pause command diff --git a/cmd/resume.go b/cmd/resume.go index 93fe3bc0733..2e8183e5f19 100644 --- a/cmd/resume.go +++ b/cmd/resume.go @@ -23,11 +23,12 @@ package cmd import ( "context" + "github.com/spf13/cobra" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/api/v1" "github.com/loadimpact/k6/api/v1/client" "github.com/loadimpact/k6/ui" - "github.com/spf13/cobra" - "gopkg.in/guregu/null.v3" ) // resumeCmd represents the resume command diff --git a/cmd/runtime_options_test.go b/cmd/runtime_options_test.go index 60b8ce983f0..28bcead9a97 100644 --- a/cmd/runtime_options_test.go +++ b/cmd/runtime_options_test.go @@ -26,13 +26,13 @@ import ( "net/url" "testing" + "github.com/spf13/afero" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" - "github.com/spf13/afero" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) type runtimeOptionsTestCase struct { diff --git a/cmd/scale.go b/cmd/scale.go index 23dbc91b0e5..0423bbadcae 100644 --- a/cmd/scale.go +++ b/cmd/scale.go @@ -23,11 +23,12 @@ package cmd import ( "context" + "github.com/pkg/errors" + "github.com/spf13/cobra" + "github.com/loadimpact/k6/api/v1" "github.com/loadimpact/k6/api/v1/client" "github.com/loadimpact/k6/ui" - "github.com/pkg/errors" - "github.com/spf13/cobra" ) // scaleCmd represents the scale command diff --git a/cmd/stats.go b/cmd/stats.go index 2a77aaaa337..54889583448 100644 --- a/cmd/stats.go +++ b/cmd/stats.go @@ -23,9 +23,10 @@ package cmd import ( "context" + "github.com/spf13/cobra" + "github.com/loadimpact/k6/api/v1/client" "github.com/loadimpact/k6/ui" - "github.com/spf13/cobra" ) // statsCmd represents the stats command diff --git a/cmd/status.go b/cmd/status.go index 39910bfd106..e0fa62c0001 100644 --- a/cmd/status.go +++ b/cmd/status.go @@ -23,9 +23,10 @@ package cmd import ( "context" + "github.com/spf13/cobra" + "github.com/loadimpact/k6/api/v1/client" "github.com/loadimpact/k6/ui" - "github.com/spf13/cobra" ) // statusCmd represents the status command diff --git a/cmd/ui.go b/cmd/ui.go index e52636680b3..661bdbb48aa 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -32,9 +32,8 @@ import ( "time" "unicode/utf8" - "golang.org/x/crypto/ssh/terminal" - "github.com/sirupsen/logrus" + "golang.org/x/crypto/ssh/terminal" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" diff --git a/cmd/version.go b/cmd/version.go index 811c44915bf..12e7d1c3ef2 100644 --- a/cmd/version.go +++ b/cmd/version.go @@ -23,8 +23,9 @@ package cmd import ( "fmt" - "github.com/loadimpact/k6/lib/consts" "github.com/spf13/cobra" + + "github.com/loadimpact/k6/lib/consts" ) // versionCmd represents the version command. diff --git a/converter/har/converter.go b/converter/har/converter.go index 14ac7d7f49d..df6b2e1cdff 100644 --- a/converter/har/converter.go +++ b/converter/har/converter.go @@ -30,9 +30,10 @@ import ( "sort" "strings" - "github.com/loadimpact/k6/lib" "github.com/pkg/errors" "github.com/tidwall/pretty" + + "github.com/loadimpact/k6/lib" ) // fprint panics when where's an error writing to the supplied io.Writer diff --git a/converter/har/converter_test.go b/converter/har/converter_test.go index fdf92cce8cc..e285a71b6f8 100644 --- a/converter/har/converter_test.go +++ b/converter/har/converter_test.go @@ -25,10 +25,11 @@ import ( "net/url" "testing" + "github.com/stretchr/testify/assert" + "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" - "github.com/stretchr/testify/assert" ) func TestBuildK6Headers(t *testing.T) { diff --git a/core/engine.go b/core/engine.go index 7db9fb2b5e7..36cd0609bbc 100644 --- a/core/engine.go +++ b/core/engine.go @@ -27,11 +27,12 @@ import ( "sync" "time" + "github.com/sirupsen/logrus" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" - "gopkg.in/guregu/null.v3" ) const ( diff --git a/core/engine_test.go b/core/engine_test.go index 4944c174bff..5e832e19f70 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -28,8 +28,10 @@ import ( "testing" "time" + "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/js" @@ -43,8 +45,6 @@ import ( "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/dummy" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) const isWindows = runtime.GOOS == "windows" diff --git a/core/local/local.go b/core/local/local.go index 0d2543a06dc..d535f63a03b 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -27,12 +27,11 @@ import ( "sync/atomic" "time" - "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" + "github.com/loadimpact/k6/ui/pb" ) // ExecutionScheduler is the local implementation of lib.ExecutionScheduler diff --git a/core/local/local_test.go b/core/local/local_test.go index bd90f856775..9620068836c 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -36,7 +36,7 @@ import ( logtest "github.com/sirupsen/logrus/hooks/test" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/js" "github.com/loadimpact/k6/lib" diff --git a/js/bundle.go b/js/bundle.go index 6a50b9833b6..58717f09e10 100644 --- a/js/bundle.go +++ b/js/bundle.go @@ -26,8 +26,6 @@ import ( "net/url" "runtime" - "github.com/loadimpact/k6/lib/consts" - "github.com/dop251/goja" "github.com/pkg/errors" "github.com/spf13/afero" @@ -36,6 +34,7 @@ import ( "github.com/loadimpact/k6/js/compiler" jslib "github.com/loadimpact/k6/js/lib" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/loader" ) diff --git a/js/bundle_test.go b/js/bundle_test.go index b25d879e6a6..ec8d517be64 100644 --- a/js/bundle_test.go +++ b/js/bundle_test.go @@ -36,7 +36,7 @@ import ( "github.com/spf13/afero" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/consts" diff --git a/js/common/util.go b/js/common/util.go index aa986c0f7e0..dc66e8c954e 100644 --- a/js/common/util.go +++ b/js/common/util.go @@ -22,6 +22,7 @@ package common import ( "github.com/dop251/goja" + "github.com/loadimpact/k6/js/compiler" ) diff --git a/js/compiler/compiler.go b/js/compiler/compiler.go index 2e5b35f7290..934b3fcd80d 100644 --- a/js/compiler/compiler.go +++ b/js/compiler/compiler.go @@ -29,9 +29,10 @@ import ( rice "github.com/GeertJohan/go.rice" "github.com/dop251/goja" "github.com/dop251/goja/parser" - "github.com/loadimpact/k6/lib" "github.com/mitchellh/mapstructure" "github.com/sirupsen/logrus" + + "github.com/loadimpact/k6/lib" ) var ( diff --git a/js/compiler/compiler_test.go b/js/compiler/compiler_test.go index 534703d3191..b8d62c572d1 100644 --- a/js/compiler/compiler_test.go +++ b/js/compiler/compiler_test.go @@ -24,8 +24,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/lib" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/lib" ) func TestTransform(t *testing.T) { diff --git a/js/console_test.go b/js/console_test.go index 2d65af9cbdf..159e329f1ae 100644 --- a/js/console_test.go +++ b/js/console_test.go @@ -33,7 +33,7 @@ import ( logtest "github.com/sirupsen/logrus/hooks/test" "github.com/spf13/afero" "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" diff --git a/js/initcontext.go b/js/initcontext.go index 47c175a19f3..cbd063605eb 100644 --- a/js/initcontext.go +++ b/js/initcontext.go @@ -28,13 +28,14 @@ import ( "strings" "github.com/dop251/goja" + "github.com/pkg/errors" + "github.com/spf13/afero" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/js/compiler" "github.com/loadimpact/k6/js/modules" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/loader" - "github.com/pkg/errors" - "github.com/spf13/afero" ) type programWithSource struct { diff --git a/js/lib/lib.go b/js/lib/lib.go index f3c6adad30f..31cff76071f 100644 --- a/js/lib/lib.go +++ b/js/lib/lib.go @@ -25,7 +25,7 @@ package lib import ( "sync" - "github.com/GeertJohan/go.rice" + rice "github.com/GeertJohan/go.rice" "github.com/dop251/goja" ) diff --git a/js/module_loading_test.go b/js/module_loading_test.go index 57b0f5728c3..a5a3fa7289e 100644 --- a/js/module_loading_test.go +++ b/js/module_loading_test.go @@ -28,7 +28,7 @@ import ( "github.com/spf13/afero" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/testutils/httpmultibin" diff --git a/js/modules/k6/crypto/crypto_test.go b/js/modules/k6/crypto/crypto_test.go index 384f8987dd2..4117df54d50 100644 --- a/js/modules/k6/crypto/crypto_test.go +++ b/js/modules/k6/crypto/crypto_test.go @@ -27,9 +27,10 @@ import ( "testing" "github.com/dop251/goja" + "github.com/stretchr/testify/assert" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" - "github.com/stretchr/testify/assert" ) type MockReader struct{} diff --git a/js/modules/k6/crypto/x509/x509.go b/js/modules/k6/crypto/x509/x509.go index 0fff2c4c95e..28480bd6aa5 100644 --- a/js/modules/k6/crypto/x509/x509.go +++ b/js/modules/k6/crypto/x509/x509.go @@ -32,8 +32,9 @@ import ( "fmt" "time" - "github.com/loadimpact/k6/js/common" "github.com/pkg/errors" + + "github.com/loadimpact/k6/js/common" ) // X509 certificate functionality diff --git a/js/modules/k6/crypto/x509/x509_test.go b/js/modules/k6/crypto/x509/x509_test.go index 18d8712bb31..7f4cce804a6 100644 --- a/js/modules/k6/crypto/x509/x509_test.go +++ b/js/modules/k6/crypto/x509/x509_test.go @@ -27,8 +27,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) func makeRuntime() *goja.Runtime { diff --git a/js/modules/k6/encoding/encoding_test.go b/js/modules/k6/encoding/encoding_test.go index fd8b7de18d4..1ae395916ac 100644 --- a/js/modules/k6/encoding/encoding_test.go +++ b/js/modules/k6/encoding/encoding_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) func TestEncodingAlgorithms(t *testing.T) { diff --git a/js/modules/k6/html/element_test.go b/js/modules/k6/html/element_test.go index 415fc133666..0f0d1704548 100644 --- a/js/modules/k6/html/element_test.go +++ b/js/modules/k6/html/element_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) const testHTMLElem = ` diff --git a/js/modules/k6/html/elements_gen_test.go b/js/modules/k6/html/elements_gen_test.go index a2a0e5534bc..5065755c6ce 100644 --- a/js/modules/k6/html/elements_gen_test.go +++ b/js/modules/k6/html/elements_gen_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) var textTests = []struct { diff --git a/js/modules/k6/html/elements_test.go b/js/modules/k6/html/elements_test.go index 72302831fae..a8aec7e15eb 100644 --- a/js/modules/k6/html/elements_test.go +++ b/js/modules/k6/html/elements_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) const testHTMLElems = ` diff --git a/js/modules/k6/html/html.go b/js/modules/k6/html/html.go index 0eec2eade45..f5efc2e11af 100644 --- a/js/modules/k6/html/html.go +++ b/js/modules/k6/html/html.go @@ -27,9 +27,10 @@ import ( "github.com/PuerkitoBio/goquery" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/pkg/errors" gohtml "golang.org/x/net/html" + + "github.com/loadimpact/k6/js/common" ) type HTML struct{} diff --git a/js/modules/k6/html/html_test.go b/js/modules/k6/html/html_test.go index 6178230c753..52d1bf16296 100644 --- a/js/modules/k6/html/html_test.go +++ b/js/modules/k6/html/html_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) const testHTML = ` diff --git a/js/modules/k6/html/serialize_test.go b/js/modules/k6/html/serialize_test.go index db5deb05051..82b06a9ee1d 100644 --- a/js/modules/k6/html/serialize_test.go +++ b/js/modules/k6/html/serialize_test.go @@ -25,8 +25,9 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/js/common" ) const testSerializeHTML = ` diff --git a/js/modules/k6/http/cookiejar.go b/js/modules/k6/http/cookiejar.go index 59e4beb9a8e..aad7b695390 100644 --- a/js/modules/k6/http/cookiejar.go +++ b/js/modules/k6/http/cookiejar.go @@ -29,8 +29,9 @@ import ( "time" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" "github.com/pkg/errors" + + "github.com/loadimpact/k6/js/common" ) // HTTPCookieJar is cookiejar.Jar wrapper to be used in js scripts diff --git a/js/modules/k6/http/http_test.go b/js/modules/k6/http/http_test.go index 19f895a1fcf..ee1c43f44fd 100644 --- a/js/modules/k6/http/http_test.go +++ b/js/modules/k6/http/http_test.go @@ -24,10 +24,11 @@ import ( "testing" "github.com/dop251/goja" - "github.com/loadimpact/k6/js/common" - "github.com/loadimpact/k6/lib/netext/httpext" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/js/common" + "github.com/loadimpact/k6/lib/netext/httpext" ) func TestTagURL(t *testing.T) { diff --git a/js/modules/k6/http/http_url.go b/js/modules/k6/http/http_url.go index be1c46e1ae9..49ac36cede3 100644 --- a/js/modules/k6/http/http_url.go +++ b/js/modules/k6/http/http_url.go @@ -24,6 +24,7 @@ import ( "fmt" "github.com/dop251/goja" + "github.com/loadimpact/k6/lib/netext/httpext" ) diff --git a/js/modules/k6/http/request.go b/js/modules/k6/http/request.go index 45cbaa6a63e..0f7a0cdd9ad 100644 --- a/js/modules/k6/http/request.go +++ b/js/modules/k6/http/request.go @@ -33,10 +33,11 @@ import ( "time" "github.com/dop251/goja" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/netext/httpext" - null "gopkg.in/guregu/null.v3" ) // ErrHTTPForbiddenInInitContext is used when a http requests was made in the init context diff --git a/js/modules/k6/http/request_test.go b/js/modules/k6/http/request_test.go index de912fbadac..c1b4a04c1de 100644 --- a/js/modules/k6/http/request_test.go +++ b/js/modules/k6/http/request_test.go @@ -40,19 +40,20 @@ import ( "github.com/andybalholm/brotli" "github.com/dop251/goja" "github.com/klauspost/compress/zstd" - "github.com/loadimpact/k6/js/common" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/metrics" - "github.com/loadimpact/k6/lib/testutils" - "github.com/loadimpact/k6/lib/testutils/httpmultibin" - "github.com/loadimpact/k6/stats" "github.com/mccutchen/go-httpbin/httpbin" "github.com/oxtoacart/bpool" "github.com/sirupsen/logrus" logtest "github.com/sirupsen/logrus/hooks/test" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/js/common" + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/httpmultibin" + "github.com/loadimpact/k6/stats" ) func assertRequestMetricsEmitted(t *testing.T, sampleContainers []stats.SampleContainer, method, url, name string, status int, group string) { diff --git a/js/modules/k6/http/response.go b/js/modules/k6/http/response.go index 987347c94ee..8deb1b2462c 100644 --- a/js/modules/k6/http/response.go +++ b/js/modules/k6/http/response.go @@ -27,6 +27,7 @@ import ( "strings" "github.com/dop251/goja" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/js/modules/k6/html" "github.com/loadimpact/k6/lib/netext/httpext" diff --git a/js/modules/k6/http/response_test.go b/js/modules/k6/http/response_test.go index 2d8fd286540..ad752a1332d 100644 --- a/js/modules/k6/http/response_test.go +++ b/js/modules/k6/http/response_test.go @@ -27,10 +27,11 @@ import ( "net/url" "testing" + "github.com/stretchr/testify/assert" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib/netext/httpext" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" ) const testGetFormHTML = ` diff --git a/js/modules/k6/http/tls_test.go b/js/modules/k6/http/tls_test.go index e4416671289..85d3f736135 100644 --- a/js/modules/k6/http/tls_test.go +++ b/js/modules/k6/http/tls_test.go @@ -27,10 +27,11 @@ import ( "net/http" "testing" + "github.com/stretchr/testify/assert" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" - "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" ) func TestTLS13Support(t *testing.T) { diff --git a/js/modules/k6/k6_test.go b/js/modules/k6/k6_test.go index 3be8a357a03..92194e77365 100644 --- a/js/modules/k6/k6_test.go +++ b/js/modules/k6/k6_test.go @@ -27,14 +27,14 @@ import ( "testing" "time" - "github.com/loadimpact/k6/stats" - "github.com/dop251/goja" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/stats" ) func TestFail(t *testing.T) { diff --git a/js/modules/k6/metrics/metrics_test.go b/js/modules/k6/metrics/metrics_test.go index cb1d00f016c..c3fe9a70d5e 100644 --- a/js/modules/k6/metrics/metrics_test.go +++ b/js/modules/k6/metrics/metrics_test.go @@ -26,11 +26,12 @@ import ( "testing" "github.com/dop251/goja" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestMetrics(t *testing.T) { diff --git a/js/modules/k6/ws/ws_test.go b/js/modules/k6/ws/ws_test.go index 04363f22dac..1acf46a90bb 100644 --- a/js/modules/k6/ws/ws_test.go +++ b/js/modules/k6/ws/ws_test.go @@ -31,12 +31,13 @@ import ( "github.com/dop251/goja" "github.com/gorilla/websocket" + "github.com/stretchr/testify/assert" + "github.com/loadimpact/k6/js/common" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" ) func assertSessionMetricsEmitted(t *testing.T, sampleContainers []stats.SampleContainer, subprotocol, url string, status int, group string) { diff --git a/js/runner_test.go b/js/runner_test.go index 6f9f4a70857..095375567ba 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -43,7 +43,7 @@ import ( "github.com/spf13/afero" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" diff --git a/lib/archive.go b/lib/archive.go index 2bb924ca415..0931c9c14d0 100644 --- a/lib/archive.go +++ b/lib/archive.go @@ -36,9 +36,10 @@ import ( "strings" "time" + "github.com/spf13/afero" + "github.com/loadimpact/k6/lib/fsext" "github.com/loadimpact/k6/loader" - "github.com/spf13/afero" ) //nolint: gochecknoglobals, lll diff --git a/lib/archive_test.go b/lib/archive_test.go index 3bcdbd55273..740927ee345 100644 --- a/lib/archive_test.go +++ b/lib/archive_test.go @@ -30,14 +30,14 @@ import ( "runtime" "testing" - "github.com/loadimpact/k6/lib/consts" - "github.com/loadimpact/k6/lib/fsext" - "github.com/loadimpact/k6/stats" - "github.com/spf13/afero" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/consts" + "github.com/loadimpact/k6/lib/fsext" + "github.com/loadimpact/k6/stats" ) func TestNormalizeAndAnonymizePath(t *testing.T) { diff --git a/lib/execution.go b/lib/execution.go index d8f4b73eb02..8a0a3666bec 100644 --- a/lib/execution.go +++ b/lib/execution.go @@ -27,9 +27,8 @@ import ( "sync/atomic" "time" - "github.com/sirupsen/logrus" - "github.com/pkg/errors" + "github.com/sirupsen/logrus" "github.com/loadimpact/k6/stats" ) diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index 86ecc92e1d4..c9405a03a8a 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -26,7 +26,7 @@ import ( "strings" "time" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib/consts" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index fb7a7df6ed7..245d92a5f10 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -30,7 +30,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 355e2bdbf94..9569c2d9675 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -31,7 +31,7 @@ import ( "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 8b618149c38..9f21a0deb5a 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -27,7 +27,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go index d45751e0ecd..f209009553c 100644 --- a/lib/executor/constant_looping_vus_test.go +++ b/lib/executor/constant_looping_vus_test.go @@ -28,7 +28,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index b0a63f24ddb..907cb9cd52f 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -21,10 +21,11 @@ package executor import ( + "github.com/sirupsen/logrus" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" - "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" ) // ExecutionConflictError is a custom error type used for all of the errors in diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 3a6bfee3dcb..2e3457133d4 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -26,11 +26,12 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/types" ) type exp struct { diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 7f515e2dcec..123def5d650 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -30,7 +30,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index fde53426880..ae5f4c6deaf 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -27,10 +27,9 @@ import ( "testing" "time" - null "gopkg.in/guregu/null.v3" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index b9e352ac988..d3e8e1d503b 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -26,12 +26,11 @@ import ( "math/big" "time" - "github.com/loadimpact/k6/ui/pb" - "github.com/sirupsen/logrus" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/ui/pb" ) func sumStagesDuration(stages []Stage) (result time.Duration) { diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 35a25e659ac..6a06167d3f3 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -28,7 +28,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 45d6b63b2db..7811a811ee6 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -29,7 +29,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 3d948d77891..92ea27f5955 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -28,7 +28,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index 61f1a448103..f0b7cd12fcd 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -29,7 +29,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index ff5a64f1370..f41f541a453 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -29,7 +29,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index c90622f60cf..1c7c8616f6d 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -32,7 +32,7 @@ import ( "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 2fd7070d579..bb9dd7fcfcf 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -28,7 +28,7 @@ import ( "time" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index c67794b6fbf..cf28bcddbd4 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -32,7 +32,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/types" diff --git a/lib/models.go b/lib/models.go index 3944eeecba6..e7b69b29183 100644 --- a/lib/models.go +++ b/lib/models.go @@ -29,9 +29,10 @@ import ( "sync" "time" - "github.com/loadimpact/k6/lib/types" "github.com/pkg/errors" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) // Separator for group IDs. diff --git a/lib/models_test.go b/lib/models_test.go index 85d020fd3b8..5211bc9eb25 100644 --- a/lib/models_test.go +++ b/lib/models_test.go @@ -26,9 +26,10 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) func TestStageJSON(t *testing.T) { diff --git a/lib/netext/dialer.go b/lib/netext/dialer.go index cecc4e735a7..aed10c4fd27 100644 --- a/lib/netext/dialer.go +++ b/lib/netext/dialer.go @@ -28,11 +28,11 @@ import ( "sync/atomic" "time" + "github.com/viki-org/dnscache" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/stats" - - "github.com/viki-org/dnscache" ) // Dialer wraps net.Dialer and provides k6 specific functionality - diff --git a/lib/netext/httpext/error_codes.go b/lib/netext/httpext/error_codes.go index b240d6841c2..1d771629d34 100644 --- a/lib/netext/httpext/error_codes.go +++ b/lib/netext/httpext/error_codes.go @@ -30,9 +30,10 @@ import ( "runtime" "syscall" - "github.com/loadimpact/k6/lib/netext" "github.com/pkg/errors" "golang.org/x/net/http2" + + "github.com/loadimpact/k6/lib/netext" ) // TODO: maybe rename the type errorCode, so we can have errCode variables? and diff --git a/lib/netext/httpext/error_codes_test.go b/lib/netext/httpext/error_codes_test.go index 76ab824a5a8..50c3d76e99d 100644 --- a/lib/netext/httpext/error_codes_test.go +++ b/lib/netext/httpext/error_codes_test.go @@ -31,10 +31,11 @@ import ( "syscall" "testing" - "github.com/loadimpact/k6/lib/netext" "github.com/pkg/errors" "github.com/stretchr/testify/require" "golang.org/x/net/http2" + + "github.com/loadimpact/k6/lib/netext" ) func TestDefaultError(t *testing.T) { diff --git a/lib/netext/httpext/request.go b/lib/netext/httpext/request.go index 9a9af70ae34..eeace3a0ba4 100644 --- a/lib/netext/httpext/request.go +++ b/lib/netext/httpext/request.go @@ -36,7 +36,7 @@ import ( "github.com/Azure/go-ntlmssp" "github.com/sirupsen/logrus" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" diff --git a/lib/netext/httpext/request_test.go b/lib/netext/httpext/request_test.go index 2ee74249bac..a0e11dfb94d 100644 --- a/lib/netext/httpext/request_test.go +++ b/lib/netext/httpext/request_test.go @@ -31,12 +31,13 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/stats" "github.com/pkg/errors" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" ) type reader func([]byte) (int, error) diff --git a/lib/netext/httpext/response.go b/lib/netext/httpext/response.go index 7aa921bd42f..11189901094 100644 --- a/lib/netext/httpext/response.go +++ b/lib/netext/httpext/response.go @@ -26,9 +26,10 @@ import ( "encoding/json" "fmt" - "github.com/loadimpact/k6/lib/netext" "github.com/pkg/errors" "github.com/tidwall/gjson" + + "github.com/loadimpact/k6/lib/netext" ) // ResponseType is used in the request to specify how the response body should be treated diff --git a/lib/netext/httpext/tracer_test.go b/lib/netext/httpext/tracer_test.go index 75b3d12fbe0..451bc239c31 100644 --- a/lib/netext/httpext/tracer_test.go +++ b/lib/netext/httpext/tracer_test.go @@ -35,13 +35,14 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib/metrics" - "github.com/loadimpact/k6/lib/netext" - "github.com/loadimpact/k6/stats" "github.com/mccutchen/go-httpbin/httpbin" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/metrics" + "github.com/loadimpact/k6/lib/netext" + "github.com/loadimpact/k6/stats" ) func TestTracer(t *testing.T) { diff --git a/lib/netext/tls.go b/lib/netext/tls.go index 223421ba747..de92a785fea 100644 --- a/lib/netext/tls.go +++ b/lib/netext/tls.go @@ -23,8 +23,9 @@ package netext import ( "crypto/tls" - "github.com/loadimpact/k6/lib" "golang.org/x/crypto/ocsp" + + "github.com/loadimpact/k6/lib" ) //nolint: golint diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index af2cdc88aa9..9c262495e05 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -29,9 +29,10 @@ import ( "path/filepath" "testing" - "github.com/loadimpact/k6/lib/fsext" "github.com/spf13/afero" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/fsext" ) func dumpMemMapFsToBuf(fs afero.Fs) (*bytes.Buffer, error) { diff --git a/lib/options_test.go b/lib/options_test.go index 76a6419fae4..1ead264d9a2 100644 --- a/lib/options_test.go +++ b/lib/options_test.go @@ -30,12 +30,13 @@ import ( "time" "github.com/kelseyhightower/envconfig" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib/testutils" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" ) func TestOptions(t *testing.T) { diff --git a/lib/runtime_options.go b/lib/runtime_options.go index 4d7e4f04617..b39d84ff3c6 100644 --- a/lib/runtime_options.go +++ b/lib/runtime_options.go @@ -24,7 +24,7 @@ import ( "fmt" "strings" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) // CompatibilityMode specifies the JS compatibility mode diff --git a/lib/testutils/httpmultibin/httpmultibin.go b/lib/testutils/httpmultibin/httpmultibin.go index e905997cdd4..3feb268bea2 100644 --- a/lib/testutils/httpmultibin/httpmultibin.go +++ b/lib/testutils/httpmultibin/httpmultibin.go @@ -40,13 +40,14 @@ import ( "github.com/andybalholm/brotli" "github.com/gorilla/websocket" "github.com/klauspost/compress/zstd" - "github.com/loadimpact/k6/lib/netext" - "github.com/loadimpact/k6/lib/netext/httpext" "github.com/mccutchen/go-httpbin/httpbin" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/net/http2" + + "github.com/loadimpact/k6/lib/netext" + "github.com/loadimpact/k6/lib/netext/httpext" ) // GetTLSClientConfig returns a TLS config that trusts the supplied diff --git a/lib/types/types.go b/lib/types/types.go index 158ea2744b0..e9ffeab037f 100644 --- a/lib/types/types.go +++ b/lib/types/types.go @@ -29,7 +29,7 @@ import ( "strings" "time" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) // NullDecoder converts data with expected type f to a guregu/null value diff --git a/lib/types/types_test.go b/lib/types/types_test.go index d7975c19646..d54f6c474c8 100644 --- a/lib/types/types_test.go +++ b/lib/types/types_test.go @@ -30,7 +30,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) func TestNullDecoder(t *testing.T) { diff --git a/loader/filesystems.go b/loader/filesystems.go index 6baabc852f5..e9e28a0ba38 100644 --- a/loader/filesystems.go +++ b/loader/filesystems.go @@ -23,8 +23,9 @@ package loader import ( "runtime" - "github.com/loadimpact/k6/lib/fsext" "github.com/spf13/afero" + + "github.com/loadimpact/k6/lib/fsext" ) // CreateFilesystems creates the correct filesystem map for the current OS diff --git a/loader/loader_test.go b/loader/loader_test.go index e01ef239043..e6e1ef733aa 100644 --- a/loader/loader_test.go +++ b/loader/loader_test.go @@ -27,11 +27,12 @@ import ( "path/filepath" "testing" - "github.com/loadimpact/k6/lib/testutils/httpmultibin" - "github.com/loadimpact/k6/loader" "github.com/spf13/afero" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/testutils/httpmultibin" + "github.com/loadimpact/k6/loader" ) func TestDir(t *testing.T) { diff --git a/loader/readsource.go b/loader/readsource.go index 89f01ffabe2..bc97326225b 100644 --- a/loader/readsource.go +++ b/loader/readsource.go @@ -26,9 +26,10 @@ import ( "net/url" "path/filepath" - "github.com/loadimpact/k6/lib/fsext" "github.com/pkg/errors" "github.com/spf13/afero" + + "github.com/loadimpact/k6/lib/fsext" ) // ReadSource Reads a source file from any supported destination. diff --git a/loader/readsource_test.go b/loader/readsource_test.go index f3a5fad83d5..72c319ec3c0 100644 --- a/loader/readsource_test.go +++ b/loader/readsource_test.go @@ -26,10 +26,11 @@ import ( "net/url" "testing" - "github.com/loadimpact/k6/lib/fsext" "github.com/pkg/errors" "github.com/spf13/afero" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/fsext" ) type errorReader string diff --git a/stats/cloud/api.go b/stats/cloud/api.go index 684bf45a846..adc0ab8885f 100644 --- a/stats/cloud/api.go +++ b/stats/cloud/api.go @@ -29,8 +29,9 @@ import ( "net/http" "strconv" - "github.com/loadimpact/k6/lib" "github.com/pkg/errors" + + "github.com/loadimpact/k6/lib" ) type ResultStatus int diff --git a/stats/cloud/api_test.go b/stats/cloud/api_test.go index 3a7f234a5b1..840be9cda99 100644 --- a/stats/cloud/api_test.go +++ b/stats/cloud/api_test.go @@ -33,9 +33,10 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/types" ) func init() { diff --git a/stats/cloud/bench_test.go b/stats/cloud/bench_test.go index 3d42c86edbe..72b62e383fd 100644 --- a/stats/cloud/bench_test.go +++ b/stats/cloud/bench_test.go @@ -31,16 +31,16 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/netext/httpext" "github.com/loadimpact/k6/lib/testutils/httpmultibin" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/loader" "github.com/loadimpact/k6/stats" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "gopkg.in/guregu/null.v3" ) // script to clean the logs: `perl -p -e "s/time=\".*\n//g"` diff --git a/stats/cloud/collector_test.go b/stats/cloud/collector_test.go index d088e29ef56..89aa7548878 100644 --- a/stats/cloud/collector_test.go +++ b/stats/cloud/collector_test.go @@ -33,11 +33,9 @@ import ( "testing" "time" - "gopkg.in/guregu/null.v3" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/lib/metrics" diff --git a/stats/cloud/config.go b/stats/cloud/config.go index 238c9b54b33..73a8e0cdc5f 100644 --- a/stats/cloud/config.go +++ b/stats/cloud/config.go @@ -23,8 +23,9 @@ package cloud import ( "time" - "github.com/loadimpact/k6/lib/types" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) // Config holds all the necessary data and options for sending metrics to the Load Impact cloud. diff --git a/stats/cloud/data_test.go b/stats/cloud/data_test.go index 53486e6d601..eb9dda7de64 100644 --- a/stats/cloud/data_test.go +++ b/stats/cloud/data_test.go @@ -27,11 +27,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/netext/httpext" "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestTimestampMarshaling(t *testing.T) { diff --git a/stats/csv/collector.go b/stats/csv/collector.go index 0f27768da5a..4669d959012 100644 --- a/stats/csv/collector.go +++ b/stats/csv/collector.go @@ -31,10 +31,11 @@ import ( "sync" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/stats" "github.com/sirupsen/logrus" "github.com/spf13/afero" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" ) // Collector saving output to csv implements the lib.Collector interface diff --git a/stats/csv/collector_test.go b/stats/csv/collector_test.go index 149eb714d43..5520d0f89a6 100644 --- a/stats/csv/collector_test.go +++ b/stats/csv/collector_test.go @@ -28,13 +28,12 @@ import ( "testing" "time" + "github.com/spf13/afero" + "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" - - "github.com/spf13/afero" - "github.com/stretchr/testify/assert" ) func TestMakeHeader(t *testing.T) { diff --git a/stats/csv/config.go b/stats/csv/config.go index 87b5f626ae0..95914d2a749 100644 --- a/stats/csv/config.go +++ b/stats/csv/config.go @@ -25,8 +25,9 @@ import ( "strings" "time" - "github.com/loadimpact/k6/lib/types" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) // Config is the config for the csv collector diff --git a/stats/datadog/collector_test.go b/stats/datadog/collector_test.go index 83bcce5a4ea..e1ff04e42f6 100644 --- a/stats/datadog/collector_test.go +++ b/stats/datadog/collector_test.go @@ -24,10 +24,11 @@ import ( "strings" "testing" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/statsd/common" "github.com/loadimpact/k6/stats/statsd/common/testutil" - "github.com/stretchr/testify/require" ) func TestCollector(t *testing.T) { diff --git a/stats/influxdb/collector.go b/stats/influxdb/collector.go index 86f8430d3d7..ac32db97643 100644 --- a/stats/influxdb/collector.go +++ b/stats/influxdb/collector.go @@ -27,9 +27,10 @@ import ( "time" client "github.com/influxdata/influxdb1-client/v2" + "github.com/sirupsen/logrus" + "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" - "github.com/sirupsen/logrus" ) // Verify that Collector implements lib.Collector diff --git a/stats/influxdb/collector_test.go b/stats/influxdb/collector_test.go index 92692426bd3..42b33f8d8a2 100644 --- a/stats/influxdb/collector_test.go +++ b/stats/influxdb/collector_test.go @@ -30,9 +30,10 @@ import ( "testing" "time" - "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" ) func TestBadConcurrentWrites(t *testing.T) { diff --git a/stats/influxdb/config.go b/stats/influxdb/config.go index 794a4ae43a0..fe4416bb06b 100644 --- a/stats/influxdb/config.go +++ b/stats/influxdb/config.go @@ -27,10 +27,11 @@ import ( "time" "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" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" ) type Config struct { diff --git a/stats/influxdb/config_test.go b/stats/influxdb/config_test.go index 115831eedf2..773815fb1c0 100644 --- a/stats/influxdb/config_test.go +++ b/stats/influxdb/config_test.go @@ -24,7 +24,7 @@ import ( "testing" "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) func TestParseArg(t *testing.T) { diff --git a/stats/influxdb/util.go b/stats/influxdb/util.go index eeaa31cf4c9..172c008d7dd 100644 --- a/stats/influxdb/util.go +++ b/stats/influxdb/util.go @@ -24,7 +24,7 @@ import ( "strings" client "github.com/influxdata/influxdb1-client/v2" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) func MakeClient(conf Config) (client.Client, error) { diff --git a/stats/influxdb/util_test.go b/stats/influxdb/util_test.go index 29bef1b7e0f..2653752d35b 100644 --- a/stats/influxdb/util_test.go +++ b/stats/influxdb/util_test.go @@ -25,7 +25,7 @@ import ( client "github.com/influxdata/influxdb1-client/v2" "github.com/stretchr/testify/assert" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" ) func TestMakeBatchConfig(t *testing.T) { diff --git a/stats/json/wrapper_test.go b/stats/json/wrapper_test.go index 490719533d0..c50e8dadc7a 100644 --- a/stats/json/wrapper_test.go +++ b/stats/json/wrapper_test.go @@ -23,8 +23,9 @@ package json import ( "testing" - "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/stats" ) func TestWrapersWithNilArg(t *testing.T) { diff --git a/stats/kafka/collector_test.go b/stats/kafka/collector_test.go index 0452857950c..1db7eae0b8b 100644 --- a/stats/kafka/collector_test.go +++ b/stats/kafka/collector_test.go @@ -26,9 +26,10 @@ import ( "testing" "github.com/Shopify/sarama" - "github.com/loadimpact/k6/stats" "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" ) func TestRun(t *testing.T) { diff --git a/stats/kafka/config.go b/stats/kafka/config.go index 49941186d61..9d391155e5e 100644 --- a/stats/kafka/config.go +++ b/stats/kafka/config.go @@ -24,10 +24,11 @@ import ( "time" "github.com/kubernetes/helm/pkg/strvals" - "github.com/loadimpact/k6/lib/types" - "github.com/loadimpact/k6/stats/influxdb" "github.com/mitchellh/mapstructure" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats/influxdb" ) // Config is the config for the kafka collector diff --git a/stats/kafka/config_test.go b/stats/kafka/config_test.go index cbced1f7e5d..e9bba9ff637 100644 --- a/stats/kafka/config_test.go +++ b/stats/kafka/config_test.go @@ -23,9 +23,10 @@ package kafka import ( "testing" - "github.com/loadimpact/k6/stats/influxdb" "github.com/stretchr/testify/assert" "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats/influxdb" ) func TestConfigParseArg(t *testing.T) { diff --git a/stats/statsd/collector_test.go b/stats/statsd/collector_test.go index 33665176923..10745cc70f0 100644 --- a/stats/statsd/collector_test.go +++ b/stats/statsd/collector_test.go @@ -23,9 +23,10 @@ package statsd import ( "testing" + "github.com/stretchr/testify/require" + "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/statsd/common/testutil" - "github.com/stretchr/testify/require" ) func TestCollector(t *testing.T) { diff --git a/stats/statsd/common/collector_test.go b/stats/statsd/common/collector_test.go index 58c2f787a7c..2b94d468862 100644 --- a/stats/statsd/common/collector_test.go +++ b/stats/statsd/common/collector_test.go @@ -23,10 +23,10 @@ package common import ( "testing" - "github.com/loadimpact/k6/stats" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" + "gopkg.in/guregu/null.v3" + + "github.com/loadimpact/k6/stats" ) func TestInitWithoutAddressErrors(t *testing.T) { diff --git a/stats/statsd/common/config.go b/stats/statsd/common/config.go index 5d7518bed58..a640f61568b 100644 --- a/stats/statsd/common/config.go +++ b/stats/statsd/common/config.go @@ -23,8 +23,9 @@ package common import ( "time" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib/types" - null "gopkg.in/guregu/null.v3" ) // Config defines the statsd configuration diff --git a/stats/statsd/common/testutil/test_helper.go b/stats/statsd/common/testutil/test_helper.go index b5bb55c8545..85b43f0adc1 100644 --- a/stats/statsd/common/testutil/test_helper.go +++ b/stats/statsd/common/testutil/test_helper.go @@ -26,11 +26,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" + "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/stats/statsd/common" - "github.com/stretchr/testify/require" - null "gopkg.in/guregu/null.v3" ) // BaseTest is a helper function to test statsd/datadog collector throughtly diff --git a/stats/thresholds_test.go b/stats/thresholds_test.go index 127407dbc6f..5c91dccf8de 100644 --- a/stats/thresholds_test.go +++ b/stats/thresholds_test.go @@ -26,8 +26,9 @@ import ( "time" "github.com/dop251/goja" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" + + "github.com/loadimpact/k6/lib/types" ) func TestNewThreshold(t *testing.T) { diff --git a/ui/pb/helpers_test.go b/ui/pb/helpers_test.go index 7d07b1636db..eb8f58a68d9 100644 --- a/ui/pb/helpers_test.go +++ b/ui/pb/helpers_test.go @@ -27,9 +27,10 @@ import ( "testing" "time" - "github.com/loadimpact/k6/lib/types" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/loadimpact/k6/lib/types" ) func TestGetFixedLengthInt(t *testing.T) { diff --git a/ui/summary.go b/ui/summary.go index 8ff03d929c1..dc981d8cc43 100644 --- a/ui/summary.go +++ b/ui/summary.go @@ -29,10 +29,11 @@ import ( "strings" "time" - "github.com/loadimpact/k6/lib" - "github.com/loadimpact/k6/stats" "github.com/pkg/errors" "golang.org/x/text/unicode/norm" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" ) const ( From c1687e0ee2fa5110a09e5254a3850f88c46a8902 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 29 May 2020 14:23:43 +0300 Subject: [PATCH 274/350] Make certain that even sneakily open can't be used outside the init context (#1475) --- js/initcontext.go | 9 ++++++++- js/runner.go | 8 ++++---- js/runner_test.go | 19 ++++++++++++++++++- 3 files changed, 30 insertions(+), 6 deletions(-) diff --git a/js/initcontext.go b/js/initcontext.go index cbd063605eb..a5a2823f5ef 100644 --- a/js/initcontext.go +++ b/js/initcontext.go @@ -44,6 +44,9 @@ type programWithSource struct { module *goja.Object } +const openCantBeUsedOutsideInitContextMsg = `The "open()" function is only available in the init stage ` + + `(i.e. the global scope), see https://k6.io/docs/using-k6/test-life-cycle for more information` + // InitContext provides APIs for use in the init context. type InitContext struct { // Bound runtime; used to instantiate objects. @@ -190,7 +193,11 @@ func (i *InitContext) compileImport(src, filename string) (*goja.Program, error) } // Open implements open() in the init context and will read and return the contents of a file -func (i *InitContext) Open(filename string, args ...string) (goja.Value, error) { +func (i *InitContext) Open(ctx context.Context, filename string, args ...string) (goja.Value, error) { + if lib.GetState(ctx) != nil { + return nil, errors.New(openCantBeUsedOutsideInitContextMsg) + } + if filename == "" { return nil, errors.New("open() can't be used with an empty filename") } diff --git a/js/runner.go b/js/runner.go index d0f95f8ee74..33a2f83b40e 100644 --- a/js/runner.go +++ b/js/runner.go @@ -197,12 +197,12 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) + + // This is here mostly so if someone tries they get a nice message + // instead of "Value is not an object: undefined ..." common.BindToGlobal(vu.Runtime, map[string]interface{}{ "open": func() { - common.Throw(vu.Runtime, errors.New( - `The "open()" function is only available to init code (aka the global scope), see `+ - ` https://k6.io/docs/using-k6/test-life-cycle for more information`, - )) + common.Throw(vu.Runtime, errors.New(openCantBeUsedOutsideInitContextMsg)) }, }) diff --git a/js/runner_test.go b/js/runner_test.go index 095375567ba..9b914cfdf23 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -1029,7 +1029,24 @@ func TestVUIntegrationOpenFunctionError(t *testing.T) { vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) err = vu.RunOnce() assert.Error(t, err) - assert.Contains(t, err.Error(), "only available to init code") + assert.Contains(t, err.Error(), "only available in the init stage") +} + +func TestVUIntegrationOpenFunctionErrorWhenSneaky(t *testing.T) { + r, err := getSimpleRunner("/script.js", ` + var sneaky = open; + exports.default = function() { sneaky("/tmp/foo") } + `) + assert.NoError(t, err) + + initVU, err := r.NewVU(1, make(chan stats.SampleContainer, 100)) + assert.NoError(t, err) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) + err = vu.RunOnce() + assert.Error(t, err) + assert.Contains(t, err.Error(), "only available in the init stage") } func TestVUIntegrationCookiesReset(t *testing.T) { From 33d65e94b65ed40639d388cc1ff09308a2299d51 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 1 Jun 2020 16:38:07 +0300 Subject: [PATCH 275/350] Dont reset startTime, as it can be racy and it wouldn't really matter --- lib/executor/constant_arrival_rate.go | 1 - 1 file changed, 1 deletion(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 245d92a5f10..45a2699b5e3 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -290,7 +290,6 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC remainingUnplannedVUs := maxVUs - preAllocatedVUs start, offsets, _ := car.et.GetStripedOffsets() - startTime = time.Now() timer := time.NewTimer(time.Hour * 24) // here the we need the not scaled one notScaledTickerPeriod := time.Duration( From 0e5e3546dbc652c1b0ed7f64fe5c64394b855fc7 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 2 Jun 2020 18:13:23 +0300 Subject: [PATCH 276/350] Fix potential data races in Engine tests --- core/engine_test.go | 67 +++++++++++++++++++++------------------------ 1 file changed, 31 insertions(+), 36 deletions(-) diff --git a/core/engine_test.go b/core/engine_test.go index 5e832e19f70..4d7e992aa47 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -51,7 +51,7 @@ const isWindows = runtime.GOOS == "windows" // Wrapper around NewEngine that applies a logger and manages the options. func newTestEngine( //nolint:golint - t *testing.T, runCtx context.Context, runner lib.Runner, opts lib.Options, + t *testing.T, runCtx context.Context, runner lib.Runner, collectors []lib.Collector, opts lib.Options, ) (engine *Engine, run func() error, wait func()) { if runner == nil { runner = &minirunner.MiniRunner{} @@ -79,6 +79,8 @@ func newTestEngine( //nolint:golint engine, err = NewEngine(execScheduler, opts, logger) require.NoError(t, err) + engine.Collectors = collectors + run, waitFn, err := engine.Init(globalCtx, runCtx) require.NoError(t, err) @@ -92,7 +94,7 @@ func newTestEngine( //nolint:golint } func TestNewEngine(t *testing.T) { - newTestEngine(t, nil, nil, lib.Options{}) + newTestEngine(t, nil, nil, nil, lib.Options{}) } func TestEngineRun(t *testing.T) { @@ -109,7 +111,7 @@ func TestEngineRun(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), duration) defer cancel() - _, run, wait := newTestEngine(t, ctx, runner, lib.Options{}) + _, run, wait := newTestEngine(t, ctx, runner, nil, lib.Options{}) defer wait() startTime := time.Now() @@ -118,7 +120,7 @@ func TestEngineRun(t *testing.T) { <-done }) t.Run("exits with executor", func(t *testing.T) { - e, run, wait := newTestEngine(t, nil, nil, lib.Options{ + e, run, wait := newTestEngine(t, nil, nil, nil, lib.Options{ VUs: null.IntFrom(10), Iterations: null.IntFrom(100), }) @@ -140,15 +142,14 @@ func TestEngineRun(t *testing.T) { return nil }} + c := &dummy.Collector{} + ctx, cancel := context.WithCancel(context.Background()) - e, run, wait := newTestEngine(t, ctx, runner, lib.Options{ + _, run, wait := newTestEngine(t, ctx, runner, []lib.Collector{c}, lib.Options{ VUs: null.IntFrom(1), Iterations: null.IntFrom(1), }) - c := &dummy.Collector{} - e.Collectors = []lib.Collector{c} - errC := make(chan error) go func() { errC <- run() }() <-signalChan @@ -171,7 +172,7 @@ func TestEngineRun(t *testing.T) { func TestEngineAtTime(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) defer cancel() - _, run, wait := newTestEngine(t, ctx, nil, lib.Options{ + _, run, wait := newTestEngine(t, ctx, nil, nil, lib.Options{ VUs: null.IntFrom(2), Duration: types.NullDurationFrom(20 * time.Second), }) @@ -183,7 +184,7 @@ func TestEngineAtTime(t *testing.T) { func TestEngineStopped(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 50*time.Millisecond) defer cancel() - e, run, wait := newTestEngine(t, ctx, nil, lib.Options{ + e, run, wait := newTestEngine(t, ctx, nil, nil, lib.Options{ VUs: null.IntFrom(1), Duration: types.NullDurationFrom(20 * time.Second), }) @@ -204,10 +205,11 @@ func TestEngineCollector(t *testing.T) { return nil }} - e, run, wait := newTestEngine(t, nil, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) - c := &dummy.Collector{} - e.Collectors = []lib.Collector{c} + e, run, wait := newTestEngine(t, nil, runner, []lib.Collector{c}, lib.Options{ + VUs: null.IntFrom(1), + Iterations: null.IntFrom(1), + }) assert.NoError(t, run()) wait() @@ -233,7 +235,7 @@ func TestEngine_processSamples(t *testing.T) { metric := stats.New("my_metric", stats.Gauge) t.Run("metric", func(t *testing.T) { - e, _, wait := newTestEngine(t, nil, nil, lib.Options{}) + e, _, wait := newTestEngine(t, nil, nil, nil, lib.Options{}) defer wait() e.processSamples( @@ -246,7 +248,7 @@ func TestEngine_processSamples(t *testing.T) { ths, err := stats.NewThresholds([]string{`1+1==2`}) assert.NoError(t, err) - e, _, wait := newTestEngine(t, nil, nil, lib.Options{ + e, _, wait := newTestEngine(t, nil, nil, nil, lib.Options{ Thresholds: map[string]stats.Thresholds{ "my_metric{a:1}": ths, }, @@ -276,7 +278,7 @@ func TestEngineThresholdsWillAbort(t *testing.T) { thresholds := map[string]stats.Thresholds{metric.Name: ths} - e, _, wait := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + e, _, wait := newTestEngine(t, nil, nil, nil, lib.Options{Thresholds: thresholds}) defer wait() e.processSamples( @@ -302,7 +304,7 @@ func TestEngineAbortedByThresholds(t *testing.T) { return nil }} - _, run, wait := newTestEngine(t, nil, runner, lib.Options{Thresholds: thresholds}) + _, run, wait := newTestEngine(t, nil, runner, nil, lib.Options{Thresholds: thresholds}) defer wait() go func() { @@ -346,7 +348,7 @@ func TestEngine_processThresholds(t *testing.T) { thresholds[m] = ths } - e, _, wait := newTestEngine(t, nil, nil, lib.Options{Thresholds: thresholds}) + e, _, wait := newTestEngine(t, nil, nil, nil, lib.Options{Thresholds: thresholds}) defer wait() e.processSamples( @@ -445,7 +447,8 @@ func TestSentReceivedMetrics(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, r, lib.Options{ + collector := &dummy.Collector{} + _, run, wait := newTestEngine(t, nil, r, []lib.Collector{collector}, lib.Options{ Iterations: null.IntFrom(tc.Iterations), VUs: null.IntFrom(tc.VUs), Hosts: tb.Dialer.Hosts, @@ -454,9 +457,6 @@ func TestSentReceivedMetrics(t *testing.T) { Batch: null.IntFrom(20), }) - collector := &dummy.Collector{} - engine.Collectors = []lib.Collector{collector} - errC := make(chan error) go func() { errC <- run() }() @@ -580,7 +580,8 @@ func TestRunTags(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, r, lib.Options{ + collector := &dummy.Collector{} + _, run, wait := newTestEngine(t, nil, r, []lib.Collector{collector}, lib.Options{ Iterations: null.IntFrom(3), VUs: null.IntFrom(2), Hosts: tb.Dialer.Hosts, @@ -589,9 +590,6 @@ func TestRunTags(t *testing.T) { InsecureSkipTLSVerify: null.BoolFrom(true), }) - collector := &dummy.Collector{} - engine.Collectors = []lib.Collector{collector} - errC := make(chan error) go func() { errC <- run() }() @@ -673,7 +671,7 @@ func TestSetupTeardownThresholds(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, runner, lib.Options{ + engine, run, wait := newTestEngine(t, nil, runner, nil, lib.Options{ SystemTags: &stats.DefaultSystemTagSet, SetupTimeout: types.NullDurationFrom(3 * time.Second), TeardownTimeout: types.NullDurationFrom(3 * time.Second), @@ -736,10 +734,8 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, runner, lib.Options{}) - collector := &dummy.Collector{} - engine.Collectors = []lib.Collector{collector} + engine, run, wait := newTestEngine(t, nil, runner, []lib.Collector{collector}, lib.Options{}) errC := make(chan error) go func() { errC <- run() }() @@ -842,10 +838,8 @@ func TestMetricsEmission(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) - collector := &dummy.Collector{} - engine.Collectors = []lib.Collector{collector} + engine, run, wait := newTestEngine(t, nil, runner, []lib.Collector{collector}, runner.GetOptions()) errC := make(chan error) go func() { errC <- run() }() @@ -929,7 +923,7 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { ) require.NoError(t, err) - engine, run, wait := newTestEngine(t, nil, runner, runner.GetOptions()) + engine, run, wait := newTestEngine(t, nil, runner, nil, runner.GetOptions()) errC := make(chan error) go func() { errC <- run() }() @@ -961,9 +955,10 @@ func TestEngineRunsTeardownEvenAfterTestRunIsAborted(t *testing.T) { }, } - e, run, wait := newTestEngine(t, ctx, runner, lib.Options{VUs: null.IntFrom(1), Iterations: null.IntFrom(1)}) c := &dummy.Collector{} - e.Collectors = []lib.Collector{c} + _, run, wait := newTestEngine(t, ctx, runner, []lib.Collector{c}, lib.Options{ + VUs: null.IntFrom(1), Iterations: null.IntFrom(1), + }) assert.NoError(t, run()) wait() From 7e636c89ce94c38695628d8b3b0cdc14f6257209 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 2 Jun 2020 18:24:16 +0300 Subject: [PATCH 277/350] Fix another data race... --- stats/dummy/collector.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/stats/dummy/collector.go b/stats/dummy/collector.go index 25534113795..fa147c699ab 100644 --- a/stats/dummy/collector.go +++ b/stats/dummy/collector.go @@ -23,8 +23,6 @@ package dummy import ( "context" - "github.com/sirupsen/logrus" - "github.com/loadimpact/k6/lib" "github.com/loadimpact/k6/stats" ) @@ -49,7 +47,6 @@ func (c *Collector) MakeConfig() interface{} { return nil } // Run just blocks until the context is done func (c *Collector) Run(ctx context.Context) { <-ctx.Done() - logrus.Debugf("finished status: %d", c.RunStatus) } // Collect just appends all of the samples passed to it to the internal sample slice. From 5974049c6cb7b8e701575aefe811b610f10b0453 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 4 Jun 2020 13:25:01 +0300 Subject: [PATCH 278/350] refactor TestIntegrationConvertCmd to be more stable (#1484) --- cmd/convert_test.go | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/cmd/convert_test.go b/cmd/convert_test.go index d27c0bd8658..7943988f004 100644 --- a/cmd/convert_test.go +++ b/cmd/convert_test.go @@ -23,13 +23,14 @@ package cmd import ( "bytes" "io/ioutil" - "os" + "path/filepath" "regexp" "testing" "github.com/pmezard/go-difflib/difflib" "github.com/spf13/afero" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) const testHAR = ` @@ -121,24 +122,19 @@ export default function() { ` func TestIntegrationConvertCmd(t *testing.T) { - var tmpFile, err = ioutil.TempFile("", "") - if err != nil { - t.Fatalf("Couldn't create temporary file: %s", err) - } - harFile := tmpFile.Name() - defer os.Remove(harFile) - tmpFile.Close() t.Run("Correlate", func(t *testing.T) { + harFile, err := filepath.Abs("correlate.har") + require.NoError(t, err) har, err := ioutil.ReadFile("testdata/example.har") - assert.NoError(t, err) + require.NoError(t, err) expectedTestPlan, err := ioutil.ReadFile("testdata/example.js") - assert.NoError(t, err) + require.NoError(t, err) defaultFs = afero.NewMemMapFs() err = afero.WriteFile(defaultFs, harFile, har, 0644) - assert.NoError(t, err) + require.NoError(t, err) buf := &bytes.Buffer{} defaultWriter = buf @@ -177,8 +173,10 @@ func TestIntegrationConvertCmd(t *testing.T) { } }) t.Run("Stdout", func(t *testing.T) { + harFile, err := filepath.Abs("stdout.har") + require.NoError(t, err) defaultFs = afero.NewMemMapFs() - err := afero.WriteFile(defaultFs, harFile, []byte(testHAR), 0644) + err = afero.WriteFile(defaultFs, harFile, []byte(testHAR), 0644) assert.NoError(t, err) buf := &bytes.Buffer{} @@ -189,11 +187,16 @@ func TestIntegrationConvertCmd(t *testing.T) { assert.Equal(t, testHARConvertResult, buf.String()) }) t.Run("Output file", func(t *testing.T) { + harFile, err := filepath.Abs("output.har") + require.NoError(t, err) defaultFs = afero.NewMemMapFs() - err := afero.WriteFile(defaultFs, harFile, []byte(testHAR), 0644) + err = afero.WriteFile(defaultFs, harFile, []byte(testHAR), 0644) assert.NoError(t, err) err = convertCmd.Flags().Set("output", "/output.js") + defer func() { + err = convertCmd.Flags().Set("output", "") + }() assert.NoError(t, err) err = convertCmd.RunE(convertCmd, []string{harFile}) assert.NoError(t, err) From bceeb6ae0f1262e69c1996da35ae91bbca03d8cc Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 4 Jun 2020 11:10:03 +0300 Subject: [PATCH 279/350] Prevent a cloud test from blocking and make it less likely it happens --- stats/cloud/collector_test.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/stats/cloud/collector_test.go b/stats/cloud/collector_test.go index 89aa7548878..8fe3b21789c 100644 --- a/stats/cloud/collector_test.go +++ b/stats/cloud/collector_test.go @@ -430,8 +430,9 @@ func TestCloudCollectorStopSendingMetric(t *testing.T) { } config := NewConfig().Apply(Config{ - Host: null.StringFrom(tb.ServerHTTP.URL), - NoCompress: null.BoolFrom(true), + Host: null.StringFrom(tb.ServerHTTP.URL), + NoCompress: null.BoolFrom(true), + MaxMetricSamplesPerPackage: null.IntFrom(50), }) collector, err := New(config, script, options, []lib.ExecutionStep{}, "1.0") require.NoError(t, err) @@ -502,8 +503,12 @@ func TestCloudCollectorStopSendingMetric(t *testing.T) { cancel() wg.Wait() require.Equal(t, lib.RunStatusQueued, collector.runStatus) - _, ok := <-collector.stopSendingMetricsCh - require.False(t, ok) + select { + case <-collector.stopSendingMetricsCh: + // all is fine + default: + t.Fatal("sending metrics wasn't stopped") + } require.Equal(t, max, count) nBufferSamples := len(collector.bufferSamples) From 24b06717280a882167f640e5e30d72a07deafd1d Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 4 Jun 2020 11:28:14 +0300 Subject: [PATCH 280/350] Increase TestEmittedMetricsWhenScalingDown timeout a bit --- core/engine_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/engine_test.go b/core/engine_test.go index 4d7e992aa47..6f2b037aead 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -741,7 +741,7 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { go func() { errC <- run() }() select { - case <-time.After(10 * time.Second): + case <-time.After(12 * time.Second): t.Fatal("Test timed out") case err := <-errC: require.NoError(t, err) From 61cd643804d27d8c879435a08f0de2945a4faaa0 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 4 Jun 2020 11:28:36 +0300 Subject: [PATCH 281/350] increase timeout and better reporting for TestExecutionSchedulerRunCustomTags --- core/local/local_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 9620068836c..173802113bc 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -341,15 +341,15 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { execScheduler, err := NewExecutionScheduler(runner, logger) require.NoError(t, err) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) defer cancel() done := make(chan struct{}) samples := make(chan stats.SampleContainer) go func() { - assert.NoError(t, execScheduler.Init(ctx, samples)) - assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) - close(done) + defer close(done) + require.NoError(t, execScheduler.Init(ctx, samples)) + require.NoError(t, execScheduler.Run(ctx, ctx, samples)) }() var gotTrailTag, gotNetTrailTag bool for { From ac4ab4e366d63ca354c90c720d9f1b949a7772bb Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 3 Jun 2020 17:18:37 +0300 Subject: [PATCH 282/350] Increase setup/teardown timeouts in some failing tests --- api/v1/setup_teardown_routes_test.go | 4 ++-- core/engine_test.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/api/v1/setup_teardown_routes_test.go b/api/v1/setup_teardown_routes_test.go index 20ba612e45c..ab7e504110d 100644 --- a/api/v1/setup_teardown_routes_test.go +++ b/api/v1/setup_teardown_routes_test.go @@ -146,8 +146,8 @@ func TestSetupData(t *testing.T) { VUs: null.IntFrom(2), Iterations: null.IntFrom(3), NoSetup: null.BoolFrom(true), - SetupTimeout: types.NullDurationFrom(1 * time.Second), - TeardownTimeout: types.NullDurationFrom(1 * time.Second), + SetupTimeout: types.NullDurationFrom(5 * time.Second), + TeardownTimeout: types.NullDurationFrom(5 * time.Second), }) execScheduler, err := local.NewExecutionScheduler(runner, logrus.StandardLogger()) require.NoError(t, err) diff --git a/core/engine_test.go b/core/engine_test.go index 6f2b037aead..e94300871c3 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -879,7 +879,7 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { gracefulStop: "0s", }, }, - setupTimeout: "2s", + setupTimeout: "3s", }; export default function () { @@ -897,7 +897,7 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { gracefulStop: "0s", }, }, - teardownTimeout: "2s", + teardownTimeout: "3s", }; export default function () { From 96cd4173b75034e06c97a1c59ce1729639ea2a57 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 5 Jun 2020 16:39:40 +0300 Subject: [PATCH 283/350] Make TestExecutionSchedulerRunCustomConfigNoCrossover more stable --- core/local/local_test.go | 70 ++++++++++++++++++---------------------- 1 file changed, 32 insertions(+), 38 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 173802113bc..573955b868e 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -403,7 +403,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { vus: 1, iterations: 1, gracefulStop: '0s', - maxDuration: '0.5s', + maxDuration: '1s', exec: 's1func', env: { TESTVAR1: 'scenario1' }, tags: { testtag1: 'scenario1' }, @@ -412,7 +412,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { type: 'shared-iterations', vus: 1, iterations: 1, - gracefulStop: '0.5s', + gracefulStop: '1s', startTime: '0.5s', maxDuration: '2s', exec: 's2func', @@ -423,7 +423,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { type: 'per-vu-iterations', vus: 1, iterations: 1, - gracefulStop: '0.5s', + gracefulStop: '1s', exec: 's3funcWS', env: { TESTVAR3: 'scenario3' }, tags: { testtag3: 'scenario3' }, @@ -505,12 +505,11 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() - done := make(chan struct{}) samples := make(chan stats.SampleContainer) go func() { assert.NoError(t, execScheduler.Init(ctx, samples)) assert.NoError(t, execScheduler.Run(ctx, ctx, samples)) - close(done) + close(samples) }() expectedTrailTags := []map[string]string{ @@ -529,49 +528,44 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { {"testtag3": "scenario3", "wstag": "scenario3"}, } var gotSampleTags int - for { - select { - case sample := <-samples: - switch s := sample.(type) { - case stats.Sample: - if s.Metric.Name == "errors" { - assert.FailNow(t, "received error sample from test") - } - if s.Metric.Name == "checks" || s.Metric.Name == "group_duration" { - tags := s.Tags.CloneTags() - for _, expTags := range expectedPlainSampleTags { - if reflect.DeepEqual(expTags, tags) { - gotSampleTags++ - } - } - } - case *httpext.Trail: + for sample := range samples { + switch s := sample.(type) { + case stats.Sample: + if s.Metric.Name == "errors" { + assert.FailNow(t, "received error sample from test") + } + if s.Metric.Name == "checks" || s.Metric.Name == "group_duration" { tags := s.Tags.CloneTags() - for _, expTags := range expectedTrailTags { + for _, expTags := range expectedPlainSampleTags { if reflect.DeepEqual(expTags, tags) { gotSampleTags++ } } - case *netext.NetTrail: - tags := s.Tags.CloneTags() - for _, expTags := range expectedNetTrailTags { - if reflect.DeepEqual(expTags, tags) { - gotSampleTags++ - } + } + case *httpext.Trail: + tags := s.Tags.CloneTags() + for _, expTags := range expectedTrailTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ } - case stats.ConnectedSamples: - for _, sm := range s.Samples { - tags := sm.Tags.CloneTags() - if reflect.DeepEqual(expectedConnSampleTags, tags) { - gotSampleTags++ - } + } + case *netext.NetTrail: + tags := s.Tags.CloneTags() + for _, expTags := range expectedNetTrailTags { + if reflect.DeepEqual(expTags, tags) { + gotSampleTags++ + } + } + case stats.ConnectedSamples: + for _, sm := range s.Samples { + tags := sm.Tags.CloneTags() + if reflect.DeepEqual(expectedConnSampleTags, tags) { + gotSampleTags++ } } - case <-done: - require.Equal(t, 8, gotSampleTags, "received wrong amount of samples with expected tags") - return } } + require.Equal(t, 8, gotSampleTags, "received wrong amount of samples with expected tags") } func TestExecutionSchedulerSetupTeardownRun(t *testing.T) { From 1f2528ccdca40ab1d220ca3fbeecfc0ca8e9ec8f Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 5 Jun 2020 16:39:57 +0300 Subject: [PATCH 284/350] Make TestConstantLoopingVUsRun more stable --- lib/executor/constant_looping_vus_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go index f209009553c..5f3e9f4d330 100644 --- a/lib/executor/constant_looping_vus_test.go +++ b/lib/executor/constant_looping_vus_test.go @@ -52,6 +52,11 @@ func TestConstantLoopingVUsRun(t *testing.T) { t, getTestConstantLoopingVUsConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(200 * time.Millisecond) + select { + case <-ctx.Done(): + return nil + default: + } state := lib.GetState(ctx) currIter, _ := result.LoadOrStore(state.Vu, uint64(0)) result.Store(state.Vu, currIter.(uint64)+1) From e798653ac8518c30b36c0fea20740b1f51bbde88 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 8 Jun 2020 17:11:16 +0300 Subject: [PATCH 285/350] Make TestConstantArrivalRateRunCorrectTiming faster and stabler --- lib/executor/constant_arrival_rate_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 9569c2d9675..926e164946c 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -191,6 +191,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { }, et, 10, 50) var count int64 config := getTestConstantArrivalRateConfig() + config.Duration.Duration = types.Duration(time.Second * 3) newET, err := es.ExecutionTuple.GetNewExecutionTupleFromValue(config.MaxVUs.Int64) require.NoError(t, err) rateScaled := newET.ScaleInt64(config.Rate.Int64) @@ -225,7 +226,7 @@ func TestConstantArrivalRateRunCorrectTiming(t *testing.T) { // check that we got around the amount of VU iterations as we would expect var currentCount int64 - for i := 0; i < 5; i++ { + for i := 0; i < 3; i++ { time.Sleep(time.Second) currentCount = atomic.LoadInt64(&count) assert.InDelta(t, int64(i+1)*rateScaled, currentCount, 3) From 6ee1625664c76e91bba411b693c50a6192a4d43b Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 8 Jun 2020 17:11:41 +0300 Subject: [PATCH 286/350] Prevent lock in cloud tests --- stats/cloud/collector_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/stats/cloud/collector_test.go b/stats/cloud/collector_test.go index 8fe3b21789c..250409962cb 100644 --- a/stats/cloud/collector_test.go +++ b/stats/cloud/collector_test.go @@ -212,6 +212,7 @@ func runCloudCollectorTestCase(t *testing.T, minSamples int) { expectedTags := stats.IntoSampleTags(&expectedTagMap) expSamples := make(chan []Sample) + defer close(expSamples) tb.Mux.HandleFunc(fmt.Sprintf("/v1/metrics/%s", collector.referenceID), getSampleChecker(t, expSamples)) tb.Mux.HandleFunc(fmt.Sprintf("/v1/tests/%s", collector.referenceID), func(rw http.ResponseWriter, _ *http.Request) { rw.WriteHeader(http.StatusOK) // silence a test warning @@ -575,6 +576,7 @@ func TestCloudCollectorAggregationPeriodZeroNoBlock(t *testing.T) { assert.Equal(t, types.Duration(5*time.Millisecond), collector.config.AggregationWaitPeriod.Duration) expSamples := make(chan []Sample) + defer close(expSamples) tb.Mux.HandleFunc(fmt.Sprintf("/v1/metrics/%s", collector.referenceID), getSampleChecker(t, expSamples)) ctx, cancel := context.WithCancel(context.Background()) From d93244641f67025b74b07a108a6c25d18d855f41 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 9 Jun 2020 11:02:58 +0300 Subject: [PATCH 287/350] Fix externally controlled executor not pausing (#1479) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The problem was that pausing used gracefulStop which in turns wait for the VU used by handle to be returned. The fix is to have another context that is used for the actual runContext of the VU so we can cancel it when we get gracefully stopped. I tried to simplify (and speed up) the looping by only going through all the channels and context stuff if there was a change, otherwise we use a int32 and some atomic.LoadInt32 to see that "there is no change" and we directly start a new iteration. Additionally now the getVU error is propagetted by the vu handle's start, which is probably badly handled in this commit. Also the function that does the iteration now returns whether it was interrupted mostly to help with faster interruption of the fast looping. This might be reverted in a future commit, if the speed up isn't wanted. Unfortunately in order to support returning the VU and not returning it and still gettign the new VU but still to get start to actually return the error from getVU it took way more code then was anticipated ... The speed up is "insignficant" too, by my proffiling we were spending around 150-250ms (of 30s) in the mutex locking/channel reading now we spend 10ms(of 30s) in loading that int32. So 15-25x, but this is some sub percentage time and this is for an empty bodied `default` function. More benchmarks to come :D. There is also a not so great benchmark :D name iterations/ns VUHandleIterations-8 0.04 ± 1% so around 40ns per iteration --- lib/executor/externally_controlled.go | 13 +- lib/executor/helpers.go | 8 +- lib/executor/variable_looping_vus.go | 3 +- lib/executor/vu_handle.go | 175 +++++++++----- lib/executor/vu_handle_test.go | 333 ++++++++++++++++++++++++++ 5 files changed, 471 insertions(+), 61 deletions(-) create mode 100644 lib/executor/vu_handle_test.go diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 123def5d650..52a7982b3a1 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -379,7 +379,7 @@ type externallyControlledRunState struct { vuHandles []*manualVUHandle // handles for manipulating and tracking all of the VUs currentlyPaused bool // whether the executor is currently paused - runIteration func(context.Context, lib.ActiveVU) // a helper closure function that runs a single iteration + runIteration func(context.Context, lib.ActiveVU) bool // a helper closure function that runs a single iteration } // retrieveStartMaxVUs gets and initializes the (scaled) number of MaxVUs @@ -456,7 +456,10 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern if oldActiveVUs < newActiveVUs { for i := oldActiveVUs; i < newActiveVUs; i++ { if !rs.currentlyPaused { - rs.vuHandles[i].start() + if err := rs.vuHandles[i].start(); err != nil { + // TODO: maybe just log it ? + return err + } } } } else { @@ -573,7 +576,11 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats } } else { for i := int64(0); i < activeVUs; i++ { - runState.vuHandles[i].start() + if err := runState.vuHandles[i].start(); err != nil { + // TODO again ... just log it? + pauseEvent.err <- err + return err + } } } runState.currentlyPaused = pauseEvent.isPaused diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index d3e8e1d503b..2df8eae53b1 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -76,13 +76,13 @@ func validateStages(stages []Stage) []error { // getIterationRunner is a helper function that returns an iteration executor // closure. It takes care of updating the execution state statistics and -// warning messages. +// warning messages. And returns whether a full iteration was finished or not // // TODO: emit the end-of-test iteration metrics here (https://github.com/loadimpact/k6/issues/1250) func getIterationRunner( executionState *lib.ExecutionState, logger *logrus.Entry, -) func(context.Context, lib.ActiveVU) { - return func(ctx context.Context, vu lib.ActiveVU) { +) func(context.Context, lib.ActiveVU) bool { + return func(ctx context.Context, vu lib.ActiveVU) bool { err := vu.RunOnce() // TODO: track (non-ramp-down) errors from script iterations as a metric, @@ -93,6 +93,7 @@ func getIterationRunner( case <-ctx.Done(): // Don't log errors or emit iterations metrics from cancelled iterations executionState.AddInterruptedIterations(1) + return false default: if err != nil { if s, ok := err.(fmt.Stringer); ok { @@ -105,6 +106,7 @@ func getIterationRunner( // TODO: move emission of end-of-iteration metrics here? executionState.AddFullIterations(1) + return true } } } diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index bb9dd7fcfcf..0350a8fae74 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -583,6 +583,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo getVU := func() (lib.InitializedVU, error) { initVU, err := vlv.executionState.GetPlannedVU(vlv.logger, false) if err != nil { + vlv.logger.WithError(err).Error("Cannot get a VU from the buffer") cancel() } else { activeVUs.Add(1) @@ -614,7 +615,7 @@ func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleCo handleNewScheduledVUs := func(newScheduledVUs uint64) { if newScheduledVUs > currentScheduledVUs { for vuNum := currentScheduledVUs; vuNum < newScheduledVUs; vuNum++ { - vuHandles[vuNum].start() + _ = vuHandles[vuNum].start() // TODO handle error vlv.executionState.ModCurrentlyActiveVUsCount(+1) } } else { diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index 1dbf6410f0b..c81749ac013 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -23,6 +23,7 @@ package executor import ( "context" "sync" + "sync/atomic" "github.com/sirupsen/logrus" @@ -35,51 +36,109 @@ import ( // // TODO: something simpler? type vuHandle struct { - mutex *sync.RWMutex + mutex *sync.Mutex parentCtx context.Context getVU func() (lib.InitializedVU, error) returnVU func(lib.InitializedVU) config *BaseConfig + initVU lib.InitializedVU + activeVU lib.ActiveVU canStartIter chan struct{} + // If change is not 0, it signals that the VU needs to be reinitialized. It must be added to and + // read with atomics and helps to skip checking all the contexts and channels all the time. + change int32 - ctx context.Context - cancel func() - logger *logrus.Entry + ctx, vuCtx context.Context + cancel, vuCancel func() + logger *logrus.Entry } func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), returnVU func(lib.InitializedVU), config *BaseConfig, logger *logrus.Entry, ) *vuHandle { - lock := &sync.RWMutex{} + lock := &sync.Mutex{} ctx, cancel := context.WithCancel(parentCtx) - return &vuHandle{ + + vh := &vuHandle{ mutex: lock, parentCtx: parentCtx, getVU: getVU, - returnVU: returnVU, config: config, canStartIter: make(chan struct{}), + change: 1, ctx: ctx, cancel: cancel, logger: logger, } + + // TODO maybe move the repeating parts in a function? + vh.returnVU = func(v lib.InitializedVU) { + // Don't return the initialized VU back + vh.mutex.Lock() + select { + case <-vh.parentCtx.Done(): + // we are done just ruturn the VU + vh.initVU = nil + vh.activeVU = nil + atomic.StoreInt32(&vh.change, 1) + vh.mutex.Unlock() + returnVU(v) + default: + select { + case <-vh.canStartIter: + // we can continue with itearting - lets not return the vu + vh.activateVU() // we still need to reactivate it to get the new context and cancel + atomic.StoreInt32(&vh.change, 1) + vh.mutex.Unlock() + default: // we actually have to return the vu + vh.initVU = nil + vh.activeVU = nil + atomic.StoreInt32(&vh.change, 1) + vh.mutex.Unlock() + returnVU(v) + } + } + } + + return vh } -func (vh *vuHandle) start() { +func (vh *vuHandle) start() (err error) { vh.mutex.Lock() vh.logger.Debug("Start") - close(vh.canStartIter) + if vh.initVU == nil { + vh.initVU, err = vh.getVU() + if err != nil { + return err + } + vh.activateVU() + atomic.AddInt32(&vh.change, 1) + } + select { + case <-vh.canStartIter: // we are alrady started do nothing + default: + close(vh.canStartIter) + } vh.mutex.Unlock() + return nil +} + +// this must be called with the mutex locked +func (vh *vuHandle) activateVU() { + vh.vuCtx, vh.vuCancel = context.WithCancel(vh.ctx) + vh.activeVU = vh.initVU.Activate(getVUActivationParams(vh.vuCtx, *vh.config, vh.returnVU)) } func (vh *vuHandle) gracefulStop() { vh.mutex.Lock() select { case <-vh.canStartIter: + atomic.AddInt32(&vh.change, 1) + vh.activeVU = nil vh.canStartIter = make(chan struct{}) vh.logger.Debug("Graceful stop") default: @@ -91,7 +150,10 @@ func (vh *vuHandle) gracefulStop() { func (vh *vuHandle) hardStop() { vh.mutex.Lock() vh.logger.Debug("Hard stop") - vh.cancel() // cancel the previous context + vh.cancel() // cancel the previous context + atomic.AddInt32(&vh.change, 1) + vh.initVU = nil + vh.activeVU = nil vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) // create a new context select { case <-vh.canStartIter: @@ -102,63 +164,68 @@ func (vh *vuHandle) hardStop() { vh.mutex.Unlock() } -//TODO: simplify this somehow - I feel like there should be a better way to -//implement this logic... maybe with sync.Cond? -func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.ActiveVU)) { - executorDone := vh.parentCtx.Done() - - var vu lib.ActiveVU +func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.ActiveVU) bool) { + // We can probably initialize here, but it's also easier to just use the slow path in the second + // part of the for loop + var ( + executorDone = vh.parentCtx.Done() + vuCtx context.Context + cancel func() + vu lib.ActiveVU + ) -mainLoop: for { - vh.mutex.RLock() + ch := atomic.LoadInt32(&vh.change) + if ch == 0 && runIter(vuCtx, vu) { // fast path + continue + } + + // slow path - something has changed - get what and wait until we can do more iterations + // TODO: I think we can skip cancelling in some cases but I doubt it will do much good in most + if cancel != nil { + cancel() // signal to return the vu before we continue + } + vh.mutex.Lock() canStartIter, ctx := vh.canStartIter, vh.ctx - vh.mutex.RUnlock() + cancel = vh.vuCancel + vh.mutex.Unlock() - // Wait for either the executor to be done, or for us to be un-paused select { - case <-canStartIter: - // Best case, we're currently running, so we do nothing here, we - // just continue straight ahead. case <-executorDone: // The whole executor is done, nothing more to do. return default: - // We're not running, but the executor isn't done yet, so we wait - // for either one of those conditions. But before that, clear - // the VU reference to ensure we get a fresh one below. - vu = nil - select { - case <-canStartIter: - // continue on, we were unblocked... - case <-ctx.Done(): - // hardStop was called, start a fresh iteration to get the new - // context and signal channel - continue mainLoop - case <-executorDone: - // The whole executor is done, nothing more to do. - return - } } - - // Probably not needed, but just in case - if both running and - // executorDone were active, check that the executor isn't done. + // We're not running, but the executor isn't done yet, so we wait + // for either one of those conditions. select { + case <-canStartIter: + vh.mutex.Lock() + select { + case <-vh.canStartIter: // we check again in case of race + // reinitialize + if vh.activeVU == nil { + // we've raced with the ReturnVU: we can continue doing iterations but + // a stop has happened and returnVU hasn't managed to run yet ... so we loop + // TODO call runtime.GoSched() in the else to give priority to possibly the returnVU goroutine + vh.mutex.Unlock() + continue + } + + vu = vh.activeVU + vuCtx = vh.vuCtx + cancel = vh.vuCancel + atomic.StoreInt32(&vh.change, 0) // clear changes here + default: + // well we got raced to here by something ... loop again ... + } + vh.mutex.Unlock() + case <-ctx.Done(): + // hardStop was called, start a fresh iteration to get the new + // context and signal channel case <-executorDone: + // The whole executor is done, nothing more to do. return - default: } - - // Ensure we have an active VU - if vu == nil { - initVU, err := vh.getVU() - if err != nil { - return - } - activationParams := getVUActivationParams(ctx, *vh.config, vh.returnVU) - vu = initVU.Activate(activationParams) - } - - runIter(ctx, vu) } } diff --git a/lib/executor/vu_handle_test.go b/lib/executor/vu_handle_test.go new file mode 100644 index 00000000000..ff688363d82 --- /dev/null +++ b/lib/executor/vu_handle_test.go @@ -0,0 +1,333 @@ +package executor + +import ( + "context" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/testutils" + "github.com/loadimpact/k6/lib/testutils/minirunner" + "github.com/loadimpact/k6/stats" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// this test is mostly interesting when -race is enabled +func TestVUHandleRace(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.DebugLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(testutils.NewTestOutput(t)) + // testLog.Level = logrus.DebugLevel + logEntry := logrus.NewEntry(testLog) + + getVU := func() (lib.InitializedVU, error) { + return &minirunner.VU{ + R: &minirunner.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + // TODO: do something + return nil + }, + }, + }, nil + } + + returnVU := func(_ lib.InitializedVU) { + // do something + } + var interruptedIter int64 + var fullIterations int64 + + runIter := func(ctx context.Context, vu lib.ActiveVU) bool { + _ = vu.RunOnce() + select { + case <-ctx.Done(): + // Don't log errors or emit iterations metrics from cancelled iterations + atomic.AddInt64(&interruptedIter, 1) + return false + default: + atomic.AddInt64(&fullIterations, 1) + return true + } + } + + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + go vuHandle.runLoopsIfPossible(runIter) + var wg sync.WaitGroup + wg.Add(3) + go func() { + defer wg.Done() + for i := 0; i < 10000; i++ { + err := vuHandle.start() + require.NoError(t, err) + } + }() + + go func() { + defer wg.Done() + for i := 0; i < 1000; i++ { + vuHandle.gracefulStop() + time.Sleep(1 * time.Nanosecond) + } + }() + + go func() { + defer wg.Done() + for i := 0; i < 100; i++ { + vuHandle.hardStop() + time.Sleep(10 * time.Nanosecond) + } + }() + wg.Wait() + vuHandle.hardStop() // STOP it + time.Sleep(time.Millisecond) + interruptedBefore := atomic.LoadInt64(&interruptedIter) + fullBefore := atomic.LoadInt64(&fullIterations) + _ = vuHandle.start() + time.Sleep(time.Millisecond * 50) // just to be sure an iteration will squeeze in + cancel() + time.Sleep(time.Millisecond * 5) + interruptedAfter := atomic.LoadInt64(&interruptedIter) + fullAfter := atomic.LoadInt64(&fullIterations) + assert.True(t, interruptedBefore >= interruptedAfter-1, + "too big of a difference %d >= %d - 1", interruptedBefore, interruptedAfter) + assert.True(t, fullBefore+1 <= fullAfter, + "too small of a difference %d + 1 <= %d", fullBefore, fullAfter) +} + +func TestVUHandleSimple(t *testing.T) { + t.Parallel() + + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.DebugLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(testutils.NewTestOutput(t)) + // testLog.Level = logrus.DebugLevel + logEntry := logrus.NewEntry(testLog) + + var ( + getVUCount uint32 + returnVUCount uint32 + interruptedIter int64 + fullIterations int64 + ) + reset := func() { + getVUCount = 0 + returnVUCount = 0 + interruptedIter = 0 + fullIterations = 0 + } + + getVU := func() (lib.InitializedVU, error) { //nolint:unparam + atomic.AddUint32(&getVUCount, 1) + + return &minirunner.VU{ + R: &minirunner.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + // TODO: do something + return nil + }, + }, + }, nil + } + + returnVU := func(_ lib.InitializedVU) { + atomic.AddUint32(&returnVUCount, 1) + } + + runIter := func(ctx context.Context, _ lib.ActiveVU) bool { + select { + case <-time.After(time.Second): + case <-ctx.Done(): + } + + select { + case <-ctx.Done(): + // Don't log errors or emit iterations metrics from cancelled iterations + atomic.AddInt64(&interruptedIter, 1) + return false + default: + atomic.AddInt64(&fullIterations, 1) + return true + } + } + t.Run("start before gracefulStop finishes", func(t *testing.T) { + reset() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + vuHandle.runLoopsIfPossible(runIter) + }() + err := vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 5) + vuHandle.gracefulStop() + time.Sleep(time.Millisecond * 5) + err = vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 1500) + assert.EqualValues(t, 1, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 0, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 0, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 1, atomic.LoadInt64(&fullIterations)) + cancel() + wg.Wait() + time.Sleep(time.Millisecond * 5) + assert.EqualValues(t, 1, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 1, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 1, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 1, atomic.LoadInt64(&fullIterations)) + }) + + t.Run("start after gracefulStop finishes", func(t *testing.T) { + reset() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + vuHandle.runLoopsIfPossible(runIter) + }() + err := vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 5) + vuHandle.gracefulStop() + time.Sleep(time.Millisecond * 1500) + assert.EqualValues(t, 1, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 1, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 0, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 1, atomic.LoadInt64(&fullIterations)) + err = vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 1500) + cancel() + wg.Wait() + + time.Sleep(time.Millisecond * 5) + assert.EqualValues(t, 2, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 2, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 1, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 2, atomic.LoadInt64(&fullIterations)) + }) + + t.Run("start after hardStop", func(t *testing.T) { + reset() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + vuHandle.runLoopsIfPossible(runIter) + }() + err := vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 5) + vuHandle.hardStop() + time.Sleep(time.Millisecond * 15) + assert.EqualValues(t, 1, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 1, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 1, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 0, atomic.LoadInt64(&fullIterations)) + err = vuHandle.start() + require.NoError(t, err) + time.Sleep(time.Millisecond * 1500) + cancel() + wg.Wait() + + time.Sleep(time.Millisecond * 5) + assert.EqualValues(t, 2, atomic.LoadUint32(&getVUCount)) + assert.EqualValues(t, 2, atomic.LoadUint32(&returnVUCount)) + assert.EqualValues(t, 2, atomic.LoadInt64(&interruptedIter)) + assert.EqualValues(t, 1, atomic.LoadInt64(&fullIterations)) + }) +} + +func BenchmarkVUHandleIterations(b *testing.B) { + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.DebugLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + // testLog.Level = logrus.DebugLevel + logEntry := logrus.NewEntry(testLog) + + var ( + getVUCount uint32 + returnVUCount uint32 + interruptedIter int64 + fullIterations int64 + ) + reset := func() { + getVUCount = 0 + returnVUCount = 0 + interruptedIter = 0 + fullIterations = 0 + } + + getVU := func() (lib.InitializedVU, error) { + atomic.AddUint32(&getVUCount, 1) + + return &minirunner.VU{ + R: &minirunner.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + // TODO: do something + return nil + }, + }, + }, nil + } + + returnVU := func(_ lib.InitializedVU) { + atomic.AddUint32(&returnVUCount, 1) + } + + runIter := func(ctx context.Context, _ lib.ActiveVU) bool { + // Do nothing + select { + case <-ctx.Done(): + // Don't log errors or emit iterations metrics from cancelled iterations + atomic.AddInt64(&interruptedIter, 1) + return false + default: + atomic.AddInt64(&fullIterations, 1) + return true + } + } + + reset() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + vuHandle.runLoopsIfPossible(runIter) + }() + start := time.Now() + b.ResetTimer() + err := vuHandle.start() + require.NoError(b, err) + time.Sleep(time.Second) + cancel() + wg.Wait() + b.StopTimer() + took := time.Since(start) + b.ReportMetric(float64(atomic.LoadInt64(&fullIterations))/float64(took), "iterations/ns") +} From 0839be833a912e89e64eef97dbc277d0cdc4ed9f Mon Sep 17 00:00:00 2001 From: michiboo Date: Thu, 28 May 2020 19:51:36 +0800 Subject: [PATCH 288/350] make JSON output logging less noisy Increase time interval between log message to 200 milliseconds Remove Writing JSON metrics message Add if statement output log based on no of count --- stats/json/collector.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/stats/json/collector.go b/stats/json/collector.go index ee1ef04772f..291679465c6 100644 --- a/stats/json/collector.go +++ b/stats/json/collector.go @@ -98,8 +98,9 @@ func (c *Collector) Init() error { func (c *Collector) SetRunStatus(status lib.RunStatus) {} func (c *Collector) Run(ctx context.Context) { + const timeout = 200 logrus.Debug("JSON output: Running!") - ticker := time.NewTicker(time.Millisecond * 100) + ticker := time.NewTicker(time.Millisecond * timeout) defer func() { _ = c.closeFn() }() @@ -142,7 +143,6 @@ func (c *Collector) commit() { samples := c.buffer c.buffer = nil c.bufferLock.Unlock() - logrus.WithField("filename", c.fname).Debug("JSON: Writing JSON metrics") var start = time.Now() var count int for _, sc := range samples { @@ -160,8 +160,10 @@ func (c *Collector) commit() { } } } - logrus.WithField("filename", c.fname).WithField("t", time.Since(start)). - WithField("count", count).Debug("JSON: Wrote JSON metrics") + if count > 0 { + logrus.WithField("filename", c.fname).WithField("t", time.Since(start)). + WithField("count", count).Debug("JSON: Wrote JSON metrics") + } } func (c *Collector) Link() string { From 08070ca9cb58dc25b266fdbc5ece3818d39759c8 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 14:58:08 +0300 Subject: [PATCH 289/350] Rename the user-facing config options This handles the executor configuration parts of https://github.com/loadimpact/k6/issues/1425, without actually changing the Go type names or structure yet. --- cmd/config_consolidation_test.go | 6 +- cmd/run.go | 2 +- core/engine_test.go | 16 +-- core/local/local_test.go | 40 ++++---- js/runner_test.go | 4 +- lib/archive.go | 2 + lib/executor/base_config.go | 2 +- lib/executor/constant_looping_vus.go | 2 +- lib/executor/executors_test.go | 142 +++++++++++++------------- lib/executor/externally_controlled.go | 2 +- lib/executor/variable_arrival_rate.go | 2 +- lib/executor/variable_looping_vus.go | 2 +- lib/executors.go | 4 +- lib/old_archive_test.go | 10 +- lib/options.go | 2 +- 15 files changed, 120 insertions(+), 118 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 9a1995ba352..a3d29c7903d 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -243,7 +243,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, }}, exp{derivationError: true}, nil, }, - {opts{fs: defaultConfig(`{"execution": {}}`)}, exp{logWarning: true}, verifyOneIterPerOneVU}, + {opts{fs: defaultConfig(`{"scenarios": {}}`)}, exp{logWarning: true}, verifyOneIterPerOneVU}, // Test if environment variable shortcuts are working as expected {opts{env: []string{"K6_VUS=5", "K6_ITERATIONS=15"}}, exp{}, verifySharedIters(I(5), I(15))}, {opts{env: []string{"K6_VUS=10", "K6_DURATION=20s"}}, exp{}, verifyConstLoopingVUs(I(10), 20*time.Second)}, @@ -316,8 +316,8 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { { opts{ fs: defaultConfig(`{ - "execution": { "someKey": { - "type": "constant-looping-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", + "scenarios": { "someKey": { + "executor": "constant-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc" }}}`), env: []string{"K6_ITERATIONS=25"}, diff --git a/cmd/run.go b/cmd/run.go index 21a4bd2e275..3ae8f18bf67 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -256,7 +256,7 @@ a commandline interface for interacting with it.`, executorConfigs := execScheduler.GetExecutorConfigs() maxDuration, _ := lib.GetEndOffset(plan) - fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprintf( + fprintf(stdout, " scenarios: %s\n", ui.ValueColor.Sprintf( "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), lib.GetMaxPossibleVUs(plan), maxDuration), diff --git a/core/engine_test.go b/core/engine_test.go index e94300871c3..853d07d6940 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -702,9 +702,9 @@ func TestEmittedMetricsWhenScalingDown(t *testing.T) { export let options = { systemTags: ["iter", "vu", "url"], - execution: { + scenarios: { we_need_hard_stop_and_ramp_down: { - type: "variable-looping-vus", + executor: "ramping-vus", // Start with 2 VUs for 4 seconds and then quickly scale down to 1 for the next 4s and then quit startVUs: 2, stages: [ @@ -818,9 +818,9 @@ func TestMetricsEmission(t *testing.T) { let testCounter = new Counter("testcounter"); export let options = { - execution: { + scenarios: { we_need_hard_stop: { - type: "constant-looping-vus", + executor: "constant-vus", vus: 4, duration: "1s", gracefulStop: "0s", @@ -871,9 +871,9 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { export let options = { minIterationDuration: "2s", - execution: { + scenarios: { we_need_hard_stop: { - type: "constant-looping-vus", + executor: "constant-vus", vus: 2, duration: "1.9s", gracefulStop: "0s", @@ -889,9 +889,9 @@ func TestMinIterationDurationInSetupTeardownStage(t *testing.T) { export let options = { minIterationDuration: "2s", - execution: { + scenarios: { we_need_hard_stop: { - type: "constant-looping-vus", + executor: "constant-vus", vus: 2, duration: "1.9s", gracefulStop: "0s", diff --git a/core/local/local_test.go b/core/local/local_test.go index 573955b868e..a2c1ad45422 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -110,9 +110,9 @@ func TestExecutionSchedulerRunNonDefault(t *testing.T) { {"defaultOK", `export default function () {}`, ""}, {"nonDefaultOK", ` export let options = { - execution: { + scenarios: { per_vu_iters: { - type: "per-vu-iterations", + executor: "per-vu-iterations", vus: 1, iterations: 1, exec: "nonDefault", @@ -170,9 +170,9 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { let errors = new Counter("errors"); export let options = { - execution: { + scenarios: { executor: { - type: "%[1]s", + executor: "%[1]s", gracefulStop: "0.5s", %[2]s } @@ -193,7 +193,7 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { duration: "0.5s", preAllocatedVUs: 1, maxVUs: 2,`, - "constant-looping-vus": ` + "constant-vus": ` vus: 1, duration: "0.5s",`, "externally-controlled": ` @@ -205,13 +205,13 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { "shared-iterations": ` vus: 1, iterations: 1,`, - "variable-arrival-rate": ` + "ramping-arrival-rate": ` startRate: 1, timeUnit: "0.5s", preAllocatedVUs: 1, maxVUs: 2, stages: [ { target: 1, duration: "0.5s" } ],`, - "variable-looping-vus": ` + "ramping-vus": ` startVUs: 1, stages: [ { target: 1, duration: "0.5s" } ],`, } @@ -275,9 +275,9 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { import http from "k6/http"; export let options = { - execution: { + scenarios: { executor: { - type: "%s", + executor: "%s", gracefulStop: "0.5s", %s } @@ -295,7 +295,7 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { duration: "0.5s", preAllocatedVUs: 1, maxVUs: 2,`, - "constant-looping-vus": ` + "constant-vus": ` vus: 1, duration: "0.5s",`, "externally-controlled": ` @@ -307,13 +307,13 @@ func TestExecutionSchedulerRunCustomTags(t *testing.T) { "shared-iterations": ` vus: 1, iterations: 1,`, - "variable-arrival-rate": ` + "ramping-arrival-rate": ` startRate: 5, timeUnit: "0.5s", preAllocatedVUs: 1, maxVUs: 2, stages: [ { target: 10, duration: "1s" } ],`, - "variable-looping-vus": ` + "ramping-vus": ` startVUs: 1, stages: [ { target: 1, duration: "0.5s" } ],`, } @@ -397,9 +397,9 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { export let options = { // Required for WS tests hosts: { 'httpbin.local': '127.0.0.1' }, - execution: { + scenarios: { scenario1: { - type: 'per-vu-iterations', + executor: 'per-vu-iterations', vus: 1, iterations: 1, gracefulStop: '0s', @@ -409,7 +409,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { tags: { testtag1: 'scenario1' }, }, scenario2: { - type: 'shared-iterations', + executor: 'shared-iterations', vus: 1, iterations: 1, gracefulStop: '1s', @@ -420,7 +420,7 @@ func TestExecutionSchedulerRunCustomConfigNoCrossover(t *testing.T) { tags: { testtag2: 'scenario2' }, }, scenario3: { - type: 'per-vu-iterations', + executor: 'per-vu-iterations', vus: 1, iterations: 1, gracefulStop: '1s', @@ -1210,19 +1210,19 @@ func TestNewExecutionSchedulerHasWork(t *testing.T) { export let options = { executionSegment: "3/4:1", executionSegmentSequence: "0,1/4,2/4,3/4,1", - execution: { + scenarios: { shared_iters1: { - type: "shared-iterations", + executor: "shared-iterations", vus: 3, iterations: 3, }, shared_iters2: { - type: "shared-iterations", + executor: "shared-iterations", vus: 4, iterations: 4, }, constant_arr_rate: { - type: "constant-arrival-rate", + executor: "constant-arrival-rate", rate: 3, timeUnit: "1s", duration: "20s", diff --git a/js/runner_test.go b/js/runner_test.go index 9b914cfdf23..3214b0bd3eb 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -252,9 +252,9 @@ func TestSetupDataIsolation(t *testing.T) { var Counter = require("k6/metrics").Counter; exports.options = { - execution: { + scenarios: { shared_iters: { - type: "shared-iterations", + executor: "shared-iterations", vus: 5, iterations: 500, }, diff --git a/lib/archive.go b/lib/archive.go index 0931c9c14d0..bdbff280546 100644 --- a/lib/archive.go +++ b/lib/archive.go @@ -117,6 +117,8 @@ func (arc *Archive) getFs(name string) afero.Fs { // 83193f8a96e06a190325b838b2cc451119d6b836. This basically means k6 v0.24.0 and // surrounding master commits. We filter these out by the value of the k6version // property, saved in the metadata.json since the previous to the above commit. +// +//TODO: delete? now that execution has been renamed to scenarios, this isn't needed func CleanUpWrongMetadataJSON(data []byte) ([]byte, error) { var tmpArc map[string]interface{} if err := json.Unmarshal(data, &tmpArc); err != nil { diff --git a/lib/executor/base_config.go b/lib/executor/base_config.go index c9405a03a8a..40bbe461ea9 100644 --- a/lib/executor/base_config.go +++ b/lib/executor/base_config.go @@ -43,7 +43,7 @@ const executorNameErr = "the executor name should contain only numbers, latin le // BaseConfig contains the common config fields for all executors type BaseConfig struct { Name string `json:"-"` // set via the JS object key - Type string `json:"type"` + Type string `json:"executor"` StartTime types.NullDuration `json:"startTime"` GracefulStop types.NullDuration `json:"gracefulStop"` Env map[string]string `json:"env"` diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 9f21a0deb5a..25902706c33 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -35,7 +35,7 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const constantLoopingVUsType = "constant-looping-vus" +const constantLoopingVUsType = "constant-vus" func init() { lib.RegisterExecutorConfigType( diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 2e3457133d4..40f75d5a6c0 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -60,12 +60,12 @@ var configMapTestCases = []configMapTestCase{ assert.Nil(t, cm) }}}, {`{"someKey": {}}`, exp{parseError: true}}, - {`{"someKey": {"type": "constant-blah-blah", "vus": 10, "duration": "60s"}}`, exp{parseError: true}}, - {`{"someKey": {"type": "constant-looping-vus", "uknownField": "should_error"}}`, exp{parseError: true}}, - {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", "env": 123}}`, exp{parseError: true}}, + {`{"someKey": {"executor": "constant-blah-blah", "vus": 10, "duration": "60s"}}`, exp{parseError: true}}, + {`{"someKey": {"executor": "constant-vus", "uknownField": "should_error"}}`, exp{parseError: true}}, + {`{"someKey": {"executor": "constant-vus", "vus": 10, "duration": "60s", "env": 123}}`, exp{parseError: true}}, - // Validation errors for constant-looping-vus and the base config - {`{"someKey": {"type": "constant-looping-vus", "vus": 10, "duration": "60s", + // Validation errors for constant-vus and the base config + {`{"someKey": {"executor": "constant-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewConstantLoopingVUsConfig("someKey") @@ -108,19 +108,19 @@ var configMapTestCases = []configMapTestCase{ }}, }, - {`{"aname": {"type": "constant-looping-vus", "duration": "60s"}}`, exp{}}, - {`{"": {"type": "constant-looping-vus", "vus": 10, "duration": "60s"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 0.5}}`, exp{parseError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 0, "duration": "60s"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": -1, "duration": "60s"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "0s"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "startTime": "-10s"}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "exec": ""}}`, exp{validationError: true}}, - {`{"aname": {"type": "constant-looping-vus", "vus": 10, "duration": "10s", "gracefulStop": "-2s"}}`, exp{validationError: true}}, - // variable-looping-vus - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", + {`{"aname": {"executor": "constant-vus", "duration": "60s"}}`, exp{}}, + {`{"": {"executor": "constant-vus", "vus": 10, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 0.5}}`, exp{parseError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 10}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 0, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": -1, "duration": "60s"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 10, "duration": "0s"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 10, "duration": "10s", "startTime": "-10s"}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 10, "duration": "10s", "exec": ""}}`, exp{validationError: true}}, + {`{"aname": {"executor": "constant-vus", "vus": 10, "duration": "10s", "gracefulStop": "-2s"}}`, exp{validationError: true}}, + // ramping-vus + {`{"varloops": {"executor": "ramping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", "startTime": "23s", "stages": [{"duration": "60s", "target": 30}, {"duration": "130s", "target": 10}]}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewVariableLoopingVUsConfig("varloops") @@ -156,7 +156,7 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, uint64(30), lib.GetMaxPossibleVUs(schedReqs)) }}, }, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "10s", + {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "10s", "stages": [{"duration": "10s", "target": 10}]}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { assert.Empty(t, cm["varloops"].Validate()) @@ -171,7 +171,7 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) }}, }, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", + {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { assert.Empty(t, cm["varloops"].Validate()) @@ -186,7 +186,7 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, uint64(10), lib.GetMaxPossibleVUs(schedReqs)) }}, }, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", + {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 11},{"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { assert.Empty(t, cm["varloops"].Validate()) @@ -201,16 +201,16 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, uint64(11), lib.GetMaxPossibleVUs(schedReqs)) }}, }, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 0, "stages": [{"duration": "60s", "target": 0}]}}`, exp{}}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": -1, "stages": [{"duration": "60s", "target": 30}]}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "-60s", "target": 30}]}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus", "startVUs": 2, "stages": [{"duration": "60s", "target": -30}]}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus", "stages": [{"duration": "60s"}]}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus", "stages": [{"target": 30}]}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus", "stages": []}}`, exp{validationError: true}}, - {`{"varloops": {"type": "variable-looping-vus"}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "startVUs": 0, "stages": [{"duration": "60s", "target": 0}]}}`, exp{}}, + {`{"varloops": {"executor": "ramping-vus", "startVUs": -1, "stages": [{"duration": "60s", "target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "startVUs": 2, "stages": [{"duration": "-60s", "target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "startVUs": 2, "stages": [{"duration": "60s", "target": -30}]}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "stages": [{"duration": "60s"}]}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "stages": [{"target": 30}]}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus", "stages": []}}`, exp{validationError: true}}, + {`{"varloops": {"executor": "ramping-vus"}}`, exp{validationError: true}}, // shared-iterations - {`{"ishared": {"type": "shared-iterations", "iterations": 22, "vus": 12, "maxDuration": "100s"}}`, + {`{"ishared": {"executor": "shared-iterations", "iterations": 22, "vus": 12, "maxDuration": "100s"}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewSharedIterationsConfig("ishared") sched.Iterations = null.IntFrom(22) @@ -275,17 +275,17 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, schedReqs, totalReqs) }}, }, - {`{"ishared": {"type": "shared-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values - {`{"ishared": {"type": "shared-iterations", "iterations": 20}}`, exp{}}, - {`{"ishared": {"type": "shared-iterations", "vus": 10}}`, exp{validationError: true}}, // error because VUs are more than iters - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "30m"}}`, exp{}}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, - {`{"ishared": {"type": "shared-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, - {`{"ishared": {"type": "shared-iterations", "iterations": 20, "vus": 30}}`, exp{validationError: true}}, + {`{"ishared": {"executor": "shared-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values + {`{"ishared": {"executor": "shared-iterations", "iterations": 20}}`, exp{}}, + {`{"ishared": {"executor": "shared-iterations", "vus": 10}}`, exp{validationError: true}}, // error because VUs are more than iters + {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "30m"}}`, exp{}}, + {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, + {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, + {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, + {`{"ishared": {"executor": "shared-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, + {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": 30}}`, exp{validationError: true}}, // per-vu-iterations - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 23, "vus": 13, "gracefulStop": 0}}`, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 23, "vus": 13, "gracefulStop": 0}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewPerVUIterationsConfig("ipervu") sched.Iterations = null.IntFrom(23) @@ -310,17 +310,17 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, schedReqs, totalReqs) }}, }, - {`{"ipervu": {"type": "per-vu-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20}}`, exp{}}, - {`{"ipervu": {"type": "per-vu-iterations", "vus": 10}}`, exp{}}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10}}`, exp{}}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, - {`{"ipervu": {"type": "per-vu-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, + {`{"ipervu": {"executor": "per-vu-iterations"}}`, exp{}}, // Has 1 VU & 1 iter default values + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 20}}`, exp{}}, + {`{"ipervu": {"executor": "per-vu-iterations", "vus": 10}}`, exp{}}, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 20, "vus": 10}}`, exp{}}, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "-3m"}}`, exp{validationError: true}}, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 20, "vus": 10, "maxDuration": "0s"}}`, exp{validationError: true}}, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 20, "vus": -10}}`, exp{validationError: true}}, + {`{"ipervu": {"executor": "per-vu-iterations", "iterations": -1, "vus": 1}}`, exp{validationError: true}}, // constant-arrival-rate - {`{"carrival": {"type": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) @@ -347,19 +347,19 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, schedReqs, totalReqs) }}, }, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30, "timeUnit": "-1s"}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "maxVUs": 30}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 0, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 15}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "0s", "preAllocatedVUs": 20, "maxVUs": 25}}`, exp{validationError: true}}, - {`{"carrival": {"type": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": -2, "maxVUs": 25}}`, exp{validationError: true}}, - // variable-arrival-rate - {`{"varrival": {"type": "variable-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30, "timeUnit": "-1s"}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "0m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 0, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 15}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "0s", "preAllocatedVUs": 20, "maxVUs": 25}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": -2, "maxVUs": 25}}`, exp{validationError: true}}, + // ramping-arrival-rate + {`{"varrival": {"executor": "ramping-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { sched := NewVariableArrivalRateConfig("varrival") @@ -391,15 +391,15 @@ var configMapTestCases = []configMapTestCase{ assert.Equal(t, schedReqs, totalReqs) }}, }, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": -20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "startRate": -1, "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}], "timeUnit": "-1s"}}`, exp{validationError: true}}, - {`{"varrival": {"type": "variable-arrival-rate", "preAllocatedVUs": 30, "maxVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": -20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "startRate": -1, "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}], "timeUnit": "-1s"}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 30, "maxVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, //TODO: more tests of mixed executors and execution plans } diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 52a7982b3a1..ce5f63e3926 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -326,7 +326,7 @@ func (mex *ExternallyControlled) stopWhenDurationIsReached(ctx context.Context, } // manualVUHandle is a wrapper around the vuHandle helper, used in the -// variable-looping-vus executor. Here, instead of using its getVU and returnVU +// ramping-vus executor. Here, instead of using its getVU and returnVU // methods to retrieve and return a VU from the global buffer, we use them to // accurately update the local and global active VU counters and to ensure that // the pausing and reducing VUs operations wait for VUs to fully finish diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index f41f541a453..696f3560a37 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -37,7 +37,7 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const variableArrivalRateType = "variable-arrival-rate" +const variableArrivalRateType = "ramping-arrival-rate" func init() { lib.RegisterExecutorConfigType( diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 0350a8fae74..9abd74b42bb 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -36,7 +36,7 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const variableLoopingVUsType = "variable-looping-vus" +const variableLoopingVUsType = "ramping-vus" func init() { lib.RegisterExecutorConfigType( diff --git a/lib/executors.go b/lib/executors.go index 81dddc70433..b9e55c65db0 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -56,7 +56,7 @@ var ( // // Keep in mind that t(i) may be exactly equal to t(i+i), when there's an abrupt // transition in the number of VUs required by an executor. For example, the -// variable-looping-vus executor may have 0-duration stages, or it may scale up +// ramping-vus executor may have 0-duration stages, or it may scale up // VUs in its last stage right until the end. These immediate transitions cannot // be ignored, since the gracefulStop/gracefulRampDown options potentially allow // any started iterations to finish. @@ -338,7 +338,7 @@ type protoExecutorConfig struct { // stores the unprocessed JSON so we can parse the full config in the next step func (pc *protoExecutorConfig) UnmarshalJSON(b []byte) error { var tmp struct { - ConfigType string `json:"type"` + ConfigType string `json:"executor"` } err := json.Unmarshal(b, &tmp) *pc = protoExecutorConfig{tmp.ConfigType, b} diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index 9c262495e05..c682e0fb3f3 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -233,7 +233,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { var emptyConfigMap ExecutorConfigMap var tests = []struct { metadata string - expExecution interface{} + expScenarios interface{} expError string }{ { @@ -241,7 +241,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { "filename": "/test.js", "pwd": "/", "options": { "execution": { "something": "invalid" } } }`, - expExecution: emptyConfigMap, + expScenarios: emptyConfigMap, }, { metadata: `{ @@ -249,7 +249,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { "k6version": "0.24.0", "options": { "execution": { "something": "invalid" } } }`, - expExecution: emptyConfigMap, + expScenarios: emptyConfigMap, }, { metadata: `blah`, @@ -274,7 +274,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { metadata: `{ "filename": "/test.js", "pwd": "/", "k6version": "0.25.0", - "options": { "execution": { "something": "invalid" } } + "options": { "scenarios": { "something": "invalid" } } }`, expError: "cannot unmarshal string", }, @@ -294,7 +294,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { require.Contains(t, err.Error(), test.expError) } else { require.NoError(t, err) - require.Equal(t, test.expExecution, arc.Options.Execution) + require.Equal(t, test.expScenarios, arc.Options.Execution) } } } diff --git a/lib/options.go b/lib/options.go index 1d92964236c..c33cb7104aa 100644 --- a/lib/options.go +++ b/lib/options.go @@ -204,7 +204,7 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Execution ExecutorConfigMap `json:"execution,omitempty" ignored:"true"` + Execution ExecutorConfigMap `json:"scenarios,omitempty" ignored:"true"` ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` ExecutionSegmentSequence *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` From dcf2be335d47e8ce09c6728a7d67daad4fc3c146 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 15:06:44 +0300 Subject: [PATCH 290/350] Remove mentions of Load Impact Insights --- SUPPORT.md | 2 +- stats/cloud/collector.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/SUPPORT.md b/SUPPORT.md index 12c068d7c7e..2cc8c52d186 100644 --- a/SUPPORT.md +++ b/SUPPORT.md @@ -9,4 +9,4 @@ Types of questions and where to ask: - Why do you? -- [community.k6.io](https://community.k6.io/) or [Slack](https://k6.io/slack) - When will you? -- [community.k6.io](https://community.k6.io/) or [Slack](https://k6.io/slack) -If your questions are about any of the commercial Load Impact services like managed cloud execution and Load Impact Insights, you can contact or write in the `#loadimpact` channel in [Slack](https://k6.io/slack). +If your questions are about any of the commercial Load Impact services like managed cloud execution and k6 cloud output, you can contact or write in the `#loadimpact` channel in [Slack](https://k6.io/slack). diff --git a/stats/cloud/collector.go b/stats/cloud/collector.go index e492af49671..013c3eb0403 100644 --- a/stats/cloud/collector.go +++ b/stats/cloud/collector.go @@ -126,7 +126,7 @@ func New( duration, testEnds := lib.GetEndOffset(executionPlan) if !testEnds { - return nil, errors.New("Tests with unspecified duration are not allowed when using Load Impact Insights") + return nil, errors.New("tests with unspecified duration are not allowed when outputting data to k6 cloud") } if !conf.Token.Valid && conf.DeprecatedToken.Valid { From 594b141b4c2e5b7116063d5ab15a2441f2c4f015 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 15:20:34 +0300 Subject: [PATCH 291/350] Remove CleanUpWrongMetadataJSON() now that we don't need it --- lib/archive.go | 53 +---------------------------------------- lib/old_archive_test.go | 15 ++++-------- 2 files changed, 6 insertions(+), 62 deletions(-) diff --git a/lib/archive.go b/lib/archive.go index bdbff280546..6671989b9a5 100644 --- a/lib/archive.go +++ b/lib/archive.go @@ -108,58 +108,7 @@ func (arc *Archive) getFs(name string) afero.Fs { return fs } -// CleanUpWrongMetadataJSON fixes issues with the metadata.json contents before -// they are unmarshalled in the Archive struct. -// -// Currently, the only fix this function performs is the discarding of the -// derived `execution` config value in the consolidated options that was wrongly -// saved by k6 in the archive metadata.json files until commit -// 83193f8a96e06a190325b838b2cc451119d6b836. This basically means k6 v0.24.0 and -// surrounding master commits. We filter these out by the value of the k6version -// property, saved in the metadata.json since the previous to the above commit. -// -//TODO: delete? now that execution has been renamed to scenarios, this isn't needed -func CleanUpWrongMetadataJSON(data []byte) ([]byte, error) { - var tmpArc map[string]interface{} - if err := json.Unmarshal(data, &tmpArc); err != nil { - return nil, err - } - - k6Version := "" - if k6RawVersion, ok := tmpArc["k6version"]; ok { - if k6Version, ok = k6RawVersion.(string); !ok { - return nil, fmt.Errorf("k6version is present in the archive metadata, but it's not a string") - } - } - - // TODO: semantically parse the k6version and compare it with the current - // one, log a warning if the current k6 version in lib/consts is lower than - // the k6 version that generated the archive. - - if k6Version != "" && k6Version != "0.24.0" { - return data, nil - } - - if rawOptions, ok := tmpArc["options"]; !ok { - return nil, fmt.Errorf("missing options key in the archive metadata.json") - } else if options, ok := rawOptions.(map[string]interface{}); !ok { - return nil, fmt.Errorf("wrong options type in metadata.json") - } else if _, hasExecution := options["execution"]; !hasExecution { - return data, nil // no need to fix anything - } else { - delete(options, "execution") - tmpArc["options"] = options - } - - return json.Marshal(tmpArc) -} - -func (arc *Archive) loadMetadataJSON(data []byte) error { - data, err := CleanUpWrongMetadataJSON(data) - if err != nil { - return err - } - +func (arc *Archive) loadMetadataJSON(data []byte) (err error) { if err = json.Unmarshal(data, &arc); err != nil { return err } diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index c682e0fb3f3..833598950e8 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -236,6 +236,8 @@ func TestDerivedExecutionDiscarding(t *testing.T) { expScenarios interface{} expError string }{ + // Tests to make sure that "execution" in the options, the old name for + // "scenarios" before #1007 was merged, doesn't mess up the options... { metadata: `{ "filename": "/test.js", "pwd": "/", @@ -255,20 +257,13 @@ func TestDerivedExecutionDiscarding(t *testing.T) { metadata: `blah`, expError: "invalid character", }, - { - metadata: `{ - "filename": "/test.js", "pwd": "/", - "k6version": "0.24.0" - }`, - expError: "missing options key", - }, { metadata: `{ "filename": "/test.js", "pwd": "/", "k6version": "0.24.0", "options": "something invalid" }`, - expError: "wrong options type in metadata.json", + expError: "cannot unmarshal string into Go struct field", }, { metadata: `{ @@ -278,7 +273,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { }`, expError: "cannot unmarshal string", }, - // TODO: test an actual execution unmarshalling, which is currently + // TODO: test an actual scenarios unmarshalling, which is currently // impossible due to import cycles... } @@ -290,7 +285,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { arc, err := ReadArchive(buf) if test.expError != "" { - require.Error(t, err) + require.Errorf(t, err, "expected error '%s' but got nil", test.expError) require.Contains(t, err.Error(), test.expError) } else { require.NoError(t, err) From 0bc0470fcc70690151a0a6db8590753aa656d41e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 15:39:53 +0300 Subject: [PATCH 292/350] Rename Go names from Execution to Scenarios --- cmd/config.go | 8 ++--- cmd/config_consolidation_test.go | 12 +++---- cmd/config_test.go | 4 +-- core/engine_test.go | 2 +- core/local/local.go | 4 +-- core/local/local_test.go | 10 +++--- lib/executor/execution_config_shortcuts.go | 42 +++++++++++----------- lib/old_archive_test.go | 2 +- lib/options.go | 12 +++---- 9 files changed, 48 insertions(+), 48 deletions(-) diff --git a/cmd/config.go b/cmd/config.go index 1183031459d..8d635548b42 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -264,7 +264,7 @@ func applyDefault(conf Config) Config { func deriveAndValidateConfig(conf Config, isExecutable func(string) bool) (result Config, err error) { result = conf - result.Options, err = executor.DeriveExecutionFromShortcuts(conf.Options) + result.Options, err = executor.DeriveScenariosFromShortcuts(conf.Options) if err != nil { return result, err } @@ -274,8 +274,8 @@ func deriveAndValidateConfig(conf Config, isExecutable func(string) bool) (resul func validateConfig(conf Config, isExecutable func(string) bool) error { errList := conf.Validate() - for _, ec := range conf.Execution { - if err := validateExecutorConfig(ec, isExecutable); err != nil { + for _, ec := range conf.Scenarios { + if err := validateScenarioConfig(ec, isExecutable); err != nil { errList = append(errList, err) } } @@ -292,7 +292,7 @@ func validateConfig(conf Config, isExecutable func(string) bool) error { return errors.New(strings.Join(errMsgParts, "\n")) } -func validateExecutorConfig(conf lib.ExecutorConfig, isExecutable func(string) bool) error { +func validateScenarioConfig(conf lib.ExecutorConfig, isExecutable func(string) bool) error { execFn := conf.GetExec() if !isExecutable(execFn) { return fmt.Errorf("executor %s: function '%s' not found in exports", conf.GetName(), execFn) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index a3d29c7903d..e44303e9d68 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -43,7 +43,7 @@ import ( func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config - exec := c.Execution[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultExecutorName] require.NotEmpty(t, exec) require.IsType(t, executor.PerVUIterationsConfig{}, exec) perVuIters, ok := exec.(executor.PerVUIterationsConfig) @@ -54,7 +54,7 @@ func verifyOneIterPerOneVU(t *testing.T, c Config) { func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Execution[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultExecutorName] require.NotEmpty(t, exec) require.IsType(t, executor.SharedIterationsConfig{}, exec) sharedIterConfig, ok := exec.(executor.SharedIterationsConfig) @@ -68,7 +68,7 @@ func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Execution[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultExecutorName] require.NotEmpty(t, exec) require.IsType(t, executor.ConstantLoopingVUsConfig{}, exec) clvc, ok := exec.(executor.ConstantLoopingVUsConfig) @@ -82,7 +82,7 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing func verifyVarLoopingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Execution[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultExecutorName] require.NotEmpty(t, exec) require.IsType(t, executor.VariableLoopingVUsConfig{}, exec) clvc, ok := exec.(executor.VariableLoopingVUsConfig) @@ -223,7 +223,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) - sharedIterConfig := c.Execution[lib.DefaultExecutorName].(executor.SharedIterationsConfig) + sharedIterConfig := c.Scenarios[lib.DefaultExecutorName].(executor.SharedIterationsConfig) assert.Equal(t, time.Duration(sharedIterConfig.MaxDuration.Duration), 10*time.Second) }}, // This should get a validation error since VUs are more than the shared iterations @@ -437,7 +437,7 @@ func runTestCase( require.NoError(t, err) derivedConfig := consolidatedConfig - derivedConfig.Options, err = executor.DeriveExecutionFromShortcuts(consolidatedConfig.Options) + derivedConfig.Options, err = executor.DeriveScenariosFromShortcuts(consolidatedConfig.Options) if testCase.expected.derivationError { require.Error(t, err) return diff --git a/cmd/config_test.go b/cmd/config_test.go index 35cf32f0cd6..714386e3218 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -152,7 +152,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { {"defaultOK", Config{}, true, ""}, {"defaultErr", Config{}, false, "executor default: function 'default' not found in exports"}, - {"nonDefaultOK", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ + {"nonDefaultOK", Config{Options: lib.Options{Scenarios: lib.ExecutorConfigMap{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefault")}, VUs: null.IntFrom(1), @@ -160,7 +160,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { MaxDuration: types.NullDurationFrom(time.Second), }}}}, true, "", }, - {"nonDefaultErr", Config{Options: lib.Options{Execution: lib.ExecutorConfigMap{ + {"nonDefaultErr", Config{Options: lib.Options{Scenarios: lib.ExecutorConfigMap{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefaultErr")}, VUs: null.IntFrom(1), diff --git a/core/engine_test.go b/core/engine_test.go index 853d07d6940..6c6d2c8f9b0 100644 --- a/core/engine_test.go +++ b/core/engine_test.go @@ -62,7 +62,7 @@ func newTestEngine( //nolint:golint runCtx, runCancel = context.WithCancel(globalCtx) } - newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ + newOpts, err := executor.DeriveScenariosFromShortcuts(lib.Options{ MetricSamplesBufferSize: null.NewInt(200, false), }.Apply(runner.GetOptions()).Apply(opts)) require.NoError(t, err) diff --git a/core/local/local.go b/core/local/local.go index d535f63a03b..0fc80a993aa 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -62,14 +62,14 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution if err != nil { return nil, err } - executionPlan := options.Execution.GetFullExecutionRequirements(et) + executionPlan := options.Scenarios.GetFullExecutionRequirements(et) maxPlannedVUs := lib.GetMaxPlannedVUs(executionPlan) maxPossibleVUs := lib.GetMaxPossibleVUs(executionPlan) executionState := lib.NewExecutionState(options, et, maxPlannedVUs, maxPossibleVUs) maxDuration, _ := lib.GetEndOffset(executionPlan) // we don't care if the end offset is final - executorConfigs := options.Execution.GetSortedConfigs() + executorConfigs := options.Scenarios.GetSortedConfigs() executors := make([]lib.Executor, 0, len(executorConfigs)) // Only take executors which have work. for _, sc := range executorConfigs { diff --git a/core/local/local_test.go b/core/local/local_test.go index a2c1ad45422..229be833606 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -59,7 +59,7 @@ func newTestExecutionScheduler( runner = &minirunner.MiniRunner{} } ctx, cancel = context.WithCancel(context.Background()) - newOpts, err := executor.DeriveExecutionFromShortcuts(lib.Options{ + newOpts, err := executor.DeriveScenariosFromShortcuts(lib.Options{ MetricSamplesBufferSize: null.NewInt(200, false), }.Apply(runner.GetOptions()).Apply(opts)) require.NoError(t, err) @@ -773,7 +773,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { return errors.New("hi") }, Options: lib.Options{ - Execution: lib.ExecutorConfigMap{exec.GetName(): exec}, + Scenarios: lib.ExecutorConfigMap{exec.GetName(): exec}, }, } logger, hook := logtest.NewNullLogger() @@ -797,7 +797,7 @@ func TestExecutionSchedulerEndIterations(t *testing.T) { t.Parallel() metric := &stats.Metric{Name: "test_metric"} - options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ + options, err := executor.DeriveScenariosFromShortcuts(lib.Options{ VUs: null.IntFrom(1), Iterations: null.IntFrom(100), }) @@ -1007,7 +1007,7 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { runner, err := js.New(&loader.SourceData{URL: &url.URL{Path: "/script.js"}, Data: script}, nil, lib.RuntimeOptions{}) require.NoError(t, err) - options, err := executor.DeriveExecutionFromShortcuts(runner.GetOptions().Apply(lib.Options{ + options, err := executor.DeriveScenariosFromShortcuts(runner.GetOptions().Apply(lib.Options{ Iterations: null.IntFrom(2), VUs: null.IntFrom(1), SystemTags: &stats.DefaultSystemTagSet, @@ -1185,7 +1185,7 @@ func TestSetPaused(t *testing.T) { t.Run("can't pause unpausable executor", func(t *testing.T) { runner := &minirunner.MiniRunner{} - options, err := executor.DeriveExecutionFromShortcuts(lib.Options{ + options, err := executor.DeriveScenariosFromShortcuts(lib.Options{ Iterations: null.IntFrom(2), VUs: null.IntFrom(1), }.Apply(runner.GetOptions())) diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index 907cb9cd52f..eabe62bb9eb 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -29,7 +29,7 @@ import ( ) // ExecutionConflictError is a custom error type used for all of the errors in -// the DeriveExecutionFromShortcuts() function. +// the DeriveScenariosFromShortcuts() function. type ExecutionConflictError string func (e ExecutionConflictError) Error() string { @@ -38,14 +38,14 @@ func (e ExecutionConflictError) Error() string { var _ error = ExecutionConflictError("") -func getConstantLoopingVUsExecution(duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { +func getConstantLoopingVUsScenario(duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { ds := NewConstantLoopingVUsConfig(lib.DefaultExecutorName) ds.VUs = vus ds.Duration = duration return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } -func getVariableLoopingVUsExecution(stages []lib.Stage, startVUs null.Int) lib.ExecutorConfigMap { +func getVariableLoopingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ExecutorConfigMap { ds := NewVariableLoopingVUsConfig(lib.DefaultExecutorName) ds.StartVUs = startVUs for _, s := range stages { @@ -56,7 +56,7 @@ func getVariableLoopingVUsExecution(stages []lib.Stage, startVUs null.Int) lib.E return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } -func getSharedIterationsExecution(iters null.Int, duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { +func getSharedIterationsScenario(iters null.Int, duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { ds := NewSharedIterationsConfig(lib.DefaultExecutorName) ds.VUs = vus ds.Iterations = iters @@ -66,10 +66,10 @@ func getSharedIterationsExecution(iters null.Int, duration types.NullDuration, v return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} } -// DeriveExecutionFromShortcuts checks for conflicting options and turns any +// DeriveScenariosFromShortcuts checks for conflicting options and turns any // shortcut options (i.e. duration, iterations, stages) into the proper -// long-form executor configuration in the execution property. -func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { +// long-form scenario/executor configuration in the scenarios property. +func DeriveScenariosFromShortcuts(opts lib.Options) (lib.Options, error) { result := opts switch { @@ -79,12 +79,12 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { "using multiple execution config shortcuts (`iterations` and `stages`) simultaneously is not allowed", ) } - if opts.Execution != nil { + if opts.Scenarios != nil { return opts, ExecutionConflictError( - "using an execution configuration shortcut (`iterations`) and `execution` simultaneously is not allowed", + "using an execution configuration shortcut (`iterations`) and `scenarios` simultaneously is not allowed", ) } - result.Execution = getSharedIterationsExecution(opts.Iterations, opts.Duration, opts.VUs) + result.Scenarios = getSharedIterationsScenario(opts.Iterations, opts.Duration, opts.VUs) case opts.Duration.Valid: if len(opts.Stages) > 0 { // stages isn't nil (not set) and isn't explicitly set to empty @@ -92,9 +92,9 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { "using multiple execution config shortcuts (`duration` and `stages`) simultaneously is not allowed", ) } - if opts.Execution != nil { + if opts.Scenarios != nil { return result, ExecutionConflictError( - "using an execution configuration shortcut (`duration`) and `execution` simultaneously is not allowed", + "using an execution configuration shortcut (`duration`) and `scenarios` simultaneously is not allowed", ) } if opts.Duration.Duration <= 0 { @@ -103,18 +103,18 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { "`duration` should be more than 0, for infinite duration use the externally-controlled executor", ) } - result.Execution = getConstantLoopingVUsExecution(opts.Duration, opts.VUs) + result.Scenarios = getConstantLoopingVUsScenario(opts.Duration, opts.VUs) case len(opts.Stages) > 0: // stages isn't nil (not set) and isn't explicitly set to empty - if opts.Execution != nil { + if opts.Scenarios != nil { return opts, ExecutionConflictError( - "using an execution configuration shortcut (`stages`) and `execution` simultaneously is not allowed", + "using an execution configuration shortcut (`stages`) and `scenarios` simultaneously is not allowed", ) } - result.Execution = getVariableLoopingVUsExecution(opts.Stages, opts.VUs) + result.Scenarios = getVariableLoopingVUsScenario(opts.Stages, opts.VUs) - case len(opts.Execution) > 0: - // Do nothing, execution was explicitly specified + case len(opts.Scenarios) > 0: + // Do nothing, scenarios was explicitly specified default: // Check if we should emit some warnings @@ -128,13 +128,13 @@ func DeriveExecutionFromShortcuts(opts lib.Options) (lib.Options, error) { // No someone explicitly set stages to empty logrus.Warnf("`stages` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") } - if opts.Execution != nil && len(opts.Execution) == 0 { + if opts.Scenarios != nil && len(opts.Scenarios) == 0 { // No shortcut, and someone explicitly set execution to empty - logrus.Warnf("`execution` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") + logrus.Warnf("`scenarios` was explicitly set to an empty value, running the script with 1 iteration in 1 VU") } // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config // with 1 VU and 1 iteration. - result.Execution = lib.ExecutorConfigMap{ + result.Scenarios = lib.ExecutorConfigMap{ lib.DefaultExecutorName: NewPerVUIterationsConfig(lib.DefaultExecutorName), } } diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index 833598950e8..384f943c524 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -289,7 +289,7 @@ func TestDerivedExecutionDiscarding(t *testing.T) { require.Contains(t, err.Error(), test.expError) } else { require.NoError(t, err) - require.Equal(t, test.expScenarios, arc.Options.Execution) + require.Equal(t, test.expScenarios, arc.Options.Scenarios) } } } diff --git a/lib/options.go b/lib/options.go index c33cb7104aa..d82b4406304 100644 --- a/lib/options.go +++ b/lib/options.go @@ -204,7 +204,7 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Execution ExecutorConfigMap `json:"scenarios,omitempty" ignored:"true"` + Scenarios ExecutorConfigMap `json:"scenarios,omitempty" ignored:"true"` ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` ExecutionSegmentSequence *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` @@ -314,12 +314,12 @@ func (o Options) Apply(opts Options) Options { // Still, if more than one of those options is simultaneously specified in the same // config tier, they will be preserved, so the validation after we've consolidated // all of the options can return an error. - if opts.Duration.Valid || opts.Iterations.Valid || opts.Stages != nil || opts.Execution != nil { + if opts.Duration.Valid || opts.Iterations.Valid || opts.Stages != nil || opts.Scenarios != nil { // TODO: emit a warning or a notice log message if overwrite lower tier config options? o.Duration = types.NewNullDuration(0, false) o.Iterations = null.NewInt(0, false) o.Stages = nil - o.Execution = nil + o.Scenarios = nil } if opts.Duration.Valid { @@ -340,8 +340,8 @@ func (o Options) Apply(opts Options) Options { // that happens after the configuration from the different sources is consolidated. It can't // happen here, because something like `K6_ITERATIONS=10 k6 run --vus 5 script.js` wont't // work correctly at this level. - if opts.Execution != nil { - o.Execution = opts.Execution + if opts.Scenarios != nil { + o.Scenarios = opts.Scenarios } if opts.ExecutionSegment != nil { o.ExecutionSegment = opts.ExecutionSegment @@ -466,7 +466,7 @@ func (o Options) Validate() []error { o.ExecutionSegment, o.ExecutionSegmentSequence)) } } - return append(errors, o.Execution.Validate()...) + return append(errors, o.Scenarios.Validate()...) } // ForEachSpecified enumerates all struct fields and calls the supplied function with each From 3fc886e4028ada29f766c4f0adb01661291b3702 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 16:04:20 +0300 Subject: [PATCH 293/350] Rename more Go names from Execution to Scenarios --- cmd/config_consolidation_test.go | 10 +++---- cmd/config_test.go | 4 +-- core/local/local_test.go | 2 +- lib/executor/execution_config_shortcuts.go | 22 +++++++------- lib/executor/executors_test.go | 34 +++++++++++----------- lib/executors.go | 16 +++++----- lib/old_archive_test.go | 2 +- lib/options.go | 6 ++-- 8 files changed, 48 insertions(+), 48 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index e44303e9d68..2703480c6e0 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -43,7 +43,7 @@ import ( func verifyOneIterPerOneVU(t *testing.T, c Config) { // No config anywhere should result in a 1 VU with a 1 iteration config - exec := c.Scenarios[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) require.IsType(t, executor.PerVUIterationsConfig{}, exec) perVuIters, ok := exec.(executor.PerVUIterationsConfig) @@ -54,7 +54,7 @@ func verifyOneIterPerOneVU(t *testing.T, c Config) { func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Scenarios[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) require.IsType(t, executor.SharedIterationsConfig{}, exec) sharedIterConfig, ok := exec.(executor.SharedIterationsConfig) @@ -68,7 +68,7 @@ func verifySharedIters(vus, iters null.Int) func(t *testing.T, c Config) { func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Scenarios[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) require.IsType(t, executor.ConstantLoopingVUsConfig{}, exec) clvc, ok := exec.(executor.ConstantLoopingVUsConfig) @@ -82,7 +82,7 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing func verifyVarLoopingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { - exec := c.Scenarios[lib.DefaultExecutorName] + exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) require.IsType(t, executor.VariableLoopingVUsConfig{}, exec) clvc, ok := exec.(executor.VariableLoopingVUsConfig) @@ -223,7 +223,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) - sharedIterConfig := c.Scenarios[lib.DefaultExecutorName].(executor.SharedIterationsConfig) + sharedIterConfig := c.Scenarios[lib.DefaultScenarioName].(executor.SharedIterationsConfig) assert.Equal(t, time.Duration(sharedIterConfig.MaxDuration.Duration), 10*time.Second) }}, // This should get a validation error since VUs are more than the shared iterations diff --git a/cmd/config_test.go b/cmd/config_test.go index 714386e3218..5a4595307ed 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -152,7 +152,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { {"defaultOK", Config{}, true, ""}, {"defaultErr", Config{}, false, "executor default: function 'default' not found in exports"}, - {"nonDefaultOK", Config{Options: lib.Options{Scenarios: lib.ExecutorConfigMap{ + {"nonDefaultOK", Config{Options: lib.Options{Scenarios: lib.ScenarioConfigs{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefault")}, VUs: null.IntFrom(1), @@ -160,7 +160,7 @@ func TestDeriveAndValidateConfig(t *testing.T) { MaxDuration: types.NullDurationFrom(time.Second), }}}}, true, "", }, - {"nonDefaultErr", Config{Options: lib.Options{Scenarios: lib.ExecutorConfigMap{ + {"nonDefaultErr", Config{Options: lib.Options{Scenarios: lib.ScenarioConfigs{ "per_vu_iters": executor.PerVUIterationsConfig{BaseConfig: executor.BaseConfig{ Name: "per_vu_iters", Type: "per-vu-iterations", Exec: null.StringFrom("nonDefaultErr")}, VUs: null.IntFrom(1), diff --git a/core/local/local_test.go b/core/local/local_test.go index 229be833606..9fd0ab6cefa 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -773,7 +773,7 @@ func TestExecutionSchedulerEndErrors(t *testing.T) { return errors.New("hi") }, Options: lib.Options{ - Scenarios: lib.ExecutorConfigMap{exec.GetName(): exec}, + Scenarios: lib.ScenarioConfigs{exec.GetName(): exec}, }, } logger, hook := logtest.NewNullLogger() diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index eabe62bb9eb..ff9ae32a313 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -38,32 +38,32 @@ func (e ExecutionConflictError) Error() string { var _ error = ExecutionConflictError("") -func getConstantLoopingVUsScenario(duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { - ds := NewConstantLoopingVUsConfig(lib.DefaultExecutorName) +func getConstantLoopingVUsScenario(duration types.NullDuration, vus null.Int) lib.ScenarioConfigs { + ds := NewConstantLoopingVUsConfig(lib.DefaultScenarioName) ds.VUs = vus ds.Duration = duration - return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} + return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} } -func getVariableLoopingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ExecutorConfigMap { - ds := NewVariableLoopingVUsConfig(lib.DefaultExecutorName) +func getVariableLoopingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ScenarioConfigs { + ds := NewVariableLoopingVUsConfig(lib.DefaultScenarioName) ds.StartVUs = startVUs for _, s := range stages { if s.Duration.Valid { ds.Stages = append(ds.Stages, Stage{Duration: s.Duration, Target: s.Target}) } } - return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} + return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} } -func getSharedIterationsScenario(iters null.Int, duration types.NullDuration, vus null.Int) lib.ExecutorConfigMap { - ds := NewSharedIterationsConfig(lib.DefaultExecutorName) +func getSharedIterationsScenario(iters null.Int, duration types.NullDuration, vus null.Int) lib.ScenarioConfigs { + ds := NewSharedIterationsConfig(lib.DefaultScenarioName) ds.VUs = vus ds.Iterations = iters if duration.Valid { ds.MaxDuration = duration } - return lib.ExecutorConfigMap{lib.DefaultExecutorName: ds} + return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} } // DeriveScenariosFromShortcuts checks for conflicting options and turns any @@ -134,8 +134,8 @@ func DeriveScenariosFromShortcuts(opts lib.Options) (lib.Options, error) { } // No execution parameters whatsoever were specified, so we'll create a per-VU iterations config // with 1 VU and 1 iteration. - result.Scenarios = lib.ExecutorConfigMap{ - lib.DefaultExecutorName: NewPerVUIterationsConfig(lib.DefaultExecutorName), + result.Scenarios = lib.ScenarioConfigs{ + lib.DefaultScenarioName: NewPerVUIterationsConfig(lib.DefaultScenarioName), } } diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 40f75d5a6c0..c88cc7b5359 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -37,7 +37,7 @@ import ( type exp struct { parseError bool validationError bool - custom func(t *testing.T, cm lib.ExecutorConfigMap) + custom func(t *testing.T, cm lib.ScenarioConfigs) } type configMapTestCase struct { @@ -52,11 +52,11 @@ var configMapTestCases = []configMapTestCase{ {"asdf", exp{parseError: true}}, {"'adsf'", exp{parseError: true}}, {"[]", exp{parseError: true}}, - {"{}", exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { - assert.Equal(t, cm, lib.ExecutorConfigMap{}) + {"{}", exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { + assert.Equal(t, cm, lib.ScenarioConfigs{}) }}}, {"{}asdf", exp{parseError: true}}, - {"null", exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + {"null", exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { assert.Nil(t, cm) }}}, {`{"someKey": {}}`, exp{parseError: true}}, @@ -67,7 +67,7 @@ var configMapTestCases = []configMapTestCase{ // Validation errors for constant-vus and the base config {`{"someKey": {"executor": "constant-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewConstantLoopingVUsConfig("someKey") sched.VUs = null.IntFrom(10) sched.Duration = types.NullDurationFrom(1 * time.Minute) @@ -75,7 +75,7 @@ var configMapTestCases = []configMapTestCase{ sched.StartTime = types.NullDurationFrom(70 * time.Second) sched.Exec = null.StringFrom("someFunc") sched.Env = map[string]string{"test": "mest"} - require.Equal(t, cm, lib.ExecutorConfigMap{"someKey": sched}) + require.Equal(t, cm, lib.ScenarioConfigs{"someKey": sched}) require.Equal(t, sched.BaseConfig.Name, cm["someKey"].GetName()) require.Equal(t, sched.BaseConfig.Type, cm["someKey"].GetType()) require.Equal(t, sched.BaseConfig.GetGracefulStop(), cm["someKey"].GetGracefulStop()) @@ -122,7 +122,7 @@ var configMapTestCases = []configMapTestCase{ // ramping-vus {`{"varloops": {"executor": "ramping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", "startTime": "23s", "stages": [{"duration": "60s", "target": 30}, {"duration": "130s", "target": 10}]}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewVariableLoopingVUsConfig("varloops") sched.GracefulStop = types.NullDurationFrom(15 * time.Second) sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) @@ -132,7 +132,7 @@ var configMapTestCases = []configMapTestCase{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(60 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(130 * time.Second)}, } - require.Equal(t, cm, lib.ExecutorConfigMap{"varloops": sched}) + require.Equal(t, cm, lib.ScenarioConfigs{"varloops": sched}) assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) @@ -158,7 +158,7 @@ var configMapTestCases = []configMapTestCase{ }, {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "10s", "stages": [{"duration": "10s", "target": 10}]}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) @@ -173,7 +173,7 @@ var configMapTestCases = []configMapTestCase{ }, {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) @@ -188,7 +188,7 @@ var configMapTestCases = []configMapTestCase{ }, {`{"varloops": {"executor": "ramping-vus", "startVUs": 1, "gracefulStop": "0s", "gracefulRampDown": "0s", "stages": [{"duration": "10s", "target": 10}, {"duration": "0s", "target": 11},{"duration": "0s", "target": 1}, {"duration": "10s", "target": 5}]}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { assert.Empty(t, cm["varloops"].Validate()) assert.Empty(t, cm.Validate()) @@ -211,7 +211,7 @@ var configMapTestCases = []configMapTestCase{ {`{"varloops": {"executor": "ramping-vus"}}`, exp{validationError: true}}, // shared-iterations {`{"ishared": {"executor": "shared-iterations", "iterations": 22, "vus": 12, "maxDuration": "100s"}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewSharedIterationsConfig("ishared") sched.Iterations = null.IntFrom(22) sched.MaxDuration = types.NullDurationFrom(100 * time.Second) @@ -286,7 +286,7 @@ var configMapTestCases = []configMapTestCase{ {`{"ishared": {"executor": "shared-iterations", "iterations": 20, "vus": 30}}`, exp{validationError: true}}, // per-vu-iterations {`{"ipervu": {"executor": "per-vu-iterations", "iterations": 23, "vus": 13, "gracefulStop": 0}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewPerVUIterationsConfig("ipervu") sched.Iterations = null.IntFrom(23) sched.GracefulStop = types.NullDurationFrom(0) @@ -321,7 +321,7 @@ var configMapTestCases = []configMapTestCase{ // constant-arrival-rate {`{"carrival": {"executor": "constant-arrival-rate", "rate": 30, "timeUnit": "1m", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) sched := NewConstantArrivalRateConfig("carrival") @@ -361,7 +361,7 @@ var configMapTestCases = []configMapTestCase{ // ramping-arrival-rate {`{"varrival": {"executor": "ramping-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, - exp{custom: func(t *testing.T, cm lib.ExecutorConfigMap) { + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewVariableArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) sched.Stages = []Stage{ @@ -371,7 +371,7 @@ var configMapTestCases = []configMapTestCase{ sched.TimeUnit = types.NullDurationFrom(30 * time.Second) sched.PreAllocatedVUs = null.IntFrom(20) sched.MaxVUs = null.IntFrom(50) - require.Equal(t, cm, lib.ExecutorConfigMap{"varrival": sched}) + require.Equal(t, cm, lib.ScenarioConfigs{"varrival": sched}) assert.Empty(t, cm["varrival"].Validate()) assert.Empty(t, cm.Validate()) @@ -409,7 +409,7 @@ func TestConfigMapParsingAndValidation(t *testing.T) { tc := tc t.Run(fmt.Sprintf("TestCase#%d", i), func(t *testing.T) { t.Logf(tc.rawJSON) - var result lib.ExecutorConfigMap + var result lib.ScenarioConfigs err := json.Unmarshal([]byte(tc.rawJSON), &result) if tc.expected.parseError { require.Error(t, err) diff --git a/lib/executors.go b/lib/executors.go index b9e55c65db0..a79df23d3d6 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -61,7 +61,7 @@ var ( // be ignored, since the gracefulStop/gracefulRampDown options potentially allow // any started iterations to finish. // -// []ExecutionStep is also used by the ExecutorConfigMap, to represent the +// []ExecutionStep is also used by the ScenarioConfigs, to represent the // amount of needed VUs among all executors, during the whole execution of a // test script. In that context, each executor's StartTime is accounted for and // included in the offsets. @@ -159,12 +159,12 @@ func RegisterExecutorConfigType(configType string, constructor ExecutorConfigCon executorConfigConstructors[configType] = constructor } -// ExecutorConfigMap can contain mixed executor config types -type ExecutorConfigMap map[string]ExecutorConfig +// ScenarioConfigs can contain mixed executor config types +type ScenarioConfigs map[string]ExecutorConfig // UnmarshalJSON implements the json.Unmarshaler interface in a two-step manner, // creating the correct type of configs based on the `type` property. -func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { +func (scs *ScenarioConfigs) UnmarshalJSON(data []byte) error { if len(data) == 0 { return nil } @@ -180,7 +180,7 @@ func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { return err } - result := make(ExecutorConfigMap, len(protoConfigs)) + result := make(ScenarioConfigs, len(protoConfigs)) for k, v := range protoConfigs { if v.configType == "" { return fmt.Errorf("execution config '%s' doesn't have a type value", k) @@ -198,7 +198,7 @@ func (scs *ExecutorConfigMap) UnmarshalJSON(data []byte) error { } // Validate checks if all of the specified executor options make sense -func (scs ExecutorConfigMap) Validate() (errors []error) { +func (scs ScenarioConfigs) Validate() (errors []error) { for name, exec := range scs { if execErr := exec.Validate(); len(execErr) != 0 { errors = append(errors, @@ -217,7 +217,7 @@ func (scs ExecutorConfigMap) Validate() (errors []error) { // The configs in the returned slice will be sorted by their start times in an // ascending order, and alphabetically by their names (which are unique) if // there are ties. -func (scs ExecutorConfigMap) GetSortedConfigs() []ExecutorConfig { +func (scs ScenarioConfigs) GetSortedConfigs() []ExecutorConfig { configs := make([]ExecutorConfig, len(scs)) // Populate the configs slice with sorted executor configs @@ -244,7 +244,7 @@ func (scs ExecutorConfigMap) GetSortedConfigs() []ExecutorConfig { // the configured executors. It takes into account their start times and their // individual VU requirements and calculates the total VU requirements for each // moment in the test execution. -func (scs ExecutorConfigMap) GetFullExecutionRequirements(et *ExecutionTuple) []ExecutionStep { +func (scs ScenarioConfigs) GetFullExecutionRequirements(et *ExecutionTuple) []ExecutionStep { sortedConfigs := scs.GetSortedConfigs() // Combine the steps and requirements from all different executors, and diff --git a/lib/old_archive_test.go b/lib/old_archive_test.go index 384f943c524..a64d930190f 100644 --- a/lib/old_archive_test.go +++ b/lib/old_archive_test.go @@ -230,7 +230,7 @@ func TestFilenamePwdResolve(t *testing.T) { } func TestDerivedExecutionDiscarding(t *testing.T) { - var emptyConfigMap ExecutorConfigMap + var emptyConfigMap ScenarioConfigs var tests = []struct { metadata string expScenarios interface{} diff --git a/lib/options.go b/lib/options.go index d82b4406304..2f51d61a538 100644 --- a/lib/options.go +++ b/lib/options.go @@ -34,10 +34,10 @@ import ( "github.com/loadimpact/k6/stats" ) -// DefaultExecutorName is used as the default key/ID of the executor config entries +// DefaultScenarioName is used as the default key/ID of the scenario config entries // that were created due to the use of the shortcut execution control options (i.e. duration+vus, // iterations+vus, or stages) -const DefaultExecutorName = "default" +const DefaultScenarioName = "default" // DefaultSummaryTrendStats are the default trend columns shown in the test summary output // nolint: gochecknoglobals @@ -204,7 +204,7 @@ type Options struct { // We should support specifying execution segments via environment // variables, but we currently can't, because envconfig has this nasty bug // (among others): https://github.com/kelseyhightower/envconfig/issues/113 - Scenarios ExecutorConfigMap `json:"scenarios,omitempty" ignored:"true"` + Scenarios ScenarioConfigs `json:"scenarios,omitempty" ignored:"true"` ExecutionSegment *ExecutionSegment `json:"executionSegment" ignored:"true"` ExecutionSegmentSequence *ExecutionSegmentSequence `json:"executionSegmentSequence" ignored:"true"` From 5020f21e574dc83b8265ca8657782e56ed660f11 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 16:21:29 +0300 Subject: [PATCH 294/350] Rename ConstantLoopingVUs to ConstantVUs --- cmd/config_consolidation_test.go | 4 +- core/local/local_test.go | 2 +- lib/executor/constant_looping_vus.go | 46 +++++++++++----------- lib/executor/constant_looping_vus_test.go | 8 ++-- lib/executor/execution_config_shortcuts.go | 6 +-- lib/executor/executors_test.go | 2 +- 6 files changed, 34 insertions(+), 34 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index 2703480c6e0..a965d3a4696 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -70,8 +70,8 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing return func(t *testing.T, c Config) { exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) - require.IsType(t, executor.ConstantLoopingVUsConfig{}, exec) - clvc, ok := exec.(executor.ConstantLoopingVUsConfig) + require.IsType(t, executor.ConstantVUsConfig{}, exec) + clvc, ok := exec.(executor.ConstantVUsConfig) require.True(t, ok) assert.Equal(t, vus, clvc.VUs) assert.Equal(t, types.NullDurationFrom(duration), clvc.Duration) diff --git a/core/local/local_test.go b/core/local/local_test.go index 9fd0ab6cefa..22a4fdd9770 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -762,7 +762,7 @@ func TestExecutionSchedulerRuntimeErrors(t *testing.T) { func TestExecutionSchedulerEndErrors(t *testing.T) { t.Parallel() - exec := executor.NewConstantLoopingVUsConfig("we_need_hard_stop") + exec := executor.NewConstantVUsConfig("we_need_hard_stop") exec.VUs = null.IntFrom(10) exec.Duration = types.NullDurationFrom(1 * time.Second) exec.GracefulStop = types.NullDurationFrom(0 * time.Second) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_looping_vus.go index 25902706c33..ba4249ea982 100644 --- a/lib/executor/constant_looping_vus.go +++ b/lib/executor/constant_looping_vus.go @@ -35,13 +35,13 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const constantLoopingVUsType = "constant-vus" +const constantVUsType = "constant-vus" func init() { lib.RegisterExecutorConfigType( - constantLoopingVUsType, + constantVUsType, func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { - config := NewConstantLoopingVUsConfig(name) + config := NewConstantVUsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err }, @@ -52,37 +52,37 @@ func init() { // configuration, where 0-duration virtual stages are allowed for instantaneous VU jumps const minDuration = 1 * time.Second -// ConstantLoopingVUsConfig stores VUs and duration -type ConstantLoopingVUsConfig struct { +// ConstantVUsConfig stores VUs and duration +type ConstantVUsConfig struct { BaseConfig VUs null.Int `json:"vus"` Duration types.NullDuration `json:"duration"` } -// NewConstantLoopingVUsConfig returns a ConstantLoopingVUsConfig with default values -func NewConstantLoopingVUsConfig(name string) ConstantLoopingVUsConfig { - return ConstantLoopingVUsConfig{ - BaseConfig: NewBaseConfig(name, constantLoopingVUsType), +// NewConstantVUsConfig returns a ConstantVUsConfig with default values +func NewConstantVUsConfig(name string) ConstantVUsConfig { + return ConstantVUsConfig{ + BaseConfig: NewBaseConfig(name, constantVUsType), VUs: null.NewInt(1, false), } } // Make sure we implement the lib.ExecutorConfig interface -var _ lib.ExecutorConfig = &ConstantLoopingVUsConfig{} +var _ lib.ExecutorConfig = &ConstantVUsConfig{} // GetVUs returns the scaled VUs for the executor. -func (clvc ConstantLoopingVUsConfig) GetVUs(et *lib.ExecutionTuple) int64 { +func (clvc ConstantVUsConfig) GetVUs(et *lib.ExecutionTuple) int64 { return et.Segment.Scale(clvc.VUs.Int64) } // GetDescription returns a human-readable description of the executor options -func (clvc ConstantLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { +func (clvc ConstantVUsConfig) GetDescription(et *lib.ExecutionTuple) string { return fmt.Sprintf("%d looping VUs for %s%s", clvc.GetVUs(et), clvc.Duration.Duration, clvc.getBaseInfo()) } // Validate makes sure all options are configured and valid -func (clvc ConstantLoopingVUsConfig) Validate() []error { +func (clvc ConstantVUsConfig) Validate() []error { errors := clvc.BaseConfig.Validate() if clvc.VUs.Int64 <= 0 { errors = append(errors, fmt.Errorf("the number of VUs should be more than 0")) @@ -104,7 +104,7 @@ func (clvc ConstantLoopingVUsConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { +func (clvc ConstantVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, @@ -118,31 +118,31 @@ func (clvc ConstantLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionT } // HasWork reports whether there is any work to be done for the given execution segment. -func (clvc ConstantLoopingVUsConfig) HasWork(et *lib.ExecutionTuple) bool { +func (clvc ConstantVUsConfig) HasWork(et *lib.ExecutionTuple) bool { return clvc.GetVUs(et) > 0 } -// NewExecutor creates a new ConstantLoopingVUs executor -func (clvc ConstantLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - return ConstantLoopingVUs{ +// NewExecutor creates a new ConstantVUs executor +func (clvc ConstantVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { + return ConstantVUs{ BaseExecutor: NewBaseExecutor(clvc, es, logger), config: clvc, }, nil } -// ConstantLoopingVUs maintains a constant number of VUs running for the +// ConstantVUs maintains a constant number of VUs running for the // specified duration. -type ConstantLoopingVUs struct { +type ConstantVUs struct { *BaseExecutor - config ConstantLoopingVUsConfig + config ConstantVUsConfig } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &ConstantLoopingVUs{} +var _ lib.Executor = &ConstantVUs{} // Run constantly loops through as many iterations as possible on a fixed number // of VUs for the specified duration. -func (clv ConstantLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (clv ConstantVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := clv.config.GetVUs(clv.executionState.ExecutionTuple) duration := time.Duration(clv.config.Duration.Duration) gracefulStop := clv.config.GetGracefulStop() diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_looping_vus_test.go index 5f3e9f4d330..7e91c36c2a8 100644 --- a/lib/executor/constant_looping_vus_test.go +++ b/lib/executor/constant_looping_vus_test.go @@ -34,22 +34,22 @@ import ( "github.com/loadimpact/k6/lib/types" ) -func getTestConstantLoopingVUsConfig() ConstantLoopingVUsConfig { - return ConstantLoopingVUsConfig{ +func getTestConstantVUsConfig() ConstantVUsConfig { + return ConstantVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, VUs: null.IntFrom(10), Duration: types.NullDurationFrom(1 * time.Second), } } -func TestConstantLoopingVUsRun(t *testing.T) { +func TestConstantVUsRun(t *testing.T) { t.Parallel() var result sync.Map et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) var ctx, cancel, executor, _ = setupExecutor( - t, getTestConstantLoopingVUsConfig(), es, + t, getTestConstantVUsConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(200 * time.Millisecond) select { diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index ff9ae32a313..8443fbdfe47 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -38,8 +38,8 @@ func (e ExecutionConflictError) Error() string { var _ error = ExecutionConflictError("") -func getConstantLoopingVUsScenario(duration types.NullDuration, vus null.Int) lib.ScenarioConfigs { - ds := NewConstantLoopingVUsConfig(lib.DefaultScenarioName) +func getConstantVUsScenario(duration types.NullDuration, vus null.Int) lib.ScenarioConfigs { + ds := NewConstantVUsConfig(lib.DefaultScenarioName) ds.VUs = vus ds.Duration = duration return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} @@ -103,7 +103,7 @@ func DeriveScenariosFromShortcuts(opts lib.Options) (lib.Options, error) { "`duration` should be more than 0, for infinite duration use the externally-controlled executor", ) } - result.Scenarios = getConstantLoopingVUsScenario(opts.Duration, opts.VUs) + result.Scenarios = getConstantVUsScenario(opts.Duration, opts.VUs) case len(opts.Stages) > 0: // stages isn't nil (not set) and isn't explicitly set to empty if opts.Scenarios != nil { diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index c88cc7b5359..7ada57ab097 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -68,7 +68,7 @@ var configMapTestCases = []configMapTestCase{ {`{"someKey": {"executor": "constant-vus", "vus": 10, "duration": "60s", "gracefulStop": "10s", "startTime": "70s", "env": {"test": "mest"}, "exec": "someFunc"}}`, exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { - sched := NewConstantLoopingVUsConfig("someKey") + sched := NewConstantVUsConfig("someKey") sched.VUs = null.IntFrom(10) sched.Duration = types.NullDurationFrom(1 * time.Minute) sched.GracefulStop = types.NullDurationFrom(10 * time.Second) From f95969a18860748e15ec80021612ec4fe961e859 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 16:41:05 +0300 Subject: [PATCH 295/350] Rename VariableArrivalRate to RampingArrivalRate --- lib/executor/constant_arrival_rate_test.go | 2 +- lib/executor/executors_test.go | 2 +- lib/executor/variable_arrival_rate.go | 51 +++++++++++----------- lib/executor/variable_arrival_rate_test.go | 28 ++++++------ 4 files changed, 42 insertions(+), 41 deletions(-) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 926e164946c..5f3cc695716 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -247,7 +247,7 @@ func TestArrivalRateCancel(t *testing.T) { testCases := map[string]lib.ExecutorConfig{ "constant": getTestConstantArrivalRateConfig(), - "variable": getTestVariableArrivalRateConfig(), + "ramping": getTestRampingArrivalRateConfig(), } for name, config := range testCases { config := config diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 7ada57ab097..28aabeaffc0 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -362,7 +362,7 @@ var configMapTestCases = []configMapTestCase{ {`{"varrival": {"executor": "ramping-arrival-rate", "startRate": 10, "timeUnit": "30s", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "3m", "target": 30}, {"duration": "5m", "target": 10}]}}`, exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { - sched := NewVariableArrivalRateConfig("varrival") + sched := NewRampingArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) sched.Stages = []Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(180 * time.Second)}, diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/variable_arrival_rate.go index 696f3560a37..5d763567f43 100644 --- a/lib/executor/variable_arrival_rate.go +++ b/lib/executor/variable_arrival_rate.go @@ -37,21 +37,22 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const variableArrivalRateType = "ramping-arrival-rate" +const rampingArrivalRateType = "ramping-arrival-rate" func init() { lib.RegisterExecutorConfigType( - variableArrivalRateType, + rampingArrivalRateType, func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { - config := NewVariableArrivalRateConfig(name) + config := NewRampingArrivalRateConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err }, ) } -// VariableArrivalRateConfig stores config for the variable arrival-rate executor -type VariableArrivalRateConfig struct { +// RampingArrivalRateConfig stores config for the ramping (i.e. variable) +// arrival-rate executor. +type RampingArrivalRateConfig struct { BaseConfig StartRate null.Int `json:"startRate"` TimeUnit types.NullDuration `json:"timeUnit"` @@ -64,29 +65,29 @@ type VariableArrivalRateConfig struct { MaxVUs null.Int `json:"maxVUs"` } -// NewVariableArrivalRateConfig returns a VariableArrivalRateConfig with default values -func NewVariableArrivalRateConfig(name string) VariableArrivalRateConfig { - return VariableArrivalRateConfig{ - BaseConfig: NewBaseConfig(name, variableArrivalRateType), +// NewRampingArrivalRateConfig returns a RampingArrivalRateConfig with default values +func NewRampingArrivalRateConfig(name string) RampingArrivalRateConfig { + return RampingArrivalRateConfig{ + BaseConfig: NewBaseConfig(name, rampingArrivalRateType), TimeUnit: types.NewNullDuration(1*time.Second, false), } } // Make sure we implement the lib.ExecutorConfig interface -var _ lib.ExecutorConfig = &VariableArrivalRateConfig{} +var _ lib.ExecutorConfig = &RampingArrivalRateConfig{} // GetPreAllocatedVUs is just a helper method that returns the scaled pre-allocated VUs. -func (varc VariableArrivalRateConfig) GetPreAllocatedVUs(et *lib.ExecutionTuple) int64 { +func (varc RampingArrivalRateConfig) GetPreAllocatedVUs(et *lib.ExecutionTuple) int64 { return et.Segment.Scale(varc.PreAllocatedVUs.Int64) } // GetMaxVUs is just a helper method that returns the scaled max VUs. -func (varc VariableArrivalRateConfig) GetMaxVUs(et *lib.ExecutionTuple) int64 { +func (varc RampingArrivalRateConfig) GetMaxVUs(et *lib.ExecutionTuple) int64 { return et.Segment.Scale(varc.MaxVUs.Int64) } // GetDescription returns a human-readable description of the executor options -func (varc VariableArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) string { +func (varc RampingArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) string { // TODO: something better? always show iterations per second? maxVUsRange := fmt.Sprintf("maxVUs: %d", et.Segment.Scale(varc.PreAllocatedVUs.Int64)) if varc.MaxVUs.Int64 > varc.PreAllocatedVUs.Int64 { @@ -103,7 +104,7 @@ func (varc VariableArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) str } // Validate makes sure all options are configured and valid -func (varc VariableArrivalRateConfig) Validate() []error { +func (varc RampingArrivalRateConfig) Validate() []error { errors := varc.BaseConfig.Validate() if varc.StartRate.Int64 < 0 { @@ -136,7 +137,7 @@ func (varc VariableArrivalRateConfig) Validate() []error { // maximum waiting time for any iterations to gracefully stop. This is used by // the execution scheduler in its VU reservation calculations, so it knows how // many VUs to pre-initialize. -func (varc VariableArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { +func (varc RampingArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { return []lib.ExecutionStep{ { TimeOffset: 0, @@ -151,31 +152,31 @@ func (varc VariableArrivalRateConfig) GetExecutionRequirements(et *lib.Execution } } -// NewExecutor creates a new VariableArrivalRate executor -func (varc VariableArrivalRateConfig) NewExecutor( +// NewExecutor creates a new RampingArrivalRate executor +func (varc RampingArrivalRateConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { - return VariableArrivalRate{ + return RampingArrivalRate{ BaseExecutor: NewBaseExecutor(varc, es, logger), config: varc, }, nil } // HasWork reports whether there is any work to be done for the given execution segment. -func (varc VariableArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { +func (varc RampingArrivalRateConfig) HasWork(et *lib.ExecutionTuple) bool { return varc.GetMaxVUs(et) > 0 } -// VariableArrivalRate tries to execute a specific number of iterations for a +// RampingArrivalRate tries to execute a specific number of iterations for a // specific period. // TODO: combine with the ConstantArrivalRate? -type VariableArrivalRate struct { +type RampingArrivalRate struct { *BaseExecutor - config VariableArrivalRateConfig + config RampingArrivalRateConfig } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &VariableArrivalRate{} +var _ lib.Executor = &RampingArrivalRate{} // cal calculates the transtitions between stages and gives the next full value produced by the // stages. In this explanation we are talking about events and in practice those events are starting @@ -235,7 +236,7 @@ var _ lib.Executor = &VariableArrivalRate{} // The specific implementation here can only go forward and does incorporate // the striping algorithm from the lib.ExecutionTuple for additional speed up but this could // possibly be refactored if need for this arises. -func (varc VariableArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time.Duration) { +func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time.Duration) { start, offsets, _ := et.GetStripedOffsets() li := -1 // TODO: move this to a utility function, or directly what GetStripedOffsets uses once we see everywhere we will use it @@ -278,7 +279,7 @@ func (varc VariableArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time } // Run executes a variable number of iterations per second. -func (varr VariableArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen +func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/variable_arrival_rate_test.go index 1c7c8616f6d..7286aeab2d7 100644 --- a/lib/executor/variable_arrival_rate_test.go +++ b/lib/executor/variable_arrival_rate_test.go @@ -39,8 +39,8 @@ import ( "github.com/loadimpact/k6/stats" ) -func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { - return VariableArrivalRateConfig{ +func getTestRampingArrivalRateConfig() RampingArrivalRateConfig { + return RampingArrivalRateConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(10), @@ -63,13 +63,13 @@ func getTestVariableArrivalRateConfig() VariableArrivalRateConfig { } } -func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { +func TestRampingArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { t.Parallel() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) ctx, cancel, executor, logHook := setupExecutor( - t, getTestVariableArrivalRateConfig(), es, + t, getTestRampingArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { time.Sleep(time.Second) return nil @@ -89,14 +89,14 @@ func TestVariableArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { } } -func TestVariableArrivalRateRunCorrectRate(t *testing.T) { +func TestRampingArrivalRateRunCorrectRate(t *testing.T) { t.Parallel() var count int64 et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) ctx, cancel, executor, logHook := setupExecutor( - t, getTestVariableArrivalRateConfig(), es, + t, getTestRampingArrivalRateConfig(), es, simpleRunner(func(ctx context.Context) error { atomic.AddInt64(&count, 1) return nil @@ -129,7 +129,7 @@ func TestVariableArrivalRateRunCorrectRate(t *testing.T) { require.Empty(t, logHook.Drain()) } -func TestVariableArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { +func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { t.Parallel() var count int64 now := time.Now() @@ -140,7 +140,7 @@ func TestVariableArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6, } ctx, cancel, executor, logHook := setupExecutor( - t, VariableArrivalRateConfig{ + t, RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Stages: []Stage{ { @@ -191,12 +191,12 @@ func mustNewExecutionTuple(seg *lib.ExecutionSegment, seq *lib.ExecutionSegmentS return et } -func TestVariableArrivalRateCal(t *testing.T) { +func TestRampingArrivalRateCal(t *testing.T) { t.Parallel() var ( defaultTimeUnit = time.Second - config = VariableArrivalRateConfig{ + config = RampingArrivalRateConfig{ StartRate: null.IntFrom(0), Stages: []Stage{ // TODO make this even bigger and longer .. will need more time { @@ -295,7 +295,7 @@ func BenchmarkCal(b *testing.B) { } { t := t b.Run(t.String(), func(b *testing.B) { - config := VariableArrivalRateConfig{ + config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), Stages: []Stage{ @@ -331,7 +331,7 @@ func BenchmarkCalRat(b *testing.B) { } { t := t b.Run(t.String(), func(b *testing.B) { - config := VariableArrivalRateConfig{ + config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), Stages: []Stage{ @@ -367,7 +367,7 @@ func TestCompareCalImplementation(t *testing.T) { // in my experiment the difference is 1(nanosecond) in 7 case for the whole test // the duration is 1 second for each stage as calRat takes way longer - a longer better test can // be done when/if it's performance is improved - config := VariableArrivalRateConfig{ + config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(0), Stages: []Stage{ @@ -452,7 +452,7 @@ func sqrtRat(x *big.Rat) *big.Rat { } // This implementation is just for reference and accuracy testing -func (varc VariableArrivalRateConfig) calRat(et *lib.ExecutionTuple, ch chan<- time.Duration) { +func (varc RampingArrivalRateConfig) calRat(et *lib.ExecutionTuple, ch chan<- time.Duration) { defer close(ch) start, offsets, _ := et.GetStripedOffsets() From 05524b614e7cfe014555b72a0afe3e4ec3412afd Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 16:59:43 +0300 Subject: [PATCH 296/350] Rename VariableLoopingVUs to RampingVUs --- cmd/config_consolidation_test.go | 20 ++++---- lib/executor/execution_config_shortcuts.go | 6 +-- lib/executor/executors_test.go | 2 +- lib/executor/variable_looping_vus.go | 60 +++++++++++----------- lib/executor/variable_looping_vus_test.go | 32 ++++++------ lib/executor/vu_handle.go | 2 +- 6 files changed, 61 insertions(+), 61 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index a965d3a4696..b11763c7197 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -80,12 +80,12 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing } } -func verifyVarLoopingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { +func verifyRampingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) - require.IsType(t, executor.VariableLoopingVUsConfig{}, exec) - clvc, ok := exec.(executor.VariableLoopingVUsConfig) + require.IsType(t, executor.RampingVUsConfig{}, exec) + clvc, ok := exec.(executor.RampingVUsConfig) require.True(t, ok) assert.Equal(t, startVus, clvc.StartVUs) assert.Equal(t, startVus, c.VUs) @@ -215,11 +215,11 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { {opts{cli: []string{"-u", "4", "--duration", "60s"}}, exp{}, verifyConstLoopingVUs(I(4), 1*time.Minute)}, { opts{cli: []string{"--stage", "20s:10", "-s", "3m:5"}}, exp{}, - verifyVarLoopingVUs(null.NewInt(1, false), buildStages(20, 10, 180, 5)), + verifyRampingVUs(null.NewInt(1, false), buildStages(20, 10, 180, 5)), }, { opts{cli: []string{"-s", "1m6s:5", "--vus", "10"}}, exp{}, - verifyVarLoopingVUs(null.NewInt(10, true), buildStages(66, 5)), + verifyRampingVUs(null.NewInt(10, true), buildStages(66, 5)), }, {opts{cli: []string{"-u", "1", "-i", "6", "-d", "10s"}}, exp{}, func(t *testing.T, c Config) { verifySharedIters(I(1), I(6))(t, c) @@ -249,11 +249,11 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { {opts{env: []string{"K6_VUS=10", "K6_DURATION=20s"}}, exp{}, verifyConstLoopingVUs(I(10), 20*time.Second)}, { opts{env: []string{"K6_STAGES=2m30s:11,1h1m:100"}}, exp{}, - verifyVarLoopingVUs(null.NewInt(1, false), buildStages(150, 11, 3660, 100)), + verifyRampingVUs(null.NewInt(1, false), buildStages(150, 11, 3660, 100)), }, { opts{env: []string{"K6_STAGES=100s:100,0m30s:0", "K6_VUS=0"}}, exp{}, - verifyVarLoopingVUs(null.NewInt(0, true), buildStages(100, 100, 30, 0)), + verifyRampingVUs(null.NewInt(0, true), buildStages(100, 100, 30, 0)), }, // Test if JSON configs work as expected {opts{fs: defaultConfig(`{"iterations": 77, "vus": 7}`)}, exp{}, verifySharedIters(I(7), I(77))}, @@ -282,7 +282,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { runner: &lib.Options{VUs: null.IntFrom(5), Duration: types.NullDurationFrom(50 * time.Second)}, cli: []string{"--stage", "5s:5"}, }, - exp{}, verifyVarLoopingVUs(I(5), buildStages(5, 5)), + exp{}, verifyRampingVUs(I(5), buildStages(5, 5)), }, { opts{ @@ -290,7 +290,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { runner: &lib.Options{VUs: null.IntFrom(5)}, }, exp{}, - verifyVarLoopingVUs(I(5), buildStages(20, 10)), + verifyRampingVUs(I(5), buildStages(20, 10)), }, { opts{ @@ -309,7 +309,7 @@ func getConfigConsolidationTestCases() []configConsolidationTestCase { cli: []string{"--stage", "44s:44", "-s", "55s:55"}, }, exp{}, - verifyVarLoopingVUs(null.NewInt(33, true), buildStages(44, 44, 55, 55)), + verifyRampingVUs(null.NewInt(33, true), buildStages(44, 44, 55, 55)), }, // TODO: test the future full overwriting of the duration/iterations/stages/execution options diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index 8443fbdfe47..418a0ec160a 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -45,8 +45,8 @@ func getConstantVUsScenario(duration types.NullDuration, vus null.Int) lib.Scena return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} } -func getVariableLoopingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ScenarioConfigs { - ds := NewVariableLoopingVUsConfig(lib.DefaultScenarioName) +func getRampingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ScenarioConfigs { + ds := NewRampingVUsConfig(lib.DefaultScenarioName) ds.StartVUs = startVUs for _, s := range stages { if s.Duration.Valid { @@ -111,7 +111,7 @@ func DeriveScenariosFromShortcuts(opts lib.Options) (lib.Options, error) { "using an execution configuration shortcut (`stages`) and `scenarios` simultaneously is not allowed", ) } - result.Scenarios = getVariableLoopingVUsScenario(opts.Stages, opts.VUs) + result.Scenarios = getRampingVUsScenario(opts.Stages, opts.VUs) case len(opts.Scenarios) > 0: // Do nothing, scenarios was explicitly specified diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 28aabeaffc0..9485fa5adf4 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -123,7 +123,7 @@ var configMapTestCases = []configMapTestCase{ {`{"varloops": {"executor": "ramping-vus", "startVUs": 20, "gracefulStop": "15s", "gracefulRampDown": "10s", "startTime": "23s", "stages": [{"duration": "60s", "target": 30}, {"duration": "130s", "target": 10}]}}`, exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { - sched := NewVariableLoopingVUsConfig("varloops") + sched := NewRampingVUsConfig("varloops") sched.GracefulStop = types.NullDurationFrom(15 * time.Second) sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) sched.StartVUs = null.IntFrom(20) diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/variable_looping_vus.go index 9abd74b42bb..28dc43970a4 100644 --- a/lib/executor/variable_looping_vus.go +++ b/lib/executor/variable_looping_vus.go @@ -36,13 +36,13 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -const variableLoopingVUsType = "ramping-vus" +const rampingVUsType = "ramping-vus" func init() { lib.RegisterExecutorConfigType( - variableLoopingVUsType, + rampingVUsType, func(name string, rawJSON []byte) (lib.ExecutorConfig, error) { - config := NewVariableLoopingVUsConfig(name) + config := NewRampingVUsConfig(name) err := lib.StrictJSONUnmarshal(rawJSON, &config) return config, err }, @@ -56,39 +56,39 @@ type Stage struct { // TODO: add a progression function? } -// VariableLoopingVUsConfig stores the configuration for the stages executor -type VariableLoopingVUsConfig struct { +// RampingVUsConfig stores the configuration for the stages executor +type RampingVUsConfig struct { BaseConfig StartVUs null.Int `json:"startVUs"` Stages []Stage `json:"stages"` GracefulRampDown types.NullDuration `json:"gracefulRampDown"` } -// NewVariableLoopingVUsConfig returns a VariableLoopingVUsConfig with its default values -func NewVariableLoopingVUsConfig(name string) VariableLoopingVUsConfig { - return VariableLoopingVUsConfig{ - BaseConfig: NewBaseConfig(name, variableLoopingVUsType), +// NewRampingVUsConfig returns a RampingVUsConfig with its default values +func NewRampingVUsConfig(name string) RampingVUsConfig { + return RampingVUsConfig{ + BaseConfig: NewBaseConfig(name, rampingVUsType), StartVUs: null.NewInt(1, false), GracefulRampDown: types.NewNullDuration(30*time.Second, false), } } // Make sure we implement the lib.ExecutorConfig interface -var _ lib.ExecutorConfig = &VariableLoopingVUsConfig{} +var _ lib.ExecutorConfig = &RampingVUsConfig{} // GetStartVUs is just a helper method that returns the scaled starting VUs. -func (vlvc VariableLoopingVUsConfig) GetStartVUs(et *lib.ExecutionTuple) int64 { +func (vlvc RampingVUsConfig) GetStartVUs(et *lib.ExecutionTuple) int64 { return et.Segment.Scale(vlvc.StartVUs.Int64) } // GetGracefulRampDown is just a helper method that returns the graceful // ramp-down period as a standard Go time.Duration value... -func (vlvc VariableLoopingVUsConfig) GetGracefulRampDown() time.Duration { +func (vlvc RampingVUsConfig) GetGracefulRampDown() time.Duration { return time.Duration(vlvc.GracefulRampDown.Duration) } // GetDescription returns a human-readable description of the executor options -func (vlvc VariableLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { +func (vlvc RampingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { maxVUs := et.Segment.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), @@ -96,7 +96,7 @@ func (vlvc VariableLoopingVUsConfig) GetDescription(et *lib.ExecutionTuple) stri } // Validate makes sure all options are configured and valid -func (vlvc VariableLoopingVUsConfig) Validate() []error { +func (vlvc RampingVUsConfig) Validate() []error { errors := vlvc.BaseConfig.Validate() if vlvc.StartVUs.Int64 < 0 { errors = append(errors, fmt.Errorf("the number of start VUs shouldn't be negative")) @@ -181,7 +181,7 @@ func (vlvc VariableLoopingVUsConfig) Validate() []error { // 00000000001111111111222 (t/10) // // More information: https://github.com/loadimpact/k6/issues/997#issuecomment-484416866 -func (vlvc VariableLoopingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple, zeroEnd bool) []lib.ExecutionStep { +func (vlvc RampingVUsConfig) getRawExecutionSteps(et *lib.ExecutionTuple, zeroEnd bool) []lib.ExecutionStep { var ( timeTillEnd time.Duration fromVUs = vlvc.StartVUs.Int64 @@ -317,7 +317,7 @@ func absInt64(a int64) int64 { return a } -func (vlvc VariableLoopingVUsConfig) precalculateTheRequiredSteps(et *lib.ExecutionTuple, zeroEnd bool) int { +func (vlvc RampingVUsConfig) precalculateTheRequiredSteps(et *lib.ExecutionTuple, zeroEnd bool) int { p := et.ScaleInt64(vlvc.StartVUs.Int64) var result int64 result++ // for the first one @@ -370,7 +370,7 @@ func (vlvc VariableLoopingVUsConfig) precalculateTheRequiredSteps(et *lib.Execut // to 1 again, scale up to 4, back to 1, and finally back down to 0. If our // gracefulStop timeout was 30s (the default), then we'll stay with 6 PlannedVUs // until t=32 in the test above, and the actual executor could run until t=52. -// See TestVariableLoopingVUsConfigExecutionPlanExample() for the above example +// See TestRampingVUsConfigExecutionPlanExample() for the above example // as a unit test. // // The algorithm we use below to reserve VUs so that ramping-down VUs can finish @@ -382,7 +382,7 @@ func (vlvc VariableLoopingVUsConfig) precalculateTheRequiredSteps(et *lib.Execut // executorEndOffset, is not handled here. Instead GetExecutionRequirements() // takes care of that. But to make its job easier, this method won't add any // steps with an offset that's greater or equal to executorEndOffset. -func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:funlen +func (vlvc RampingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:funlen rawSteps []lib.ExecutionStep, executorEndOffset time.Duration, ) []lib.ExecutionStep { rawStepsLen := len(rawSteps) @@ -485,7 +485,7 @@ func (vlvc VariableLoopingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:fu // last stage's target is 0), then this will have no effect. // - If the last stage's target is more than 0, the VUs at the end of the // executor's life will have more time to finish their last iterations. -func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { +func (vlvc RampingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { steps := vlvc.getRawExecutionSteps(et, false) executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) @@ -502,29 +502,29 @@ func (vlvc VariableLoopingVUsConfig) GetExecutionRequirements(et *lib.ExecutionT return steps } -// NewExecutor creates a new VariableLoopingVUs executor -func (vlvc VariableLoopingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { - return VariableLoopingVUs{ +// NewExecutor creates a new RampingVUs executor +func (vlvc RampingVUsConfig) NewExecutor(es *lib.ExecutionState, logger *logrus.Entry) (lib.Executor, error) { + return RampingVUs{ BaseExecutor: NewBaseExecutor(vlvc, es, logger), config: vlvc, }, nil } // HasWork reports whether there is any work to be done for the given execution segment. -func (vlvc VariableLoopingVUsConfig) HasWork(et *lib.ExecutionTuple) bool { +func (vlvc RampingVUsConfig) HasWork(et *lib.ExecutionTuple) bool { return lib.GetMaxPlannedVUs(vlvc.GetExecutionRequirements(et)) > 0 } -// VariableLoopingVUs handles the old "stages" execution configuration - it -// loops iterations with a variable number of VUs for the sum of all of the -// specified stages' duration. -type VariableLoopingVUs struct { +// RampingVUs handles the old "stages" execution configuration - it loops +// iterations with a variable number of VUs for the sum of all of the specified +// stages' duration. +type RampingVUs struct { *BaseExecutor - config VariableLoopingVUsConfig + config RampingVUsConfig } // Make sure we implement the lib.Executor interface. -var _ lib.Executor = &VariableLoopingVUs{} +var _ lib.Executor = &RampingVUs{} // Run constantly loops through as many iterations as possible on a variable // number of VUs for the specified stages. @@ -533,7 +533,7 @@ var _ lib.Executor = &VariableLoopingVUs{} // of a less complex way to implement it (besides the old "increment by 100ms // and see what happens)... :/ so maybe see how it can be split? // nolint:funlen,gocognit -func (vlv VariableLoopingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { rawExecutionSteps := vlv.config.getRawExecutionSteps(vlv.executionState.ExecutionTuple, true) regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) if !isFinal { diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/variable_looping_vus_test.go index cf28bcddbd4..91af6c8bc71 100644 --- a/lib/executor/variable_looping_vus_test.go +++ b/lib/executor/variable_looping_vus_test.go @@ -38,10 +38,10 @@ import ( "github.com/loadimpact/k6/lib/types" ) -func TestVariableLoopingVUsRun(t *testing.T) { +func TestRampingVUsRun(t *testing.T) { t.Parallel() - config := VariableLoopingVUsConfig{ + config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(0), StartVUs: null.IntFrom(5), @@ -101,10 +101,10 @@ func TestVariableLoopingVUsRun(t *testing.T) { // Ensure there's no wobble of VUs during graceful ramp-down, without segments. // See https://github.com/loadimpact/k6/issues/1296 -func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { +func TestRampingVUsRampDownNoWobble(t *testing.T) { t.Parallel() - config := VariableLoopingVUsConfig{ + config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(1 * time.Second), StartVUs: null.IntFrom(0), @@ -170,11 +170,11 @@ func TestVariableLoopingVUsRampDownNoWobble(t *testing.T) { assert.Equal(t, []int64{10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}, vuChanges) } -func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { +func TestRampingVUsConfigExecutionPlanExample(t *testing.T) { t.Parallel() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) - conf := NewVariableLoopingVUsConfig("test") + conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -270,11 +270,11 @@ func TestVariableLoopingVUsConfigExecutionPlanExample(t *testing.T) { assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) } -func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { +func TestRampingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { t.Parallel() et, err := lib.NewExecutionTuple(newExecutionSegmentFromString("0:1/3"), nil) require.NoError(t, err) - conf := NewVariableLoopingVUsConfig("test") + conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -349,10 +349,10 @@ func TestVariableLoopingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { assert.Equal(t, rawStepsZeroEnd, conf.GetExecutionRequirements(et)) } -func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { +func TestRampingVUsExecutionTupleTests(t *testing.T) { t.Parallel() - conf := NewVariableLoopingVUsConfig("test") + conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -572,7 +572,7 @@ func TestVariableLoopingVUsExecutionTupleTests(t *testing.T) { } } -func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { +func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { t.Parallel() testCases := []struct { @@ -735,7 +735,7 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { } for _, testCase := range testCases { - conf := NewVariableLoopingVUsConfig("test") + conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(testCase.start) conf.Stages = testCase.stages et := testCase.et @@ -751,7 +751,7 @@ func TestVariableLoopingVUsGetRawExecutionStepsCornerCases(t *testing.T) { } } -func BenchmarkVariableLoopingVUsGetRawExecutionSteps(b *testing.B) { +func BenchmarkRampingVUsGetRawExecutionSteps(b *testing.B) { testCases := []struct { seq string seg string @@ -801,7 +801,7 @@ func BenchmarkVariableLoopingVUsGetRawExecutionSteps(b *testing.B) { for _, stageCase := range stageCases { var st []Stage require.NoError(b, json.Unmarshal([]byte(stageCase.stages), &st)) - vlvc := VariableLoopingVUsConfig{ + vlvc := RampingVUsConfig{ Stages: st, } b.Run(stageCase.name, func(b *testing.B) { @@ -1044,7 +1044,7 @@ func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { segmentSeqMaxLen = 15 maxNumerator = 300 ) - getTestConfig := func(name string) VariableLoopingVUsConfig { + getTestConfig := func(name string) RampingVUsConfig { stagesCount := 1 + r.Int31n(maxStages) stages := make([]Stage, stagesCount) for s := int32(0); s < stagesCount; s++ { @@ -1052,7 +1052,7 @@ func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { stages[s] = Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} } - c := NewVariableLoopingVUsConfig(name) + c := NewRampingVUsConfig(name) c.GracefulRampDown = types.NullDurationFrom(0) c.GracefulStop = types.NullDurationFrom(0) c.StartVUs = null.IntFrom(r.Int63n(maxVUs)) diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index c81749ac013..f3984260cdf 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -32,7 +32,7 @@ import ( // This is a helper type used in executors where we have to dynamically control // the number of VUs that are simultaneously running. For the moment, it is used -// in the VariableLoopingVUs and the ExternallyControlled executors. +// in the RampingVUs and the ExternallyControlled executors. // // TODO: something simpler? type vuHandle struct { From 54a1a45448801b9ba835bec9b63881289265e7b2 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 9 Jun 2020 17:58:13 +0300 Subject: [PATCH 297/350] Rename executor files to reflect their new names --- lib/executor/{constant_looping_vus.go => constant_vus.go} | 0 .../{constant_looping_vus_test.go => constant_vus_test.go} | 0 .../{variable_arrival_rate.go => ramping_arrival_rate.go} | 0 ...variable_arrival_rate_test.go => ramping_arrival_rate_test.go} | 0 lib/executor/{variable_looping_vus.go => ramping_vus.go} | 0 .../{variable_looping_vus_test.go => ramping_vus_test.go} | 0 6 files changed, 0 insertions(+), 0 deletions(-) rename lib/executor/{constant_looping_vus.go => constant_vus.go} (100%) rename lib/executor/{constant_looping_vus_test.go => constant_vus_test.go} (100%) rename lib/executor/{variable_arrival_rate.go => ramping_arrival_rate.go} (100%) rename lib/executor/{variable_arrival_rate_test.go => ramping_arrival_rate_test.go} (100%) rename lib/executor/{variable_looping_vus.go => ramping_vus.go} (100%) rename lib/executor/{variable_looping_vus_test.go => ramping_vus_test.go} (100%) diff --git a/lib/executor/constant_looping_vus.go b/lib/executor/constant_vus.go similarity index 100% rename from lib/executor/constant_looping_vus.go rename to lib/executor/constant_vus.go diff --git a/lib/executor/constant_looping_vus_test.go b/lib/executor/constant_vus_test.go similarity index 100% rename from lib/executor/constant_looping_vus_test.go rename to lib/executor/constant_vus_test.go diff --git a/lib/executor/variable_arrival_rate.go b/lib/executor/ramping_arrival_rate.go similarity index 100% rename from lib/executor/variable_arrival_rate.go rename to lib/executor/ramping_arrival_rate.go diff --git a/lib/executor/variable_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go similarity index 100% rename from lib/executor/variable_arrival_rate_test.go rename to lib/executor/ramping_arrival_rate_test.go diff --git a/lib/executor/variable_looping_vus.go b/lib/executor/ramping_vus.go similarity index 100% rename from lib/executor/variable_looping_vus.go rename to lib/executor/ramping_vus.go diff --git a/lib/executor/variable_looping_vus_test.go b/lib/executor/ramping_vus_test.go similarity index 100% rename from lib/executor/variable_looping_vus_test.go rename to lib/executor/ramping_vus_test.go From 0486c11b8ad17f8465885d66ad18c57e893db1f9 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 10 Jun 2020 19:22:15 +0300 Subject: [PATCH 298/350] merge steps in VLV executors for more stability (#1496) Also fix VLV not respecting gracefulStop since 270fd91c, add test for it and try to make some other VLV tests more stable. The original code (pre 270fd91c) did go through all the raw steps (ramp up/down) and the graceful stop ones concurrently and when the raw steps ended it will return from Run, but will start a goroutine to go through the remainign graceful stop ones. The most important of which ( at least in my current understanding) being the last one which is the actuall gracefulStop end which will make all VUs stop. The reason why this actually worked is that it also waiting on all activeVUs to have ended being active, being returned in the new terminology, before it actually cancels the context for all of them. So if the VUs manage to end iterations in this time the executor will end earlier if not the gracefuStop will make them. 270fd91c broke the returning of VUs which lead to the executor never returning and moving the cancel "before" the waiting for activeVUs fixed that at the cost of gracefulStop not being taken into account, but with no tests, nobody noticed. Here I basically revert that especially because vuHandle now returns VUs when stopped. fixes #1473 Co-authored-by: na-- --- lib/executor/ramping_vus.go | 90 +++++++++------- lib/executor/ramping_vus_test.go | 172 ++++++++++++++++++++++++++++++- lib/helpers.go | 45 -------- 3 files changed, 221 insertions(+), 86 deletions(-) diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index 28dc43970a4..fe4e0be4390 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -548,12 +548,12 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) maxVUs := lib.GetMaxPlannedVUs(gracefulExecutionSteps) gracefulStop := maxDuration - regularDuration - activeVUs := &sync.WaitGroup{} - defer activeVUs.Wait() - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, regularDuration, gracefulStop) defer cancel() + activeVUs := &sync.WaitGroup{} + defer activeVUs.Wait() + // Make sure the log and the progress bar have accurate information vlv.logger.WithFields(logrus.Fields{ "type": vlv.config.GetType(), "startVUs": vlv.config.GetStartVUs(vlv.executionState.ExecutionTuple), "maxVUs": maxVUs, @@ -588,6 +588,7 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) } else { activeVUs.Add(1) atomic.AddInt64(activeVUsCount, 1) + vlv.executionState.ModCurrentlyActiveVUsCount(+1) } return initVU, err } @@ -595,6 +596,7 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) vlv.executionState.ReturnVU(initVU, false) atomic.AddInt64(activeVUsCount, -1) activeVUs.Done() + vlv.executionState.ModCurrentlyActiveVUsCount(-1) } vuHandles := make([]*vuHandle, maxVUs) @@ -606,9 +608,6 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) vuHandles[i] = vuHandle } - rawStepEvents := lib.StreamExecutionSteps(ctx, startTime, rawExecutionSteps, true) - gracefulLimitEvents := lib.StreamExecutionSteps(ctx, startTime, gracefulExecutionSteps, false) - // 0 <= currentScheduledVUs <= currentMaxAllowedVUs <= maxVUs var currentScheduledVUs, currentMaxAllowedVUs uint64 @@ -616,12 +615,10 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) if newScheduledVUs > currentScheduledVUs { for vuNum := currentScheduledVUs; vuNum < newScheduledVUs; vuNum++ { _ = vuHandles[vuNum].start() // TODO handle error - vlv.executionState.ModCurrentlyActiveVUsCount(+1) } } else { for vuNum := newScheduledVUs; vuNum < currentScheduledVUs; vuNum++ { vuHandles[vuNum].gracefulStop() - vlv.executionState.ModCurrentlyActiveVUsCount(-1) } } currentScheduledVUs = newScheduledVUs @@ -636,40 +633,59 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) currentMaxAllowedVUs = newMaxAllowedVUs } - handleAllRawSteps := func() bool { - for { - select { - case step, ok := <-rawStepEvents: - if !ok { - return true - } - handleNewScheduledVUs(step.PlannedVUs) - case step := <-gracefulLimitEvents: - if step.PlannedVUs > currentMaxAllowedVUs { - // Handle the case where a value is read from the - // gracefulLimitEvents channel before rawStepEvents - handleNewScheduledVUs(step.PlannedVUs) - } - handleNewMaxAllowedVUs(step.PlannedVUs) - case <-ctx.Done(): - return false + wait := waiter(ctx, startTime) + // iterate over rawExecutionSteps and gracefulExecutionSteps in order by TimeOffset + // giving rawExecutionSteps precedence. + // we stop iterating once rawExecutionSteps are over as we need to run the remaining + // gracefulExecutionSteps concurrently while waiting for VUs to stop in order to not wait until + // the end of gracefulStop timeouts + i, j := 0, 0 + for i != len(rawExecutionSteps) { + if rawExecutionSteps[i].TimeOffset > gracefulExecutionSteps[j].TimeOffset { + if wait(gracefulExecutionSteps[j].TimeOffset) { + return + } + handleNewMaxAllowedVUs(gracefulExecutionSteps[j].PlannedVUs) + j++ + } else { + if wait(rawExecutionSteps[i].TimeOffset) { + return } + handleNewScheduledVUs(rawExecutionSteps[i].PlannedVUs) + i++ } } - if handleAllRawSteps() { - // Handle any remaining graceful stops - go func() { - for { - select { - case step := <-gracefulLimitEvents: - handleNewMaxAllowedVUs(step.PlannedVUs) - case <-maxDurationCtx.Done(): - return - } + go func() { // iterate over the remaining gracefulExecutionSteps + for _, step := range gracefulExecutionSteps[j:] { + if wait(step.TimeOffset) { + return } - }() - } + handleNewMaxAllowedVUs(step.PlannedVUs) + } + }() return nil } + +// waiter returns a function that will sleep/wait for the required time since the startTime and then +// return. If the context was done before that it will return true otherwise it will return false +// TODO use elsewhere +// TODO set startTime here? +// TODO move it to a struct type or something and benchmark if that makes a difference +func waiter(ctx context.Context, startTime time.Time) func(offset time.Duration) bool { + timer := time.NewTimer(time.Hour * 24) + return func(offset time.Duration) bool { + offsetDiff := offset - time.Since(startTime) + if offsetDiff > 0 { // wait until time of event arrives // TODO have a mininum + timer.Reset(offsetDiff) + select { + case <-ctx.Done(): + return true // exit if context is cancelled + case <-timer.C: + // now we do a step + } + } + return false + } +} diff --git a/lib/executor/ramping_vus_test.go b/lib/executor/ramping_vus_test.go index 91af6c8bc71..bdd2d710f4d 100644 --- a/lib/executor/ramping_vus_test.go +++ b/lib/executor/ramping_vus_test.go @@ -81,7 +81,7 @@ func TestRampingVUsRun(t *testing.T) { sampleTimes := []time.Duration{ 500 * time.Millisecond, 1000 * time.Millisecond, - 800 * time.Millisecond, + 900 * time.Millisecond, } errCh := make(chan error) @@ -99,6 +99,167 @@ func TestRampingVUsRun(t *testing.T) { assert.Equal(t, int64(29), atomic.LoadInt64(&iterCount)) } +func TestRampingVUsGracefulStopWaits(t *testing.T) { + t.Parallel() + + config := RampingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, + StartVUs: null.IntFrom(1), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(1), + }, + }, + } + + var ( + started = make(chan struct{}) // the iteration started + stopped = make(chan struct{}) // the iteration stopped + stop = make(chan struct{}) // the itearation should stop + ) + + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, _ := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + close(started) + defer close(stopped) + select { + case <-ctx.Done(): + t.Fatal("The iterations should've ended before the context") + case <-stop: + } + return nil + }), + ) + defer cancel() + errCh := make(chan error) + go func() { errCh <- executor.Run(ctx, nil) }() + + <-started + // 500 milliseconds more then the duration and 500 less then the gracefulStop + time.Sleep(time.Millisecond * 1500) + close(stop) + <-stopped + + require.NoError(t, <-errCh) +} + +func TestRampingVUsGracefulStopStops(t *testing.T) { + t.Parallel() + + config := RampingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, + StartVUs: null.IntFrom(1), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(1), + }, + }, + } + + var ( + started = make(chan struct{}) // the iteration started + stopped = make(chan struct{}) // the iteration stopped + stop = make(chan struct{}) // the itearation should stop + ) + + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, _ := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + close(started) + defer close(stopped) + select { + case <-ctx.Done(): + case <-stop: + t.Fatal("The iterations shouldn't have ended before the context") + } + return nil + }), + ) + defer cancel() + errCh := make(chan error) + go func() { errCh <- executor.Run(ctx, nil) }() + + <-started + // 500 milliseconds more then the gracefulStop + duration + time.Sleep(time.Millisecond * 2500) + close(stop) + <-stopped + + require.NoError(t, <-errCh) +} + +func TestRampingVUsGracefulRampDown(t *testing.T) { + t.Parallel() + + config := RampingVUsConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(5 * time.Second)}, + StartVUs: null.IntFrom(2), + GracefulRampDown: types.NullDurationFrom(5 * time.Second), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(2), + }, + { + Duration: types.NullDurationFrom(1 * time.Second), + Target: null.IntFrom(0), + }, + }, + } + + var ( + started = make(chan struct{}) // the iteration started + stopped = make(chan struct{}) // the iteration stopped + stop = make(chan struct{}) // the itearation should stop + ) + + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, _ := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + if lib.GetState(ctx).Vu == 1 { // the first VU will wait here to do stuff + close(started) + defer close(stopped) + select { + case <-ctx.Done(): + t.Fatal("The iterations shouldn't have ended before the context") + case <-stop: + } + } else { // all other (1) VUs will just sleep long enough + time.Sleep(2500 * time.Millisecond) + } + return nil + }), + ) + defer cancel() + errCh := make(chan error) + go func() { errCh <- executor.Run(ctx, nil) }() + + <-started + // 500 milliseconds more then the gracefulRampDown + duration + time.Sleep(2500 * time.Millisecond) + close(stop) + <-stopped + + select { + case err := <-errCh: + require.NoError(t, err) + case <-time.After(time.Second): // way too much time + t.Fatal("Execution should've ended already") + } +} + // Ensure there's no wobble of VUs during graceful ramp-down, without segments. // See https://github.com/loadimpact/k6/issues/1296 func TestRampingVUsRampDownNoWobble(t *testing.T) { @@ -126,7 +287,7 @@ func TestRampingVUsRampDownNoWobble(t *testing.T) { ctx, cancel, executor, _ := setupExecutor( t, config, es, simpleRunner(func(ctx context.Context) error { - time.Sleep(1 * time.Second) + time.Sleep(500 * time.Millisecond) return nil }), ) @@ -136,7 +297,10 @@ func TestRampingVUsRampDownNoWobble(t *testing.T) { 100 * time.Millisecond, 3000 * time.Millisecond, } - const rampDownSamples = 50 + const rampDownSampleTime = 50 * time.Millisecond + var rampDownSamples = int(time.Duration( + config.Stages[len(config.Stages)-1].Duration.Duration+config.GracefulRampDown.Duration, + ) / rampDownSampleTime) errCh := make(chan error) go func() { errCh <- executor.Run(ctx, nil) }() @@ -149,7 +313,7 @@ func TestRampingVUsRampDownNoWobble(t *testing.T) { // Sample ramp-down at a higher rate for i := len(sampleTimes); i < rampDownSamples; i++ { - time.Sleep(50 * time.Millisecond) + time.Sleep(rampDownSampleTime) result[i] = es.GetCurrentlyActiveVUsCount() } diff --git a/lib/helpers.go b/lib/helpers.go index cedcbe0fa7e..8d2e133528a 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -22,7 +22,6 @@ package lib import ( "bytes" - "context" "encoding/json" "fmt" "strings" @@ -113,47 +112,3 @@ func ConcatErrors(errors []error, separator string) string { } return strings.Join(errStrings, separator) } - -// StreamExecutionSteps launches a new goroutine and emits all execution steps -// at their appropriate time offsets over the returned unbuffered channel. If -// closeChanWhenDone is specified, it will close the channel after it sends the -// last step. If it isn't, or if the context is cancelled, the internal -// goroutine will be stopped, *but the channel will remain open*! -// -// As usual, steps in the supplied slice have to be sorted by their TimeOffset -// values in an ascending order. Of course, multiple events can have the same -// time offset (incl. 0). -func StreamExecutionSteps( - ctx context.Context, startTime time.Time, steps []ExecutionStep, closeChanWhenDone bool, -) <-chan ExecutionStep { - ch := make(chan ExecutionStep) - go func() { - for _, step := range steps { - offsetDiff := step.TimeOffset - time.Since(startTime) - if offsetDiff > 0 { // wait until time of event arrives - select { - case <-ctx.Done(): - return // exit if context is cancelled - case <-time.After(offsetDiff): //TODO: reuse a timer? - // do nothing - } - } - select { - case <-ctx.Done(): - // exit if context is cancelled - return - case ch <- step: - // ... otherwise, just send the step - the out channel is - // unbuffered, so we don't need to worry whether the other side - // will keep reading after the context is done. - } - } - - // Close the channel only if all steps were sent successfully (i.e. the - // parent context didn't die) and we were instructed to do so. - if closeChanWhenDone { - close(ch) - } - }() - return ch -} From c8bfbcbc0f6413567148702aa84afb3df5d10271 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 12 Jun 2020 12:45:21 +0300 Subject: [PATCH 299/350] fix js package http benchmark and add one with base compatibility-mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit name time/op HTTPRequests-8 668µs ± 2% HTTPRequestsBase-8 664µs ± 3% name alloc/op HTTPRequests-8 27.6kB ± 0% HTTPRequestsBase-8 27.6kB ± 0% name allocs/op HTTPRequests-8 365 ± 0% HTTPRequestsBase-8 365 ± 0% as can be seen the diffence is negligable and within the margin of error --- js/http_bench_test.go | 48 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 47 insertions(+), 1 deletion(-) diff --git a/js/http_bench_test.go b/js/http_bench_test.go index aeb5cf50613..3bf41d1009f 100644 --- a/js/http_bench_test.go +++ b/js/http_bench_test.go @@ -25,6 +25,7 @@ import ( "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" @@ -44,16 +45,61 @@ func BenchmarkHTTPRequests(b *testing.B) { let res = http.get(url + "/cookies/set?k2=v2&k1=v1"); if (res.status != 200) { throw new Error("wrong status: " + res.status) } } + `), lib.RuntimeOptions{CompatibilityMode: null.StringFrom("extended")}) + if !assert.NoError(b, err) { + return + } + err = r.SetOptions(lib.Options{ + Throw: null.BoolFrom(true), + MaxRedirects: null.IntFrom(10), + Hosts: tb.Dialer.Hosts, + NoCookiesReset: null.BoolFrom(true), + }) + require.NoError(b, err) + + var ch = make(chan stats.SampleContainer, 100) + go func() { // read the channel so it doesn't block + for { + <-ch + } + }() + initVU, err := r.NewVU(1, ch) + if !assert.NoError(b, err) { + return + } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) + b.StartTimer() + for i := 0; i < b.N; i++ { + err = vu.RunOnce() + assert.NoError(b, err) + } +} + +func BenchmarkHTTPRequestsBase(b *testing.B) { + b.StopTimer() + tb := httpmultibin.NewHTTPMultiBin(b) + defer tb.Cleanup() + + r, err := getSimpleRunner("/script.js", tb.Replacer.Replace(` + var http = require("k6/http"); + exports.default = function() { + var url = "HTTPBIN_URL"; + var res = http.get(url + "/cookies/set?k2=v2&k1=v1"); + if (res.status != 200) { throw new Error("wrong status: " + res.status) } + } `)) if !assert.NoError(b, err) { return } - r.SetOptions(lib.Options{ + err = r.SetOptions(lib.Options{ Throw: null.BoolFrom(true), MaxRedirects: null.IntFrom(10), Hosts: tb.Dialer.Hosts, NoCookiesReset: null.BoolFrom(true), }) + require.NoError(b, err) var ch = make(chan stats.SampleContainer, 100) go func() { // read the channel so it doesn't block From 1841d359d525e734f017684471a2e9599526b1fb Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 12 Jun 2020 13:57:50 +0300 Subject: [PATCH 300/350] Skip bridge benchmarks atleast one of which is broken since 4cfbe1612303d9f335fbfa3265c4a4ad4eb309f6 --- js/common/bridge_test.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/js/common/bridge_test.go b/js/common/bridge_test.go index 4ce64637370..f31ddc77284 100644 --- a/js/common/bridge_test.go +++ b/js/common/bridge_test.go @@ -702,6 +702,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"AddError", "addWithError", bridgeTestAddWithErrorType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(int, int) int) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -709,6 +710,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"Context", "context", bridgeTestContextType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func()) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -716,6 +718,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"ContextAdd", "contextAdd", bridgeTestContextAddType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(int, int) int) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -723,6 +726,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"ContextAddError", "contextAddWithError", bridgeTestContextAddWithErrorType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(int, int) int) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -737,6 +741,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"SumContext", "sumWithContext", bridgeTestSumWithContextType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(...int) int) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -744,6 +749,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"SumError", "sumWithError", bridgeTestSumWithErrorType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(...int) int) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -751,6 +757,7 @@ func BenchmarkProxy(b *testing.B) { } }}, {"SumContextError", "sumWithContextAndError", bridgeTestSumWithContextAndErrorType{}, func(b *testing.B, fn interface{}) { + b.Skip() f := fn.(func(...int) int) b.ResetTimer() for i := 0; i < b.N; i++ { From 4e4d5d8776b3723f7c9c180d5648aa662b61620d Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Fri, 12 Jun 2020 14:03:42 +0300 Subject: [PATCH 301/350] Skip BenchmarkResponseJson which was apperantly broken when spliting http to httpext --- js/modules/k6/http/response_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/js/modules/k6/http/response_test.go b/js/modules/k6/http/response_test.go index ad752a1332d..a8691c148cd 100644 --- a/js/modules/k6/http/response_test.go +++ b/js/modules/k6/http/response_test.go @@ -397,6 +397,7 @@ func TestResponse(t *testing.T) { } func BenchmarkResponseJson(b *testing.B) { + b.Skipf("We need to have context in the response") testCases := []struct { selector string }{ @@ -410,6 +411,7 @@ func BenchmarkResponseJson(b *testing.B) { {"glossary"}, } for _, tc := range testCases { + tc := tc b.Run(fmt.Sprintf("Selector %s ", tc.selector), func(b *testing.B) { for n := 0; n < b.N; n++ { resp := responseFromHttpext(&httpext.Response{Body: jsonData}) From 40d5c1a3f9516f4f70657466c0d885f0e9953892 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 11 Jun 2020 14:19:46 +0300 Subject: [PATCH 302/350] Make the version a constant --- lib/consts/consts.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/consts/consts.go b/lib/consts/consts.go index 6437a97175a..c4d5ce92792 100644 --- a/lib/consts/consts.go +++ b/lib/consts/consts.go @@ -28,7 +28,7 @@ import ( ) // Version contains the current semantic version of k6. -var Version = "0.27.0-dev" //nolint:gochecknoglobals +const Version = "0.27.0-dev" // VersionDetails can be set externally as part of the build process var VersionDetails = "" // nolint:gochecknoglobals From 1ff05a46504f5386ca26a118420ff2268d09887e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 11 Jun 2020 14:19:58 +0300 Subject: [PATCH 303/350] Update the used Go version in appveyor --- appveyor.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/appveyor.yml b/appveyor.yml index 24090f366e9..56a9b3adc25 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -10,7 +10,7 @@ environment: # specific to go VERSION: "%APPVEYOR_REPO_TAG_NAME:v=%" GOPATH: c:\gopath - GOVERSION: 1.14.2 + GOVERSION: 1.14.4 GOMAXPROCS: 2 CGO_ENABLED: '0' GOARCH: amd64 From 06c82e2019178526665b5c8fe128192b30e2e65e Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 11 Jun 2020 14:20:31 +0300 Subject: [PATCH 304/350] Fix config validation messages after the renames --- lib/executors.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/executors.go b/lib/executors.go index a79df23d3d6..c08e7b5f60b 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -183,7 +183,7 @@ func (scs *ScenarioConfigs) UnmarshalJSON(data []byte) error { result := make(ScenarioConfigs, len(protoConfigs)) for k, v := range protoConfigs { if v.configType == "" { - return fmt.Errorf("execution config '%s' doesn't have a type value", k) + return fmt.Errorf("scenario '%s' doesn't have a specified executor type", k) } config, err := GetParsedExecutorConfig(k, v.configType, v.rawJSON) if err != nil { @@ -202,7 +202,7 @@ func (scs ScenarioConfigs) Validate() (errors []error) { for name, exec := range scs { if execErr := exec.Validate(); len(execErr) != 0 { errors = append(errors, - fmt.Errorf("executor %s has errors: %s", name, ConcatErrors(execErr, ", "))) + fmt.Errorf("scenario %s has configuration errors: %s", name, ConcatErrors(execErr, ", "))) } } return errors From 987c903c0b01533a04b4b65dc8dc3f53cf348b9c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 16 Jun 2020 15:33:01 +0300 Subject: [PATCH 305/350] Make errors starting the API server fatal, if adddress was explicitly set --- cmd/run.go | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/cmd/run.go b/cmd/run.go index 3ae8f18bf67..0bfc75cda30 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -61,6 +61,7 @@ const ( genericEngineErrorCode = 103 invalidConfigErrorCode = 104 externalAbortErrorCode = 105 + cannotStartRESTAPIErrorCode = 106 ) // TODO: fix this, global variables are not very testable... @@ -220,12 +221,19 @@ a commandline interface for interacting with it.`, engine.Collectors = append(engine.Collectors, collector) } - // Create an API server. + // Spin up the REST API server, if not disabled. if address != "" { initBar.Modify(pb.WithConstProgress(0, "Init API server")) go func() { - if aerr := api.ListenAndServe(address, engine); err != nil { - logger.WithError(aerr).Warn("Error from API server") + logger.Debugf("Starting the REST API server on %s", address) + if aerr := api.ListenAndServe(address, engine); aerr != nil { + // Only exit k6 if the user has explicitly set the REST API address + if cmd.Flags().Lookup("address").Changed { + logger.WithError(aerr).Error("Error from API server") + os.Exit(cannotStartRESTAPIErrorCode) + } else { + logger.WithError(aerr).Warn("Error from API server") + } } }() } From 56a6285c1d3fedd62db1a669f013c7894acd6835 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 17 Jun 2020 14:11:48 +0300 Subject: [PATCH 306/350] Rename configType to executorType --- lib/executors.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/lib/executors.go b/lib/executors.go index c08e7b5f60b..2912c1d58b3 100644 --- a/lib/executors.go +++ b/lib/executors.go @@ -182,10 +182,10 @@ func (scs *ScenarioConfigs) UnmarshalJSON(data []byte) error { result := make(ScenarioConfigs, len(protoConfigs)) for k, v := range protoConfigs { - if v.configType == "" { + if v.executorType == "" { return fmt.Errorf("scenario '%s' doesn't have a specified executor type", k) } - config, err := GetParsedExecutorConfig(k, v.configType, v.rawJSON) + config, err := GetParsedExecutorConfig(k, v.executorType, v.rawJSON) if err != nil { return err } @@ -330,17 +330,17 @@ func GetParsedExecutorConfig(name, configType string, rawJSON []byte) (result Ex } type protoExecutorConfig struct { - configType string - rawJSON json.RawMessage + executorType string + rawJSON json.RawMessage } // UnmarshalJSON unmarshals the base config (to get the type), but it also // stores the unprocessed JSON so we can parse the full config in the next step func (pc *protoExecutorConfig) UnmarshalJSON(b []byte) error { var tmp struct { - ConfigType string `json:"executor"` + ExecutorType string `json:"executor"` } err := json.Unmarshal(b, &tmp) - *pc = protoExecutorConfig{tmp.ConfigType, b} + *pc = protoExecutorConfig{tmp.ExecutorType, b} return err } From e0eab8fb682548f204e715ca8ff56f7b318e9225 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 17 Jun 2020 15:59:24 +0300 Subject: [PATCH 307/350] rewrite vuHandle as a state machine and fix some more races (#1506) this is ... mostly a complete rewrite of vuHandle now using an atomically updated state to work as a state machine. There is still a fast path which provides all the speed, but now it also always returns any VU it has got and doesn't return it in cases where this isn't required - there was a start after a gracefulStop, but before it actually stopped. closes #1492 --- lib/executor/vu_handle.go | 258 +++++++++++++++++++-------------- lib/executor/vu_handle_test.go | 91 +++++++++++- 2 files changed, 241 insertions(+), 108 deletions(-) diff --git a/lib/executor/vu_handle.go b/lib/executor/vu_handle.go index f3984260cdf..ef894b13bd8 100644 --- a/lib/executor/vu_handle.go +++ b/lib/executor/vu_handle.go @@ -30,11 +30,63 @@ import ( "github.com/loadimpact/k6/lib" ) +type stateType int32 + +// states +const ( + stopped stateType = iota + starting + running + toGracefulStop + toHardStop +) + +/* +the below is a state transition table (https://en.wikipedia.org/wiki/State-transition_table) +short names for input: +- start is the method start +- loop is a loop of runLoopsIfPossible +- grace is the method gracefulStop +- hard is the method hardStop ++-------+-------------------------------------+---------------------------------------------------+ +| input | current | next state | notes | ++-------+-------------------------------------+---------------------------------------------------+ +| start | stopped | starting | normal | +| start | starting | starting | nothing | +| start | running | running | nothing | +| start | toGracefulStop | running | we raced with the loop stopping, just continue | +| start | toHardStop | starting | same as stopped really | +| loop | stopped | stopped | we actually are blocked on canStartIter | +| loop | starting | running | get new VU and context | +| loop | running | running | usually fast path | +| loop | toGracefulStop | stopped | cancel the context and make new one | +| loop | toHardStop | stopped | cancel the context and make new one | +| grace | stopped | stopped | nothing | +| grace | starting | stopped | cancel the context to return the VU | +| grace | running | toGracefulStop | normal one, the actual work is in the loop | +| grace | toGracefulStop | toGracefulStop | nothing | +| grace | toHardSTop | toHardStop | nothing | +| hard | stopped | stopped | nothing | +| hard | starting | stopped | short circuit as in the grace case, not necessary | +| hard | running | toHardStop | normal, cancel context and reinitialize it | +| hard | toGracefulStop | toHardStop | normal, cancel context and reinitialize it | +| hard | toHardStop | toHardStop | nothing | ++-------+-----------------+-------------------+----------------------------------------------------+ +*/ + // This is a helper type used in executors where we have to dynamically control // the number of VUs that are simultaneously running. For the moment, it is used // in the RampingVUs and the ExternallyControlled executors. -// -// TODO: something simpler? +// Notes on the implementation requirements: +// - it needs to be able to start and stop VUs in thread safe fashion +// - for each call to getVU there must be 1 (and only 1) call to returnVU +// - gracefulStop must let an iteration which has started to finish. For reasons of ease of +// implementation and lack of good evidence it's not required to let a not started iteration to +// finish in other words if you call start and then gracefulStop, there is no requirement for +// 1 iteration to have been started. +// - hardStop must stop an iteration in process +// - it's not required but preferable, if where possible to not reactivate VUs and to reuse context +// as this speed ups the execution type vuHandle struct { mutex *sync.Mutex parentCtx context.Context @@ -45,177 +97,173 @@ type vuHandle struct { initVU lib.InitializedVU activeVU lib.ActiveVU canStartIter chan struct{} - // If change is not 0, it signals that the VU needs to be reinitialized. It must be added to and - // read with atomics and helps to skip checking all the contexts and channels all the time. - change int32 - ctx, vuCtx context.Context - cancel, vuCancel func() - logger *logrus.Entry + state stateType // see the table above for meanings + // stateH []int32 // helper for debugging + + ctx context.Context + cancel func() + logger *logrus.Entry } func newStoppedVUHandle( parentCtx context.Context, getVU func() (lib.InitializedVU, error), returnVU func(lib.InitializedVU), config *BaseConfig, logger *logrus.Entry, ) *vuHandle { - lock := &sync.Mutex{} ctx, cancel := context.WithCancel(parentCtx) - vh := &vuHandle{ - mutex: lock, + return &vuHandle{ + mutex: &sync.Mutex{}, parentCtx: parentCtx, getVU: getVU, config: config, canStartIter: make(chan struct{}), - change: 1, - - ctx: ctx, - cancel: cancel, - logger: logger, - } + state: stopped, - // TODO maybe move the repeating parts in a function? - vh.returnVU = func(v lib.InitializedVU) { - // Don't return the initialized VU back - vh.mutex.Lock() - select { - case <-vh.parentCtx.Done(): - // we are done just ruturn the VU - vh.initVU = nil - vh.activeVU = nil - atomic.StoreInt32(&vh.change, 1) - vh.mutex.Unlock() - returnVU(v) - default: - select { - case <-vh.canStartIter: - // we can continue with itearting - lets not return the vu - vh.activateVU() // we still need to reactivate it to get the new context and cancel - atomic.StoreInt32(&vh.change, 1) - vh.mutex.Unlock() - default: // we actually have to return the vu - vh.initVU = nil - vh.activeVU = nil - atomic.StoreInt32(&vh.change, 1) - vh.mutex.Unlock() - returnVU(v) - } - } + ctx: ctx, + cancel: cancel, + logger: logger, + returnVU: returnVU, } - - return vh } func (vh *vuHandle) start() (err error) { vh.mutex.Lock() - vh.logger.Debug("Start") - if vh.initVU == nil { + defer vh.mutex.Unlock() + + switch vh.state { + case starting, running: + return nil // nothing to do + case toGracefulStop: // we raced with the loop, lets not return the vu just to get it back + vh.logger.Debug("Start") + close(vh.canStartIter) + vh.changeState(running) + case stopped, toHardStop: // we need to reactivate the VU and remake the context for it + vh.logger.Debug("Start") vh.initVU, err = vh.getVU() if err != nil { return err } - vh.activateVU() - atomic.AddInt32(&vh.change, 1) - } - select { - case <-vh.canStartIter: // we are alrady started do nothing - default: + + vh.activeVU = vh.initVU.Activate(getVUActivationParams(vh.ctx, *vh.config, vh.returnVU)) close(vh.canStartIter) + vh.changeState(starting) } - vh.mutex.Unlock() return nil } -// this must be called with the mutex locked -func (vh *vuHandle) activateVU() { - vh.vuCtx, vh.vuCancel = context.WithCancel(vh.ctx) - vh.activeVU = vh.initVU.Activate(getVUActivationParams(vh.vuCtx, *vh.config, vh.returnVU)) +// just a helper function for debugging +func (vh *vuHandle) changeState(newState stateType) { + // vh.stateH = append(vh.stateH, newState) + atomic.StoreInt32((*int32)(&vh.state), int32(newState)) } func (vh *vuHandle) gracefulStop() { vh.mutex.Lock() - select { - case <-vh.canStartIter: - atomic.AddInt32(&vh.change, 1) - vh.activeVU = nil - vh.canStartIter = make(chan struct{}) - vh.logger.Debug("Graceful stop") - default: - // do nothing, the signalling channel was already initialized by hardStop() + defer vh.mutex.Unlock() + switch vh.state { + case toGracefulStop, toHardStop, stopped: + return // nothing to do + case starting: // we raced with the loop and apparently it won't do a single iteration + vh.cancel() + vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) + vh.changeState(stopped) + case running: + vh.changeState(toGracefulStop) } - vh.mutex.Unlock() + + vh.logger.Debug("Graceful stop") + vh.canStartIter = make(chan struct{}) } func (vh *vuHandle) hardStop() { vh.mutex.Lock() - vh.logger.Debug("Hard stop") - vh.cancel() // cancel the previous context - atomic.AddInt32(&vh.change, 1) - vh.initVU = nil - vh.activeVU = nil - vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) // create a new context - select { - case <-vh.canStartIter: - vh.canStartIter = make(chan struct{}) - default: - // do nothing, the signalling channel was already initialized by gracefulStop() + defer vh.mutex.Unlock() + + switch vh.state { + case toHardStop, stopped: + return // nothing to do + case starting: // we raced with the loop and apparently it won't do a single iteration + vh.changeState(stopped) + case running, toGracefulStop: + vh.changeState(toHardStop) } - vh.mutex.Unlock() + vh.logger.Debug("Hard stop") + vh.cancel() + vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) + vh.canStartIter = make(chan struct{}) } +// runLoopsIfPossible is where all the fun is :D. Unfortunately somewhere we need to check most +// of the cases and this is where this happens. func (vh *vuHandle) runLoopsIfPossible(runIter func(context.Context, lib.ActiveVU) bool) { // We can probably initialize here, but it's also easier to just use the slow path in the second // part of the for loop + defer func() { + // not sure if this is needed, because here the parentCtx is canceled and I would argue it doesn't matter + // if we set the correct state + vh.mutex.Lock() + vh.changeState(stopped) + vh.mutex.Unlock() + }() + var ( executorDone = vh.parentCtx.Done() - vuCtx context.Context + ctx context.Context cancel func() vu lib.ActiveVU ) for { - ch := atomic.LoadInt32(&vh.change) - if ch == 0 && runIter(vuCtx, vu) { // fast path + state := stateType(atomic.LoadInt32((*int32)(&vh.state))) + if state == running && runIter(ctx, vu) { // fast path continue } // slow path - something has changed - get what and wait until we can do more iterations - // TODO: I think we can skip cancelling in some cases but I doubt it will do much good in most - if cancel != nil { - cancel() // signal to return the vu before we continue - } vh.mutex.Lock() - canStartIter, ctx := vh.canStartIter, vh.ctx - cancel = vh.vuCancel - vh.mutex.Unlock() - select { case <-executorDone: // The whole executor is done, nothing more to do. + vh.mutex.Unlock() return default: } - // We're not running, but the executor isn't done yet, so we wait + + switch vh.state { + case running: // start raced us toGracefulStop + vh.mutex.Unlock() + continue + case toGracefulStop: + if cancel != nil { + // we need to cancel the context, to return the vu + // and because *we* did, lets reinitialize it + cancel() + vh.ctx, vh.cancel = context.WithCancel(vh.parentCtx) + } + fallthrough // to set the state + case toHardStop: + // we have *now* stopped + vh.changeState(stopped) + case stopped, starting: + // there is nothing to do + } + + canStartIter := vh.canStartIter + ctx = vh.ctx + vh.mutex.Unlock() + + // We're are stopped, but the executor isn't done yet, so we wait // for either one of those conditions. select { - case <-canStartIter: + case <-canStartIter: // we can start again vh.mutex.Lock() select { case <-vh.canStartIter: // we check again in case of race // reinitialize - if vh.activeVU == nil { - // we've raced with the ReturnVU: we can continue doing iterations but - // a stop has happened and returnVU hasn't managed to run yet ... so we loop - // TODO call runtime.GoSched() in the else to give priority to possibly the returnVU goroutine - vh.mutex.Unlock() - continue - } - - vu = vh.activeVU - vuCtx = vh.vuCtx - cancel = vh.vuCancel - atomic.StoreInt32(&vh.change, 0) // clear changes here + vu, ctx, cancel = vh.activeVU, vh.ctx, vh.cancel + vh.changeState(running) default: // well we got raced to here by something ... loop again ... } diff --git a/lib/executor/vu_handle_test.go b/lib/executor/vu_handle_test.go index ff688363d82..0198a265776 100644 --- a/lib/executor/vu_handle_test.go +++ b/lib/executor/vu_handle_test.go @@ -29,7 +29,10 @@ func TestVUHandleRace(t *testing.T) { // testLog.Level = logrus.DebugLevel logEntry := logrus.NewEntry(testLog) + var getVUCount int64 + var returnVUCount int64 getVU := func() (lib.InitializedVU, error) { + atomic.AddInt64(&getVUCount, 1) return &minirunner.VU{ R: &minirunner.MiniRunner{ Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { @@ -41,6 +44,7 @@ func TestVUHandleRace(t *testing.T) { } returnVU := func(_ lib.InitializedVU) { + atomic.AddInt64(&returnVUCount, 1) // do something } var interruptedIter int64 @@ -88,7 +92,88 @@ func TestVUHandleRace(t *testing.T) { }() wg.Wait() vuHandle.hardStop() // STOP it - time.Sleep(time.Millisecond) + time.Sleep(time.Millisecond * 50) + interruptedBefore := atomic.LoadInt64(&interruptedIter) + fullBefore := atomic.LoadInt64(&fullIterations) + _ = vuHandle.start() + time.Sleep(time.Millisecond * 50) // just to be sure an iteration will squeeze in + cancel() + time.Sleep(time.Millisecond * 50) + interruptedAfter := atomic.LoadInt64(&interruptedIter) + fullAfter := atomic.LoadInt64(&fullIterations) + assert.True(t, interruptedBefore >= interruptedAfter-1, + "too big of a difference %d >= %d - 1", interruptedBefore, interruptedAfter) + assert.True(t, fullBefore+1 <= fullAfter, + "too small of a difference %d + 1 <= %d", fullBefore, fullAfter) + require.Equal(t, atomic.LoadInt64(&getVUCount), atomic.LoadInt64(&returnVUCount)) +} + +// this test is mostly interesting when -race is enabled +func TestVUHandleStartStopRace(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + logHook := &testutils.SimpleLogrusHook{HookedLevels: []logrus.Level{logrus.DebugLevel}} + testLog := logrus.New() + testLog.AddHook(logHook) + testLog.SetOutput(testutils.NewTestOutput(t)) + // testLog.Level = logrus.DebugLevel + logEntry := logrus.NewEntry(testLog) + + var vuID int64 = -1 + + var testIterations = 10000 + returned := make(chan struct{}) + getVU := func() (lib.InitializedVU, error) { + returned = make(chan struct{}) + return &minirunner.VU{ + ID: atomic.AddInt64(&vuID, 1), + R: &minirunner.MiniRunner{ + Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { + // TODO: do something + return nil + }, + }, + }, nil + } + + returnVU := func(v lib.InitializedVU) { + require.Equal(t, atomic.LoadInt64(&vuID), v.(*minirunner.VU).ID) + close(returned) + } + var interruptedIter int64 + var fullIterations int64 + + runIter := func(ctx context.Context, vu lib.ActiveVU) bool { + _ = vu.RunOnce() + select { + case <-ctx.Done(): + // Don't log errors or emit iterations metrics from cancelled iterations + atomic.AddInt64(&interruptedIter, 1) + return false + default: + atomic.AddInt64(&fullIterations, 1) + return true + } + } + + vuHandle := newStoppedVUHandle(ctx, getVU, returnVU, &BaseConfig{}, logEntry) + go vuHandle.runLoopsIfPossible(runIter) + for i := 0; i < testIterations; i++ { + err := vuHandle.start() + vuHandle.gracefulStop() + require.NoError(t, err) + select { + case <-returned: + case <-time.After(100 * time.Millisecond): + go panic("returning took too long") + time.Sleep(time.Second) + } + } + + vuHandle.hardStop() // STOP it + time.Sleep(time.Millisecond * 5) interruptedBefore := atomic.LoadInt64(&interruptedIter) fullBefore := atomic.LoadInt64(&fullIterations) _ = vuHandle.start() @@ -205,7 +290,7 @@ func TestVUHandleSimple(t *testing.T) { }() err := vuHandle.start() require.NoError(t, err) - time.Sleep(time.Millisecond * 5) + time.Sleep(time.Millisecond * 50) vuHandle.gracefulStop() time.Sleep(time.Millisecond * 1500) assert.EqualValues(t, 1, atomic.LoadUint32(&getVUCount)) @@ -218,7 +303,7 @@ func TestVUHandleSimple(t *testing.T) { cancel() wg.Wait() - time.Sleep(time.Millisecond * 5) + time.Sleep(time.Millisecond * 50) assert.EqualValues(t, 2, atomic.LoadUint32(&getVUCount)) assert.EqualValues(t, 2, atomic.LoadUint32(&returnVUCount)) assert.EqualValues(t, 1, atomic.LoadInt64(&interruptedIter)) From 56a03ae4b9195e97c06e6f9a046354d82406fa06 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 23 Jun 2020 15:11:56 +0300 Subject: [PATCH 308/350] remove space from logrus Field --- loader/loader.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/loader/loader.go b/loader/loader.go index 3f73e378b7f..f72ae4b2773 100644 --- a/loader/loader.go +++ b/loader/loader.go @@ -171,8 +171,8 @@ func Load( ) (*SourceData, error) { logrus.WithFields( logrus.Fields{ - "moduleSpecifier": moduleSpecifier, - "original moduleSpecifier": originalModuleSpecifier, + "moduleSpecifier": moduleSpecifier, + "originalModuleSpecifier": originalModuleSpecifier, }).Debug("Loading...") var pathOnFs string From 986f67d6b9d5fcfc9df1662610b6a0eb6e1a0b91 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 24 Jun 2020 11:53:08 +0300 Subject: [PATCH 309/350] Don't leak goroutines everywhere in the ws code Previous to this past running k6 compiled with the race detector and ws code will most definetely leak enough goroutines in 10 seconds that it will stop because of the 8k limit ... After this it seems to not leak anymore :confetti: --- js/modules/k6/ws/ws.go | 35 +++++++++++++++++++++++++++-------- js/modules/k6/ws/ws_test.go | 3 ++- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/js/modules/k6/ws/ws.go b/js/modules/k6/ws/ws.go index 078b976ac94..55a973ac18d 100644 --- a/js/modules/k6/ws/ws.go +++ b/js/modules/k6/ws/ws.go @@ -247,7 +247,7 @@ func (*WS) Connect(ctx context.Context, url string, args ...goja.Value) (*WSHTTP readErrChan := make(chan error) // Wraps a couple of channels around conn.ReadMessage - go readPump(conn, readDataChan, readErrChan, readCloseChan) + go socket.readPump(readDataChan, readErrChan, readCloseChan) // This is the main control loop. All JS code (including error handlers) // should only be executed by this thread to avoid race conditions @@ -384,7 +384,11 @@ func (s *Socket) SetTimeout(fn goja.Callable, timeoutMs int) { go func() { select { case <-time.After(time.Duration(timeoutMs) * time.Millisecond): - s.scheduled <- fn + select { + case s.scheduled <- fn: + case <-s.done: + return + } case <-s.done: return @@ -402,7 +406,11 @@ func (s *Socket) SetInterval(fn goja.Callable, intervalMs int) { for { select { case <-ticker.C: - s.scheduled <- fn + select { + case s.scheduled <- fn: + case <-s.done: + return + } case <-s.done: return @@ -450,24 +458,35 @@ func (s *Socket) closeConnection(code int) error { } // Wraps conn.ReadMessage in a channel -func readPump(conn *websocket.Conn, readChan chan []byte, errorChan chan error, closeChan chan int) { +func (s *Socket) readPump(readChan chan []byte, errorChan chan error, closeChan chan int) { for { - _, message, err := conn.ReadMessage() + _, message, err := s.conn.ReadMessage() if err != nil { if websocket.IsUnexpectedCloseError( err, websocket.CloseNormalClosure, websocket.CloseGoingAway) { // Report an unexpected closure - errorChan <- err + select { + case errorChan <- err: + case <-s.done: + return + } } code := websocket.CloseGoingAway if e, ok := err.(*websocket.CloseError); ok { code = e.Code } - closeChan <- code + select { + case closeChan <- code: + case <-s.done: + } return } - readChan <- message + select { + case readChan <- message: + case <-s.done: + return + } } } diff --git a/js/modules/k6/ws/ws_test.go b/js/modules/k6/ws/ws_test.go index 1acf46a90bb..174d270dc7f 100644 --- a/js/modules/k6/ws/ws_test.go +++ b/js/modules/k6/ws/ws_test.go @@ -581,7 +581,8 @@ func TestReadPump(t *testing.T) { msgChan := make(chan []byte) errChan := make(chan error) closeChan := make(chan int) - go readPump(conn, msgChan, errChan, closeChan) + s := &Socket{conn: conn} + go s.readPump(msgChan, errChan, closeChan) readChans: for { From a6bd7513d4b7feadfd9d1a040002b1d01be5cc4d Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 24 Jun 2020 13:50:41 +0300 Subject: [PATCH 310/350] update github.com/gorilla/websocket to 1.4.2 --- Gopkg.lock | 6 +- vendor/github.com/gorilla/websocket/AUTHORS | 1 + vendor/github.com/gorilla/websocket/client.go | 245 ++++----- vendor/github.com/gorilla/websocket/conn.go | 284 ++++++----- .../gorilla/websocket/conn_read_legacy.go | 21 - .../websocket/{conn_read.go => conn_write.go} | 15 +- .../gorilla/websocket/conn_write_legacy.go | 18 + vendor/github.com/gorilla/websocket/doc.go | 105 ++-- vendor/github.com/gorilla/websocket/join.go | 42 ++ vendor/github.com/gorilla/websocket/json.go | 11 +- vendor/github.com/gorilla/websocket/mask.go | 1 - .../github.com/gorilla/websocket/prepared.go | 5 +- vendor/github.com/gorilla/websocket/proxy.go | 77 +++ vendor/github.com/gorilla/websocket/server.go | 138 +++-- vendor/github.com/gorilla/websocket/trace.go | 19 + .../github.com/gorilla/websocket/trace_17.go | 12 + vendor/github.com/gorilla/websocket/util.go | 165 ++++-- .../gorilla/websocket/x_net_proxy.go | 473 ++++++++++++++++++ 18 files changed, 1251 insertions(+), 387 deletions(-) delete mode 100644 vendor/github.com/gorilla/websocket/conn_read_legacy.go rename vendor/github.com/gorilla/websocket/{conn_read.go => conn_write.go} (52%) create mode 100644 vendor/github.com/gorilla/websocket/conn_write_legacy.go create mode 100644 vendor/github.com/gorilla/websocket/join.go create mode 100644 vendor/github.com/gorilla/websocket/proxy.go create mode 100644 vendor/github.com/gorilla/websocket/trace.go create mode 100644 vendor/github.com/gorilla/websocket/trace_17.go create mode 100644 vendor/github.com/gorilla/websocket/x_net_proxy.go diff --git a/Gopkg.lock b/Gopkg.lock index 2243c01a8ac..783c26ad9b3 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -240,12 +240,12 @@ version = "v1.6.1" [[projects]] - digest = "1:3b708ebf63bfa9ba3313bedb8526bc0bb284e51474e65e958481476a9d4a12aa" + digest = "1:c9c619da0235710875bae079870a396912a475402f9ccd50667f6ad309476259" name = "github.com/gorilla/websocket" packages = ["."] pruneopts = "NUT" - revision = "ea4d1f681babbce9545c9c5f3d5194a789c89f5b" - version = "v1.2.0" + revision = "b65e62901fc1c0d968042419e74789f6af455eb9" + version = "v1.4.2" [[projects]] digest = "1:406338ad39ab2e37b7f4452906442a3dbf0eb3379dd1f06aafb5c07e769a5fbb" diff --git a/vendor/github.com/gorilla/websocket/AUTHORS b/vendor/github.com/gorilla/websocket/AUTHORS index b003eca0ca1..1931f400682 100644 --- a/vendor/github.com/gorilla/websocket/AUTHORS +++ b/vendor/github.com/gorilla/websocket/AUTHORS @@ -4,5 +4,6 @@ # Please keep the list sorted. Gary Burd +Google LLC (https://opensource.google.com/) Joachim Bauch diff --git a/vendor/github.com/gorilla/websocket/client.go b/vendor/github.com/gorilla/websocket/client.go index 43a87c753bf..962c06a391c 100644 --- a/vendor/github.com/gorilla/websocket/client.go +++ b/vendor/github.com/gorilla/websocket/client.go @@ -5,15 +5,15 @@ package websocket import ( - "bufio" "bytes" + "context" "crypto/tls" - "encoding/base64" "errors" "io" "io/ioutil" "net" "net/http" + "net/http/httptrace" "net/url" "strings" "time" @@ -53,6 +53,10 @@ type Dialer struct { // NetDial is nil, net.Dial is used. NetDial func(network, addr string) (net.Conn, error) + // NetDialContext specifies the dial function for creating TCP connections. If + // NetDialContext is nil, net.DialContext is used. + NetDialContext func(ctx context.Context, network, addr string) (net.Conn, error) + // Proxy specifies a function to return a proxy for a given // Request. If the function returns a non-nil error, the // request is aborted with the provided error. @@ -66,11 +70,22 @@ type Dialer struct { // HandshakeTimeout specifies the duration for the handshake to complete. HandshakeTimeout time.Duration - // ReadBufferSize and WriteBufferSize specify I/O buffer sizes. If a buffer + // ReadBufferSize and WriteBufferSize specify I/O buffer sizes in bytes. If a buffer // size is zero, then a useful default size is used. The I/O buffer sizes // do not limit the size of the messages that can be sent or received. ReadBufferSize, WriteBufferSize int + // WriteBufferPool is a pool of buffers for write operations. If the value + // is not set, then write buffers are allocated to the connection for the + // lifetime of the connection. + // + // A pool is most useful when the application has a modest volume of writes + // across a large number of connections. + // + // Applications should use a single pool for each unique value of + // WriteBufferSize. + WriteBufferPool BufferPool + // Subprotocols specifies the client's requested subprotocols. Subprotocols []string @@ -86,52 +101,13 @@ type Dialer struct { Jar http.CookieJar } -var errMalformedURL = errors.New("malformed ws or wss URL") - -// parseURL parses the URL. -// -// This function is a replacement for the standard library url.Parse function. -// In Go 1.4 and earlier, url.Parse loses information from the path. -func parseURL(s string) (*url.URL, error) { - // From the RFC: - // - // ws-URI = "ws:" "//" host [ ":" port ] path [ "?" query ] - // wss-URI = "wss:" "//" host [ ":" port ] path [ "?" query ] - var u url.URL - switch { - case strings.HasPrefix(s, "ws://"): - u.Scheme = "ws" - s = s[len("ws://"):] - case strings.HasPrefix(s, "wss://"): - u.Scheme = "wss" - s = s[len("wss://"):] - default: - return nil, errMalformedURL - } - - if i := strings.Index(s, "?"); i >= 0 { - u.RawQuery = s[i+1:] - s = s[:i] - } - - if i := strings.Index(s, "/"); i >= 0 { - u.Opaque = s[i:] - s = s[:i] - } else { - u.Opaque = "/" - } - - u.Host = s - - if strings.Contains(u.Host, "@") { - // Don't bother parsing user information because user information is - // not allowed in websocket URIs. - return nil, errMalformedURL - } - - return &u, nil +// Dial creates a new client connection by calling DialContext with a background context. +func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Response, error) { + return d.DialContext(context.Background(), urlStr, requestHeader) } +var errMalformedURL = errors.New("malformed ws or wss URL") + func hostPortNoPort(u *url.URL) (hostPort, hostNoPort string) { hostPort = u.Host hostNoPort = u.Host @@ -150,26 +126,29 @@ func hostPortNoPort(u *url.URL) (hostPort, hostNoPort string) { return hostPort, hostNoPort } -// DefaultDialer is a dialer with all fields set to the default zero values. +// DefaultDialer is a dialer with all fields set to the default values. var DefaultDialer = &Dialer{ - Proxy: http.ProxyFromEnvironment, + Proxy: http.ProxyFromEnvironment, + HandshakeTimeout: 45 * time.Second, } -// Dial creates a new client connection. Use requestHeader to specify the +// nilDialer is dialer to use when receiver is nil. +var nilDialer = *DefaultDialer + +// DialContext creates a new client connection. Use requestHeader to specify the // origin (Origin), subprotocols (Sec-WebSocket-Protocol) and cookies (Cookie). // Use the response.Header to get the selected subprotocol // (Sec-WebSocket-Protocol) and cookies (Set-Cookie). // +// The context will be used in the request and in the Dialer. +// // If the WebSocket handshake fails, ErrBadHandshake is returned along with a // non-nil *http.Response so that callers can handle redirects, authentication, // etcetera. The response body may not contain the entire response and does not // need to be closed by the application. -func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Response, error) { - +func (d *Dialer) DialContext(ctx context.Context, urlStr string, requestHeader http.Header) (*Conn, *http.Response, error) { if d == nil { - d = &Dialer{ - Proxy: http.ProxyFromEnvironment, - } + d = &nilDialer } challengeKey, err := generateChallengeKey() @@ -177,7 +156,7 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re return nil, nil, err } - u, err := parseURL(urlStr) + u, err := url.Parse(urlStr) if err != nil { return nil, nil, err } @@ -205,6 +184,7 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re Header: make(http.Header), Host: u.Host, } + req = req.WithContext(ctx) // Set the cookies present in the cookie jar of the dialer if d.Jar != nil { @@ -237,45 +217,83 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re k == "Sec-Websocket-Extensions" || (k == "Sec-Websocket-Protocol" && len(d.Subprotocols) > 0): return nil, nil, errors.New("websocket: duplicate header not allowed: " + k) + case k == "Sec-Websocket-Protocol": + req.Header["Sec-WebSocket-Protocol"] = vs default: req.Header[k] = vs } } if d.EnableCompression { - req.Header.Set("Sec-Websocket-Extensions", "permessage-deflate; server_no_context_takeover; client_no_context_takeover") + req.Header["Sec-WebSocket-Extensions"] = []string{"permessage-deflate; server_no_context_takeover; client_no_context_takeover"} } - hostPort, hostNoPort := hostPortNoPort(u) - - var proxyURL *url.URL - // Check wether the proxy method has been configured - if d.Proxy != nil { - proxyURL, err = d.Proxy(req) - } - if err != nil { - return nil, nil, err + if d.HandshakeTimeout != 0 { + var cancel func() + ctx, cancel = context.WithTimeout(ctx, d.HandshakeTimeout) + defer cancel() } - var targetHostPort string - if proxyURL != nil { - targetHostPort, _ = hostPortNoPort(proxyURL) + // Get network dial function. + var netDial func(network, add string) (net.Conn, error) + + if d.NetDialContext != nil { + netDial = func(network, addr string) (net.Conn, error) { + return d.NetDialContext(ctx, network, addr) + } + } else if d.NetDial != nil { + netDial = d.NetDial } else { - targetHostPort = hostPort + netDialer := &net.Dialer{} + netDial = func(network, addr string) (net.Conn, error) { + return netDialer.DialContext(ctx, network, addr) + } } - var deadline time.Time - if d.HandshakeTimeout != 0 { - deadline = time.Now().Add(d.HandshakeTimeout) + // If needed, wrap the dial function to set the connection deadline. + if deadline, ok := ctx.Deadline(); ok { + forwardDial := netDial + netDial = func(network, addr string) (net.Conn, error) { + c, err := forwardDial(network, addr) + if err != nil { + return nil, err + } + err = c.SetDeadline(deadline) + if err != nil { + c.Close() + return nil, err + } + return c, nil + } } - netDial := d.NetDial - if netDial == nil { - netDialer := &net.Dialer{Deadline: deadline} - netDial = netDialer.Dial + // If needed, wrap the dial function to connect through a proxy. + if d.Proxy != nil { + proxyURL, err := d.Proxy(req) + if err != nil { + return nil, nil, err + } + if proxyURL != nil { + dialer, err := proxy_FromURL(proxyURL, netDialerFunc(netDial)) + if err != nil { + return nil, nil, err + } + netDial = dialer.Dial + } + } + + hostPort, hostNoPort := hostPortNoPort(u) + trace := httptrace.ContextClientTrace(ctx) + if trace != nil && trace.GetConn != nil { + trace.GetConn(hostPort) } - netConn, err := netDial("tcp", targetHostPort) + netConn, err := netDial("tcp", hostPort) + if trace != nil && trace.GotConn != nil { + trace.GotConn(httptrace.GotConnInfo{ + Conn: netConn, + }) + } if err != nil { return nil, nil, err } @@ -286,42 +304,6 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re } }() - if err := netConn.SetDeadline(deadline); err != nil { - return nil, nil, err - } - - if proxyURL != nil { - connectHeader := make(http.Header) - if user := proxyURL.User; user != nil { - proxyUser := user.Username() - if proxyPassword, passwordSet := user.Password(); passwordSet { - credential := base64.StdEncoding.EncodeToString([]byte(proxyUser + ":" + proxyPassword)) - connectHeader.Set("Proxy-Authorization", "Basic "+credential) - } - } - connectReq := &http.Request{ - Method: "CONNECT", - URL: &url.URL{Opaque: hostPort}, - Host: hostPort, - Header: connectHeader, - } - - connectReq.Write(netConn) - - // Read response. - // Okay to use and discard buffered reader here, because - // TLS server will not speak until spoken to. - br := bufio.NewReader(netConn) - resp, err := http.ReadResponse(br, connectReq) - if err != nil { - return nil, nil, err - } - if resp.StatusCode != 200 { - f := strings.SplitN(resp.Status, " ", 2) - return nil, nil, errors.New(f[1]) - } - } - if u.Scheme == "https" { cfg := cloneTLSConfig(d.TLSClientConfig) if cfg.ServerName == "" { @@ -329,22 +311,31 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re } tlsConn := tls.Client(netConn, cfg) netConn = tlsConn - if err := tlsConn.Handshake(); err != nil { - return nil, nil, err + + var err error + if trace != nil { + err = doHandshakeWithTrace(trace, tlsConn, cfg) + } else { + err = doHandshake(tlsConn, cfg) } - if !cfg.InsecureSkipVerify { - if err := tlsConn.VerifyHostname(cfg.ServerName); err != nil { - return nil, nil, err - } + + if err != nil { + return nil, nil, err } } - conn := newConn(netConn, false, d.ReadBufferSize, d.WriteBufferSize) + conn := newConn(netConn, false, d.ReadBufferSize, d.WriteBufferSize, d.WriteBufferPool, nil, nil) if err := req.Write(netConn); err != nil { return nil, nil, err } + if trace != nil && trace.GotFirstResponseByte != nil { + if peek, err := conn.br.Peek(1); err == nil && len(peek) == 1 { + trace.GotFirstResponseByte() + } + } + resp, err := http.ReadResponse(conn.br, req) if err != nil { return nil, nil, err @@ -390,3 +381,15 @@ func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Re netConn = nil // to avoid close in defer. return conn, resp, nil } + +func doHandshake(tlsConn *tls.Conn, cfg *tls.Config) error { + if err := tlsConn.Handshake(); err != nil { + return err + } + if !cfg.InsecureSkipVerify { + if err := tlsConn.VerifyHostname(cfg.ServerName); err != nil { + return err + } + } + return nil +} diff --git a/vendor/github.com/gorilla/websocket/conn.go b/vendor/github.com/gorilla/websocket/conn.go index 97e1dbacb12..ca46d2f793c 100644 --- a/vendor/github.com/gorilla/websocket/conn.go +++ b/vendor/github.com/gorilla/websocket/conn.go @@ -76,7 +76,7 @@ const ( // is UTF-8 encoded text. PingMessage = 9 - // PongMessage denotes a ping control message. The optional message payload + // PongMessage denotes a pong control message. The optional message payload // is UTF-8 encoded text. PongMessage = 10 ) @@ -100,9 +100,8 @@ func (e *netError) Error() string { return e.msg } func (e *netError) Temporary() bool { return e.temporary } func (e *netError) Timeout() bool { return e.timeout } -// CloseError represents close frame. +// CloseError represents a close message. type CloseError struct { - // Code is defined in RFC 6455, section 11.7. Code int @@ -224,6 +223,20 @@ func isValidReceivedCloseCode(code int) bool { return validReceivedCloseCodes[code] || (code >= 3000 && code <= 4999) } +// BufferPool represents a pool of buffers. The *sync.Pool type satisfies this +// interface. The type of the value stored in a pool is not specified. +type BufferPool interface { + // Get gets a value from the pool or returns nil if the pool is empty. + Get() interface{} + // Put adds a value to the pool. + Put(interface{}) +} + +// writePoolData is the type added to the write buffer pool. This wrapper is +// used to prevent applications from peeking at and depending on the values +// added to the pool. +type writePoolData struct{ buf []byte } + // The Conn type represents a WebSocket connection. type Conn struct { conn net.Conn @@ -231,8 +244,10 @@ type Conn struct { subprotocol string // Write fields - mu chan bool // used as mutex to protect write to conn - writeBuf []byte // frame is constructed in this buffer. + mu chan struct{} // used as mutex to protect write to conn + writeBuf []byte // frame is constructed in this buffer. + writePool BufferPool + writeBufSize int writeDeadline time.Time writer io.WriteCloser // the current writer returned to the application isWriting bool // for best-effort concurrent write detection @@ -245,10 +260,12 @@ type Conn struct { newCompressionWriter func(io.WriteCloser, int) io.WriteCloser // Read fields - reader io.ReadCloser // the current reader returned to the application - readErr error - br *bufio.Reader - readRemaining int64 // bytes remaining in current frame. + reader io.ReadCloser // the current reader returned to the application + readErr error + br *bufio.Reader + // bytes remaining in current frame. + // set setReadRemaining to safely update this value and prevent overflow + readRemaining int64 readFinal bool // true the current message has more frames. readLength int64 // Message size. readLimit int64 // Maximum message size. @@ -264,64 +281,29 @@ type Conn struct { newDecompressionReader func(io.Reader) io.ReadCloser } -func newConn(conn net.Conn, isServer bool, readBufferSize, writeBufferSize int) *Conn { - return newConnBRW(conn, isServer, readBufferSize, writeBufferSize, nil) -} - -type writeHook struct { - p []byte -} - -func (wh *writeHook) Write(p []byte) (int, error) { - wh.p = p - return len(p), nil -} +func newConn(conn net.Conn, isServer bool, readBufferSize, writeBufferSize int, writeBufferPool BufferPool, br *bufio.Reader, writeBuf []byte) *Conn { -func newConnBRW(conn net.Conn, isServer bool, readBufferSize, writeBufferSize int, brw *bufio.ReadWriter) *Conn { - mu := make(chan bool, 1) - mu <- true - - var br *bufio.Reader - if readBufferSize == 0 && brw != nil && brw.Reader != nil { - // Reuse the supplied bufio.Reader if the buffer has a useful size. - // This code assumes that peek on a reader returns - // bufio.Reader.buf[:0]. - brw.Reader.Reset(conn) - if p, err := brw.Reader.Peek(0); err == nil && cap(p) >= 256 { - br = brw.Reader - } - } if br == nil { if readBufferSize == 0 { readBufferSize = defaultReadBufferSize - } - if readBufferSize < maxControlFramePayloadSize { + } else if readBufferSize < maxControlFramePayloadSize { + // must be large enough for control frame readBufferSize = maxControlFramePayloadSize } br = bufio.NewReaderSize(conn, readBufferSize) } - var writeBuf []byte - if writeBufferSize == 0 && brw != nil && brw.Writer != nil { - // Use the bufio.Writer's buffer if the buffer has a useful size. This - // code assumes that bufio.Writer.buf[:1] is passed to the - // bufio.Writer's underlying writer. - var wh writeHook - brw.Writer.Reset(&wh) - brw.Writer.WriteByte(0) - brw.Flush() - if cap(wh.p) >= maxFrameHeaderSize+256 { - writeBuf = wh.p[:cap(wh.p)] - } + if writeBufferSize <= 0 { + writeBufferSize = defaultWriteBufferSize } + writeBufferSize += maxFrameHeaderSize - if writeBuf == nil { - if writeBufferSize == 0 { - writeBufferSize = defaultWriteBufferSize - } - writeBuf = make([]byte, writeBufferSize+maxFrameHeaderSize) + if writeBuf == nil && writeBufferPool == nil { + writeBuf = make([]byte, writeBufferSize) } + mu := make(chan struct{}, 1) + mu <- struct{}{} c := &Conn{ isServer: isServer, br: br, @@ -329,6 +311,8 @@ func newConnBRW(conn net.Conn, isServer bool, readBufferSize, writeBufferSize in mu: mu, readFinal: true, writeBuf: writeBuf, + writePool: writeBufferPool, + writeBufSize: writeBufferSize, enableWriteCompression: true, compressionLevel: defaultCompressionLevel, } @@ -338,12 +322,24 @@ func newConnBRW(conn net.Conn, isServer bool, readBufferSize, writeBufferSize in return c } +// setReadRemaining tracks the number of bytes remaining on the connection. If n +// overflows, an ErrReadLimit is returned. +func (c *Conn) setReadRemaining(n int64) error { + if n < 0 { + return ErrReadLimit + } + + c.readRemaining = n + return nil +} + // Subprotocol returns the negotiated protocol for the connection. func (c *Conn) Subprotocol() string { return c.subprotocol } -// Close closes the underlying network connection without sending or waiting for a close frame. +// Close closes the underlying network connection without sending or waiting +// for a close message. func (c *Conn) Close() error { return c.conn.Close() } @@ -370,9 +366,18 @@ func (c *Conn) writeFatal(err error) error { return err } -func (c *Conn) write(frameType int, deadline time.Time, bufs ...[]byte) error { +func (c *Conn) read(n int) ([]byte, error) { + p, err := c.br.Peek(n) + if err == io.EOF { + err = errUnexpectedEOF + } + c.br.Discard(len(p)) + return p, err +} + +func (c *Conn) write(frameType int, deadline time.Time, buf0, buf1 []byte) error { <-c.mu - defer func() { c.mu <- true }() + defer func() { c.mu <- struct{}{} }() c.writeErrMu.Lock() err := c.writeErr @@ -382,15 +387,14 @@ func (c *Conn) write(frameType int, deadline time.Time, bufs ...[]byte) error { } c.conn.SetWriteDeadline(deadline) - for _, buf := range bufs { - if len(buf) > 0 { - _, err := c.conn.Write(buf) - if err != nil { - return c.writeFatal(err) - } - } + if len(buf1) == 0 { + _, err = c.conn.Write(buf0) + } else { + err = c.writeBufs(buf0, buf1) + } + if err != nil { + return c.writeFatal(err) } - if frameType == CloseMessage { c.writeFatal(ErrCloseSent) } @@ -425,7 +429,7 @@ func (c *Conn) WriteControl(messageType int, data []byte, deadline time.Time) er maskBytes(key, 0, buf[6:]) } - d := time.Hour * 1000 + d := 1000 * time.Hour if !deadline.IsZero() { d = deadline.Sub(time.Now()) if d < 0 { @@ -440,7 +444,7 @@ func (c *Conn) WriteControl(messageType int, data []byte, deadline time.Time) er case <-timer.C: return errWriteTimeout } - defer func() { c.mu <- true }() + defer func() { c.mu <- struct{}{} }() c.writeErrMu.Lock() err := c.writeErr @@ -460,7 +464,8 @@ func (c *Conn) WriteControl(messageType int, data []byte, deadline time.Time) er return err } -func (c *Conn) prepWrite(messageType int) error { +// beginMessage prepares a connection and message writer for a new message. +func (c *Conn) beginMessage(mw *messageWriter, messageType int) error { // Close previous writer if not already closed by the application. It's // probably better to return an error in this situation, but we cannot // change this without breaking existing applications. @@ -476,7 +481,23 @@ func (c *Conn) prepWrite(messageType int) error { c.writeErrMu.Lock() err := c.writeErr c.writeErrMu.Unlock() - return err + if err != nil { + return err + } + + mw.c = c + mw.frameType = messageType + mw.pos = maxFrameHeaderSize + + if c.writeBuf == nil { + wpd, ok := c.writePool.Get().(writePoolData) + if ok { + c.writeBuf = wpd.buf + } else { + c.writeBuf = make([]byte, c.writeBufSize) + } + } + return nil } // NextWriter returns a writer for the next message to send. The writer's Close @@ -484,17 +505,15 @@ func (c *Conn) prepWrite(messageType int) error { // // There can be at most one open writer on a connection. NextWriter closes the // previous writer if the application has not already done so. +// +// All message types (TextMessage, BinaryMessage, CloseMessage, PingMessage and +// PongMessage) are supported. func (c *Conn) NextWriter(messageType int) (io.WriteCloser, error) { - if err := c.prepWrite(messageType); err != nil { + var mw messageWriter + if err := c.beginMessage(&mw, messageType); err != nil { return nil, err } - - mw := &messageWriter{ - c: c, - frameType: messageType, - pos: maxFrameHeaderSize, - } - c.writer = mw + c.writer = &mw if c.newCompressionWriter != nil && c.enableWriteCompression && isData(messageType) { w := c.newCompressionWriter(c.writer, c.compressionLevel) mw.compress = true @@ -511,10 +530,16 @@ type messageWriter struct { err error } -func (w *messageWriter) fatal(err error) error { +func (w *messageWriter) endMessage(err error) error { if w.err != nil { - w.err = err - w.c.writer = nil + return err + } + c := w.c + w.err = err + c.writer = nil + if c.writePool != nil { + c.writePool.Put(writePoolData{buf: c.writeBuf}) + c.writeBuf = nil } return err } @@ -528,7 +553,7 @@ func (w *messageWriter) flushFrame(final bool, extra []byte) error { // Check for invalid control frames. if isControl(w.frameType) && (!final || length > maxControlFramePayloadSize) { - return w.fatal(errInvalidControlFrame) + return w.endMessage(errInvalidControlFrame) } b0 := byte(w.frameType) @@ -573,7 +598,7 @@ func (w *messageWriter) flushFrame(final bool, extra []byte) error { copy(c.writeBuf[maxFrameHeaderSize-4:], key[:]) maskBytes(key, 0, c.writeBuf[maxFrameHeaderSize:w.pos]) if len(extra) > 0 { - return c.writeFatal(errors.New("websocket: internal error, extra used in client mode")) + return w.endMessage(c.writeFatal(errors.New("websocket: internal error, extra used in client mode"))) } } @@ -594,11 +619,11 @@ func (w *messageWriter) flushFrame(final bool, extra []byte) error { c.isWriting = false if err != nil { - return w.fatal(err) + return w.endMessage(err) } if final { - c.writer = nil + w.endMessage(errWriteClosed) return nil } @@ -696,11 +721,7 @@ func (w *messageWriter) Close() error { if w.err != nil { return w.err } - if err := w.flushFrame(true, nil); err != nil { - return err - } - w.err = errWriteClosed - return nil + return w.flushFrame(true, nil) } // WritePreparedMessage writes prepared message into connection. @@ -732,10 +753,10 @@ func (c *Conn) WriteMessage(messageType int, data []byte) error { if c.isServer && (c.newCompressionWriter == nil || !c.enableWriteCompression) { // Fast path with no allocations and single frame. - if err := c.prepWrite(messageType); err != nil { + var mw messageWriter + if err := c.beginMessage(&mw, messageType); err != nil { return err } - mw := messageWriter{c: c, frameType: messageType, pos: maxFrameHeaderSize} n := copy(c.writeBuf[mw.pos:], data) mw.pos += n data = data[n:] @@ -764,7 +785,6 @@ func (c *Conn) SetWriteDeadline(t time.Time) error { // Read methods func (c *Conn) advanceFrame() (int, error) { - // 1. Skip remainder of previous frame. if c.readRemaining > 0 { @@ -783,7 +803,7 @@ func (c *Conn) advanceFrame() (int, error) { final := p[0]&finalBit != 0 frameType := int(p[0] & 0xf) mask := p[1]&maskBit != 0 - c.readRemaining = int64(p[1] & 0x7f) + c.setReadRemaining(int64(p[1] & 0x7f)) c.readDecompress = false if c.newDecompressionReader != nil && (p[0]&rsv1Bit) != 0 { @@ -817,7 +837,17 @@ func (c *Conn) advanceFrame() (int, error) { return noFrame, c.handleProtocolError("unknown opcode " + strconv.Itoa(frameType)) } - // 3. Read and parse frame length. + // 3. Read and parse frame length as per + // https://tools.ietf.org/html/rfc6455#section-5.2 + // + // The length of the "Payload data", in bytes: if 0-125, that is the payload + // length. + // - If 126, the following 2 bytes interpreted as a 16-bit unsigned + // integer are the payload length. + // - If 127, the following 8 bytes interpreted as + // a 64-bit unsigned integer (the most significant bit MUST be 0) are the + // payload length. Multibyte length quantities are expressed in network byte + // order. switch c.readRemaining { case 126: @@ -825,13 +855,19 @@ func (c *Conn) advanceFrame() (int, error) { if err != nil { return noFrame, err } - c.readRemaining = int64(binary.BigEndian.Uint16(p)) + + if err := c.setReadRemaining(int64(binary.BigEndian.Uint16(p))); err != nil { + return noFrame, err + } case 127: p, err := c.read(8) if err != nil { return noFrame, err } - c.readRemaining = int64(binary.BigEndian.Uint64(p)) + + if err := c.setReadRemaining(int64(binary.BigEndian.Uint64(p))); err != nil { + return noFrame, err + } } // 4. Handle frame masking. @@ -854,6 +890,12 @@ func (c *Conn) advanceFrame() (int, error) { if frameType == continuationFrame || frameType == TextMessage || frameType == BinaryMessage { c.readLength += c.readRemaining + // Don't allow readLength to overflow in the presence of a large readRemaining + // counter. + if c.readLength < 0 { + return noFrame, ErrReadLimit + } + if c.readLimit > 0 && c.readLength > c.readLimit { c.WriteControl(CloseMessage, FormatCloseMessage(CloseMessageTooBig, ""), time.Now().Add(writeWait)) return noFrame, ErrReadLimit @@ -867,7 +909,7 @@ func (c *Conn) advanceFrame() (int, error) { var payload []byte if c.readRemaining > 0 { payload, err = c.read(int(c.readRemaining)) - c.readRemaining = 0 + c.setReadRemaining(0) if err != nil { return noFrame, err } @@ -940,6 +982,7 @@ func (c *Conn) NextReader() (messageType int, r io.Reader, err error) { c.readErr = hideTempErr(err) break } + if frameType == TextMessage || frameType == BinaryMessage { c.messageReader = &messageReader{c} c.reader = c.messageReader @@ -980,7 +1023,9 @@ func (r *messageReader) Read(b []byte) (int, error) { if c.isServer { c.readMaskPos = maskBytes(c.readMaskKey, c.readMaskPos, b[:n]) } - c.readRemaining -= int64(n) + rem := c.readRemaining + rem -= int64(n) + c.setReadRemaining(rem) if c.readRemaining > 0 && c.readErr == io.EOF { c.readErr = errUnexpectedEOF } @@ -1032,8 +1077,8 @@ func (c *Conn) SetReadDeadline(t time.Time) error { return c.conn.SetReadDeadline(t) } -// SetReadLimit sets the maximum size for a message read from the peer. If a -// message exceeds the limit, the connection sends a close frame to the peer +// SetReadLimit sets the maximum size in bytes for a message read from the peer. If a +// message exceeds the limit, the connection sends a close message to the peer // and returns ErrReadLimit to the application. func (c *Conn) SetReadLimit(limit int64) { c.readLimit = limit @@ -1046,24 +1091,22 @@ func (c *Conn) CloseHandler() func(code int, text string) error { // SetCloseHandler sets the handler for close messages received from the peer. // The code argument to h is the received close code or CloseNoStatusReceived -// if the close message is empty. The default close handler sends a close frame -// back to the peer. +// if the close message is empty. The default close handler sends a close +// message back to the peer. // -// The application must read the connection to process close messages as -// described in the section on Control Frames above. +// The handler function is called from the NextReader, ReadMessage and message +// reader Read methods. The application must read the connection to process +// close messages as described in the section on Control Messages above. // -// The connection read methods return a CloseError when a close frame is +// The connection read methods return a CloseError when a close message is // received. Most applications should handle close messages as part of their // normal error handling. Applications should only set a close handler when the -// application must perform some action before sending a close frame back to +// application must perform some action before sending a close message back to // the peer. func (c *Conn) SetCloseHandler(h func(code int, text string) error) { if h == nil { h = func(code int, text string) error { - message := []byte{} - if code != CloseNoStatusReceived { - message = FormatCloseMessage(code, "") - } + message := FormatCloseMessage(code, "") c.WriteControl(CloseMessage, message, time.Now().Add(writeWait)) return nil } @@ -1077,11 +1120,12 @@ func (c *Conn) PingHandler() func(appData string) error { } // SetPingHandler sets the handler for ping messages received from the peer. -// The appData argument to h is the PING frame application data. The default +// The appData argument to h is the PING message application data. The default // ping handler sends a pong to the peer. // -// The application must read the connection to process ping messages as -// described in the section on Control Frames above. +// The handler function is called from the NextReader, ReadMessage and message +// reader Read methods. The application must read the connection to process +// ping messages as described in the section on Control Messages above. func (c *Conn) SetPingHandler(h func(appData string) error) { if h == nil { h = func(message string) error { @@ -1103,11 +1147,12 @@ func (c *Conn) PongHandler() func(appData string) error { } // SetPongHandler sets the handler for pong messages received from the peer. -// The appData argument to h is the PONG frame application data. The default +// The appData argument to h is the PONG message application data. The default // pong handler does nothing. // -// The application must read the connection to process ping messages as -// described in the section on Control Frames above. +// The handler function is called from the NextReader, ReadMessage and message +// reader Read methods. The application must read the connection to process +// pong messages as described in the section on Control Messages above. func (c *Conn) SetPongHandler(h func(appData string) error) { if h == nil { h = func(string) error { return nil } @@ -1141,7 +1186,14 @@ func (c *Conn) SetCompressionLevel(level int) error { } // FormatCloseMessage formats closeCode and text as a WebSocket close message. +// An empty message is returned for code CloseNoStatusReceived. func FormatCloseMessage(closeCode int, text string) []byte { + if closeCode == CloseNoStatusReceived { + // Return empty message because it's illegal to send + // CloseNoStatusReceived. Return non-nil value in case application + // checks for nil. + return []byte{} + } buf := make([]byte, 2+len(text)) binary.BigEndian.PutUint16(buf, uint16(closeCode)) copy(buf[2:], text) diff --git a/vendor/github.com/gorilla/websocket/conn_read_legacy.go b/vendor/github.com/gorilla/websocket/conn_read_legacy.go deleted file mode 100644 index 018541cf6cb..00000000000 --- a/vendor/github.com/gorilla/websocket/conn_read_legacy.go +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -// +build !go1.5 - -package websocket - -import "io" - -func (c *Conn) read(n int) ([]byte, error) { - p, err := c.br.Peek(n) - if err == io.EOF { - err = errUnexpectedEOF - } - if len(p) > 0 { - // advance over the bytes just read - io.ReadFull(c.br, p) - } - return p, err -} diff --git a/vendor/github.com/gorilla/websocket/conn_read.go b/vendor/github.com/gorilla/websocket/conn_write.go similarity index 52% rename from vendor/github.com/gorilla/websocket/conn_read.go rename to vendor/github.com/gorilla/websocket/conn_write.go index 1ea15059ee1..a509a21f87a 100644 --- a/vendor/github.com/gorilla/websocket/conn_read.go +++ b/vendor/github.com/gorilla/websocket/conn_write.go @@ -2,17 +2,14 @@ // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. -// +build go1.5 +// +build go1.8 package websocket -import "io" +import "net" -func (c *Conn) read(n int) ([]byte, error) { - p, err := c.br.Peek(n) - if err == io.EOF { - err = errUnexpectedEOF - } - c.br.Discard(len(p)) - return p, err +func (c *Conn) writeBufs(bufs ...[]byte) error { + b := net.Buffers(bufs) + _, err := b.WriteTo(c.conn) + return err } diff --git a/vendor/github.com/gorilla/websocket/conn_write_legacy.go b/vendor/github.com/gorilla/websocket/conn_write_legacy.go new file mode 100644 index 00000000000..37edaff5a57 --- /dev/null +++ b/vendor/github.com/gorilla/websocket/conn_write_legacy.go @@ -0,0 +1,18 @@ +// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !go1.8 + +package websocket + +func (c *Conn) writeBufs(bufs ...[]byte) error { + for _, buf := range bufs { + if len(buf) > 0 { + if _, err := c.conn.Write(buf); err != nil { + return err + } + } + } + return nil +} diff --git a/vendor/github.com/gorilla/websocket/doc.go b/vendor/github.com/gorilla/websocket/doc.go index e291a952c1e..8db0cef95a2 100644 --- a/vendor/github.com/gorilla/websocket/doc.go +++ b/vendor/github.com/gorilla/websocket/doc.go @@ -6,9 +6,8 @@ // // Overview // -// The Conn type represents a WebSocket connection. A server application uses -// the Upgrade function from an Upgrader object with a HTTP request handler -// to get a pointer to a Conn: +// The Conn type represents a WebSocket connection. A server application calls +// the Upgrader.Upgrade method from an HTTP request handler to get a *Conn: // // var upgrader = websocket.Upgrader{ // ReadBufferSize: 1024, @@ -31,10 +30,12 @@ // for { // messageType, p, err := conn.ReadMessage() // if err != nil { +// log.Println(err) // return // } -// if err = conn.WriteMessage(messageType, p); err != nil { -// return err +// if err := conn.WriteMessage(messageType, p); err != nil { +// log.Println(err) +// return // } // } // @@ -85,20 +86,26 @@ // and pong. Call the connection WriteControl, WriteMessage or NextWriter // methods to send a control message to the peer. // -// Connections handle received close messages by sending a close message to the -// peer and returning a *CloseError from the the NextReader, ReadMessage or the -// message Read method. +// Connections handle received close messages by calling the handler function +// set with the SetCloseHandler method and by returning a *CloseError from the +// NextReader, ReadMessage or the message Read method. The default close +// handler sends a close message to the peer. // -// Connections handle received ping and pong messages by invoking callback -// functions set with SetPingHandler and SetPongHandler methods. The callback -// functions are called from the NextReader, ReadMessage and the message Read -// methods. +// Connections handle received ping messages by calling the handler function +// set with the SetPingHandler method. The default ping handler sends a pong +// message to the peer. +// +// Connections handle received pong messages by calling the handler function +// set with the SetPongHandler method. The default pong handler does nothing. +// If an application sends ping messages, then the application should set a +// pong handler to receive the corresponding pong. // -// The default ping handler sends a pong to the peer. The application's reading -// goroutine can block for a short time while the handler writes the pong data -// to the connection. +// The control message handler functions are called from the NextReader, +// ReadMessage and message reader Read methods. The default close and ping +// handlers can block these methods for a short time when the handler writes to +// the connection. // -// The application must read the connection to process ping, pong and close +// The application must read the connection to process close, ping and pong // messages sent from the peer. If the application is not otherwise interested // in messages from the peer, then the application should start a goroutine to // read and discard messages from the peer. A simple example is: @@ -137,19 +144,59 @@ // method fails the WebSocket handshake with HTTP status 403. // // If the CheckOrigin field is nil, then the Upgrader uses a safe default: fail -// the handshake if the Origin request header is present and not equal to the -// Host request header. -// -// An application can allow connections from any origin by specifying a -// function that always returns true: -// -// var upgrader = websocket.Upgrader{ -// CheckOrigin: func(r *http.Request) bool { return true }, -// } -// -// The deprecated Upgrade function does not enforce an origin policy. It's the -// application's responsibility to check the Origin header before calling -// Upgrade. +// the handshake if the Origin request header is present and the Origin host is +// not equal to the Host request header. +// +// The deprecated package-level Upgrade function does not perform origin +// checking. The application is responsible for checking the Origin header +// before calling the Upgrade function. +// +// Buffers +// +// Connections buffer network input and output to reduce the number +// of system calls when reading or writing messages. +// +// Write buffers are also used for constructing WebSocket frames. See RFC 6455, +// Section 5 for a discussion of message framing. A WebSocket frame header is +// written to the network each time a write buffer is flushed to the network. +// Decreasing the size of the write buffer can increase the amount of framing +// overhead on the connection. +// +// The buffer sizes in bytes are specified by the ReadBufferSize and +// WriteBufferSize fields in the Dialer and Upgrader. The Dialer uses a default +// size of 4096 when a buffer size field is set to zero. The Upgrader reuses +// buffers created by the HTTP server when a buffer size field is set to zero. +// The HTTP server buffers have a size of 4096 at the time of this writing. +// +// The buffer sizes do not limit the size of a message that can be read or +// written by a connection. +// +// Buffers are held for the lifetime of the connection by default. If the +// Dialer or Upgrader WriteBufferPool field is set, then a connection holds the +// write buffer only when writing a message. +// +// Applications should tune the buffer sizes to balance memory use and +// performance. Increasing the buffer size uses more memory, but can reduce the +// number of system calls to read or write the network. In the case of writing, +// increasing the buffer size can reduce the number of frame headers written to +// the network. +// +// Some guidelines for setting buffer parameters are: +// +// Limit the buffer sizes to the maximum expected message size. Buffers larger +// than the largest message do not provide any benefit. +// +// Depending on the distribution of message sizes, setting the buffer size to +// a value less than the maximum expected message size can greatly reduce memory +// use with a small impact on performance. Here's an example: If 99% of the +// messages are smaller than 256 bytes and the maximum message size is 512 +// bytes, then a buffer size of 256 bytes will result in 1.01 more system calls +// than a buffer size of 512 bytes. The memory savings is 50%. +// +// A write buffer pool is useful when the application has a modest number +// writes over a large number of connections. when buffers are pooled, a larger +// buffer size has a reduced impact on total memory use and has the benefit of +// reducing system calls and frame overhead. // // Compression EXPERIMENTAL // diff --git a/vendor/github.com/gorilla/websocket/join.go b/vendor/github.com/gorilla/websocket/join.go new file mode 100644 index 00000000000..c64f8c82901 --- /dev/null +++ b/vendor/github.com/gorilla/websocket/join.go @@ -0,0 +1,42 @@ +// Copyright 2019 The Gorilla WebSocket Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package websocket + +import ( + "io" + "strings" +) + +// JoinMessages concatenates received messages to create a single io.Reader. +// The string term is appended to each message. The returned reader does not +// support concurrent calls to the Read method. +func JoinMessages(c *Conn, term string) io.Reader { + return &joinReader{c: c, term: term} +} + +type joinReader struct { + c *Conn + term string + r io.Reader +} + +func (r *joinReader) Read(p []byte) (int, error) { + if r.r == nil { + var err error + _, r.r, err = r.c.NextReader() + if err != nil { + return 0, err + } + if r.term != "" { + r.r = io.MultiReader(r.r, strings.NewReader(r.term)) + } + } + n, err := r.r.Read(p) + if err == io.EOF { + err = nil + r.r = nil + } + return n, err +} diff --git a/vendor/github.com/gorilla/websocket/json.go b/vendor/github.com/gorilla/websocket/json.go index 4f0e36875a5..dc2c1f6415f 100644 --- a/vendor/github.com/gorilla/websocket/json.go +++ b/vendor/github.com/gorilla/websocket/json.go @@ -9,12 +9,14 @@ import ( "io" ) -// WriteJSON is deprecated, use c.WriteJSON instead. +// WriteJSON writes the JSON encoding of v as a message. +// +// Deprecated: Use c.WriteJSON instead. func WriteJSON(c *Conn, v interface{}) error { return c.WriteJSON(v) } -// WriteJSON writes the JSON encoding of v to the connection. +// WriteJSON writes the JSON encoding of v as a message. // // See the documentation for encoding/json Marshal for details about the // conversion of Go values to JSON. @@ -31,7 +33,10 @@ func (c *Conn) WriteJSON(v interface{}) error { return err2 } -// ReadJSON is deprecated, use c.ReadJSON instead. +// ReadJSON reads the next JSON-encoded message from the connection and stores +// it in the value pointed to by v. +// +// Deprecated: Use c.ReadJSON instead. func ReadJSON(c *Conn, v interface{}) error { return c.ReadJSON(v) } diff --git a/vendor/github.com/gorilla/websocket/mask.go b/vendor/github.com/gorilla/websocket/mask.go index 6a88bbc7434..577fce9efd7 100644 --- a/vendor/github.com/gorilla/websocket/mask.go +++ b/vendor/github.com/gorilla/websocket/mask.go @@ -11,7 +11,6 @@ import "unsafe" const wordSize = int(unsafe.Sizeof(uintptr(0))) func maskBytes(key [4]byte, pos int, b []byte) int { - // Mask one byte at a time for small buffers. if len(b) < 2*wordSize { for i := range b { diff --git a/vendor/github.com/gorilla/websocket/prepared.go b/vendor/github.com/gorilla/websocket/prepared.go index 1efffbd1ebe..c854225e967 100644 --- a/vendor/github.com/gorilla/websocket/prepared.go +++ b/vendor/github.com/gorilla/websocket/prepared.go @@ -19,7 +19,6 @@ import ( type PreparedMessage struct { messageType int data []byte - err error mu sync.Mutex frames map[prepareKey]*preparedFrame } @@ -74,8 +73,8 @@ func (pm *PreparedMessage) frame(key prepareKey) (int, []byte, error) { // Prepare a frame using a 'fake' connection. // TODO: Refactor code in conn.go to allow more direct construction of // the frame. - mu := make(chan bool, 1) - mu <- true + mu := make(chan struct{}, 1) + mu <- struct{}{} var nc prepareConn c := &Conn{ conn: &nc, diff --git a/vendor/github.com/gorilla/websocket/proxy.go b/vendor/github.com/gorilla/websocket/proxy.go new file mode 100644 index 00000000000..e87a8c9f0c9 --- /dev/null +++ b/vendor/github.com/gorilla/websocket/proxy.go @@ -0,0 +1,77 @@ +// Copyright 2017 The Gorilla WebSocket Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package websocket + +import ( + "bufio" + "encoding/base64" + "errors" + "net" + "net/http" + "net/url" + "strings" +) + +type netDialerFunc func(network, addr string) (net.Conn, error) + +func (fn netDialerFunc) Dial(network, addr string) (net.Conn, error) { + return fn(network, addr) +} + +func init() { + proxy_RegisterDialerType("http", func(proxyURL *url.URL, forwardDialer proxy_Dialer) (proxy_Dialer, error) { + return &httpProxyDialer{proxyURL: proxyURL, forwardDial: forwardDialer.Dial}, nil + }) +} + +type httpProxyDialer struct { + proxyURL *url.URL + forwardDial func(network, addr string) (net.Conn, error) +} + +func (hpd *httpProxyDialer) Dial(network string, addr string) (net.Conn, error) { + hostPort, _ := hostPortNoPort(hpd.proxyURL) + conn, err := hpd.forwardDial(network, hostPort) + if err != nil { + return nil, err + } + + connectHeader := make(http.Header) + if user := hpd.proxyURL.User; user != nil { + proxyUser := user.Username() + if proxyPassword, passwordSet := user.Password(); passwordSet { + credential := base64.StdEncoding.EncodeToString([]byte(proxyUser + ":" + proxyPassword)) + connectHeader.Set("Proxy-Authorization", "Basic "+credential) + } + } + + connectReq := &http.Request{ + Method: "CONNECT", + URL: &url.URL{Opaque: addr}, + Host: addr, + Header: connectHeader, + } + + if err := connectReq.Write(conn); err != nil { + conn.Close() + return nil, err + } + + // Read response. It's OK to use and discard buffered reader here becaue + // the remote server does not speak until spoken to. + br := bufio.NewReader(conn) + resp, err := http.ReadResponse(br, connectReq) + if err != nil { + conn.Close() + return nil, err + } + + if resp.StatusCode != 200 { + conn.Close() + f := strings.SplitN(resp.Status, " ", 2) + return nil, errors.New(f[1]) + } + return conn, nil +} diff --git a/vendor/github.com/gorilla/websocket/server.go b/vendor/github.com/gorilla/websocket/server.go index 3495e0f1abc..887d558918c 100644 --- a/vendor/github.com/gorilla/websocket/server.go +++ b/vendor/github.com/gorilla/websocket/server.go @@ -7,7 +7,7 @@ package websocket import ( "bufio" "errors" - "net" + "io" "net/http" "net/url" "strings" @@ -27,16 +27,29 @@ type Upgrader struct { // HandshakeTimeout specifies the duration for the handshake to complete. HandshakeTimeout time.Duration - // ReadBufferSize and WriteBufferSize specify I/O buffer sizes. If a buffer + // ReadBufferSize and WriteBufferSize specify I/O buffer sizes in bytes. If a buffer // size is zero, then buffers allocated by the HTTP server are used. The // I/O buffer sizes do not limit the size of the messages that can be sent // or received. ReadBufferSize, WriteBufferSize int + // WriteBufferPool is a pool of buffers for write operations. If the value + // is not set, then write buffers are allocated to the connection for the + // lifetime of the connection. + // + // A pool is most useful when the application has a modest volume of writes + // across a large number of connections. + // + // Applications should use a single pool for each unique value of + // WriteBufferSize. + WriteBufferPool BufferPool + // Subprotocols specifies the server's supported protocols in order of - // preference. If this field is set, then the Upgrade method negotiates a + // preference. If this field is not nil, then the Upgrade method negotiates a // subprotocol by selecting the first match in this list with a protocol - // requested by the client. + // requested by the client. If there's no match, then no protocol is + // negotiated (the Sec-Websocket-Protocol header is not included in the + // handshake response). Subprotocols []string // Error specifies the function for generating HTTP error responses. If Error @@ -44,8 +57,12 @@ type Upgrader struct { Error func(w http.ResponseWriter, r *http.Request, status int, reason error) // CheckOrigin returns true if the request Origin header is acceptable. If - // CheckOrigin is nil, the host in the Origin header must not be set or - // must match the host of the request. + // CheckOrigin is nil, then a safe default is used: return false if the + // Origin request header is present and the origin host is not equal to + // request Host header. + // + // A CheckOrigin function should carefully validate the request origin to + // prevent cross-site request forgery. CheckOrigin func(r *http.Request) bool // EnableCompression specify if the server should attempt to negotiate per @@ -76,7 +93,7 @@ func checkSameOrigin(r *http.Request) bool { if err != nil { return false } - return u.Host == r.Host + return equalASCIIFold(u.Host, r.Host) } func (u *Upgrader) selectSubprotocol(r *http.Request, responseHeader http.Header) string { @@ -99,42 +116,44 @@ func (u *Upgrader) selectSubprotocol(r *http.Request, responseHeader http.Header // // The responseHeader is included in the response to the client's upgrade // request. Use the responseHeader to specify cookies (Set-Cookie) and the -// application negotiated subprotocol (Sec-Websocket-Protocol). +// application negotiated subprotocol (Sec-WebSocket-Protocol). // // If the upgrade fails, then Upgrade replies to the client with an HTTP error // response. func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeader http.Header) (*Conn, error) { - if r.Method != "GET" { - return u.returnError(w, r, http.StatusMethodNotAllowed, "websocket: not a websocket handshake: request method is not GET") - } - - if _, ok := responseHeader["Sec-Websocket-Extensions"]; ok { - return u.returnError(w, r, http.StatusInternalServerError, "websocket: application specific 'Sec-Websocket-Extensions' headers are unsupported") - } + const badHandshake = "websocket: the client is not using the websocket protocol: " if !tokenListContainsValue(r.Header, "Connection", "upgrade") { - return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: 'upgrade' token not found in 'Connection' header") + return u.returnError(w, r, http.StatusBadRequest, badHandshake+"'upgrade' token not found in 'Connection' header") } if !tokenListContainsValue(r.Header, "Upgrade", "websocket") { - return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: 'websocket' token not found in 'Upgrade' header") + return u.returnError(w, r, http.StatusBadRequest, badHandshake+"'websocket' token not found in 'Upgrade' header") + } + + if r.Method != "GET" { + return u.returnError(w, r, http.StatusMethodNotAllowed, badHandshake+"request method is not GET") } if !tokenListContainsValue(r.Header, "Sec-Websocket-Version", "13") { return u.returnError(w, r, http.StatusBadRequest, "websocket: unsupported version: 13 not found in 'Sec-Websocket-Version' header") } + if _, ok := responseHeader["Sec-Websocket-Extensions"]; ok { + return u.returnError(w, r, http.StatusInternalServerError, "websocket: application specific 'Sec-WebSocket-Extensions' headers are unsupported") + } + checkOrigin := u.CheckOrigin if checkOrigin == nil { checkOrigin = checkSameOrigin } if !checkOrigin(r) { - return u.returnError(w, r, http.StatusForbidden, "websocket: 'Origin' header value not allowed") + return u.returnError(w, r, http.StatusForbidden, "websocket: request origin not allowed by Upgrader.CheckOrigin") } challengeKey := r.Header.Get("Sec-Websocket-Key") if challengeKey == "" { - return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: `Sec-Websocket-Key' header is missing or blank") + return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: 'Sec-WebSocket-Key' header is missing or blank") } subprotocol := u.selectSubprotocol(r, responseHeader) @@ -151,17 +170,12 @@ func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeade } } - var ( - netConn net.Conn - err error - ) - h, ok := w.(http.Hijacker) if !ok { return u.returnError(w, r, http.StatusInternalServerError, "websocket: response does not implement http.Hijacker") } var brw *bufio.ReadWriter - netConn, brw, err = h.Hijack() + netConn, brw, err := h.Hijack() if err != nil { return u.returnError(w, r, http.StatusInternalServerError, err.Error()) } @@ -171,7 +185,21 @@ func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeade return nil, errors.New("websocket: client sent data before handshake is complete") } - c := newConnBRW(netConn, true, u.ReadBufferSize, u.WriteBufferSize, brw) + var br *bufio.Reader + if u.ReadBufferSize == 0 && bufioReaderSize(netConn, brw.Reader) > 256 { + // Reuse hijacked buffered reader as connection reader. + br = brw.Reader + } + + buf := bufioWriterBuffer(netConn, brw.Writer) + + var writeBuf []byte + if u.WriteBufferPool == nil && u.WriteBufferSize == 0 && len(buf) >= maxFrameHeaderSize+256 { + // Reuse hijacked write buffer as connection buffer. + writeBuf = buf + } + + c := newConn(netConn, true, u.ReadBufferSize, u.WriteBufferSize, u.WriteBufferPool, br, writeBuf) c.subprotocol = subprotocol if compress { @@ -179,17 +207,23 @@ func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeade c.newDecompressionReader = decompressNoContextTakeover } - p := c.writeBuf[:0] + // Use larger of hijacked buffer and connection write buffer for header. + p := buf + if len(c.writeBuf) > len(p) { + p = c.writeBuf + } + p = p[:0] + p = append(p, "HTTP/1.1 101 Switching Protocols\r\nUpgrade: websocket\r\nConnection: Upgrade\r\nSec-WebSocket-Accept: "...) p = append(p, computeAcceptKey(challengeKey)...) p = append(p, "\r\n"...) if c.subprotocol != "" { - p = append(p, "Sec-Websocket-Protocol: "...) + p = append(p, "Sec-WebSocket-Protocol: "...) p = append(p, c.subprotocol...) p = append(p, "\r\n"...) } if compress { - p = append(p, "Sec-Websocket-Extensions: permessage-deflate; server_no_context_takeover; client_no_context_takeover\r\n"...) + p = append(p, "Sec-WebSocket-Extensions: permessage-deflate; server_no_context_takeover; client_no_context_takeover\r\n"...) } for k, vs := range responseHeader { if k == "Sec-Websocket-Protocol" { @@ -230,13 +264,14 @@ func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeade // Upgrade upgrades the HTTP server connection to the WebSocket protocol. // -// This function is deprecated, use websocket.Upgrader instead. +// Deprecated: Use websocket.Upgrader instead. // -// The application is responsible for checking the request origin before -// calling Upgrade. An example implementation of the same origin policy is: +// Upgrade does not perform origin checking. The application is responsible for +// checking the Origin header before calling Upgrade. An example implementation +// of the same origin policy check is: // // if req.Header.Get("Origin") != "http://"+req.Host { -// http.Error(w, "Origin not allowed", 403) +// http.Error(w, "Origin not allowed", http.StatusForbidden) // return // } // @@ -289,3 +324,40 @@ func IsWebSocketUpgrade(r *http.Request) bool { return tokenListContainsValue(r.Header, "Connection", "upgrade") && tokenListContainsValue(r.Header, "Upgrade", "websocket") } + +// bufioReaderSize size returns the size of a bufio.Reader. +func bufioReaderSize(originalReader io.Reader, br *bufio.Reader) int { + // This code assumes that peek on a reset reader returns + // bufio.Reader.buf[:0]. + // TODO: Use bufio.Reader.Size() after Go 1.10 + br.Reset(originalReader) + if p, err := br.Peek(0); err == nil { + return cap(p) + } + return 0 +} + +// writeHook is an io.Writer that records the last slice passed to it vio +// io.Writer.Write. +type writeHook struct { + p []byte +} + +func (wh *writeHook) Write(p []byte) (int, error) { + wh.p = p + return len(p), nil +} + +// bufioWriterBuffer grabs the buffer from a bufio.Writer. +func bufioWriterBuffer(originalWriter io.Writer, bw *bufio.Writer) []byte { + // This code assumes that bufio.Writer.buf[:1] is passed to the + // bufio.Writer's underlying writer. + var wh writeHook + bw.Reset(&wh) + bw.WriteByte(0) + bw.Flush() + + bw.Reset(originalWriter) + + return wh.p[:cap(wh.p)] +} diff --git a/vendor/github.com/gorilla/websocket/trace.go b/vendor/github.com/gorilla/websocket/trace.go new file mode 100644 index 00000000000..834f122a00d --- /dev/null +++ b/vendor/github.com/gorilla/websocket/trace.go @@ -0,0 +1,19 @@ +// +build go1.8 + +package websocket + +import ( + "crypto/tls" + "net/http/httptrace" +) + +func doHandshakeWithTrace(trace *httptrace.ClientTrace, tlsConn *tls.Conn, cfg *tls.Config) error { + if trace.TLSHandshakeStart != nil { + trace.TLSHandshakeStart() + } + err := doHandshake(tlsConn, cfg) + if trace.TLSHandshakeDone != nil { + trace.TLSHandshakeDone(tlsConn.ConnectionState(), err) + } + return err +} diff --git a/vendor/github.com/gorilla/websocket/trace_17.go b/vendor/github.com/gorilla/websocket/trace_17.go new file mode 100644 index 00000000000..77d05a0b574 --- /dev/null +++ b/vendor/github.com/gorilla/websocket/trace_17.go @@ -0,0 +1,12 @@ +// +build !go1.8 + +package websocket + +import ( + "crypto/tls" + "net/http/httptrace" +) + +func doHandshakeWithTrace(trace *httptrace.ClientTrace, tlsConn *tls.Conn, cfg *tls.Config) error { + return doHandshake(tlsConn, cfg) +} diff --git a/vendor/github.com/gorilla/websocket/util.go b/vendor/github.com/gorilla/websocket/util.go index 9a4908df2ee..7bf2f66c674 100644 --- a/vendor/github.com/gorilla/websocket/util.go +++ b/vendor/github.com/gorilla/websocket/util.go @@ -11,6 +11,7 @@ import ( "io" "net/http" "strings" + "unicode/utf8" ) var keyGUID = []byte("258EAFA5-E914-47DA-95CA-C5AB0DC85B11") @@ -30,68 +31,113 @@ func generateChallengeKey() (string, error) { return base64.StdEncoding.EncodeToString(p), nil } -// Octet types from RFC 2616. -var octetTypes [256]byte - -const ( - isTokenOctet = 1 << iota - isSpaceOctet -) - -func init() { - // From RFC 2616 - // - // OCTET = - // CHAR = - // CTL = - // CR = - // LF = - // SP = - // HT = - // <"> = - // CRLF = CR LF - // LWS = [CRLF] 1*( SP | HT ) - // TEXT = - // separators = "(" | ")" | "<" | ">" | "@" | "," | ";" | ":" | "\" | <"> - // | "/" | "[" | "]" | "?" | "=" | "{" | "}" | SP | HT - // token = 1* - // qdtext = > - - for c := 0; c < 256; c++ { - var t byte - isCtl := c <= 31 || c == 127 - isChar := 0 <= c && c <= 127 - isSeparator := strings.IndexRune(" \t\"(),/:;<=>?@[]\\{}", rune(c)) >= 0 - if strings.IndexRune(" \t\r\n", rune(c)) >= 0 { - t |= isSpaceOctet - } - if isChar && !isCtl && !isSeparator { - t |= isTokenOctet - } - octetTypes[c] = t - } +// Token octets per RFC 2616. +var isTokenOctet = [256]bool{ + '!': true, + '#': true, + '$': true, + '%': true, + '&': true, + '\'': true, + '*': true, + '+': true, + '-': true, + '.': true, + '0': true, + '1': true, + '2': true, + '3': true, + '4': true, + '5': true, + '6': true, + '7': true, + '8': true, + '9': true, + 'A': true, + 'B': true, + 'C': true, + 'D': true, + 'E': true, + 'F': true, + 'G': true, + 'H': true, + 'I': true, + 'J': true, + 'K': true, + 'L': true, + 'M': true, + 'N': true, + 'O': true, + 'P': true, + 'Q': true, + 'R': true, + 'S': true, + 'T': true, + 'U': true, + 'W': true, + 'V': true, + 'X': true, + 'Y': true, + 'Z': true, + '^': true, + '_': true, + '`': true, + 'a': true, + 'b': true, + 'c': true, + 'd': true, + 'e': true, + 'f': true, + 'g': true, + 'h': true, + 'i': true, + 'j': true, + 'k': true, + 'l': true, + 'm': true, + 'n': true, + 'o': true, + 'p': true, + 'q': true, + 'r': true, + 's': true, + 't': true, + 'u': true, + 'v': true, + 'w': true, + 'x': true, + 'y': true, + 'z': true, + '|': true, + '~': true, } +// skipSpace returns a slice of the string s with all leading RFC 2616 linear +// whitespace removed. func skipSpace(s string) (rest string) { i := 0 for ; i < len(s); i++ { - if octetTypes[s[i]]&isSpaceOctet == 0 { + if b := s[i]; b != ' ' && b != '\t' { break } } return s[i:] } +// nextToken returns the leading RFC 2616 token of s and the string following +// the token. func nextToken(s string) (token, rest string) { i := 0 for ; i < len(s); i++ { - if octetTypes[s[i]]&isTokenOctet == 0 { + if !isTokenOctet[s[i]] { break } } return s[:i], s[i:] } +// nextTokenOrQuoted returns the leading token or quoted string per RFC 2616 +// and the string following the token or quoted string. func nextTokenOrQuoted(s string) (value string, rest string) { if !strings.HasPrefix(s, "\"") { return nextToken(s) @@ -111,14 +157,14 @@ func nextTokenOrQuoted(s string) (value string, rest string) { case escape: escape = false p[j] = b - j += 1 + j++ case b == '\\': escape = true case b == '"': return string(p[:j]), s[i+1:] default: p[j] = b - j += 1 + j++ } } return "", "" @@ -127,8 +173,32 @@ func nextTokenOrQuoted(s string) (value string, rest string) { return "", "" } +// equalASCIIFold returns true if s is equal to t with ASCII case folding as +// defined in RFC 4790. +func equalASCIIFold(s, t string) bool { + for s != "" && t != "" { + sr, size := utf8.DecodeRuneInString(s) + s = s[size:] + tr, size := utf8.DecodeRuneInString(t) + t = t[size:] + if sr == tr { + continue + } + if 'A' <= sr && sr <= 'Z' { + sr = sr + 'a' - 'A' + } + if 'A' <= tr && tr <= 'Z' { + tr = tr + 'a' - 'A' + } + if sr != tr { + return false + } + } + return s == t +} + // tokenListContainsValue returns true if the 1#token header with the given -// name contains token. +// name contains a token equal to value with ASCII case folding. func tokenListContainsValue(header http.Header, name string, value string) bool { headers: for _, s := range header[name] { @@ -142,7 +212,7 @@ headers: if s != "" && s[0] != ',' { continue headers } - if strings.EqualFold(t, value) { + if equalASCIIFold(t, value) { return true } if s == "" { @@ -154,9 +224,8 @@ headers: return false } -// parseExtensiosn parses WebSocket extensions from a header. +// parseExtensions parses WebSocket extensions from a header. func parseExtensions(header http.Header) []map[string]string { - // From RFC 6455: // // Sec-WebSocket-Extensions = extension-list diff --git a/vendor/github.com/gorilla/websocket/x_net_proxy.go b/vendor/github.com/gorilla/websocket/x_net_proxy.go new file mode 100644 index 00000000000..2e668f6b882 --- /dev/null +++ b/vendor/github.com/gorilla/websocket/x_net_proxy.go @@ -0,0 +1,473 @@ +// Code generated by golang.org/x/tools/cmd/bundle. DO NOT EDIT. +//go:generate bundle -o x_net_proxy.go golang.org/x/net/proxy + +// Package proxy provides support for a variety of protocols to proxy network +// data. +// + +package websocket + +import ( + "errors" + "io" + "net" + "net/url" + "os" + "strconv" + "strings" + "sync" +) + +type proxy_direct struct{} + +// Direct is a direct proxy: one that makes network connections directly. +var proxy_Direct = proxy_direct{} + +func (proxy_direct) Dial(network, addr string) (net.Conn, error) { + return net.Dial(network, addr) +} + +// A PerHost directs connections to a default Dialer unless the host name +// requested matches one of a number of exceptions. +type proxy_PerHost struct { + def, bypass proxy_Dialer + + bypassNetworks []*net.IPNet + bypassIPs []net.IP + bypassZones []string + bypassHosts []string +} + +// NewPerHost returns a PerHost Dialer that directs connections to either +// defaultDialer or bypass, depending on whether the connection matches one of +// the configured rules. +func proxy_NewPerHost(defaultDialer, bypass proxy_Dialer) *proxy_PerHost { + return &proxy_PerHost{ + def: defaultDialer, + bypass: bypass, + } +} + +// Dial connects to the address addr on the given network through either +// defaultDialer or bypass. +func (p *proxy_PerHost) Dial(network, addr string) (c net.Conn, err error) { + host, _, err := net.SplitHostPort(addr) + if err != nil { + return nil, err + } + + return p.dialerForRequest(host).Dial(network, addr) +} + +func (p *proxy_PerHost) dialerForRequest(host string) proxy_Dialer { + if ip := net.ParseIP(host); ip != nil { + for _, net := range p.bypassNetworks { + if net.Contains(ip) { + return p.bypass + } + } + for _, bypassIP := range p.bypassIPs { + if bypassIP.Equal(ip) { + return p.bypass + } + } + return p.def + } + + for _, zone := range p.bypassZones { + if strings.HasSuffix(host, zone) { + return p.bypass + } + if host == zone[1:] { + // For a zone ".example.com", we match "example.com" + // too. + return p.bypass + } + } + for _, bypassHost := range p.bypassHosts { + if bypassHost == host { + return p.bypass + } + } + return p.def +} + +// AddFromString parses a string that contains comma-separated values +// specifying hosts that should use the bypass proxy. Each value is either an +// IP address, a CIDR range, a zone (*.example.com) or a host name +// (localhost). A best effort is made to parse the string and errors are +// ignored. +func (p *proxy_PerHost) AddFromString(s string) { + hosts := strings.Split(s, ",") + for _, host := range hosts { + host = strings.TrimSpace(host) + if len(host) == 0 { + continue + } + if strings.Contains(host, "/") { + // We assume that it's a CIDR address like 127.0.0.0/8 + if _, net, err := net.ParseCIDR(host); err == nil { + p.AddNetwork(net) + } + continue + } + if ip := net.ParseIP(host); ip != nil { + p.AddIP(ip) + continue + } + if strings.HasPrefix(host, "*.") { + p.AddZone(host[1:]) + continue + } + p.AddHost(host) + } +} + +// AddIP specifies an IP address that will use the bypass proxy. Note that +// this will only take effect if a literal IP address is dialed. A connection +// to a named host will never match an IP. +func (p *proxy_PerHost) AddIP(ip net.IP) { + p.bypassIPs = append(p.bypassIPs, ip) +} + +// AddNetwork specifies an IP range that will use the bypass proxy. Note that +// this will only take effect if a literal IP address is dialed. A connection +// to a named host will never match. +func (p *proxy_PerHost) AddNetwork(net *net.IPNet) { + p.bypassNetworks = append(p.bypassNetworks, net) +} + +// AddZone specifies a DNS suffix that will use the bypass proxy. A zone of +// "example.com" matches "example.com" and all of its subdomains. +func (p *proxy_PerHost) AddZone(zone string) { + if strings.HasSuffix(zone, ".") { + zone = zone[:len(zone)-1] + } + if !strings.HasPrefix(zone, ".") { + zone = "." + zone + } + p.bypassZones = append(p.bypassZones, zone) +} + +// AddHost specifies a host name that will use the bypass proxy. +func (p *proxy_PerHost) AddHost(host string) { + if strings.HasSuffix(host, ".") { + host = host[:len(host)-1] + } + p.bypassHosts = append(p.bypassHosts, host) +} + +// A Dialer is a means to establish a connection. +type proxy_Dialer interface { + // Dial connects to the given address via the proxy. + Dial(network, addr string) (c net.Conn, err error) +} + +// Auth contains authentication parameters that specific Dialers may require. +type proxy_Auth struct { + User, Password string +} + +// FromEnvironment returns the dialer specified by the proxy related variables in +// the environment. +func proxy_FromEnvironment() proxy_Dialer { + allProxy := proxy_allProxyEnv.Get() + if len(allProxy) == 0 { + return proxy_Direct + } + + proxyURL, err := url.Parse(allProxy) + if err != nil { + return proxy_Direct + } + proxy, err := proxy_FromURL(proxyURL, proxy_Direct) + if err != nil { + return proxy_Direct + } + + noProxy := proxy_noProxyEnv.Get() + if len(noProxy) == 0 { + return proxy + } + + perHost := proxy_NewPerHost(proxy, proxy_Direct) + perHost.AddFromString(noProxy) + return perHost +} + +// proxySchemes is a map from URL schemes to a function that creates a Dialer +// from a URL with such a scheme. +var proxy_proxySchemes map[string]func(*url.URL, proxy_Dialer) (proxy_Dialer, error) + +// RegisterDialerType takes a URL scheme and a function to generate Dialers from +// a URL with that scheme and a forwarding Dialer. Registered schemes are used +// by FromURL. +func proxy_RegisterDialerType(scheme string, f func(*url.URL, proxy_Dialer) (proxy_Dialer, error)) { + if proxy_proxySchemes == nil { + proxy_proxySchemes = make(map[string]func(*url.URL, proxy_Dialer) (proxy_Dialer, error)) + } + proxy_proxySchemes[scheme] = f +} + +// FromURL returns a Dialer given a URL specification and an underlying +// Dialer for it to make network requests. +func proxy_FromURL(u *url.URL, forward proxy_Dialer) (proxy_Dialer, error) { + var auth *proxy_Auth + if u.User != nil { + auth = new(proxy_Auth) + auth.User = u.User.Username() + if p, ok := u.User.Password(); ok { + auth.Password = p + } + } + + switch u.Scheme { + case "socks5": + return proxy_SOCKS5("tcp", u.Host, auth, forward) + } + + // If the scheme doesn't match any of the built-in schemes, see if it + // was registered by another package. + if proxy_proxySchemes != nil { + if f, ok := proxy_proxySchemes[u.Scheme]; ok { + return f(u, forward) + } + } + + return nil, errors.New("proxy: unknown scheme: " + u.Scheme) +} + +var ( + proxy_allProxyEnv = &proxy_envOnce{ + names: []string{"ALL_PROXY", "all_proxy"}, + } + proxy_noProxyEnv = &proxy_envOnce{ + names: []string{"NO_PROXY", "no_proxy"}, + } +) + +// envOnce looks up an environment variable (optionally by multiple +// names) once. It mitigates expensive lookups on some platforms +// (e.g. Windows). +// (Borrowed from net/http/transport.go) +type proxy_envOnce struct { + names []string + once sync.Once + val string +} + +func (e *proxy_envOnce) Get() string { + e.once.Do(e.init) + return e.val +} + +func (e *proxy_envOnce) init() { + for _, n := range e.names { + e.val = os.Getenv(n) + if e.val != "" { + return + } + } +} + +// SOCKS5 returns a Dialer that makes SOCKSv5 connections to the given address +// with an optional username and password. See RFC 1928 and RFC 1929. +func proxy_SOCKS5(network, addr string, auth *proxy_Auth, forward proxy_Dialer) (proxy_Dialer, error) { + s := &proxy_socks5{ + network: network, + addr: addr, + forward: forward, + } + if auth != nil { + s.user = auth.User + s.password = auth.Password + } + + return s, nil +} + +type proxy_socks5 struct { + user, password string + network, addr string + forward proxy_Dialer +} + +const proxy_socks5Version = 5 + +const ( + proxy_socks5AuthNone = 0 + proxy_socks5AuthPassword = 2 +) + +const proxy_socks5Connect = 1 + +const ( + proxy_socks5IP4 = 1 + proxy_socks5Domain = 3 + proxy_socks5IP6 = 4 +) + +var proxy_socks5Errors = []string{ + "", + "general failure", + "connection forbidden", + "network unreachable", + "host unreachable", + "connection refused", + "TTL expired", + "command not supported", + "address type not supported", +} + +// Dial connects to the address addr on the given network via the SOCKS5 proxy. +func (s *proxy_socks5) Dial(network, addr string) (net.Conn, error) { + switch network { + case "tcp", "tcp6", "tcp4": + default: + return nil, errors.New("proxy: no support for SOCKS5 proxy connections of type " + network) + } + + conn, err := s.forward.Dial(s.network, s.addr) + if err != nil { + return nil, err + } + if err := s.connect(conn, addr); err != nil { + conn.Close() + return nil, err + } + return conn, nil +} + +// connect takes an existing connection to a socks5 proxy server, +// and commands the server to extend that connection to target, +// which must be a canonical address with a host and port. +func (s *proxy_socks5) connect(conn net.Conn, target string) error { + host, portStr, err := net.SplitHostPort(target) + if err != nil { + return err + } + + port, err := strconv.Atoi(portStr) + if err != nil { + return errors.New("proxy: failed to parse port number: " + portStr) + } + if port < 1 || port > 0xffff { + return errors.New("proxy: port number out of range: " + portStr) + } + + // the size here is just an estimate + buf := make([]byte, 0, 6+len(host)) + + buf = append(buf, proxy_socks5Version) + if len(s.user) > 0 && len(s.user) < 256 && len(s.password) < 256 { + buf = append(buf, 2 /* num auth methods */, proxy_socks5AuthNone, proxy_socks5AuthPassword) + } else { + buf = append(buf, 1 /* num auth methods */, proxy_socks5AuthNone) + } + + if _, err := conn.Write(buf); err != nil { + return errors.New("proxy: failed to write greeting to SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + if _, err := io.ReadFull(conn, buf[:2]); err != nil { + return errors.New("proxy: failed to read greeting from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + if buf[0] != 5 { + return errors.New("proxy: SOCKS5 proxy at " + s.addr + " has unexpected version " + strconv.Itoa(int(buf[0]))) + } + if buf[1] == 0xff { + return errors.New("proxy: SOCKS5 proxy at " + s.addr + " requires authentication") + } + + // See RFC 1929 + if buf[1] == proxy_socks5AuthPassword { + buf = buf[:0] + buf = append(buf, 1 /* password protocol version */) + buf = append(buf, uint8(len(s.user))) + buf = append(buf, s.user...) + buf = append(buf, uint8(len(s.password))) + buf = append(buf, s.password...) + + if _, err := conn.Write(buf); err != nil { + return errors.New("proxy: failed to write authentication request to SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + if _, err := io.ReadFull(conn, buf[:2]); err != nil { + return errors.New("proxy: failed to read authentication reply from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + if buf[1] != 0 { + return errors.New("proxy: SOCKS5 proxy at " + s.addr + " rejected username/password") + } + } + + buf = buf[:0] + buf = append(buf, proxy_socks5Version, proxy_socks5Connect, 0 /* reserved */) + + if ip := net.ParseIP(host); ip != nil { + if ip4 := ip.To4(); ip4 != nil { + buf = append(buf, proxy_socks5IP4) + ip = ip4 + } else { + buf = append(buf, proxy_socks5IP6) + } + buf = append(buf, ip...) + } else { + if len(host) > 255 { + return errors.New("proxy: destination host name too long: " + host) + } + buf = append(buf, proxy_socks5Domain) + buf = append(buf, byte(len(host))) + buf = append(buf, host...) + } + buf = append(buf, byte(port>>8), byte(port)) + + if _, err := conn.Write(buf); err != nil { + return errors.New("proxy: failed to write connect request to SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + if _, err := io.ReadFull(conn, buf[:4]); err != nil { + return errors.New("proxy: failed to read connect reply from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + failure := "unknown error" + if int(buf[1]) < len(proxy_socks5Errors) { + failure = proxy_socks5Errors[buf[1]] + } + + if len(failure) > 0 { + return errors.New("proxy: SOCKS5 proxy at " + s.addr + " failed to connect: " + failure) + } + + bytesToDiscard := 0 + switch buf[3] { + case proxy_socks5IP4: + bytesToDiscard = net.IPv4len + case proxy_socks5IP6: + bytesToDiscard = net.IPv6len + case proxy_socks5Domain: + _, err := io.ReadFull(conn, buf[:1]) + if err != nil { + return errors.New("proxy: failed to read domain length from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + bytesToDiscard = int(buf[0]) + default: + return errors.New("proxy: got unknown address type " + strconv.Itoa(int(buf[3])) + " from SOCKS5 proxy at " + s.addr) + } + + if cap(buf) < bytesToDiscard { + buf = make([]byte, bytesToDiscard) + } else { + buf = buf[:bytesToDiscard] + } + if _, err := io.ReadFull(conn, buf); err != nil { + return errors.New("proxy: failed to read address from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + // Also need to discard the port number + if _, err := io.ReadFull(conn, buf[:2]); err != nil { + return errors.New("proxy: failed to read port from SOCKS5 proxy at " + s.addr + ": " + err.Error()) + } + + return nil +} From fc34f91e2160e8c5dd7308fc337b22340ad2dffd Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Wed, 24 Jun 2020 14:25:34 +0300 Subject: [PATCH 311/350] Fix ws_ping not being formatted as time --- lib/metrics/metrics.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lib/metrics/metrics.go b/lib/metrics/metrics.go index d550d762bd1..33e1f579eff 100644 --- a/lib/metrics/metrics.go +++ b/lib/metrics/metrics.go @@ -26,6 +26,7 @@ import ( //TODO: refactor this, using non thread-safe global variables seems like a bad idea for various reasons... +//nolint:gochecknoglobals var ( // Engine-emitted. VUs = stats.New("vus", stats.Gauge) @@ -52,7 +53,7 @@ var ( WSSessions = stats.New("ws_sessions", stats.Counter) WSMessagesSent = stats.New("ws_msgs_sent", stats.Counter) WSMessagesReceived = stats.New("ws_msgs_received", stats.Counter) - WSPing = stats.New("ws_ping", stats.Trend) + WSPing = stats.New("ws_ping", stats.Trend, stats.Time) WSSessionDuration = stats.New("ws_session_duration", stats.Trend, stats.Time) WSConnecting = stats.New("ws_connecting", stats.Trend, stats.Time) From 33ca3af500603d8a1cbedfc2f402d7a8592b202b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Wed, 24 Jun 2020 13:09:44 +0200 Subject: [PATCH 312/350] Fix externally-controlled maxVUs scaling Closes #1511 --- api/v1/status_routes.go | 6 ++--- api/v1/status_routes_test.go | 39 ++++++++++++++++++++------- lib/executor/externally_controlled.go | 5 ++-- 3 files changed, 35 insertions(+), 15 deletions(-) diff --git a/api/v1/status_routes.go b/api/v1/status_routes.go index c3c3e8b44aa..ee9a62e9e66 100644 --- a/api/v1/status_routes.go +++ b/api/v1/status_routes.go @@ -54,7 +54,7 @@ func getFirstExternallyControlledExecutor( return mex, nil } } - return nil, errors.New("a externally-controlled executor needs to be configured for live configuration updates") + return nil, errors.New("an externally-controlled executor needs to be configured for live configuration updates") } func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Params) { @@ -98,8 +98,8 @@ func HandlePatchStatus(rw http.ResponseWriter, r *http.Request, p httprouter.Par if status.VUs.Valid { newConfig.VUs = status.VUs } - if updateErr := executor.UpdateConfig(r.Context(), newConfig); err != nil { - apiError(rw, "Config update error", updateErr.Error(), http.StatusInternalServerError) + if updateErr := executor.UpdateConfig(r.Context(), newConfig); updateErr != nil { + apiError(rw, "Config update error", updateErr.Error(), http.StatusBadRequest) return } } diff --git a/api/v1/status_routes_test.go b/api/v1/status_routes_test.go index 321dba0e713..79e1bb5478a 100644 --- a/api/v1/status_routes_test.go +++ b/api/v1/status_routes_test.go @@ -21,15 +21,19 @@ package v1 import ( + "bytes" + "context" "encoding/json" "net/http" "net/http/httptest" "testing" + "time" "github.com/manyminds/api2go/jsonapi" "github.com/sirupsen/logrus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/core" "github.com/loadimpact/k6/core/local" @@ -68,24 +72,40 @@ func TestGetStatus(t *testing.T) { }) } -// TODO: fix after the externally-controlled executor -/* func TestPatchStatus(t *testing.T) { testdata := map[string]struct { StatusCode int Status Status }{ - "nothing": {200, Status{}}, - "paused": {200, Status{Paused: null.BoolFrom(true)}}, - "max vus": {200, Status{VUsMax: null.IntFrom(10)}}, - "too many vus": {400, Status{VUs: null.IntFrom(10), VUsMax: null.IntFrom(0)}}, - "vus": {200, Status{VUs: null.IntFrom(10), VUsMax: null.IntFrom(10)}}, + "nothing": {200, Status{}}, + "paused": {200, Status{Paused: null.BoolFrom(true)}}, + "max vus": {200, Status{VUsMax: null.IntFrom(20)}}, + "max vus below initial": {400, Status{VUsMax: null.IntFrom(5)}}, + "too many vus": {400, Status{VUs: null.IntFrom(10), VUsMax: null.IntFrom(0)}}, + "vus": {200, Status{VUs: null.IntFrom(10), VUsMax: null.IntFrom(10)}}, } + scenarios := lib.ScenarioConfigs{} + err := json.Unmarshal([]byte(` + {"external": {"executor": "externally-controlled", + "vus": 0, "maxVUs": 10, "duration": "1s"}}`), &scenarios) + require.NoError(t, err) + options := lib.Options{Scenarios: scenarios} + for name, indata := range testdata { t.Run(name, func(t *testing.T) { - engine, err := core.NewEngine(nil, lib.Options{}) - assert.NoError(t, err) + execScheduler, err := local.NewExecutionScheduler(&minirunner.MiniRunner{Options: options}, logrus.StandardLogger()) + require.NoError(t, err) + engine, err := core.NewEngine(execScheduler, options, logrus.StandardLogger()) + require.NoError(t, err) + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + run, _, err := engine.Init(ctx, ctx) + require.NoError(t, err) + + go func() { _ = run() }() + // wait for the executor to initialize to avoid a potential data race below + time.Sleep(100 * time.Millisecond) body, err := jsonapi.Marshal(indata.Status) if !assert.NoError(t, err) { @@ -116,4 +136,3 @@ func TestPatchStatus(t *testing.T) { }) } } -*/ diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index ce5f63e3926..973108881ca 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -73,8 +73,8 @@ func (mecc ExternallyControlledConfigParams) Validate() (errors []error) { if mecc.MaxVUs.Int64 < mecc.VUs.Int64 { errors = append(errors, fmt.Errorf( - "the specified maxVUs (%d) should be more than or equal to the the number of active VUs (%d)", - mecc.MaxVUs.Int64, mecc.VUs.Int64, + "the number of active VUs (%d) must be less than or equal to the number of maxVUs (%d)", + mecc.VUs.Int64, mecc.MaxVUs.Int64, )) } @@ -479,6 +479,7 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern executionState.ReturnVU(rs.vuHandles[i].initVU, false) } rs.vuHandles[i] = nil + executionState.ModInitializedVUsCount(-1) } rs.vuHandles = rs.vuHandles[:newMaxVUs] } From 1dc8222c4fca2fffe85d22f93d4dad4d28d91bcd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 25 Jun 2020 11:27:01 +0200 Subject: [PATCH 313/350] Fix double increment of initializedVUs in executor tests This replicates more closely what the local ExecutionScheduler does, and is needed for correct assertions in externally-controlled tests. --- lib/executor/common_test.go | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 2bd27d09600..652c994944e 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -54,15 +54,16 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta testLog.SetOutput(ioutil.Discard) logEntry := logrus.NewEntry(testLog) - es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { + initVUFunc := func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { return runner.NewVU(int64(es.GetUniqueVUIdentifier()), engineOut) - }) + } + es.SetInitVUFunc(initVUFunc) et, err := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) require.NoError(t, err) maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(et)) - initializeVUs(ctx, t, logEntry, es, maxVUs) + initializeVUs(ctx, t, logEntry, es, maxVUs, initVUFunc) executor, err := config.NewExecutor(es, logEntry) require.NoError(t, err) @@ -73,11 +74,13 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta } func initializeVUs( - ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number uint64, + ctx context.Context, t testing.TB, logEntry *logrus.Entry, es *lib.ExecutionState, number uint64, initVU lib.InitVUFunc, ) { // This is not how the local ExecutionScheduler initializes VUs, but should do the same job for i := uint64(0); i < number; i++ { - vu, err := es.InitializeNewVU(ctx, logEntry) + // Not calling es.InitializeNewVU() here to avoid a double increment of initializedVUs, + // which is done in es.AddInitializedVU(). + vu, err := initVU(ctx, logEntry) require.NoError(t, err) es.AddInitializedVU(vu) } From 06f66c532ba2c264f271c70b9c0a2ce575a21600 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 25 Jun 2020 11:32:49 +0200 Subject: [PATCH 314/350] Update TestExternallyControlledRun to check for maxVUs and scaling down --- lib/executor/externally_controlled_test.go | 37 ++++++++++++++-------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index ae5f4c6deaf..70ddee2dc82 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -40,7 +40,7 @@ func getTestExternallyControlledConfig() ExternallyControlledConfig { ExternallyControlledConfigParams: ExternallyControlledConfigParams{ VUs: null.IntFrom(2), MaxVUs: null.IntFrom(10), - Duration: types.NullDurationFrom(2 * time.Second), + Duration: types.NullDurationFrom(3 * time.Second), }, } } @@ -65,7 +65,7 @@ func TestExternallyControlledRun(t *testing.T) { wg sync.WaitGroup errCh = make(chan error, 1) doneCh = make(chan struct{}) - resultVUCount []int64 + resultVUCount [][]int64 ) wg.Add(1) go func() { @@ -74,38 +74,47 @@ func TestExternallyControlledRun(t *testing.T) { close(doneCh) }() - updateConfig := func(vus int) { + updateConfig := func(vus, maxVUs int) { newConfig := ExternallyControlledConfigParams{ VUs: null.IntFrom(int64(vus)), - MaxVUs: null.IntFrom(10), - Duration: types.NullDurationFrom(2 * time.Second), + MaxVUs: null.IntFrom(int64(maxVUs)), + Duration: types.NullDurationFrom(3 * time.Second), } err := executor.(*ExternallyControlled).UpdateConfig(ctx, newConfig) assert.NoError(t, err) } + snapshot := func() { + resultVUCount = append(resultVUCount, + []int64{es.GetCurrentlyActiveVUsCount(), es.GetInitializedVUsCount()}) + } + wg.Add(1) go func() { defer wg.Done() es.MarkStarted() time.Sleep(150 * time.Millisecond) // wait for startup - resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + snapshot() time.Sleep(500 * time.Millisecond) - updateConfig(4) + updateConfig(4, 10) time.Sleep(100 * time.Millisecond) - resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + snapshot() time.Sleep(500 * time.Millisecond) - updateConfig(8) - time.Sleep(100 * time.Millisecond) - resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + updateConfig(8, 20) + time.Sleep(500 * time.Millisecond) + snapshot() + time.Sleep(500 * time.Millisecond) + updateConfig(4, 10) + time.Sleep(500 * time.Millisecond) + snapshot() time.Sleep(1 * time.Second) - resultVUCount = append(resultVUCount, es.GetCurrentlyActiveVUsCount()) + snapshot() es.MarkEnded() }() <-doneCh wg.Wait() require.NoError(t, <-errCh) - assert.Equal(t, uint64(50), atomic.LoadUint64(doneIters)) - assert.Equal(t, []int64{2, 4, 8, 0}, resultVUCount) + assert.InDelta(t, uint64(75), atomic.LoadUint64(doneIters), 1) + assert.Equal(t, [][]int64{{2, 10}, {4, 10}, {8, 20}, {4, 10}, {0, 0}}, resultVUCount) } From fc48c46ea133f5d4abcf2f17a2f94a1d32706146 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Thu, 25 Jun 2020 16:06:41 +0200 Subject: [PATCH 315/350] Rewrite TestExternallyControlledRun to hopefully fix flakiness --- lib/executor/externally_controlled_test.go | 58 ++++++++++++---------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index 70ddee2dc82..2da34259285 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -40,17 +40,19 @@ func getTestExternallyControlledConfig() ExternallyControlledConfig { ExternallyControlledConfigParams: ExternallyControlledConfigParams{ VUs: null.IntFrom(2), MaxVUs: null.IntFrom(10), - Duration: types.NullDurationFrom(3 * time.Second), + Duration: types.NullDurationFrom(2 * time.Second), }, } } func TestExternallyControlledRun(t *testing.T) { t.Parallel() - doneIters := new(uint64) + et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + + doneIters := new(uint64) var ctx, cancel, executor, _ = setupExecutor( t, getTestExternallyControlledConfig(), es, simpleRunner(func(ctx context.Context) error { @@ -62,15 +64,16 @@ func TestExternallyControlledRun(t *testing.T) { defer cancel() var ( - wg sync.WaitGroup - errCh = make(chan error, 1) - doneCh = make(chan struct{}) - resultVUCount [][]int64 + wg sync.WaitGroup + errCh = make(chan error, 1) + doneCh = make(chan struct{}) ) wg.Add(1) go func() { defer wg.Done() + es.MarkStarted() errCh <- executor.Run(ctx, nil) + es.MarkEnded() close(doneCh) }() @@ -78,12 +81,13 @@ func TestExternallyControlledRun(t *testing.T) { newConfig := ExternallyControlledConfigParams{ VUs: null.IntFrom(int64(vus)), MaxVUs: null.IntFrom(int64(maxVUs)), - Duration: types.NullDurationFrom(3 * time.Second), + Duration: types.NullDurationFrom(2 * time.Second), } err := executor.(*ExternallyControlled).UpdateConfig(ctx, newConfig) assert.NoError(t, err) } + var resultVUCount [][]int64 snapshot := func() { resultVUCount = append(resultVUCount, []int64{es.GetCurrentlyActiveVUsCount(), es.GetInitializedVUsCount()}) @@ -92,29 +96,29 @@ func TestExternallyControlledRun(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - es.MarkStarted() - time.Sleep(150 * time.Millisecond) // wait for startup - snapshot() - time.Sleep(500 * time.Millisecond) - updateConfig(4, 10) - time.Sleep(100 * time.Millisecond) - snapshot() - time.Sleep(500 * time.Millisecond) - updateConfig(8, 20) - time.Sleep(500 * time.Millisecond) - snapshot() - time.Sleep(500 * time.Millisecond) - updateConfig(4, 10) - time.Sleep(500 * time.Millisecond) - snapshot() - time.Sleep(1 * time.Second) - snapshot() - es.MarkEnded() + snapshotTicker := time.NewTicker(500 * time.Millisecond) + ticks := 0 + for { + select { + case <-snapshotTicker.C: + snapshot() + switch ticks { + case 0, 2: + updateConfig(4, 10) + case 1: + updateConfig(8, 20) + } + ticks++ + case <-doneCh: + snapshotTicker.Stop() + snapshot() + return + } + } }() - <-doneCh wg.Wait() require.NoError(t, <-errCh) - assert.InDelta(t, uint64(75), atomic.LoadUint64(doneIters), 1) + assert.Equal(t, uint64(48), atomic.LoadUint64(doneIters)) assert.Equal(t, [][]int64{{2, 10}, {4, 10}, {8, 20}, {4, 10}, {0, 0}}, resultVUCount) } From b9939c7de1caac3d17114b3714d3d1673aa9e750 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 29 Jun 2020 15:59:44 +0200 Subject: [PATCH 316/350] Remove old ExecutionSchedulerSetVUs test, integrate in TestExternallyControlledRun --- core/local/local_test.go | 97 ---------------------- lib/executor/externally_controlled_test.go | 18 +++- 2 files changed, 14 insertions(+), 101 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 22a4fdd9770..3aedeb776dc 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -865,103 +865,6 @@ func TestExecutionSchedulerIsRunning(t *testing.T) { assert.NoError(t, <-err) } -/* -//TODO: convert for the externally-controlled scheduler -func TestExecutionSchedulerSetVUs(t *testing.T) { - t.Run("Negative", func(t *testing.T) { - assert.EqualError(t, New(nil).SetVUs(-1), "vu count can't be negative") - }) - - t.Run("Too High", func(t *testing.T) { - assert.EqualError(t, New(nil).SetVUs(100), "can't raise vu count (to 100) above vu cap (0)") - }) - - t.Run("Raise", func(t *testing.T) { - e := New(&minirunner.MiniRunner{Fn: func(ctx context.Context, out chan<- stats.SampleContainer) error { - return nil - }}) - e.ctx = context.Background() - - assert.NoError(t, e.SetVUsMax(100)) - assert.Equal(t, int64(100), e.GetVUsMax()) - if assert.Len(t, e.vus, 100) { - num := 0 - for i, handle := range e.vus { - num++ - if assert.NotNil(t, handle.vu, "vu %d lacks impl", i) { - assert.Equal(t, int64(0), handle.vu.(*minirunner.VU).ID) - } - assert.Nil(t, handle.ctx, "vu %d has ctx", i) - assert.Nil(t, handle.cancel, "vu %d has cancel", i) - } - assert.Equal(t, 100, num) - } - - assert.NoError(t, e.SetVUs(50)) - assert.Equal(t, int64(50), e.GetVUs()) - if assert.Len(t, e.vus, 100) { - num := 0 - for i, handle := range e.vus { - if i < 50 { - assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) - num++ - } else { - assert.Nil(t, handle.cancel, "vu %d has cancel", i) - assert.Equal(t, int64(0), handle.vu.(*minirunner.VU).ID) - } - } - assert.Equal(t, 50, num) - } - - assert.NoError(t, e.SetVUs(100)) - assert.Equal(t, int64(100), e.GetVUs()) - if assert.Len(t, e.vus, 100) { - num := 0 - for i, handle := range e.vus { - assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) - num++ - } - assert.Equal(t, 100, num) - } - - t.Run("Lower", func(t *testing.T) { - assert.NoError(t, e.SetVUs(50)) - assert.Equal(t, int64(50), e.GetVUs()) - if assert.Len(t, e.vus, 100) { - num := 0 - for i, handle := range e.vus { - if i < 50 { - assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - num++ - } else { - assert.Nil(t, handle.cancel, "vu %d has cancel", i) - } - assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) - } - assert.Equal(t, 50, num) - } - - t.Run("Raise", func(t *testing.T) { - assert.NoError(t, e.SetVUs(100)) - assert.Equal(t, int64(100), e.GetVUs()) - if assert.Len(t, e.vus, 100) { - for i, handle := range e.vus { - assert.NotNil(t, handle.cancel, "vu %d lacks cancel", i) - if i < 50 { - assert.Equal(t, int64(i+1), handle.vu.(*minirunner.VU).ID) - } else { - assert.Equal(t, int64(50+i+1), handle.vu.(*minirunner.VU).ID) - } - } - } - }) - }) - }) -} -*/ - func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { if runtime.GOOS == "windows" { t.Skip() diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index 2da34259285..6cdf7d78d87 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -77,14 +77,18 @@ func TestExternallyControlledRun(t *testing.T) { close(doneCh) }() - updateConfig := func(vus, maxVUs int) { + updateConfig := func(vus, maxVUs int, errMsg string) { newConfig := ExternallyControlledConfigParams{ VUs: null.IntFrom(int64(vus)), MaxVUs: null.IntFrom(int64(maxVUs)), Duration: types.NullDurationFrom(2 * time.Second), } err := executor.(*ExternallyControlled).UpdateConfig(ctx, newConfig) - assert.NoError(t, err) + if errMsg != "" { + assert.EqualError(t, err, errMsg) + } else { + assert.NoError(t, err) + } } var resultVUCount [][]int64 @@ -104,9 +108,15 @@ func TestExternallyControlledRun(t *testing.T) { snapshot() switch ticks { case 0, 2: - updateConfig(4, 10) + updateConfig(4, 10, "") case 1: - updateConfig(8, 20) + updateConfig(8, 20, "") + case 3: + updateConfig(15, 10, + "invalid configuration supplied: the number of active VUs (15)"+ + " must be less than or equal to the number of maxVUs (10)") + updateConfig(-1, 10, + "invalid configuration supplied: the number of VUs shouldn't be negative") } ticks++ case <-doneCh: From d3894cb0ddaa29578baa084bb1ae7c394715a547 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Mon, 29 Jun 2020 22:39:05 +0300 Subject: [PATCH 317/350] Update goja --- Gopkg.lock | 4 +- vendor/github.com/dop251/goja/compiler.go | 3 +- .../github.com/dop251/goja/compiler_expr.go | 21 +++++- .../github.com/dop251/goja/compiler_stmt.go | 32 ++++---- .../dop251/goja/object_goreflect.go | 57 ++++++++++++++ vendor/github.com/dop251/goja/parser/lexer.go | 4 + vendor/github.com/dop251/goja/vm.go | 74 ++++++++++++++----- 7 files changed, 158 insertions(+), 37 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 783c26ad9b3..e90fa5a83fc 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -97,7 +97,7 @@ [[projects]] branch = "master" - digest = "1:fda79eedf88d4aedeccb193dfeae71522bd806f5fc07ae70f05187cbbd4b1146" + digest = "1:dd49b520ab3ef9766f486a5771c1aef117e77b4d09bd8e2690df02cfc8e607b9" name = "github.com/dop251/goja" packages = [ ".", @@ -107,7 +107,7 @@ "token", ] pruneopts = "NUT" - revision = "0cd29d81c048b6c79afb6ba6a1305bcb70c28740" + revision = "bfd59704b500581bf75771f79cc2579741cf3d2c" [[projects]] branch = "master" diff --git a/vendor/github.com/dop251/goja/compiler.go b/vendor/github.com/dop251/goja/compiler.go index 385ac0dc254..47d7fb74fcd 100644 --- a/vendor/github.com/dop251/goja/compiler.go +++ b/vendor/github.com/dop251/goja/compiler.go @@ -10,6 +10,7 @@ import ( const ( blockLoop = iota + blockLoopEnum blockTry blockBranch blockSwitch @@ -85,7 +86,7 @@ func (c *compiler) leaveBlock() { for _, item := range c.block.breaks { c.p.code[item] = jump(lbl - item) } - if c.block.typ == blockLoop { + if t := c.block.typ; t == blockLoop || t == blockLoopEnum { for _, item := range c.block.conts { c.p.code[item] = jump(c.block.cont - item) } diff --git a/vendor/github.com/dop251/goja/compiler_expr.go b/vendor/github.com/dop251/goja/compiler_expr.go index c8e9fce78ea..79f9731c34c 100644 --- a/vendor/github.com/dop251/goja/compiler_expr.go +++ b/vendor/github.com/dop251/goja/compiler_expr.go @@ -308,6 +308,24 @@ func (e *compiledIdentifierExpr) emitGetterOrRef() { } else { if found { e.c.emit(getVar{name: e.name, idx: idx, ref: true}) + } else { + e.c.emit(getVar1Ref(e.name)) + } + } +} + +func (e *compiledIdentifierExpr) emitGetterAndCallee() { + e.addSrcMap() + if idx, found, noDynamics := e.c.scope.lookupName(e.name); noDynamics { + if found { + e.c.emit(loadUndef) + e.c.emit(getLocal(idx)) + } else { + panic("No dynamics and not found") + } + } else { + if found { + e.c.emit(getVar{name: e.name, idx: idx, ref: true, callee: true}) } else { e.c.emit(getVar1Callee(e.name)) } @@ -1423,9 +1441,8 @@ func (e *compiledCallExpr) emitGetter(putOnStack bool) { callee.member.emitGetter(true) e.c.emit(getElemCallee) case *compiledIdentifierExpr: - e.c.emit(loadUndef) calleeName = callee.name - callee.emitGetterOrRef() + callee.emitGetterAndCallee() default: e.c.emit(loadUndef) callee.emitGetter(true) diff --git a/vendor/github.com/dop251/goja/compiler_stmt.go b/vendor/github.com/dop251/goja/compiler_stmt.go index a98ac2c44a0..2f546ecb065 100644 --- a/vendor/github.com/dop251/goja/compiler_stmt.go +++ b/vendor/github.com/dop251/goja/compiler_stmt.go @@ -140,13 +140,10 @@ func (c *compiler) compileTryStatement(v *ast.TryStatement) { code[pc] = setLocalP(remap(uint32(instr))) } } + c.p.code[start+1] = pop if catchVarIdx, exists := m[0]; exists { c.p.code[start] = setLocal(catchVarIdx) - c.p.code[start+1] = pop catchOffset-- - } else { - c.p.code[start+1] = nil - catchOffset++ } } else { c.scope.accessed = true @@ -304,7 +301,7 @@ func (c *compiler) compileForInStatement(v *ast.ForInStatement, needResult bool) func (c *compiler) compileLabeledForInStatement(v *ast.ForInStatement, needResult bool, label string) { c.block = &block{ - typ: blockLoop, + typ: blockLoopEnum, outer: c.block, label: label, needResult: needResult, @@ -421,7 +418,7 @@ func (c *compiler) findBranchBlock(st *ast.BranchStatement) *block { func (c *compiler) findContinueBlock(label *ast.Identifier) (block *block) { if label != nil { for b := c.block; b != nil; b = b.outer { - if b.typ == blockLoop && b.label == label.Name { + if (b.typ == blockLoop || b.typ == blockLoopEnum) && b.label == label.Name { block = b break } @@ -429,7 +426,7 @@ func (c *compiler) findContinueBlock(label *ast.Identifier) (block *block) { } else { // find the nearest loop for b := c.block; b != nil; b = b.outer { - if b.typ == blockLoop { + if b.typ == blockLoop || b.typ == blockLoopEnum { block = b break } @@ -452,7 +449,7 @@ func (c *compiler) findBreakBlock(label *ast.Identifier) (block *block) { L: for b := c.block; b != nil; b = b.outer { switch b.typ { - case blockLoop, blockSwitch: + case blockLoop, blockLoopEnum, blockSwitch: block = b break L } @@ -486,7 +483,7 @@ func (c *compiler) compileBreak(label *ast.Identifier, idx file.Idx) { c.emit(halt) case blockWith: c.emit(leaveWith) - case blockLoop, blockSwitch: + case blockLoop, blockLoopEnum, blockSwitch: block = b break L } @@ -510,7 +507,7 @@ func (c *compiler) compileContinue(label *ast.Identifier, idx file.Idx) { for b := c.block; b != nil; b = b.outer { if b.typ == blockTry { c.emit(halt) - } else if b.typ == blockLoop && b.label == label.Name { + } else if (b.typ == blockLoop || b.typ == blockLoopEnum) && b.label == label.Name { block = b break } @@ -520,7 +517,7 @@ func (c *compiler) compileContinue(label *ast.Identifier, idx file.Idx) { for b := c.block; b != nil; b = b.outer { if b.typ == blockTry { c.emit(halt) - } else if b.typ == blockLoop { + } else if b.typ == blockLoop || b.typ == blockLoopEnum { block = b break } @@ -587,10 +584,14 @@ func (c *compiler) compileIfStatement(v *ast.IfStatement, needResult bool) { c.p.code[jmp1] = jump(len(c.p.code) - jmp1) c.markBlockStart() } else { - c.p.code[jmp] = jne(len(c.p.code) - jmp) - c.markBlockStart() if needResult { + c.emit(jump(2)) + c.p.code[jmp] = jne(len(c.p.code) - jmp) c.emit(loadUndef) + c.markBlockStart() + } else { + c.p.code[jmp] = jne(len(c.p.code) - jmp) + c.markBlockStart() } } } @@ -603,8 +604,11 @@ func (c *compiler) compileReturnStatement(v *ast.ReturnStatement) { c.emit(loadUndef) } for b := c.block; b != nil; b = b.outer { - if b.typ == blockTry { + switch b.typ { + case blockTry: c.emit(halt) + case blockLoopEnum: + c.emit(enumPop) } } c.emit(ret) diff --git a/vendor/github.com/dop251/goja/object_goreflect.go b/vendor/github.com/dop251/goja/object_goreflect.go index 8d9ca69135c..1068e4ae287 100644 --- a/vendor/github.com/dop251/goja/object_goreflect.go +++ b/vendor/github.com/dop251/goja/object_goreflect.go @@ -2,8 +2,10 @@ package goja import ( "fmt" + "github.com/dop251/goja/parser" "go/ast" "reflect" + "strings" ) // JsonEncodable allows custom JSON encoding by JSON.stringify() @@ -23,6 +25,44 @@ type FieldNameMapper interface { MethodName(t reflect.Type, m reflect.Method) string } +type tagFieldNameMapper struct { + tagName string + uncapMethods bool +} + +func (tfm tagFieldNameMapper) FieldName(_ reflect.Type, f reflect.StructField) string { + tag := f.Tag.Get(tfm.tagName) + if idx := strings.IndexByte(tag, ','); idx != -1 { + tag = tag[:idx] + } + if parser.IsIdentifier(tag) { + return tag + } + return "" +} + +func uncapitalize(s string) string { + return strings.ToLower(s[0:1]) + s[1:] +} + +func (tfm tagFieldNameMapper) MethodName(_ reflect.Type, m reflect.Method) string { + if tfm.uncapMethods { + return uncapitalize(m.Name) + } + return m.Name +} + +type uncapFieldNameMapper struct { +} + +func (u uncapFieldNameMapper) FieldName(_ reflect.Type, f reflect.StructField) string { + return uncapitalize(f.Name) +} + +func (u uncapFieldNameMapper) MethodName(_ reflect.Type, m reflect.Method) string { + return uncapitalize(m.Name) +} + type reflectFieldInfo struct { Index []int Anonymous bool @@ -512,3 +552,20 @@ func (r *Runtime) SetFieldNameMapper(mapper FieldNameMapper) { r.fieldNameMapper = mapper r.typeInfoCache = nil } + +// TagFieldNameMapper returns a FieldNameMapper that uses the given tagName for struct fields and optionally +// uncapitalises (making the first letter lower case) method names. +// The common tag value syntax is supported (name[,options]), however options are ignored. +// Setting name to anything other than a valid ECMAScript identifier makes the field hidden. +func TagFieldNameMapper(tagName string, uncapMethods bool) FieldNameMapper { + return tagFieldNameMapper{ + tagName: tagName, + uncapMethods: uncapMethods, + } +} + +// UncapFieldNameMapper returns a FieldNameMapper that uncapitalises struct field and method names +// making the first letter lower case. +func UncapFieldNameMapper() FieldNameMapper { + return uncapFieldNameMapper{} +} diff --git a/vendor/github.com/dop251/goja/parser/lexer.go b/vendor/github.com/dop251/goja/parser/lexer.go index 0626b007c34..0786efaca73 100644 --- a/vendor/github.com/dop251/goja/parser/lexer.go +++ b/vendor/github.com/dop251/goja/parser/lexer.go @@ -26,6 +26,10 @@ func isDecimalDigit(chr rune) bool { return '0' <= chr && chr <= '9' } +func IsIdentifier(s string) bool { + return matchIdentifier.MatchString(s) +} + func digitValue(chr rune) int { switch { case '0' <= chr && chr <= '9': diff --git a/vendor/github.com/dop251/goja/vm.go b/vendor/github.com/dop251/goja/vm.go index 95ed61f62df..8739945ada3 100644 --- a/vendor/github.com/dop251/goja/vm.go +++ b/vendor/github.com/dop251/goja/vm.go @@ -19,7 +19,7 @@ type stash struct { values valueStack extraArgs valueStack names map[string]uint32 - obj objectImpl + obj *Object outer *stash } @@ -200,8 +200,8 @@ func (s *valueStack) expand(idx int) { func (s *stash) put(name string, v Value) bool { if s.obj != nil { - if found := s.obj.getStr(name); found != nil { - s.obj.putStr(name, v, false) + if found := s.obj.self.getStr(name); found != nil { + s.obj.self.putStr(name, v, false) return true } return false @@ -232,7 +232,7 @@ func (s *stash) getByIdx(idx uint32) Value { func (s *stash) getByName(name string, _ *vm) (v Value, exists bool) { if s.obj != nil { - v = s.obj.getStr(name) + v = s.obj.self.getStr(name) if v == nil { return nil, false //return valueUnresolved{r: vm.r, ref: name}, false @@ -258,7 +258,7 @@ func (s *stash) createBinding(name string) { func (s *stash) deleteBinding(name string) bool { if s.obj != nil { - return s.obj.deleteStr(name, false) + return s.obj.self.deleteStr(name, false) } if idx, found := s.names[name]; found { s.values[idx] = nil @@ -1332,9 +1332,9 @@ func (s resolveVar1) exec(vm *vm) { var ref ref for stash := vm.stash; stash != nil; stash = stash.outer { if stash.obj != nil { - if stash.obj.hasPropertyStr(name) { + if stash.obj.self.hasPropertyStr(name) { ref = &objRef{ - base: stash.obj, + base: stash.obj.self, name: name, } goto end @@ -1366,8 +1366,8 @@ func (d deleteVar) exec(vm *vm) { ret := true for stash := vm.stash; stash != nil; stash = stash.outer { if stash.obj != nil { - if stash.obj.hasPropertyStr(name) { - ret = stash.obj.deleteStr(name, false) + if stash.obj.self.hasPropertyStr(name) { + ret = stash.obj.self.deleteStr(name, false) goto end } } else { @@ -1416,9 +1416,9 @@ func (s resolveVar1Strict) exec(vm *vm) { var ref ref for stash := vm.stash; stash != nil; stash = stash.outer { if stash.obj != nil { - if stash.obj.hasPropertyStr(name) { + if stash.obj.self.hasPropertyStr(name) { ref = &objRef{ - base: stash.obj, + base: stash.obj.self, name: name, strict: true, } @@ -1492,23 +1492,33 @@ func (g getLocal) exec(vm *vm) { } type getVar struct { - name string - idx uint32 - ref bool + name string + idx uint32 + ref, callee bool } func (g getVar) exec(vm *vm) { level := int(g.idx >> 24) - idx := uint32(g.idx & 0x00FFFFFF) + idx := g.idx & 0x00FFFFFF stash := vm.stash name := g.name for i := 0; i < level; i++ { if v, found := stash.getByName(name, vm); found { + if g.callee { + if stash.obj != nil { + vm.push(stash.obj) + } else { + vm.push(_undefined) + } + } vm.push(v) goto end } stash = stash.outer } + if g.callee { + vm.push(_undefined) + } if stash != nil { vm.push(stash.getByIdx(idx)) } else { @@ -1539,9 +1549,9 @@ func (r resolveVar) exec(vm *vm) { var ref ref for i := 0; i < level; i++ { if stash.obj != nil { - if stash.obj.hasPropertyStr(r.name) { + if stash.obj.self.hasPropertyStr(r.name) { ref = &objRef{ - base: stash.obj, + base: stash.obj.self, name: r.name, strict: r.strict, } @@ -1632,13 +1642,36 @@ func (n getVar1) exec(vm *vm) { vm.pc++ } +type getVar1Ref string + +func (n getVar1Ref) exec(vm *vm) { + name := string(n) + var val Value + for stash := vm.stash; stash != nil; stash = stash.outer { + if v, exists := stash.getByName(name, vm); exists { + val = v + break + } + } + if val == nil { + val = vm.r.globalObject.self.getStr(name) + if val == nil { + val = valueUnresolved{r: vm.r, ref: name} + } + } + vm.push(val) + vm.pc++ +} + type getVar1Callee string func (n getVar1Callee) exec(vm *vm) { name := string(n) var val Value + var callee *Object for stash := vm.stash; stash != nil; stash = stash.outer { if v, exists := stash.getByName(name, vm); exists { + callee = stash.obj val = v break } @@ -1649,6 +1682,11 @@ func (n getVar1Callee) exec(vm *vm) { val = valueUnresolved{r: vm.r, ref: name} } } + if callee != nil { + vm.push(callee) + } else { + vm.push(_undefined) + } vm.push(val) vm.pc++ } @@ -2376,7 +2414,7 @@ var enterWith _enterWith func (_enterWith) exec(vm *vm) { vm.newStash() - vm.stash.obj = vm.stack[vm.sp-1].ToObject(vm.r).self + vm.stash.obj = vm.stack[vm.sp-1].ToObject(vm.r) vm.sp-- vm.pc++ } From bea60a43fd553172daf2f2d591ae4a5f6e815ce2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 29 Jun 2020 10:54:48 +0200 Subject: [PATCH 318/350] Update outdated comment --- js/runner.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/js/runner.go b/js/runner.go index 33a2f83b40e..ab8caac963d 100644 --- a/js/runner.go +++ b/js/runner.go @@ -456,7 +456,7 @@ func (u *ActiveVU) RunOnce() error { fn, ok := u.exports[u.Exec] if !ok { - // Shouldn't happen; this is validated in ExecutionScheduler.Init() + // Shouldn't happen; this is validated in cmd.validateScenarioConfig() panic(fmt.Sprintf("function '%s' not found in exports", u.Exec)) } From 0411270bd47f96632c68297900d589be415768df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 29 Jun 2020 10:59:49 +0200 Subject: [PATCH 319/350] Add "scenario" as default system tag This tags all metrics emitted in the main "exec" function with a "scenario" tag set to the scenario name or "default" if no custom scenarios were defined. It can be disabled with the `--system-tags` option. There's no issue for this specific feature, but see #796 and #1300. --- core/local/local_test.go | 118 +++++++++++++++++++++++++++++++++--- js/runner.go | 11 ++-- js/runner_test.go | 2 + lib/executor/helpers.go | 1 + lib/runner.go | 2 +- stats/system_tag.go | 3 +- stats/system_tag_set_gen.go | 22 ++++--- 7 files changed, 135 insertions(+), 24 deletions(-) diff --git a/core/local/local_test.go b/core/local/local_test.go index 22a4fdd9770..4e701eb8e5c 100644 --- a/core/local/local_test.go +++ b/core/local/local_test.go @@ -265,6 +265,106 @@ func TestExecutionSchedulerRunEnv(t *testing.T) { } } +func TestExecutionSchedulerSystemTags(t *testing.T) { + t.Parallel() + tb := httpmultibin.NewHTTPMultiBin(t) + defer tb.Cleanup() + sr := tb.Replacer.Replace + + script := sr(` + import http from "k6/http"; + + export let options = { + scenarios: { + per_vu_test: { + executor: "per-vu-iterations", + gracefulStop: "0s", + vus: 1, + iterations: 1, + }, + shared_test: { + executor: "shared-iterations", + gracefulStop: "0s", + vus: 1, + iterations: 1, + } + } + } + + export default function () { + http.get("HTTPBIN_IP_URL/"); + }`) + + runner, err := js.New(&loader.SourceData{ + URL: &url.URL{Path: "/script.js"}, + Data: []byte(script)}, + nil, lib.RuntimeOptions{}) + require.NoError(t, err) + + require.NoError(t, runner.SetOptions(runner.GetOptions().Apply(lib.Options{ + SystemTags: &stats.DefaultSystemTagSet, + }))) + + logger := logrus.New() + logger.SetOutput(testutils.NewTestOutput(t)) + execScheduler, err := NewExecutionScheduler(runner, logger) + require.NoError(t, err) + + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) + defer cancel() + + samples := make(chan stats.SampleContainer) + done := make(chan struct{}) + go func() { + defer close(done) + require.NoError(t, execScheduler.Init(ctx, samples)) + require.NoError(t, execScheduler.Run(ctx, ctx, samples)) + }() + + expCommonTrailTags := stats.IntoSampleTags(&map[string]string{ + "group": "", + "method": "GET", + "name": sr("HTTPBIN_IP_URL/"), + "url": sr("HTTPBIN_IP_URL/"), + "proto": "HTTP/1.1", + "status": "200", + }) + expTrailPVUTagsRaw := expCommonTrailTags.CloneTags() + expTrailPVUTagsRaw["scenario"] = "per_vu_test" + expTrailPVUTags := stats.IntoSampleTags(&expTrailPVUTagsRaw) + expTrailSITagsRaw := expCommonTrailTags.CloneTags() + expTrailSITagsRaw["scenario"] = "shared_test" + expTrailSITags := stats.IntoSampleTags(&expTrailSITagsRaw) + expNetTrailPVUTags := stats.IntoSampleTags(&map[string]string{ + "group": "", + "scenario": "per_vu_test", + }) + expNetTrailSITags := stats.IntoSampleTags(&map[string]string{ + "group": "", + "scenario": "shared_test", + }) + + var gotCorrectTags int + for { + select { + case sample := <-samples: + switch s := sample.(type) { + case *httpext.Trail: + if s.Tags.IsEqual(expTrailPVUTags) || s.Tags.IsEqual(expTrailSITags) { + gotCorrectTags++ + } + case *netext.NetTrail: + if s.Tags.IsEqual(expNetTrailPVUTags) || s.Tags.IsEqual(expNetTrailSITags) { + gotCorrectTags++ + } + } + case <-done: + require.Equal(t, 4, gotCorrectTags, "received wrong amount of samples with expected tags") + return + } + } +} + func TestExecutionSchedulerRunCustomTags(t *testing.T) { t.Parallel() tb := httpmultibin.NewHTTPMultiBin(t) @@ -1085,9 +1185,11 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { Value: expValue, } } - getDummyTrail := func(group string, emitIterations bool) stats.SampleContainer { + getDummyTrail := func(group string, emitIterations bool, addExpTags ...string) stats.SampleContainer { + expTags := []string{"group", group} + expTags = append(expTags, addExpTags...) return netext.NewDialer(net.Dialer{}).GetTrail(time.Now(), time.Now(), - true, emitIterations, getTags("group", group)) + true, emitIterations, getTags(expTags...)) } // Initially give a long time (5s) for the execScheduler to start @@ -1095,13 +1197,13 @@ func TestRealTimeAndSetupTeardownMetrics(t *testing.T) { expectIn(900, 1100, getSample(2, testCounter, "group", "::setup", "place", "setupAfterSleep")) expectIn(0, 100, getDummyTrail("::setup", false)) - expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep")) - expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep")) - expectIn(0, 100, getDummyTrail("", true)) + expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep", "scenario", "default")) + expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep", "scenario", "default")) + expectIn(0, 100, getDummyTrail("", true, "scenario", "default")) - expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep")) - expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep")) - expectIn(0, 100, getDummyTrail("", true)) + expectIn(0, 100, getSample(5, testCounter, "group", "", "place", "defaultBeforeSleep", "scenario", "default")) + expectIn(900, 1100, getSample(6, testCounter, "group", "", "place", "defaultAfterSleep", "scenario", "default")) + expectIn(0, 100, getDummyTrail("", true, "scenario", "default")) expectIn(0, 1000, getSample(3, testCounter, "group", "::teardown", "place", "teardownBeforeSleep")) expectIn(900, 1100, getSample(4, testCounter, "group", "::teardown", "place", "teardownAfterSleep")) diff --git a/js/runner.go b/js/runner.go index ab8caac963d..04df47e63d9 100644 --- a/js/runner.go +++ b/js/runner.go @@ -328,7 +328,7 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, return goja.Undefined(), err } - v, _, _, err := vu.runFn(ctx, group, false, nil, fn, vu.Runtime.ToValue(arg)) + v, _, _, err := vu.runFn(ctx, "", group, false, nil, fn, vu.Runtime.ToValue(arg)) // deadline is reached so we have timeouted but this might've not been registered correctly if deadline, ok := ctx.Deadline(); ok && time.Now().After(deadline) { @@ -462,7 +462,7 @@ func (u *ActiveVU) RunOnce() error { // Call the exported function. _, isFullIteration, totalTime, err := u.runFn( - u.RunContext, u.Runner.defaultGroup, true, u.Tags, fn, u.setupData, + u.RunContext, u.Scenario, u.Runner.defaultGroup, true, u.Tags, fn, u.setupData, ) // If MinIterationDuration is specified and the iteration wasn't cancelled @@ -478,8 +478,8 @@ func (u *ActiveVU) RunOnce() error { } func (u *VU) runFn( - ctx context.Context, group *lib.Group, isDefault bool, customTags map[string]string, - fn goja.Callable, args ...goja.Value, + ctx context.Context, scenario string, group *lib.Group, isDefault bool, + customTags map[string]string, fn goja.Callable, args ...goja.Value, ) (goja.Value, bool, time.Duration, error) { cookieJar := u.CookieJar if !u.Runner.Bundle.Options.NoCookiesReset.ValueOrZero() { @@ -504,6 +504,9 @@ func (u *VU) runFn( if opts.SystemTags.Has(stats.TagGroup) { tags["group"] = group.Path } + if scenario != "" && opts.SystemTags.Has(stats.TagScenario) { + tags["scenario"] = scenario + } state := &lib.State{ Logger: u.Runner.Logger, diff --git a/js/runner_test.go b/js/runner_test.go index 3214b0bd3eb..27276da42c2 100644 --- a/js/runner_test.go +++ b/js/runner_test.go @@ -1626,6 +1626,7 @@ func TestSystemTags(t *testing.T) { {"ocsp_status", "https_get", "unknown"}, {"error", "bad_url_get", `dial: connection refused`}, {"error_code", "bad_url_get", "1212"}, + {"scenario", "http_get", "default"}, //TODO: add more tests } @@ -1645,6 +1646,7 @@ func TestSystemTags(t *testing.T) { activeVU := vu.Activate(&lib.VUActivationParams{ RunContext: context.Background(), Exec: tc.exec, + Scenario: "default", }) require.NoError(t, activeVU.RunOnce()) diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 2df8eae53b1..46fdf3c432d 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -223,6 +223,7 @@ func getVUActivationParams( ) *lib.VUActivationParams { return &lib.VUActivationParams{ RunContext: ctx, + Scenario: conf.Name, Exec: conf.GetExec(), Env: conf.GetEnv(), Tags: conf.GetTags(), diff --git a/lib/runner.go b/lib/runner.go index 2bc44522828..e444734d5a2 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -49,7 +49,7 @@ type VUActivationParams struct { RunContext context.Context DeactivateCallback func(InitializedVU) Env, Tags map[string]string - Exec string + Exec, Scenario string } // A Runner is a factory for VUs. It should precompute as much as possible upon diff --git a/stats/system_tag.go b/stats/system_tag.go index fde913d34cd..9d98f64a366 100644 --- a/stats/system_tag.go +++ b/stats/system_tag.go @@ -49,6 +49,7 @@ const ( TagError TagErrorCode TagTLSVersion + TagScenario // System tags not enabled by default. TagIter @@ -61,7 +62,7 @@ const ( // Other tags that are not enabled by default include: iter, vu, ocsp_status, ip //nolint:gochecknoglobals var DefaultSystemTagSet = TagProto | TagSubproto | TagStatus | TagMethod | TagURL | TagName | TagGroup | - TagCheck | TagCheck | TagError | TagErrorCode | TagTLSVersion + TagCheck | TagCheck | TagError | TagErrorCode | TagTLSVersion | TagScenario // Add adds a tag to tag set. func (i *SystemTagSet) Add(tag SystemTagSet) { diff --git a/stats/system_tag_set_gen.go b/stats/system_tag_set_gen.go index c4f12aa872c..5514e31cf94 100644 --- a/stats/system_tag_set_gen.go +++ b/stats/system_tag_set_gen.go @@ -7,7 +7,7 @@ import ( "fmt" ) -const _SystemTagSetName = "protosubprotostatusmethodurlnamegroupcheckerrorerror_codetls_versionitervuocsp_statusip" +const _SystemTagSetName = "protosubprotostatusmethodurlnamegroupcheckerrorerror_codetls_versionscenarioitervuocsp_statusip" var _SystemTagSetMap = map[SystemTagSet]string{ 1: _SystemTagSetName[0:5], @@ -21,10 +21,11 @@ var _SystemTagSetMap = map[SystemTagSet]string{ 256: _SystemTagSetName[42:47], 512: _SystemTagSetName[47:57], 1024: _SystemTagSetName[57:68], - 2048: _SystemTagSetName[68:72], - 4096: _SystemTagSetName[72:74], - 8192: _SystemTagSetName[74:85], - 16384: _SystemTagSetName[85:87], + 2048: _SystemTagSetName[68:76], + 4096: _SystemTagSetName[76:80], + 8192: _SystemTagSetName[80:82], + 16384: _SystemTagSetName[82:93], + 32768: _SystemTagSetName[93:95], } func (i SystemTagSet) String() string { @@ -34,7 +35,7 @@ func (i SystemTagSet) String() string { return fmt.Sprintf("SystemTagSet(%d)", i) } -var _SystemTagSetValues = []SystemTagSet{1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384} +var _SystemTagSetValues = []SystemTagSet{1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768} var _SystemTagSetNameToValueMap = map[string]SystemTagSet{ _SystemTagSetName[0:5]: 1, @@ -48,10 +49,11 @@ var _SystemTagSetNameToValueMap = map[string]SystemTagSet{ _SystemTagSetName[42:47]: 256, _SystemTagSetName[47:57]: 512, _SystemTagSetName[57:68]: 1024, - _SystemTagSetName[68:72]: 2048, - _SystemTagSetName[72:74]: 4096, - _SystemTagSetName[74:85]: 8192, - _SystemTagSetName[85:87]: 16384, + _SystemTagSetName[68:76]: 2048, + _SystemTagSetName[76:80]: 4096, + _SystemTagSetName[80:82]: 8192, + _SystemTagSetName[82:93]: 16384, + _SystemTagSetName[93:95]: 32768, } // SystemTagSetString retrieves an enum value from the enum constants string name. From 64f30fcdacefd06419178c8f586506404d30ecbc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 30 Jun 2020 10:19:20 +0200 Subject: [PATCH 320/350] Decrement initialized VUs only when not returning VUs to the global pool Resolves https://github.com/loadimpact/k6/pull/1517#discussion_r447435267 --- lib/executor/externally_controlled.go | 3 ++- lib/executor/externally_controlled_test.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 973108881ca..52e4c252d8e 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -477,9 +477,10 @@ func (rs *externallyControlledRunState) handleConfigChange(oldCfg, newCfg Extern if i < rs.startMaxVUs { // return the initial planned VUs to the common buffer executionState.ReturnVU(rs.vuHandles[i].initVU, false) + } else { + executionState.ModInitializedVUsCount(-1) } rs.vuHandles[i] = nil - executionState.ModInitializedVUsCount(-1) } rs.vuHandles = rs.vuHandles[:newMaxVUs] } diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index 6cdf7d78d87..30f3b8aeeb4 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -130,5 +130,5 @@ func TestExternallyControlledRun(t *testing.T) { wg.Wait() require.NoError(t, <-errCh) assert.Equal(t, uint64(48), atomic.LoadUint64(doneIters)) - assert.Equal(t, [][]int64{{2, 10}, {4, 10}, {8, 20}, {4, 10}, {0, 0}}, resultVUCount) + assert.Equal(t, [][]int64{{2, 10}, {4, 10}, {8, 20}, {4, 10}, {0, 10}}, resultVUCount) } From d84fe0ee2c5f64b2a5db43e14b6d85daa96ef7e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 30 Jun 2020 11:05:10 +0200 Subject: [PATCH 321/350] Minor type cleanup in TestExternallyControlledRun Resolves https://github.com/loadimpact/k6/pull/1517#discussion_r447524462 --- lib/executor/externally_controlled_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/lib/executor/externally_controlled_test.go b/lib/executor/externally_controlled_test.go index 30f3b8aeeb4..ed39557f701 100644 --- a/lib/executor/externally_controlled_test.go +++ b/lib/executor/externally_controlled_test.go @@ -77,10 +77,10 @@ func TestExternallyControlledRun(t *testing.T) { close(doneCh) }() - updateConfig := func(vus, maxVUs int, errMsg string) { + updateConfig := func(vus, maxVUs int64, errMsg string) { newConfig := ExternallyControlledConfigParams{ - VUs: null.IntFrom(int64(vus)), - MaxVUs: null.IntFrom(int64(maxVUs)), + VUs: null.IntFrom(vus), + MaxVUs: null.IntFrom(maxVUs), Duration: types.NullDurationFrom(2 * time.Second), } err := executor.(*ExternallyControlled).UpdateConfig(ctx, newConfig) From 61158c85904f13aab5a3a81f85bed5ba7bb22de8 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 30 Jun 2020 13:48:18 +0300 Subject: [PATCH 322/350] Reuse the context and state in js.VU as much as possible (#1478) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This took way longer then I anticipated mostly because of ... how strange some things work. The basic idea came from that for each iteration we constantly remake the iterations which looked (and probably is) the lowest hanging fruit in k6 about creating too much objects. Unfortunately because of how .. tags work and that iteration is a tag I need to still copy the tags for GetTrail .. (it gets copied either way for other metrics). When running with a empty body did this change the amount of cpu taken by scanobject and decreased it from 7.98s to 5.81s (while most other not GC related had difference withing 0.3s). This is very small step ... having tags being immutable so we don't have to constantly copy them will likely be much more significant or if we can reduce the calls to runtime.nanotime which are literally now the top CPU usage with empty iterations and are the top CPU usage for simple scripts even before that :D. Comparing before reusing and after: name old time/op new time/op delta EmptyIteration-8 11.5µs ± 1% 7.5µs ± 2% -34.72% (p=0.000 n=9+10) HTTPRequests-8 632µs ± 1% 624µs ± 2% -1.23% (p=0.002 n=9+9) HTTPRequestsBase-8 633µs ± 2% 627µs ± 1% ~ (p=0.075 n=10+10) name old alloc/op new alloc/op delta EmptyIteration-8 1.31kB ± 0% 0.53kB ± 0% -59.81% (p=0.000 n=10+8) HTTPRequests-8 27.6kB ± 0% 26.8kB ± 0% -2.84% (p=0.000 n=10+10) HTTPRequestsBase-8 27.6kB ± 0% 26.8kB ± 0% -2.81% (p=0.000 n=9+8) name old allocs/op new allocs/op delta EmptyIteration-8 11.0 ± 0% 7.0 ± 0% -36.36% (p=0.000 n=10+10) HTTPRequests-8 365 ± 0% 361 ± 0% -1.10% (p=0.000 n=10+10) HTTPRequestsBase-8 365 ± 0% 361 ± 0% -1.10% (p=0.000 n=10+10) --- js/empty_iteartions_bench_test.go | 40 ++++++++++++ js/http_bench_test.go | 8 +-- js/runner.go | 101 +++++++++++++++++------------- 3 files changed, 102 insertions(+), 47 deletions(-) create mode 100644 js/empty_iteartions_bench_test.go diff --git a/js/empty_iteartions_bench_test.go b/js/empty_iteartions_bench_test.go new file mode 100644 index 00000000000..a38a91db218 --- /dev/null +++ b/js/empty_iteartions_bench_test.go @@ -0,0 +1,40 @@ +package js + +import ( + "context" + "testing" + + "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func BenchmarkEmptyIteration(b *testing.B) { + b.StopTimer() + + r, err := getSimpleRunner("/script.js", `exports.default = function() { }`) + if !assert.NoError(b, err) { + return + } + require.NoError(b, err) + + var ch = make(chan stats.SampleContainer, 100) + defer close(ch) + go func() { // read the channel so it doesn't block + for range ch { + } + }() + initVU, err := r.NewVU(1, ch) + if !assert.NoError(b, err) { + return + } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + vu := initVU.Activate(&lib.VUActivationParams{RunContext: ctx}) + b.StartTimer() + for i := 0; i < b.N; i++ { + err = vu.RunOnce() + assert.NoError(b, err) + } +} diff --git a/js/http_bench_test.go b/js/http_bench_test.go index 3bf41d1009f..e3a828fb6fc 100644 --- a/js/http_bench_test.go +++ b/js/http_bench_test.go @@ -58,9 +58,9 @@ func BenchmarkHTTPRequests(b *testing.B) { require.NoError(b, err) var ch = make(chan stats.SampleContainer, 100) + defer close(ch) go func() { // read the channel so it doesn't block - for { - <-ch + for range ch { } }() initVU, err := r.NewVU(1, ch) @@ -102,9 +102,9 @@ func BenchmarkHTTPRequestsBase(b *testing.B) { require.NoError(b, err) var ch = make(chan stats.SampleContainer, 100) + defer close(ch) go func() { // read the channel so it doesn't block - for { - <-ch + for range ch { } }() initVU, err := r.NewVU(1, ch) diff --git a/js/runner.go b/js/runner.go index 04df47e63d9..ce6c80d4060 100644 --- a/js/runner.go +++ b/js/runner.go @@ -195,6 +195,22 @@ func (r *Runner) newVU(id int64, samplesOut chan<- stats.SampleContainer) (*VU, BPool: bpool.NewBufferPool(100), Samples: samplesOut, } + + vu.state = &lib.State{ + Logger: vu.Runner.Logger, + Options: vu.Runner.Bundle.Options, + Transport: vu.Transport, + Dialer: vu.Dialer, + TLSConfig: vu.TLSConfig, + CookieJar: cookieJar, + RPSLimit: vu.Runner.RPSLimit, + BPool: vu.BPool, + Vu: vu.ID, + Samples: vu.Samples, + Iteration: vu.Iteration, + Tags: vu.Runner.Bundle.Options.RunTags.CloneTags(), + Group: r.defaultGroup, + } vu.Runtime.Set("__VU", vu.ID) vu.Runtime.Set("console", common.Bind(vu.Runtime, vu.Console, vu.Context)) @@ -316,19 +332,27 @@ func (r *Runner) runPart(ctx context.Context, out chan<- stats.SampleContainer, return goja.Undefined(), nil } + ctx = common.WithRuntime(ctx, vu.Runtime) + ctx = lib.WithState(ctx, vu.state) ctx, cancel := context.WithCancel(ctx) defer cancel() go func() { <-ctx.Done() vu.Runtime.Interrupt(errInterrupt) }() + *vu.Context = ctx group, err := lib.NewGroup(name, r.GetDefaultGroup()) if err != nil { return goja.Undefined(), err } - v, _, _, err := vu.runFn(ctx, "", group, false, nil, fn, vu.Runtime.ToValue(arg)) + if r.Bundle.Options.SystemTags.Has(stats.TagGroup) { + vu.state.Tags["group"] = group.Path + } + vu.state.Group = group + + v, _, _, err := vu.runFn(ctx, false, fn, vu.Runtime.ToValue(arg)) // deadline is reached so we have timeouted but this might've not been registered correctly if deadline, ok := ctx.Deadline(); ok && time.Now().After(deadline) { @@ -372,6 +396,8 @@ type VU struct { Samples chan<- stats.SampleContainer setupData goja.Value + + state *lib.State } // Verify that interfaces are implemented @@ -405,6 +431,29 @@ func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { } u.Runtime.Set("__ENV", env) + opts := u.Runner.Bundle.Options + // TODO: maybe we can cache the original tags only clone them and add (if any) new tags on top ? + u.state.Tags = opts.RunTags.CloneTags() + for k, v := range params.Tags { + u.state.Tags[k] = v + } + if opts.SystemTags.Has(stats.TagVU) { + u.state.Tags["vu"] = strconv.FormatInt(u.ID, 10) + } + if opts.SystemTags.Has(stats.TagIter) { + u.state.Tags["iter"] = strconv.FormatInt(u.Iteration, 10) + } + if opts.SystemTags.Has(stats.TagGroup) { + u.state.Tags["group"] = u.state.Group.Path + } + if opts.SystemTags.Has(stats.TagScenario) { + u.state.Tags["scenario"] = params.Scenario + } + + params.RunContext = common.WithRuntime(params.RunContext, u.Runtime) + params.RunContext = lib.WithState(params.RunContext, u.state) + *u.Context = params.RunContext + avu := &ActiveVU{ VU: u, VUActivationParams: params, @@ -461,9 +510,7 @@ func (u *ActiveVU) RunOnce() error { } // Call the exported function. - _, isFullIteration, totalTime, err := u.runFn( - u.RunContext, u.Scenario, u.Runner.defaultGroup, true, u.Tags, fn, u.setupData, - ) + _, isFullIteration, totalTime, err := u.runFn(u.RunContext, true, fn, u.setupData) // If MinIterationDuration is specified and the iteration wasn't cancelled // and was less than it, sleep for the remainder @@ -478,56 +525,24 @@ func (u *ActiveVU) RunOnce() error { } func (u *VU) runFn( - ctx context.Context, scenario string, group *lib.Group, isDefault bool, - customTags map[string]string, fn goja.Callable, args ...goja.Value, + ctx context.Context, isDefault bool, fn goja.Callable, args ...goja.Value, ) (goja.Value, bool, time.Duration, error) { - cookieJar := u.CookieJar if !u.Runner.Bundle.Options.NoCookiesReset.ValueOrZero() { var err error - cookieJar, err = cookiejar.New(nil) + u.state.CookieJar, err = cookiejar.New(nil) if err != nil { return goja.Undefined(), false, time.Duration(0), err } } opts := &u.Runner.Bundle.Options - tags := opts.RunTags.CloneTags() - for k, v := range customTags { - tags[k] = v - } - if opts.SystemTags.Has(stats.TagVU) { - tags["vu"] = strconv.FormatInt(u.ID, 10) - } if opts.SystemTags.Has(stats.TagIter) { - tags["iter"] = strconv.FormatInt(u.Iteration, 10) + u.state.Tags["iter"] = strconv.FormatInt(u.Iteration, 10) } - if opts.SystemTags.Has(stats.TagGroup) { - tags["group"] = group.Path - } - if scenario != "" && opts.SystemTags.Has(stats.TagScenario) { - tags["scenario"] = scenario - } - - state := &lib.State{ - Logger: u.Runner.Logger, - Options: u.Runner.Bundle.Options, - Group: group, - Transport: u.Transport, - Dialer: u.Dialer, - TLSConfig: u.TLSConfig, - CookieJar: cookieJar, - RPSLimit: u.Runner.RPSLimit, - BPool: u.BPool, - Vu: u.ID, - Samples: u.Samples, - Iteration: u.Iteration, - Tags: tags, - } - - newctx := common.WithRuntime(ctx, u.Runtime) - newctx = lib.WithState(newctx, state) - *u.Context = newctx + // TODO: this seems like the wrong place for the iteration incrementation + // also this means that teardown and setup have __ITER defined + // maybe move it to RunOnce ? u.Runtime.Set("__ITER", u.Iteration) u.Iteration++ @@ -547,7 +562,7 @@ func (u *VU) runFn( u.Transport.CloseIdleConnections() } - state.Samples <- u.Dialer.GetTrail(startTime, endTime, isFullIteration, isDefault, stats.IntoSampleTags(&tags)) + u.state.Samples <- u.Dialer.GetTrail(startTime, endTime, isFullIteration, isDefault, stats.NewSampleTags(u.state.Tags)) return v, isFullIteration, endTime.Sub(startTime), err } From e2dd28236cd0f448c1b26797012e1d9f90bd9ef1 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Tue, 30 Jun 2020 16:53:53 +0300 Subject: [PATCH 323/350] Fix startTime>0 messing up the externally-controlled executor --- lib/executor/externally_controlled.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index 52e4c252d8e..d6e733bf035 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -315,9 +315,11 @@ func (mex *ExternallyControlled) stopWhenDurationIsReached(ctx context.Context, checkInterval.Stop() return - // TODO: something more optimized that sleeps for pauses? + // TODO: something saner and more optimized that sleeps for pauses and + // doesn't depend on the global execution state? case <-checkInterval.C: - if mex.executionState.GetCurrentTestRunDuration() >= duration { + elapsed := mex.executionState.GetCurrentTestRunDuration() - time.Duration(mex.config.StartTime.Duration) + if elapsed >= duration { cancel() return } @@ -408,17 +410,20 @@ func (rs *externallyControlledRunState) progresFn() (float64, []string) { right := []string{progVUs, rs.duration.String(), ""} - spent := rs.executor.executionState.GetCurrentTestRunDuration() - if spent > rs.duration { + // TODO: use a saner way to calculate the elapsed time, without relying on + // the global execution state... + elapsed := rs.executor.executionState.GetCurrentTestRunDuration() - time.Duration( + rs.executor.config.StartTime.Duration) + if elapsed > rs.duration { return 1, right } progress := 0.0 if rs.duration > 0 { - progress = math.Min(1, float64(spent)/float64(rs.duration)) + progress = math.Min(1, float64(elapsed)/float64(rs.duration)) } - spentDuration := pb.GetFixedLengthDuration(spent, rs.duration) + spentDuration := pb.GetFixedLengthDuration(elapsed, rs.duration) progDur := fmt.Sprintf("%s/%s", spentDuration, rs.duration) right[1] = progDur From 15c100047b76facb9a3ba08876feaf4dd698d6f3 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 30 Jun 2020 17:09:00 +0300 Subject: [PATCH 324/350] chore: drop workaround for pre go1.13 fixes #1165 --- cmd/configdir_go112.go | 69 ------------------------------------------ cmd/configdir_go113.go | 29 ------------------ cmd/root.go | 2 +- 3 files changed, 1 insertion(+), 99 deletions(-) delete mode 100644 cmd/configdir_go112.go delete mode 100644 cmd/configdir_go113.go diff --git a/cmd/configdir_go112.go b/cmd/configdir_go112.go deleted file mode 100644 index 4b1eae33f2b..00000000000 --- a/cmd/configdir_go112.go +++ /dev/null @@ -1,69 +0,0 @@ -// +build !go1.13 - -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2019 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package cmd - -import ( - "errors" - "os" - "runtime" -) - -// This code is copied from os.UserConfigDir() as of go1.13 -// https://github.com/golang/go/blob/release-branch.go1.13/src/os/file.go#L419 -func configDir() (string, error) { - var dir string - - switch runtime.GOOS { - case "windows": - dir = os.Getenv("AppData") - if dir == "" { - return "", errors.New("%AppData% is not defined") - } - - case "darwin": - dir = os.Getenv("HOME") - if dir == "" { - return "", errors.New("$HOME is not defined") - } - dir += "/Library/Application Support" - - case "plan9": - dir = os.Getenv("home") - if dir == "" { - return "", errors.New("$home is not defined") - } - dir += "/lib" - - default: // Unix - dir = os.Getenv("XDG_CONFIG_HOME") - if dir == "" { - dir = os.Getenv("HOME") - if dir == "" { - return "", errors.New("neither $XDG_CONFIG_HOME nor $HOME are defined") - } - dir += "/.config" - } - } - - return dir, nil -} diff --git a/cmd/configdir_go113.go b/cmd/configdir_go113.go deleted file mode 100644 index 66281fcddf5..00000000000 --- a/cmd/configdir_go113.go +++ /dev/null @@ -1,29 +0,0 @@ -// +build go1.13 - -/* - * - * k6 - a next-generation load testing tool - * Copyright (C) 2019 Load Impact - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as - * published by the Free Software Foundation, either version 3 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - */ - -package cmd - -import "os" - -func configDir() (string, error) { - return os.UserConfigDir() -} diff --git a/cmd/root.go b/cmd/root.go index e2806d0e9f4..1929546fd06 100644 --- a/cmd/root.go +++ b/cmd/root.go @@ -133,7 +133,7 @@ func rootCmdPersistentFlagSet() *pflag.FlagSet { } func init() { - confDir, err := configDir() + confDir, err := os.UserConfigDir() if err != nil { logrus.WithError(err).Warn("could not get config directory") confDir = ".config" From ecf5e1ad1b7a4f3ff328f0127c6b95517146a697 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 30 Jun 2020 18:39:46 +0300 Subject: [PATCH 325/350] Make MaxVUs equal to preallocatedVUs when missing for arrival rate executors (#1525) fixes #1317 --- lib/executor/constant_arrival_rate.go | 11 ++++++----- lib/executor/constant_arrival_rate_test.go | 4 ++-- lib/executor/executors_test.go | 14 ++++++++++++-- lib/executor/ramping_arrival_rate.go | 11 ++++++----- lib/executor/ramping_arrival_rate_test.go | 6 +++--- 5 files changed, 29 insertions(+), 17 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 45a2699b5e3..17873fcacec 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -66,8 +66,8 @@ type ConstantArrivalRateConfig struct { } // NewConstantArrivalRateConfig returns a ConstantArrivalRateConfig with default values -func NewConstantArrivalRateConfig(name string) ConstantArrivalRateConfig { - return ConstantArrivalRateConfig{ +func NewConstantArrivalRateConfig(name string) *ConstantArrivalRateConfig { + return &ConstantArrivalRateConfig{ BaseConfig: NewBaseConfig(name, constantArrivalRateType), TimeUnit: types.NewNullDuration(1*time.Second, false), } @@ -108,7 +108,7 @@ func (carc ConstantArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) str } // Validate makes sure all options are configured and valid -func (carc ConstantArrivalRateConfig) Validate() []error { +func (carc *ConstantArrivalRateConfig) Validate() []error { errors := carc.BaseConfig.Validate() if !carc.Rate.Valid { errors = append(errors, fmt.Errorf("the iteration rate isn't specified")) @@ -135,7 +135,8 @@ func (carc ConstantArrivalRateConfig) Validate() []error { } if !carc.MaxVUs.Valid { - errors = append(errors, fmt.Errorf("the number of maxVUs isn't specified")) + // TODO: don't change the config while validating + carc.MaxVUs.Int64 = carc.PreAllocatedVUs.Int64 } else if carc.MaxVUs.Int64 < carc.PreAllocatedVUs.Int64 { errors = append(errors, fmt.Errorf("maxVUs shouldn't be less than preAllocatedVUs")) } @@ -167,7 +168,7 @@ func (carc ConstantArrivalRateConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { return &ConstantArrivalRate{ - BaseExecutor: NewBaseExecutor(carc, es, logger), + BaseExecutor: NewBaseExecutor(&carc, es, logger), config: carc, }, nil } diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 5f3cc695716..84c0071e585 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -54,8 +54,8 @@ func newExecutionSegmentSequenceFromString(str string) *lib.ExecutionSegmentSequ return &r } -func getTestConstantArrivalRateConfig() ConstantArrivalRateConfig { - return ConstantArrivalRateConfig{ +func getTestConstantArrivalRateConfig() *ConstantArrivalRateConfig { + return &ConstantArrivalRateConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), Rate: null.IntFrom(50), diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 9485fa5adf4..222ccda8e02 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -349,7 +349,12 @@ var configMapTestCases = []configMapTestCase{ }, {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{}}, {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30, "timeUnit": "-1s"}}`, exp{validationError: true}}, - {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, exp{validationError: true}}, + {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "preAllocatedVUs": 20}}`, + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { + assert.Empty(t, cm["carrival"].Validate()) + require.EqualValues(t, 20, cm["carrival"].(*ConstantArrivalRateConfig).MaxVUs.Int64) + }}, + }, {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "duration": "10m", "maxVUs": 30}}`, exp{validationError: true}}, {`{"carrival": {"executor": "constant-arrival-rate", "rate": 10, "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, {`{"carrival": {"executor": "constant-arrival-rate", "duration": "10m", "preAllocatedVUs": 20, "maxVUs": 30}}`, exp{validationError: true}}, @@ -394,7 +399,12 @@ var configMapTestCases = []configMapTestCase{ {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{}}, {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": -20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, {`{"varrival": {"executor": "ramping-arrival-rate", "startRate": -1, "preAllocatedVUs": 20, "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, - {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, + {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "stages": [{"duration": "5m", "target": 10}]}}`, + exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { + assert.Empty(t, cm["varrival"].Validate()) + require.EqualValues(t, 20, cm["varrival"].(*RampingArrivalRateConfig).MaxVUs.Int64) + }}, + }, {`{"varrival": {"executor": "ramping-arrival-rate", "maxVUs": 50, "stages": [{"duration": "5m", "target": 10}]}}`, exp{validationError: true}}, {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50}}`, exp{validationError: true}}, {`{"varrival": {"executor": "ramping-arrival-rate", "preAllocatedVUs": 20, "maxVUs": 50, "stages": []}}`, exp{validationError: true}}, diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 5d763567f43..eb18b331be3 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -66,8 +66,8 @@ type RampingArrivalRateConfig struct { } // NewRampingArrivalRateConfig returns a RampingArrivalRateConfig with default values -func NewRampingArrivalRateConfig(name string) RampingArrivalRateConfig { - return RampingArrivalRateConfig{ +func NewRampingArrivalRateConfig(name string) *RampingArrivalRateConfig { + return &RampingArrivalRateConfig{ BaseConfig: NewBaseConfig(name, rampingArrivalRateType), TimeUnit: types.NewNullDuration(1*time.Second, false), } @@ -104,7 +104,7 @@ func (varc RampingArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) stri } // Validate makes sure all options are configured and valid -func (varc RampingArrivalRateConfig) Validate() []error { +func (varc *RampingArrivalRateConfig) Validate() []error { errors := varc.BaseConfig.Validate() if varc.StartRate.Int64 < 0 { @@ -124,7 +124,8 @@ func (varc RampingArrivalRateConfig) Validate() []error { } if !varc.MaxVUs.Valid { - errors = append(errors, fmt.Errorf("the number of maxVUs isn't specified")) + // TODO: don't change the config while validating + varc.MaxVUs.Int64 = varc.PreAllocatedVUs.Int64 } else if varc.MaxVUs.Int64 < varc.PreAllocatedVUs.Int64 { errors = append(errors, fmt.Errorf("maxVUs shouldn't be less than preAllocatedVUs")) } @@ -157,7 +158,7 @@ func (varc RampingArrivalRateConfig) NewExecutor( es *lib.ExecutionState, logger *logrus.Entry, ) (lib.Executor, error) { return RampingArrivalRate{ - BaseExecutor: NewBaseExecutor(varc, es, logger), + BaseExecutor: NewBaseExecutor(&varc, es, logger), config: varc, }, nil } diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 7286aeab2d7..8ff06c31288 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -39,8 +39,8 @@ import ( "github.com/loadimpact/k6/stats" ) -func getTestRampingArrivalRateConfig() RampingArrivalRateConfig { - return RampingArrivalRateConfig{ +func getTestRampingArrivalRateConfig() *RampingArrivalRateConfig { + return &RampingArrivalRateConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(10), @@ -140,7 +140,7 @@ func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6, } ctx, cancel, executor, logHook := setupExecutor( - t, RampingArrivalRateConfig{ + t, &RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Stages: []Stage{ { From 447f892cc2c4073290e25b220307851d14aaae95 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Wed, 1 Jul 2020 10:06:35 +0300 Subject: [PATCH 326/350] Tag the scenario logs properly --- core/local/local.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/local/local.go b/core/local/local.go index 0fc80a993aa..ed1d9d66232 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -80,7 +80,10 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution ) continue } - s, err := sc.NewExecutor(executionState, logger.WithField("executor", sc.GetName())) + s, err := sc.NewExecutor(executionState, logger.WithFields(logrus.Fields{ + "scenario": sc.GetName(), + "executor": sc.GetType(), + })) if err != nil { return nil, err } From 94b88a0a927faf06bdd13b0822bf550a3a1bcdb2 Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Tue, 5 May 2020 15:57:52 +0300 Subject: [PATCH 327/350] VariableArrivalRate: don't block on unplannedVU init if possible This make it so that if an unplannedVU needs to be started we wait for ANY free VU instead of only for that unplannedVU to start the next iteration. Also puts in place some guards against initing multiple unplannedVUs --- lib/executor/ramping_arrival_rate.go | 26 ++++++++-- lib/executor/ramping_arrival_rate_test.go | 58 +++++++++++++++++++++++ 2 files changed, 79 insertions(+), 5 deletions(-) diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index eb18b331be3..69a4228f9c7 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -382,6 +382,19 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } remainingUnplannedVUs := maxVUs - preAllocatedVUs + makeUnplannedVUCh := make(chan struct{}) + defer close(makeUnplannedVUCh) + + go func() { + for range makeUnplannedVUCh { + initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) + if err != nil { + // TODO figure out how to return it to the Run goroutine + varr.logger.WithError(err).Error("Error while allocating unplanned VU") + } + activeVUs <- activateVU(initVU) + } + }() timer := time.NewTimer(time.Hour) start := time.Now() @@ -416,12 +429,15 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) continue } - initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) - if err != nil { - return err + + select { + case makeUnplannedVUCh <- struct{}{}: + // this is the only goroutine that touches remainingUnplannedVUs and if we didn't + // send on the channel no new unplannedVU will be stared so no need to decrease it + remainingUnplannedVUs-- + vu = <-activeVUs // just get any VU that gets activated, whether it is the unplanned or not doesn't matter + case vu = <-activeVUs: // a VU got freed while were waiting to start a new unplanned one } - vu = activateVU(initVU) - remainingUnplannedVUs-- } go runIteration(vu) } diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 8ff06c31288..808daa7f9c6 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -129,6 +129,64 @@ func TestRampingArrivalRateRunCorrectRate(t *testing.T) { require.Empty(t, logHook.Drain()) } +func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { + t.Parallel() + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + var count int64 + var ch = make(chan struct{}) // closed when new unplannedVU is started and signal to get to next iterations + var ch2 = make(chan struct{}) // closed when a second iteration was started on an old VU in order to test it won't start a second unplanned VU in parallel or at all + runner := simpleRunner(func(ctx context.Context) error { + cur := atomic.AddInt64(&count, 1) + if cur == 1 { + <-ch // wait to start again + } else if cur == 2 { + <-ch2 // wait to start again + } + + return nil + }) + var ctx, cancel, executor, logHook = setupExecutor( + t, RampingArrivalRateConfig{ + TimeUnit: types.NullDurationFrom(time.Second), + Stages: []Stage{ + { + Duration: types.NullDurationFrom(time.Second * 2), + Target: null.IntFrom(10), + }, + }, + PreAllocatedVUs: null.IntFrom(1), + MaxVUs: null.IntFrom(3), + }, + es, runner) + defer cancel() + var engineOut = make(chan stats.SampleContainer, 1000) + es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { + cur := atomic.LoadInt64(&count) + require.Equal(t, cur, int64(1)) + time.Sleep(time.Second / 2) + close(ch) + time.Sleep(time.Millisecond * 10) + + cur = atomic.LoadInt64(&count) + require.Equal(t, cur, int64(2)) + time.Sleep(time.Millisecond * 10) + + cur = atomic.LoadInt64(&count) + require.Equal(t, cur, int64(2)) + close(ch2) + time.Sleep(time.Millisecond * 10) + cur = atomic.LoadInt64(&count) + require.NotEqual(t, cur, int64(2)) + return runner.NewVU(int64(es.GetUniqueVUIdentifier()), engineOut) + }) + err = executor.Run(ctx, engineOut) + assert.NoError(t, err) + assert.Empty(t, logHook.Drain()) + assert.Equal(t, count, int64(9)) +} + func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { t.Parallel() var count int64 From 6a23e8654c6d44d0c7d8bd0f5434a2615bf1731a Mon Sep 17 00:00:00 2001 From: Mihail Stoykov Date: Thu, 7 May 2020 16:49:28 +0300 Subject: [PATCH 328/350] Make certain no unitilized VU will be left unreturned --- lib/executor/common_test.go | 6 +- lib/executor/ramping_arrival_rate.go | 59 +++++++++-------- lib/executor/ramping_arrival_rate_test.go | 79 +++++++++++------------ 3 files changed, 74 insertions(+), 70 deletions(-) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 652c994944e..0996a83c36f 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -62,8 +62,8 @@ func setupExecutor(t *testing.T, config lib.ExecutorConfig, es *lib.ExecutionSta et, err := lib.NewExecutionTuple(es.Options.ExecutionSegment, es.Options.ExecutionSegmentSequence) require.NoError(t, err) - maxVUs := lib.GetMaxPossibleVUs(config.GetExecutionRequirements(et)) - initializeVUs(ctx, t, logEntry, es, maxVUs, initVUFunc) + maxPlannedVUs := lib.GetMaxPlannedVUs(config.GetExecutionRequirements(et)) + initializeVUs(ctx, t, logEntry, es, maxPlannedVUs, initVUFunc) executor, err := config.NewExecutor(es, logEntry) require.NoError(t, err) @@ -82,6 +82,6 @@ func initializeVUs( // which is done in es.AddInitializedVU(). vu, err := initVU(ctx, logEntry) require.NoError(t, err) - es.AddInitializedVU(vu) + es.ReturnVU(vu, false) } } diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 69a4228f9c7..09432365b7e 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -280,7 +280,8 @@ func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time. } // Run executes a variable number of iterations per second. -func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen +//nolint:funlen,gocognit +func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) @@ -301,21 +302,20 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC }).Debug("Starting executor run...") activeVUsWg := &sync.WaitGroup{} - defer activeVUsWg.Wait() + returnedVUs := make(chan struct{}) startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) - defer cancel() - // Pre-allocate the VUs local shared buffer - activeVUs := make(chan lib.ActiveVU, maxVUs) - activeVUsCount := uint64(0) - // Make sure all VUs aren't executing iterations anymore, for the cancel() - // above to deactivate them. + defer func() { - for i := uint64(0); i < activeVUsCount; i++ { - <-activeVUs - } + // Make sure all VUs aren't executing iterations anymore, for the cancel() + // above to deactivate them. + <-returnedVUs + cancel() + activeVUsWg.Wait() }() + activeVUs := make(chan lib.ActiveVU, maxVUs) + activeVUsCount := uint64(0) activationParams := getVUActivationParams(maxDurationCtx, varr.config.BaseConfig, func(u lib.InitializedVU) { @@ -330,6 +330,28 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC return activeVU } + remainingUnplannedVUs := maxVUs - preAllocatedVUs + makeUnplannedVUCh := make(chan struct{}) + defer close(makeUnplannedVUCh) + go func() { + defer close(returnedVUs) + defer func() { + // this is done here as to not have an unplannedVU in the middle of initialization when + // starting to return activeVUs + for i := uint64(0); i < atomic.LoadUint64(&activeVUsCount); i++ { + <-activeVUs + } + }() + for range makeUnplannedVUCh { + initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) + if err != nil { + // TODO figure out how to return it to the Run goroutine + varr.logger.WithError(err).Error("Error while allocating unplanned VU") + } + activeVUs <- activateVU(initVU) + } + }() + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { initVU, err := varr.executionState.GetPlannedVU(varr.logger, false) @@ -381,21 +403,6 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC activeVUs <- vu } - remainingUnplannedVUs := maxVUs - preAllocatedVUs - makeUnplannedVUCh := make(chan struct{}) - defer close(makeUnplannedVUCh) - - go func() { - for range makeUnplannedVUCh { - initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) - if err != nil { - // TODO figure out how to return it to the Run goroutine - varr.logger.WithError(err).Error("Error while allocating unplanned VU") - } - activeVUs <- activateVU(initVU) - } - }() - timer := time.NewTimer(time.Hour) start := time.Now() ch := make(chan time.Duration, 10) // buffer 10 iteration times ahead diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 808daa7f9c6..3e281c77212 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -148,11 +148,13 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { return nil }) var ctx, cancel, executor, logHook = setupExecutor( - t, RampingArrivalRateConfig{ + t, &RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Stages: []Stage{ { - Duration: types.NullDurationFrom(time.Second * 2), + // the minus one makes it so only 9 iterations will be started instead of 10 + // as the 10th happens to be just at the end and sometimes doesn't get executed :( + Duration: types.NullDurationFrom(time.Second*2 - 1), Target: null.IntFrom(10), }, }, @@ -189,56 +191,51 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { t.Parallel() - var count int64 - now := time.Now() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) - es := lib.NewExecutionState(lib.Options{}, et, 10, 50) - expectedTimes := []time.Duration{ - time.Millisecond * 3464, time.Millisecond * 4898, time.Second * 6, - } - ctx, cancel, executor, logHook := setupExecutor( + es := lib.NewExecutionState(lib.Options{}, et, 1, 2) + var count int64 + var ch = make(chan struct{}) // closed when new unplannedVU is started and signal to get to next iterations + runner := simpleRunner(func(ctx context.Context) error { + cur := atomic.AddInt64(&count, 1) + if cur == 1 { + <-ch // wait to start again + } + + return nil + }) + var ctx, cancel, executor, logHook = setupExecutor( t, &RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), Stages: []Stage{ { - Duration: types.NullDurationFrom(time.Second * 6), - Target: null.IntFrom(1), - }, - { - Duration: types.NullDurationFrom(time.Second * 0), - Target: null.IntFrom(0), - }, - { - Duration: types.NullDurationFrom(time.Second * 1), - Target: null.IntFrom(0), + Duration: types.NullDurationFrom(time.Second * 2), + Target: null.IntFrom(10), }, }, - PreAllocatedVUs: null.IntFrom(10), - MaxVUs: null.IntFrom(20), + PreAllocatedVUs: null.IntFrom(1), + MaxVUs: null.IntFrom(3), }, - es, - simpleRunner(func(ctx context.Context) error { - current := atomic.AddInt64(&count, 1) - if !assert.True(t, int(current) <= len(expectedTimes)) { - return nil - } - expectedTime := expectedTimes[current-1] - assert.WithinDuration(t, - now.Add(expectedTime), - time.Now(), - time.Millisecond*100, - "%d expectedTime %s", current, expectedTime, - ) - return nil - }), - ) + es, runner) defer cancel() - engineOut := make(chan stats.SampleContainer, 1000) + var engineOut = make(chan stats.SampleContainer, 1000) + es.SetInitVUFunc(func(_ context.Context, logger *logrus.Entry) (lib.InitializedVU, error) { + t.Log("init") + cur := atomic.LoadInt64(&count) + require.Equal(t, cur, int64(1)) + time.Sleep(time.Millisecond * 200) + close(ch) + time.Sleep(time.Millisecond * 20) + cur = atomic.LoadInt64(&count) + require.NotEqual(t, cur, int64(1)) + + return runner.NewVU(int64(es.GetUniqueVUIdentifier()), engineOut) + }) err = executor.Run(ctx, engineOut) - require.NoError(t, err) - require.Equal(t, int64(len(expectedTimes)), count) - require.Empty(t, logHook.Drain()) + assert.NoError(t, err) + assert.Empty(t, logHook.Drain()) + assert.Equal(t, int64(0), es.GetCurrentlyActiveVUsCount()) + assert.Equal(t, int64(2), es.GetInitializedVUsCount()) } func mustNewExecutionTuple(seg *lib.ExecutionSegment, seq *lib.ExecutionSegmentSequence) *lib.ExecutionTuple { From 502a550c7a7328f04f6810dceefb8f7b3decb8c8 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 29 Jun 2020 13:11:22 +0300 Subject: [PATCH 329/350] Fix ramping VU initialization --- lib/executor/common_test.go | 2 +- lib/executor/ramping_arrival_rate.go | 40 +++++++++++++---------- lib/executor/ramping_arrival_rate_test.go | 2 +- 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/lib/executor/common_test.go b/lib/executor/common_test.go index 0996a83c36f..280ac996e7e 100644 --- a/lib/executor/common_test.go +++ b/lib/executor/common_test.go @@ -82,6 +82,6 @@ func initializeVUs( // which is done in es.AddInitializedVU(). vu, err := initVU(ctx, logEntry) require.NoError(t, err) - es.ReturnVU(vu, false) + es.AddInitializedVU(vu) } } diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 09432365b7e..4cf290b0162 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -305,11 +305,10 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC returnedVUs := make(chan struct{}) startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) - // Pre-allocate the VUs local shared buffer defer func() { // Make sure all VUs aren't executing iterations anymore, for the cancel() - // above to deactivate them. + // below to deactivate them. <-returnedVUs cancel() activeVUsWg.Wait() @@ -407,6 +406,7 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC start := time.Now() ch := make(chan time.Duration, 10) // buffer 10 iteration times ahead var prevTime time.Duration + shownWarning := false go varr.config.cal(varr.executionState.ExecutionTuple, ch) for nextTime := range ch { select { @@ -428,25 +428,31 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC var vu lib.ActiveVU select { - case vu = <-activeVUs: - // ideally, we get the VU from the buffer without any issues - default: - if remainingUnplannedVUs == 0 { - // TODO: emit an error metric? + case vu = <-activeVUs: // ideally, we get the VU from the buffer without any issues + go runIteration(vu) + continue + default: // no free VUs currently available + } + // Since there aren't any free VUs available, consider this iteration + // dropped - we aren't going to try to recover it, but + + // TODO: emit a dropped_iteration metric + + // We'll try to start allocating another VU in the background, + // non-blockingly, if we have remainingUnplannedVUs... + if remainingUnplannedVUs == 0 { + if !shownWarning { varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) - continue + shownWarning = true } + continue + } - select { - case makeUnplannedVUCh <- struct{}{}: - // this is the only goroutine that touches remainingUnplannedVUs and if we didn't - // send on the channel no new unplannedVU will be stared so no need to decrease it - remainingUnplannedVUs-- - vu = <-activeVUs // just get any VU that gets activated, whether it is the unplanned or not doesn't matter - case vu = <-activeVUs: // a VU got freed while were waiting to start a new unplanned one - } + select { + case makeUnplannedVUCh <- struct{}{}: // great! + remainingUnplannedVUs-- + default: // we're already allocating a new VU } - go runIteration(vu) } return nil } diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 3e281c77212..1bf0705c1e9 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -193,7 +193,7 @@ func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { t.Parallel() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) - es := lib.NewExecutionState(lib.Options{}, et, 1, 2) + es := lib.NewExecutionState(lib.Options{}, et, 1, 3) var count int64 var ch = make(chan struct{}) // closed when new unplannedVU is started and signal to get to next iterations runner := simpleRunner(func(ctx context.Context) error { From 223ed80fd5df6b74b93bae3dd90144e5c574873c Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 2 Jul 2020 12:39:29 +0300 Subject: [PATCH 330/350] Fix tests and constant-arrival-rate VU init --- lib/executor/constant_arrival_rate.go | 92 +++++++++++++++------- lib/executor/constant_arrival_rate_test.go | 2 +- lib/executor/ramping_arrival_rate.go | 20 +++-- lib/executor/ramping_arrival_rate_test.go | 17 ++-- 4 files changed, 91 insertions(+), 40 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 17873fcacec..98a869b2cb7 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -200,7 +200,14 @@ func (car *ConstantArrivalRate) Init(ctx context.Context) error { // Run executes a constant number of iterations per second. // -// TODO: Reuse the variable arrival rate method? +// TODO: Split this up and make an independent component that can be reused +// between the constant and ramping arrival rate executors - that way we can +// keep the complexity in one well-architected part (with short methods and few +// lambdas :D), while having both config frontends still be present for maximum +// UX benefits. Basically, keep the progress bars and scheduling (i.e. at what +// time should iteration X begin) different, but keep everyhing else the same. +// This will allow us to implement https://github.com/loadimpact/k6/issues/1386 +// and things like all of the TODOs below in one place only. func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) @@ -217,24 +224,20 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC "tickerPeriod": tickerPeriod, "type": car.config.GetType(), }).Debug("Starting executor run...") - // Pre-allocate the VUs local shared buffer - activeVUs := make(chan lib.ActiveVU, maxVUs) - activeVUsCount := uint64(0) - activeVUsWg := &sync.WaitGroup{} - defer activeVUsWg.Wait() + returnedVUs := make(chan struct{}) startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) - defer cancel() - // Make sure all VUs aren't executing iterations anymore, for the cancel() - // above to deactivate them. defer func() { - // activeVUsCount is modified only in the loop below, which is done here - for i := uint64(0); i < activeVUsCount; i++ { - <-activeVUs - } + // Make sure all VUs aren't executing iterations anymore, for the cancel() + // below to deactivate them. + <-returnedVUs + cancel() + activeVUsWg.Wait() }() + activeVUs := make(chan lib.ActiveVU, maxVUs) + activeVUsCount := uint64(0) activationParams := getVUActivationParams(maxDurationCtx, car.config.BaseConfig, func(u lib.InitializedVU) { @@ -249,6 +252,30 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC return activeVU } + remainingUnplannedVUs := maxVUs - preAllocatedVUs + makeUnplannedVUCh := make(chan struct{}) + defer close(makeUnplannedVUCh) + go func() { + defer close(returnedVUs) + defer func() { + // this is done here as to not have an unplannedVU in the middle of initialization when + // starting to return activeVUs + for i := uint64(0); i < atomic.LoadUint64(&activeVUsCount); i++ { + <-activeVUs + } + }() + for range makeUnplannedVUCh { + car.logger.Debug("Starting initialization of an unplanned VU...") + initVU, err := car.executionState.GetUnplannedVU(maxDurationCtx, car.logger) + if err != nil { + // TODO figure out how to return it to the Run goroutine + car.logger.WithError(err).Error("Error while allocating unplanned VU") + } + car.logger.Debug("The unplanned VU finished initializing successfully!") + activeVUs <- activateVU(initVU) + } + }() + // Get the pre-allocated VUs in the local buffer for i := int64(0); i < preAllocatedVUs; i++ { initVU, err := car.executionState.GetPlannedVU(car.logger, false) @@ -289,7 +316,6 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC activeVUs <- vu } - remainingUnplannedVUs := maxVUs - preAllocatedVUs start, offsets, _ := car.et.GetStripedOffsets() timer := time.NewTimer(time.Hour * 24) // here the we need the not scaled one @@ -300,28 +326,40 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC int64(time.Duration(car.config.TimeUnit.Duration)), )).Duration) + shownWarning := false for li, gi := 0, start; ; li, gi = li+1, gi+offsets[li%len(offsets)] { t := notScaledTickerPeriod*time.Duration(gi) - time.Since(startTime) timer.Reset(t) select { case <-timer.C: select { - case vu := <-activeVUs: - // ideally, we get the VU from the buffer without any issues - go runIteration(vu) - default: - if remainingUnplannedVUs == 0 { - // TODO: emit an error metric? - car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) - break - } - initVU, err := car.executionState.GetUnplannedVU(maxDurationCtx, car.logger) - if err != nil { - return err + case vu := <-activeVUs: // ideally, we get the VU from the buffer without any issues + go runIteration(vu) //TODO: refactor so we dont spin up a goroutine for each iteration + continue + default: // no free VUs currently available + } + + // Since there aren't any free VUs available, consider this iteration + // dropped - we aren't going to try to recover it, but + + // TODO: emit a dropped_iterations metric + + // We'll try to start allocating another VU in the background, + // non-blockingly, if we have remainingUnplannedVUs... + if remainingUnplannedVUs == 0 { + if !shownWarning { + car.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot initialize more", maxVUs) + shownWarning = true } + continue + } + + select { + case makeUnplannedVUCh <- struct{}{}: // great! remainingUnplannedVUs-- - go runIteration(activateVU(initVU)) + default: // we're already allocating a new VU } + case <-regDurationCtx.Done(): return nil } diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 84c0071e585..36eca62e41a 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -85,7 +85,7 @@ func TestConstantArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { require.NotEmpty(t, entries) for _, entry := range entries { require.Equal(t, - "Insufficient VUs, reached 20 active VUs and cannot allocate more", + "Insufficient VUs, reached 20 active VUs and cannot initialize more", entry.Message) require.Equal(t, logrus.WarnLevel, entry.Level) } diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 4cf290b0162..76a3ef5f488 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -280,6 +280,15 @@ func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time. } // Run executes a variable number of iterations per second. +// +// TODO: Split this up and make an independent component that can be reused +// between the constant and ramping arrival rate executors - that way we can +// keep the complexity in one well-architected part (with short methods and few +// lambdas :D), while having both config frontends still be present for maximum +// UX benefits. Basically, keep the progress bars and scheduling (i.e. at what +// time should iteration X begin) different, but keep everyhing else the same. +// This will allow us to implement https://github.com/loadimpact/k6/issues/1386 +// and things like all of the TODOs below in one place only. //nolint:funlen,gocognit func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executionState.ExecutionTuple.Segment @@ -342,11 +351,13 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } }() for range makeUnplannedVUCh { + varr.logger.Debug("Starting initialization of an unplanned VU...") initVU, err := varr.executionState.GetUnplannedVU(maxDurationCtx, varr.logger) if err != nil { // TODO figure out how to return it to the Run goroutine varr.logger.WithError(err).Error("Error while allocating unplanned VU") } + varr.logger.Debug("The unplanned VU finished initializing successfully!") activeVUs <- activateVU(initVU) } }() @@ -426,23 +437,22 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } } - var vu lib.ActiveVU select { - case vu = <-activeVUs: // ideally, we get the VU from the buffer without any issues - go runIteration(vu) + case vu := <-activeVUs: // ideally, we get the VU from the buffer without any issues + go runIteration(vu) //TODO: refactor so we dont spin up a goroutine for each iteration continue default: // no free VUs currently available } // Since there aren't any free VUs available, consider this iteration // dropped - we aren't going to try to recover it, but - // TODO: emit a dropped_iteration metric + // TODO: emit a dropped_iterations metric // We'll try to start allocating another VU in the background, // non-blockingly, if we have remainingUnplannedVUs... if remainingUnplannedVUs == 0 { if !shownWarning { - varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot allocate more", maxVUs) + varr.logger.Warningf("Insufficient VUs, reached %d active VUs and cannot initialize more", maxVUs) shownWarning = true } continue diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 1bf0705c1e9..9a91a803772 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -83,7 +83,7 @@ func TestRampingArrivalRateRunNotEnoughAllocatedVUsWarn(t *testing.T) { require.NotEmpty(t, entries) for _, entry := range entries { require.Equal(t, - "Insufficient VUs, reached 20 active VUs and cannot allocate more", + "Insufficient VUs, reached 20 active VUs and cannot initialize more", entry.Message) require.Equal(t, logrus.WarnLevel, entry.Level) } @@ -133,7 +133,7 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { t.Parallel() et, err := lib.NewExecutionTuple(nil, nil) require.NoError(t, err) - es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + es := lib.NewExecutionState(lib.Options{}, et, 1, 3) var count int64 var ch = make(chan struct{}) // closed when new unplannedVU is started and signal to get to next iterations var ch2 = make(chan struct{}) // closed when a second iteration was started on an old VU in order to test it won't start a second unplanned VU in parallel or at all @@ -168,17 +168,19 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { cur := atomic.LoadInt64(&count) require.Equal(t, cur, int64(1)) time.Sleep(time.Second / 2) + close(ch) - time.Sleep(time.Millisecond * 10) + time.Sleep(time.Millisecond * 50) cur = atomic.LoadInt64(&count) require.Equal(t, cur, int64(2)) - time.Sleep(time.Millisecond * 10) + time.Sleep(time.Millisecond * 50) cur = atomic.LoadInt64(&count) require.Equal(t, cur, int64(2)) + close(ch2) - time.Sleep(time.Millisecond * 10) + time.Sleep(time.Millisecond * 100) cur = atomic.LoadInt64(&count) require.NotEqual(t, cur, int64(2)) return runner.NewVU(int64(es.GetUniqueVUIdentifier()), engineOut) @@ -186,7 +188,8 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { err = executor.Run(ctx, engineOut) assert.NoError(t, err) assert.Empty(t, logHook.Drain()) - assert.Equal(t, count, int64(9)) + //TODO: test that the sum of dropped_iteartions and count is 9 + // assert.Equal(t, count, int64(9)) } func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { @@ -225,7 +228,7 @@ func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { require.Equal(t, cur, int64(1)) time.Sleep(time.Millisecond * 200) close(ch) - time.Sleep(time.Millisecond * 20) + time.Sleep(time.Millisecond * 100) cur = atomic.LoadInt64(&count) require.NotEqual(t, cur, int64(1)) From 9780e4377e9e1ef1be92ea98d1a2027935d385da Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 2 Jul 2020 15:35:36 +0300 Subject: [PATCH 331/350] Fix a typo and a test --- lib/executor/constant_arrival_rate.go | 2 +- lib/executor/ramping_arrival_rate_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 98a869b2cb7..6d4f5e55d63 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -205,7 +205,7 @@ func (car *ConstantArrivalRate) Init(ctx context.Context) error { // keep the complexity in one well-architected part (with short methods and few // lambdas :D), while having both config frontends still be present for maximum // UX benefits. Basically, keep the progress bars and scheduling (i.e. at what -// time should iteration X begin) different, but keep everyhing else the same. +// time should iteration X begin) different, but keep everything else the same. // This will allow us to implement https://github.com/loadimpact/k6/issues/1386 // and things like all of the TODOs below in one place only. func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 9a91a803772..7499d058023 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -228,7 +228,7 @@ func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { require.Equal(t, cur, int64(1)) time.Sleep(time.Millisecond * 200) close(ch) - time.Sleep(time.Millisecond * 100) + time.Sleep(time.Millisecond * 200) cur = atomic.LoadInt64(&count) require.NotEqual(t, cur, int64(1)) From 07dea1284be2c9a58c8d06f8399dd4519bc50b61 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 3 Jul 2020 08:23:41 +0300 Subject: [PATCH 332/350] Handle unplanned VU init errors better --- lib/executor/constant_arrival_rate.go | 5 +++-- lib/executor/ramping_arrival_rate.go | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 6d4f5e55d63..b8f44c86777 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -270,9 +270,10 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC if err != nil { // TODO figure out how to return it to the Run goroutine car.logger.WithError(err).Error("Error while allocating unplanned VU") + } else { + car.logger.Debug("The unplanned VU finished initializing successfully!") + activeVUs <- activateVU(initVU) } - car.logger.Debug("The unplanned VU finished initializing successfully!") - activeVUs <- activateVU(initVU) } }() diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 76a3ef5f488..321ca87da06 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -356,9 +356,10 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC if err != nil { // TODO figure out how to return it to the Run goroutine varr.logger.WithError(err).Error("Error while allocating unplanned VU") + } else { + varr.logger.Debug("The unplanned VU finished initializing successfully!") + activeVUs <- activateVU(initVU) } - varr.logger.Debug("The unplanned VU finished initializing successfully!") - activeVUs <- activateVU(initVU) } }() From e0c93e8c17f3eeaddfc770c7dc5278a44c5672f6 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Thu, 2 Jul 2020 15:37:17 +0300 Subject: [PATCH 333/350] Add a new dropped_iterations metric --- js/runner.go | 5 +++++ lib/executor/base_executor.go | 15 +++++++++++++++ lib/executor/constant_arrival_rate.go | 7 ++++++- lib/executor/per_vu_iterations.go | 6 ++++++ lib/executor/ramping_arrival_rate.go | 7 ++++++- lib/executor/shared_iterations.go | 17 +++++++++++++---- lib/metrics/metrics.go | 1 + lib/runner.go | 3 +++ lib/testutils/minirunner/minirunner.go | 5 +++++ 9 files changed, 60 insertions(+), 6 deletions(-) diff --git a/js/runner.go b/js/runner.go index ce6c80d4060..a25ce877c69 100644 --- a/js/runner.go +++ b/js/runner.go @@ -413,6 +413,11 @@ type ActiveVU struct { busy chan struct{} } +// GetID returns the unique VU ID. +func (u *VU) GetID() int64 { + return u.ID +} + // Activate the VU so it will be able to run code. func (u *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { u.Runtime.ClearInterrupt() diff --git a/lib/executor/base_executor.go b/lib/executor/base_executor.go index 95a7b04b286..f43377983e6 100644 --- a/lib/executor/base_executor.go +++ b/lib/executor/base_executor.go @@ -22,10 +22,12 @@ package executor import ( "context" + "strconv" "github.com/sirupsen/logrus" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" ) @@ -73,3 +75,16 @@ func (bs BaseExecutor) GetLogger() *logrus.Entry { func (bs BaseExecutor) GetProgress() *pb.ProgressBar { return bs.progress } + +// getMetricTags returns a tag set that can be used to emit metrics by the +// executor. The VU ID is optional. +func (bs BaseExecutor) getMetricTags(vuID *int64) *stats.SampleTags { + tags := bs.executionState.Options.RunTags.CloneTags() + if bs.executionState.Options.SystemTags.Has(stats.TagScenario) { + tags["scenario"] = bs.config.GetName() + } + if vuID != nil && bs.executionState.Options.SystemTags.Has(stats.TagVU) { + tags["vu"] = strconv.FormatInt(*vuID, 10) + } + return stats.IntoSampleTags(&tags) +} diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index b8f44c86777..c2df4554df6 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -33,6 +33,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" @@ -328,6 +329,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC )).Duration) shownWarning := false + metricTags := car.getMetricTags(nil) for li, gi := 0, start; ; li, gi = li+1, gi+offsets[li%len(offsets)] { t := notScaledTickerPeriod*time.Duration(gi) - time.Since(startTime) timer.Reset(t) @@ -343,7 +345,10 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC // Since there aren't any free VUs available, consider this iteration // dropped - we aren't going to try to recover it, but - // TODO: emit a dropped_iterations metric + stats.PushIfNotDone(ctx, out, stats.Sample{ + Value: 1, Metric: metrics.DroppedIterations, + Tags: metricTags, Time: time.Now(), + }) // We'll try to start allocating another VU in the background, // non-blockingly, if we have remainingUnplannedVUs... diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 6a06167d3f3..58167fb2e36 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -31,6 +31,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" @@ -209,11 +210,16 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta newParams := *activationParams newParams.RunContext = ctx + vuID := initVU.GetID() activeVU := initVU.Activate(&newParams) for i := int64(0); i < iterations; i++ { select { case <-regDurationDone: + stats.PushIfNotDone(ctx, out, stats.Sample{ + Value: float64(iterations - i), Metric: metrics.DroppedIterations, + Tags: pvi.getMetricTags(&vuID), Time: time.Now(), + }) return // don't make more iterations default: // continue looping diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 321ca87da06..e72c8861b1e 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -32,6 +32,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" @@ -419,6 +420,7 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC ch := make(chan time.Duration, 10) // buffer 10 iteration times ahead var prevTime time.Duration shownWarning := false + metricTags := varr.getMetricTags(nil) go varr.config.cal(varr.executionState.ExecutionTuple, ch) for nextTime := range ch { select { @@ -447,7 +449,10 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC // Since there aren't any free VUs available, consider this iteration // dropped - we aren't going to try to recover it, but - // TODO: emit a dropped_iterations metric + stats.PushIfNotDone(ctx, out, stats.Sample{ + Value: 1, Metric: metrics.DroppedIterations, + Tags: metricTags, Time: time.Now(), + }) // We'll try to start allocating another VU in the background, // non-blockingly, if we have remainingUnplannedVUs... diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 92ea27f5955..0ba54fcbfaa 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -31,6 +31,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" "github.com/loadimpact/k6/ui/pb" @@ -214,15 +215,23 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta si.progress.Modify(pb.WithProgress(progresFn)) go trackProgress(ctx, maxDurationCtx, regDurationCtx, &si, progresFn) + var attemptedIters uint64 + // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} - defer activeVUs.Wait() + defer func() { + activeVUs.Wait() + if attemptedIters < totalIters { + stats.PushIfNotDone(ctx, out, stats.Sample{ + Value: float64(totalIters - attemptedIters), Metric: metrics.DroppedIterations, + Tags: si.getMetricTags(nil), Time: time.Now(), + }) + } + }() regDurationDone := regDurationCtx.Done() runIteration := getIterationRunner(si.executionState, si.logger) - attemptedIters := new(uint64) - activationParams := getVUActivationParams(maxDurationCtx, si.config.BaseConfig, func(u lib.InitializedVU) { si.executionState.ReturnVU(u, true) @@ -245,7 +254,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta // continue looping } - attemptedIterNumber := atomic.AddUint64(attemptedIters, 1) + attemptedIterNumber := atomic.AddUint64(&attemptedIters, 1) if attemptedIterNumber > totalIters { return } diff --git a/lib/metrics/metrics.go b/lib/metrics/metrics.go index 33e1f579eff..4c0d025ca53 100644 --- a/lib/metrics/metrics.go +++ b/lib/metrics/metrics.go @@ -33,6 +33,7 @@ var ( VUsMax = stats.New("vus_max", stats.Gauge) Iterations = stats.New("iterations", stats.Counter) IterationDuration = stats.New("iteration_duration", stats.Trend, stats.Time) + DroppedIterations = stats.New("dropped_iterations", stats.Counter) Errors = stats.New("errors", stats.Counter) // Runner-emitted. diff --git a/lib/runner.go b/lib/runner.go index e444734d5a2..b874f3ca1c8 100644 --- a/lib/runner.go +++ b/lib/runner.go @@ -41,6 +41,9 @@ type ActiveVU interface { type InitializedVU interface { // Fully activate the VU so it will be able to run code Activate(*VUActivationParams) ActiveVU + + // GetID returns the unique VU ID + GetID() int64 } // VUActivationParams are supplied by each executor when it retrieves a VU from diff --git a/lib/testutils/minirunner/minirunner.go b/lib/testutils/minirunner/minirunner.go index 63dcce0fbed..6ca17adc406 100644 --- a/lib/testutils/minirunner/minirunner.go +++ b/lib/testutils/minirunner/minirunner.go @@ -127,6 +127,11 @@ type ActiveVU struct { busy chan struct{} } +// GetID returns the unique VU ID. +func (vu *VU) GetID() int64 { + return vu.ID +} + // Activate the VU so it will be able to run code. func (vu *VU) Activate(params *lib.VUActivationParams) lib.ActiveVU { avu := &ActiveVU{ From 29168ee0a77437560466b08771659cd81a280e13 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 3 Jul 2020 09:25:24 +0300 Subject: [PATCH 334/350] Fix a context bug and add tests --- lib/executor/constant_arrival_rate_test.go | 36 ++++++++++++++++++++++ lib/executor/helpers_test.go | 36 ++++++++++++++++++++++ lib/executor/per_vu_iterations.go | 8 ++--- lib/executor/per_vu_iterations_test.go | 32 +++++++++++++++++++ lib/executor/ramping_arrival_rate_test.go | 6 ++-- lib/executor/shared_iterations.go | 8 ++--- lib/executor/shared_iterations_test.go | 32 +++++++++++++++++++ 7 files changed, 148 insertions(+), 10 deletions(-) create mode 100644 lib/executor/helpers_test.go diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 36eca62e41a..7f20ec26844 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -34,6 +34,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" ) @@ -296,3 +297,38 @@ func TestArrivalRateCancel(t *testing.T) { }) } } + +func TestConstantArrivalRateDroppedIterations(t *testing.T) { + t.Parallel() + var count int64 + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + + config := &ConstantArrivalRateConfig{ + BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0 * time.Second)}, + TimeUnit: types.NullDurationFrom(time.Second), + Rate: null.IntFrom(20), + Duration: types.NullDurationFrom(1 * time.Second), + PreAllocatedVUs: null.IntFrom(10), + MaxVUs: null.IntFrom(10), + } + + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, logHook := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + atomic.AddInt64(&count, 1) + <-ctx.Done() + return nil + }), + ) + defer cancel() + engineOut := make(chan stats.SampleContainer, 1000) + err = executor.Run(ctx, engineOut) + require.NoError(t, err) + logs := logHook.Drain() + require.Len(t, logs, 1) + assert.Contains(t, logs[0].Message, "cannot initialize more") + assert.Equal(t, int64(10), count) + assert.Equal(t, float64(10), sumMetricValues(engineOut, metrics.DroppedIterations.Name)) +} diff --git a/lib/executor/helpers_test.go b/lib/executor/helpers_test.go new file mode 100644 index 00000000000..a377a116798 --- /dev/null +++ b/lib/executor/helpers_test.go @@ -0,0 +1,36 @@ +/* + * + * k6 - a next-generation load testing tool + * Copyright (C) 2020 Load Impact + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +package executor + +import "github.com/loadimpact/k6/stats" + +func sumMetricValues(samples chan stats.SampleContainer, metricName string) (sum float64) { + bufferedSmaples := stats.GetBufferedSamples(samples) + for _, sc := range bufferedSmaples { + samples := sc.GetSamples() + for _, s := range samples { + if s.Metric.Name == metricName { + sum += s.Value + } + } + } + return sum +} diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 58167fb2e36..15419fad6b6 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -152,13 +152,13 @@ var _ lib.Executor = &PerVUIterations{} // Run executes a specific number of iterations with each configured VU. // nolint:funlen -func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (pvi PerVUIterations) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := pvi.config.GetVUs(pvi.executionState.ExecutionTuple) iterations := pvi.config.GetIterations() duration := time.Duration(pvi.config.MaxDuration.Duration) gracefulStop := pvi.config.GetGracefulStop() - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, duration, gracefulStop) defer cancel() // Make sure the log and the progress bar have accurate information @@ -189,7 +189,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta return float64(currentDoneIters) / float64(totalIters), right } pvi.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, pvi, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, pvi, progresFn) // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} @@ -216,7 +216,7 @@ func (pvi PerVUIterations) Run(ctx context.Context, out chan<- stats.SampleConta for i := int64(0); i < iterations; i++ { select { case <-regDurationDone: - stats.PushIfNotDone(ctx, out, stats.Sample{ + stats.PushIfNotDone(parentCtx, out, stats.Sample{ Value: float64(iterations - i), Metric: metrics.DroppedIterations, Tags: pvi.getMetricTags(&vuID), Time: time.Now(), }) diff --git a/lib/executor/per_vu_iterations_test.go b/lib/executor/per_vu_iterations_test.go index 7811a811ee6..4eb57303390 100644 --- a/lib/executor/per_vu_iterations_test.go +++ b/lib/executor/per_vu_iterations_test.go @@ -32,7 +32,9 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" ) func getTestPerVUIterationsConfig() PerVUIterationsConfig { @@ -124,3 +126,33 @@ func TestPerVUIterationsRunVariableVU(t *testing.T) { assert.Equal(t, uint64(16), val) assert.Equal(t, uint64(916), totalIters) } + +func TestPerVuIterationsEmitDroppedIterations(t *testing.T) { + t.Parallel() + var count int64 + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + + config := PerVUIterationsConfig{ + VUs: null.IntFrom(5), + Iterations: null.IntFrom(20), + MaxDuration: types.NullDurationFrom(1 * time.Second), + } + + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, logHook := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + atomic.AddInt64(&count, 1) + <-ctx.Done() + return nil + }), + ) + defer cancel() + engineOut := make(chan stats.SampleContainer, 1000) + err = executor.Run(ctx, engineOut) + require.NoError(t, err) + assert.Empty(t, logHook.Drain()) + assert.Equal(t, int64(5), count) + assert.Equal(t, float64(95), sumMetricValues(engineOut, metrics.DroppedIterations.Name)) +} diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 7499d058023..0b830ef62a1 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -35,6 +35,7 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" "github.com/loadimpact/k6/stats" ) @@ -188,8 +189,9 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { err = executor.Run(ctx, engineOut) assert.NoError(t, err) assert.Empty(t, logHook.Drain()) - //TODO: test that the sum of dropped_iteartions and count is 9 - // assert.Equal(t, count, int64(9)) + + droppedIters := sumMetricValues(engineOut, metrics.DroppedIterations.Name) + assert.Equal(t, count+int64(droppedIters), int64(9)) } func TestRampingArrivalRateRunCorrectRateWithSlowRate(t *testing.T) { diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 0ba54fcbfaa..10cfa164861 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -182,13 +182,13 @@ func (si *SharedIterations) Init(ctx context.Context) error { // Run executes a specific total number of iterations, which are all shared by // the configured VUs. // nolint:funlen -func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (si SharedIterations) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := si.config.GetVUs(si.executionState.ExecutionTuple) iterations := si.et.ScaleInt64(si.config.Iterations.Int64) duration := time.Duration(si.config.MaxDuration.Duration) gracefulStop := si.config.GetGracefulStop() - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, duration, gracefulStop) defer cancel() // Make sure the log and the progress bar have accurate information @@ -213,7 +213,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta return float64(currentDoneIters) / float64(totalIters), right } si.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, &si, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &si, progresFn) var attemptedIters uint64 @@ -222,7 +222,7 @@ func (si SharedIterations) Run(ctx context.Context, out chan<- stats.SampleConta defer func() { activeVUs.Wait() if attemptedIters < totalIters { - stats.PushIfNotDone(ctx, out, stats.Sample{ + stats.PushIfNotDone(parentCtx, out, stats.Sample{ Value: float64(totalIters - attemptedIters), Metric: metrics.DroppedIterations, Tags: si.getMetricTags(nil), Time: time.Now(), }) diff --git a/lib/executor/shared_iterations_test.go b/lib/executor/shared_iterations_test.go index f0b7cd12fcd..65f0d73bc1c 100644 --- a/lib/executor/shared_iterations_test.go +++ b/lib/executor/shared_iterations_test.go @@ -32,7 +32,9 @@ import ( "gopkg.in/guregu/null.v3" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/lib/metrics" "github.com/loadimpact/k6/lib/types" + "github.com/loadimpact/k6/stats" ) func getTestSharedIterationsConfig() SharedIterationsConfig { @@ -109,3 +111,33 @@ func TestSharedIterationsRunVariableVU(t *testing.T) { assert.Equal(t, uint64(2), val) assert.Equal(t, uint64(100), totalIters) } + +func TestSharedIterationsEmitDroppedIterations(t *testing.T) { + t.Parallel() + var count int64 + et, err := lib.NewExecutionTuple(nil, nil) + require.NoError(t, err) + + config := &SharedIterationsConfig{ + VUs: null.IntFrom(5), + Iterations: null.IntFrom(100), + MaxDuration: types.NullDurationFrom(1 * time.Second), + } + + es := lib.NewExecutionState(lib.Options{}, et, 10, 50) + ctx, cancel, executor, logHook := setupExecutor( + t, config, es, + simpleRunner(func(ctx context.Context) error { + atomic.AddInt64(&count, 1) + <-ctx.Done() + return nil + }), + ) + defer cancel() + engineOut := make(chan stats.SampleContainer, 1000) + err = executor.Run(ctx, engineOut) + require.NoError(t, err) + assert.Empty(t, logHook.Drain()) + assert.Equal(t, int64(5), count) + assert.Equal(t, float64(95), sumMetricValues(engineOut, metrics.DroppedIterations.Name)) +} From 9d6cffbf6232f1df890c60d03af36028169ac817 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 3 Jul 2020 10:11:55 +0300 Subject: [PATCH 335/350] Fix a flaky test --- lib/executor/constant_arrival_rate_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/executor/constant_arrival_rate_test.go b/lib/executor/constant_arrival_rate_test.go index 7f20ec26844..050d1b430d0 100644 --- a/lib/executor/constant_arrival_rate_test.go +++ b/lib/executor/constant_arrival_rate_test.go @@ -308,7 +308,7 @@ func TestConstantArrivalRateDroppedIterations(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), Rate: null.IntFrom(20), - Duration: types.NullDurationFrom(1 * time.Second), + Duration: types.NullDurationFrom(950 * time.Millisecond), PreAllocatedVUs: null.IntFrom(10), MaxVUs: null.IntFrom(10), } From 6b308c5ecde906553352f11821b16d1160a28e90 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 3 Jul 2020 10:49:59 +0300 Subject: [PATCH 336/350] Fix a typo by simplifying the code --- lib/executor/helpers_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/lib/executor/helpers_test.go b/lib/executor/helpers_test.go index a377a116798..81dd825b8d3 100644 --- a/lib/executor/helpers_test.go +++ b/lib/executor/helpers_test.go @@ -23,8 +23,7 @@ package executor import "github.com/loadimpact/k6/stats" func sumMetricValues(samples chan stats.SampleContainer, metricName string) (sum float64) { - bufferedSmaples := stats.GetBufferedSamples(samples) - for _, sc := range bufferedSmaples { + for _, sc := range stats.GetBufferedSamples(samples) { samples := sc.GetSamples() for _, s := range samples { if s.Metric.Name == metricName { From 41a4f730ad5db5dfbe399a6384351565460e1100 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Fri, 3 Jul 2020 16:16:37 +0300 Subject: [PATCH 337/350] Consistently use parentCtx in the executors' Run() methods --- lib/executor/constant_arrival_rate.go | 9 +++++---- lib/executor/constant_vus.go | 6 +++--- lib/executor/ramping_arrival_rate.go | 8 ++++---- lib/executor/ramping_vus.go | 8 ++++---- 4 files changed, 16 insertions(+), 15 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index c2df4554df6..751b424c39d 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -209,7 +209,8 @@ func (car *ConstantArrivalRate) Init(ctx context.Context) error { // time should iteration X begin) different, but keep everything else the same. // This will allow us to implement https://github.com/loadimpact/k6/issues/1386 // and things like all of the TODOs below in one place only. -func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { //nolint:funlen +//nolint:funlen +func (car ConstantArrivalRate) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { gracefulStop := car.config.GetGracefulStop() duration := time.Duration(car.config.Duration.Duration) preAllocatedVUs := car.config.GetPreAllocatedVUs(car.executionState.ExecutionTuple) @@ -228,7 +229,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC activeVUsWg := &sync.WaitGroup{} returnedVUs := make(chan struct{}) - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, duration, gracefulStop) defer func() { // Make sure all VUs aren't executing iterations anymore, for the cancel() @@ -310,7 +311,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC return math.Min(1, float64(spent)/float64(duration)), right } car.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, &car, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &car, progresFn) runIterationBasic := getIterationRunner(car.executionState, car.logger) runIteration := func(vu lib.ActiveVU) { @@ -345,7 +346,7 @@ func (car ConstantArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC // Since there aren't any free VUs available, consider this iteration // dropped - we aren't going to try to recover it, but - stats.PushIfNotDone(ctx, out, stats.Sample{ + stats.PushIfNotDone(parentCtx, out, stats.Sample{ Value: 1, Metric: metrics.DroppedIterations, Tags: metricTags, Time: time.Now(), }) diff --git a/lib/executor/constant_vus.go b/lib/executor/constant_vus.go index ba4249ea982..be21030b285 100644 --- a/lib/executor/constant_vus.go +++ b/lib/executor/constant_vus.go @@ -142,12 +142,12 @@ var _ lib.Executor = &ConstantVUs{} // Run constantly loops through as many iterations as possible on a fixed number // of VUs for the specified duration. -func (clv ConstantVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (clv ConstantVUs) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { numVUs := clv.config.GetVUs(clv.executionState.ExecutionTuple) duration := time.Duration(clv.config.Duration.Duration) gracefulStop := clv.config.GetGracefulStop() - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, duration, gracefulStop) defer cancel() // Make sure the log and the progress bar have accurate information @@ -167,7 +167,7 @@ func (clv ConstantVUs) Run(ctx context.Context, out chan<- stats.SampleContainer return float64(spent) / float64(duration), right } clv.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, clv, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, clv, progresFn) // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index e72c8861b1e..98503ce944e 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -291,7 +291,7 @@ func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time. // This will allow us to implement https://github.com/loadimpact/k6/issues/1386 // and things like all of the TODOs below in one place only. //nolint:funlen,gocognit -func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (varr RampingArrivalRate) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() duration := sumStagesDuration(varr.config.Stages) @@ -314,7 +314,7 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC activeVUsWg := &sync.WaitGroup{} returnedVUs := make(chan struct{}) - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, duration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, duration, gracefulStop) defer func() { // Make sure all VUs aren't executing iterations anymore, for the cancel() @@ -406,7 +406,7 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC } varr.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, varr, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, varr, progresFn) regDurationDone := regDurationCtx.Done() runIterationBasic := getIterationRunner(varr.executionState, varr.logger) @@ -449,7 +449,7 @@ func (varr RampingArrivalRate) Run(ctx context.Context, out chan<- stats.SampleC // Since there aren't any free VUs available, consider this iteration // dropped - we aren't going to try to recover it, but - stats.PushIfNotDone(ctx, out, stats.Sample{ + stats.PushIfNotDone(parentCtx, out, stats.Sample{ Value: 1, Metric: metrics.DroppedIterations, Tags: metricTags, Time: time.Now(), }) diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index fe4e0be4390..3320b2095c6 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -533,7 +533,7 @@ var _ lib.Executor = &RampingVUs{} // of a less complex way to implement it (besides the old "increment by 100ms // and see what happens)... :/ so maybe see how it can be split? // nolint:funlen,gocognit -func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) (err error) { +func (vlv RampingVUs) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { rawExecutionSteps := vlv.config.getRawExecutionSteps(vlv.executionState.ExecutionTuple, true) regularDuration, isFinal := lib.GetEndOffset(rawExecutionSteps) if !isFinal { @@ -548,7 +548,7 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) maxVUs := lib.GetMaxPlannedVUs(gracefulExecutionSteps) gracefulStop := maxDuration - regularDuration - startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(ctx, regularDuration, gracefulStop) + startTime, maxDurationCtx, regDurationCtx, cancel := getDurationContexts(parentCtx, regularDuration, gracefulStop) defer cancel() activeVUs := &sync.WaitGroup{} @@ -575,7 +575,7 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) return float64(spent) / float64(regularDuration), []string{progVUs, progDur} } vlv.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(ctx, maxDurationCtx, regDurationCtx, vlv, progresFn) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, vlv, progresFn) // Actually schedule the VUs and iterations, likely the most complicated // executor among all of them... @@ -633,7 +633,7 @@ func (vlv RampingVUs) Run(ctx context.Context, out chan<- stats.SampleContainer) currentMaxAllowedVUs = newMaxAllowedVUs } - wait := waiter(ctx, startTime) + wait := waiter(parentCtx, startTime) // iterate over rawExecutionSteps and gracefulExecutionSteps in order by TimeOffset // giving rawExecutionSteps precedence. // we stop iterating once rawExecutionSteps are over as we need to run the remaining From ba12f049cfaf02d98f98b7326d1f2002edf91cd5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:10:20 +0200 Subject: [PATCH 338/350] Fix cloud test progress status in CLI Closes #1488 --- cmd/cloud.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index d908799c138..de59e047605 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -211,15 +211,16 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud var progressErr error testProgress := &cloud.TestProgressResponse{} - percentageFmt := "[" + pb.GetFixedLengthFloatFormat(100, 2) + "%%] %s" + percentageFmt := "[" + pb.GetFixedLengthFloatFormat(100, 2) + "%%]" progressBar.Modify( pb.WithProgress(func() (float64, []string) { if testProgress.RunStatus < lib.RunStatusRunning { return 0, []string{testProgress.RunStatusText} } return testProgress.Progress, []string{ - fmt.Sprintf(percentageFmt, testProgress.Progress*100, testProgress.RunStatusText)} + fmt.Sprintf(percentageFmt, testProgress.Progress*100)} }), + pb.WithLeft(func() string { return " " + testProgress.RunStatusText }), ) ticker := time.NewTicker(time.Millisecond * 2000) From 089e416792669ad14c16a53bcdfb1c88839765c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:10:29 +0200 Subject: [PATCH 339/350] Move cloud status output to right side of progressbar, show time during Run --- cmd/cloud.go | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index de59e047605..c96c9edbd09 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -220,12 +220,14 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return testProgress.Progress, []string{ fmt.Sprintf(percentageFmt, testProgress.Progress*100)} }), - pb.WithLeft(func() string { return " " + testProgress.RunStatusText }), ) ticker := time.NewTicker(time.Millisecond * 2000) shouldExitLoop := false + var startTime time.Time + maxDuration := time.Duration(derivedConf.Duration.Duration) + runningLoop: for { select { @@ -235,7 +237,19 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } - printBar(progressBar, "") + statusText := testProgress.RunStatusText + if testProgress.RunStatus == lib.RunStatusRunning { + if startTime.IsZero() { + startTime = time.Now() + } + spent := time.Since(startTime) + if spent > maxDuration { + statusText = maxDuration.String() + } else { + statusText = fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, maxDuration), maxDuration) + } + } + printBar(progressBar, statusText) } else { logrus.WithError(progressErr).Error("Test progress error") } From 74cf03144ea5e05d2359cb6de0ce9d138b5d4e68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 9 Jun 2020 11:53:33 +0200 Subject: [PATCH 340/350] Fix: show stages and new executor duration during cloud test run In order to reuse SumStagesDuration and move it to lib, I had to remove the duplicate executor.Stage definition, and reuse the one in lib, which might be cleaner anyway. --- cmd/cloud.go | 15 ++++++-- cmd/config_consolidation_test.go | 8 ++--- lib/executor/execution_config_shortcuts.go | 2 +- lib/executor/executors_test.go | 4 +-- lib/executor/helpers.go | 11 ++---- lib/executor/ramping_arrival_rate.go | 8 ++--- lib/executor/ramping_arrival_rate_test.go | 12 +++---- lib/executor/ramping_vus.go | 13 ++----- lib/executor/ramping_vus_test.go | 42 +++++++++++----------- lib/helpers.go | 8 +++++ 10 files changed, 64 insertions(+), 59 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index c96c9edbd09..5bf39b63a38 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -225,8 +225,19 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud ticker := time.NewTicker(time.Millisecond * 2000) shouldExitLoop := false - var startTime time.Time - maxDuration := time.Duration(derivedConf.Duration.Duration) + var ( + startTime time.Time + maxDuration time.Duration + ) + if derivedConf.Duration.Valid { + maxDuration = time.Duration(derivedConf.Duration.Duration) + } else if derivedConf.Stages != nil { + maxDuration = lib.SumStagesDuration(derivedConf.Stages) + } else if derivedConf.Scenarios != nil { + et, _ := lib.NewExecutionTuple(nil, nil) + execReqs := derivedConf.Scenarios.GetFullExecutionRequirements(et) + maxDuration, _ = lib.GetEndOffset(execReqs) + } runningLoop: for { diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index b11763c7197..fca0d91411c 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -80,7 +80,7 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing } } -func verifyRampingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { +func verifyRampingVUs(startVus null.Int, stages []lib.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) @@ -100,14 +100,14 @@ func verifyRampingVUs(startVus null.Int, stages []executor.Stage) func(t *testin // A helper function that accepts (duration in second, VUs) pairs and returns // a valid slice of stage structs -func buildStages(durationsAndVUs ...int64) []executor.Stage { +func buildStages(durationsAndVUs ...int64) []lib.Stage { l := len(durationsAndVUs) if l%2 != 0 { panic("wrong len") } - result := make([]executor.Stage, 0, l/2) + result := make([]lib.Stage, 0, l/2) for i := 0; i < l; i += 2 { - result = append(result, executor.Stage{ + result = append(result, lib.Stage{ Duration: types.NullDurationFrom(time.Duration(durationsAndVUs[i]) * time.Second), Target: null.IntFrom(durationsAndVUs[i+1]), }) diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index 418a0ec160a..20d7de3afb6 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -50,7 +50,7 @@ func getRampingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ScenarioCo ds.StartVUs = startVUs for _, s := range stages { if s.Duration.Valid { - ds.Stages = append(ds.Stages, Stage{Duration: s.Duration, Target: s.Target}) + ds.Stages = append(ds.Stages, lib.Stage{Duration: s.Duration, Target: s.Target}) } } return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 222ccda8e02..326d71c0fc8 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -128,7 +128,7 @@ var configMapTestCases = []configMapTestCase{ sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) sched.StartVUs = null.IntFrom(20) sched.StartTime = types.NullDurationFrom(23 * time.Second) - sched.Stages = []Stage{ + sched.Stages = []lib.Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(60 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(130 * time.Second)}, } @@ -369,7 +369,7 @@ var configMapTestCases = []configMapTestCase{ exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewRampingArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) - sched.Stages = []Stage{ + sched.Stages = []lib.Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(180 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(300 * time.Second)}, } diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 46fdf3c432d..02e44898066 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -33,14 +33,7 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -func sumStagesDuration(stages []Stage) (result time.Duration) { - for _, s := range stages { - result += time.Duration(s.Duration.Duration) - } - return -} - -func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 { +func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []lib.Stage) int64 { max := unscaledStartValue for _, s := range stages { if s.Target.Int64 > max { @@ -51,7 +44,7 @@ func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 } // A helper function to avoid code duplication -func validateStages(stages []Stage) []error { +func validateStages(stages []lib.Stage) []error { var errors []error if len(stages) == 0 { errors = append(errors, fmt.Errorf("at least one stage has to be specified")) diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 98503ce944e..e8c447799ce 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -57,7 +57,7 @@ type RampingArrivalRateConfig struct { BaseConfig StartRate null.Int `json:"startRate"` TimeUnit types.NullDuration `json:"timeUnit"` - Stages []Stage `json:"stages"` + Stages []lib.Stage `json:"stages"` // Initialize `PreAllocatedVUs` number of VUs, and if more than that are needed, // they will be dynamically allocated, until `MaxVUs` is reached, which is an @@ -100,7 +100,7 @@ func (varc RampingArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) stri ).Float64() return fmt.Sprintf("Up to %.2f iterations/s for %s over %d stages%s", - maxArrRatePerSec, sumStagesDuration(varc.Stages), + maxArrRatePerSec, lib.SumStagesDuration(varc.Stages), len(varc.Stages), varc.getBaseInfo(maxVUsRange)) } @@ -147,7 +147,7 @@ func (varc RampingArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionT MaxUnplannedVUs: uint64(et.Segment.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), }, { - TimeOffset: sumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), + TimeOffset: lib.SumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), PlannedVUs: 0, MaxUnplannedVUs: 0, }, @@ -294,7 +294,7 @@ func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time. func (varr RampingArrivalRate) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() - duration := sumStagesDuration(varr.config.Stages) + duration := lib.SumStagesDuration(varr.config.Stages) preAllocatedVUs := varr.config.GetPreAllocatedVUs(varr.executionState.ExecutionTuple) maxVUs := varr.config.GetMaxVUs(varr.executionState.ExecutionTuple) diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 0b830ef62a1..173b464efbd 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -45,7 +45,7 @@ func getTestRampingArrivalRateConfig() *RampingArrivalRateConfig { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(10), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(time.Second * 1), Target: null.IntFrom(10), @@ -151,7 +151,7 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { var ctx, cancel, executor, logHook = setupExecutor( t, &RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), - Stages: []Stage{ + Stages: []lib.Stage{ { // the minus one makes it so only 9 iterations will be started instead of 10 // as the 10th happens to be just at the end and sometimes doesn't get executed :( @@ -258,7 +258,7 @@ func TestRampingArrivalRateCal(t *testing.T) { defaultTimeUnit = time.Second config = RampingArrivalRateConfig{ StartRate: null.IntFrom(0), - Stages: []Stage{ // TODO make this even bigger and longer .. will need more time + Stages: []lib.Stage{ // TODO make this even bigger and longer .. will need more time { Duration: types.NullDurationFrom(time.Second * 5), Target: null.IntFrom(1), @@ -358,7 +358,7 @@ func BenchmarkCal(b *testing.B) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(t), Target: null.IntFrom(49), @@ -394,7 +394,7 @@ func BenchmarkCalRat(b *testing.B) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(t), Target: null.IntFrom(49), @@ -430,7 +430,7 @@ func TestCompareCalImplementation(t *testing.T) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(0), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(200), diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index 3320b2095c6..8777c834efc 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -49,18 +49,11 @@ func init() { ) } -// Stage contains -type Stage struct { - Duration types.NullDuration `json:"duration"` - Target null.Int `json:"target"` // TODO: maybe rename this to endVUs? something else? - // TODO: add a progression function? -} - // RampingVUsConfig stores the configuration for the stages executor type RampingVUsConfig struct { BaseConfig StartVUs null.Int `json:"startVUs"` - Stages []Stage `json:"stages"` + Stages []lib.Stage `json:"stages"` GracefulRampDown types.NullDuration `json:"gracefulRampDown"` } @@ -91,7 +84,7 @@ func (vlvc RampingVUsConfig) GetGracefulRampDown() time.Duration { func (vlvc RampingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { maxVUs := et.Segment.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", - maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), + maxVUs, lib.SumStagesDuration(vlvc.Stages), len(vlvc.Stages), vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) } @@ -488,7 +481,7 @@ func (vlvc RampingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:funlen func (vlvc RampingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { steps := vlvc.getRawExecutionSteps(et, false) - executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) + executorEndOffset := lib.SumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) // Handle graceful ramp-downs, if we have them if vlvc.GracefulRampDown.Duration > 0 { steps = vlvc.reserveVUsForGracefulRampDowns(steps, executorEndOffset) diff --git a/lib/executor/ramping_vus_test.go b/lib/executor/ramping_vus_test.go index bdd2d710f4d..e45c8b0f9a0 100644 --- a/lib/executor/ramping_vus_test.go +++ b/lib/executor/ramping_vus_test.go @@ -45,7 +45,7 @@ func TestRampingVUsRun(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(0), StartVUs: null.IntFrom(5), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(5), @@ -105,7 +105,7 @@ func TestRampingVUsGracefulStopWaits(t *testing.T) { config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, StartVUs: null.IntFrom(1), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1), @@ -154,7 +154,7 @@ func TestRampingVUsGracefulStopStops(t *testing.T) { config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, StartVUs: null.IntFrom(1), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1), @@ -204,7 +204,7 @@ func TestRampingVUsGracefulRampDown(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(5 * time.Second)}, StartVUs: null.IntFrom(2), GracefulRampDown: types.NullDurationFrom(5 * time.Second), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(2), @@ -269,7 +269,7 @@ func TestRampingVUsRampDownNoWobble(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(1 * time.Second), StartVUs: null.IntFrom(0), - Stages: []Stage{ + Stages: []lib.Stage{ { Duration: types.NullDurationFrom(3 * time.Second), Target: null.IntFrom(10), @@ -340,7 +340,7 @@ func TestRampingVUsConfigExecutionPlanExample(t *testing.T) { require.NoError(t, err) conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []Stage{ + conf.Stages = []lib.Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -440,7 +440,7 @@ func TestRampingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { require.NoError(t, err) conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []Stage{ + conf.Stages = []lib.Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -518,7 +518,7 @@ func TestRampingVUsExecutionTupleTests(t *testing.T) { conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []Stage{ + conf.Stages = []lib.Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -743,7 +743,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { name string expectedSteps []lib.ExecutionStep et *lib.ExecutionTuple - stages []Stage + stages []lib.Stage start int64 }{ { @@ -754,7 +754,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 4}, {TimeOffset: 2 * time.Second, PlannedVUs: 3}, }, - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(0 * time.Second)}, {Target: null.IntFrom(3), Duration: types.NullDurationFrom(2 * time.Second)}, }, @@ -767,7 +767,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 4}, {TimeOffset: 2 * time.Second, PlannedVUs: 5}, }, - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(2 * time.Second)}, }, start: 3, @@ -785,7 +785,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 7 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -802,7 +802,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/2"), nil), - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -819,7 +819,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 7 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("1/2:1"), nil), - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -832,7 +832,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -849,7 +849,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1/2,2/3,1")), - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -871,7 +871,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 44 * time.Second, PlannedVUs: 9}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:0.3"), newExecutionSegmentSequenceFromString("0,0.3,0.6,0.9,1")), - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(20), Duration: types.NullDurationFrom(20 * time.Second)}, {Target: null.IntFrom(30), Duration: types.NullDurationFrom(30 * time.Second)}, }, @@ -891,7 +891,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 9 * time.Second, PlannedVUs: 1}, {TimeOffset: 10 * time.Second, PlannedVUs: 0}, }, - stages: []Stage{ + stages: []lib.Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(5 * time.Second)}, }, @@ -963,7 +963,7 @@ func BenchmarkRampingVUsGetRawExecutionSteps(b *testing.B) { et, err := lib.NewExecutionTuple(segment, &ess) require.NoError(b, err) for _, stageCase := range stageCases { - var st []Stage + var st []lib.Stage require.NoError(b, json.Unmarshal([]byte(stageCase.stages), &st)) vlvc := RampingVUsConfig{ Stages: st, @@ -1210,10 +1210,10 @@ func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { ) getTestConfig := func(name string) RampingVUsConfig { stagesCount := 1 + r.Int31n(maxStages) - stages := make([]Stage, stagesCount) + stages := make([]lib.Stage, stagesCount) for s := int32(0); s < stagesCount; s++ { dur := (minStageDuration + time.Duration(r.Int63n(int64(maxStageDuration-minStageDuration)))).Round(time.Second) - stages[s] = Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} + stages[s] = lib.Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} } c := NewRampingVUsConfig(name) diff --git a/lib/helpers.go b/lib/helpers.go index 8d2e133528a..5a9b4d71465 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -112,3 +112,11 @@ func ConcatErrors(errors []error, separator string) string { } return strings.Join(errStrings, separator) } + +// SumStagesDuration returns the total duration across all stages. +func SumStagesDuration(stages []Stage) (result time.Duration) { + for _, s := range stages { + result += time.Duration(s.Duration.Duration) + } + return +} From f69cdeb7ca984d9475306d3f1139878d5a956c25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Tue, 16 Jun 2020 17:26:02 +0200 Subject: [PATCH 341/350] Ensure every progress bar change is printed, better align run and cloud PBs Maybe resolves https://github.com/loadimpact/k6/issues/1488#issuecomment-644697431 --- cmd/cloud.go | 79 ++++++++++++++++++++++++--------------------- cmd/run.go | 18 ++++++++--- cmd/ui.go | 4 +-- core/local/local.go | 4 +-- 4 files changed, 61 insertions(+), 44 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 5bf39b63a38..d73838ba0a9 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -67,8 +67,10 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud //TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) - progressBar := pb.New(pb.WithConstLeft(" Init")) - printBar(progressBar, "Parsing script") + progressBar := pb.New( + pb.WithConstLeft(" Init"), + pb.WithConstProgress(0, "Parsing script"), + ) // Runner pwd, err := os.Getwd() @@ -88,13 +90,15 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - printBar(progressBar, "Getting script options") + progressBar.Modify(pb.WithConstProgress(0, "Getting script options")) + printBar(progressBar) r, err := newRunner(src, runType, filesystems, runtimeOptions) if err != nil { return err } - printBar(progressBar, "Consolidating options") + progressBar.Modify(pb.WithConstProgress(0, "Consolidating options")) + printBar(progressBar) cliOpts, err := getOptions(cmd.Flags()) if err != nil { return err @@ -127,7 +131,8 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return errors.New("Not logged in, please use `k6 login cloud`.") } - printBar(progressBar, "Building the archive") + progressBar.Modify(pb.WithConstProgress(0, "Building the archive")) + printBar(progressBar) arc := r.MakeArchive() // TODO: Fix this // We reuse cloud.Config for parsing options.ext.loadimpact, but this probably shouldn't be @@ -176,19 +181,19 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud } // Start cloud test run - printBar(progressBar, "Validating script options") + progressBar.Modify(pb.WithConstProgress(0, "Validating script options")) + printBar(progressBar) client := cloud.NewClient(cloudConfig.Token.String, cloudConfig.Host.String, consts.Version) if err := client.ValidateOptions(arc.Options); err != nil { return err } - printBar(progressBar, "Uploading archive") + progressBar.Modify(pb.WithConstProgress(0, "Uploading archive")) + printBar(progressBar) refID, err := client.StartCloudTestRun(name, cloudConfig.ProjectID.Int64, arc) if err != nil { return err } - progressBar.Modify(pb.WithConstLeft(" Run")) - printBar(progressBar, "Initializing the cloud test") testURL := cloud.URLForResults(refID, cloudConfig) fprintf(stdout, "\n\n") @@ -197,7 +202,12 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) //TODO: print executors information fprintf(stdout, "\n") - printBar(progressBar, "Initializing the cloud test") + + progressBar.Modify( + pb.WithConstLeft(" Run "), + pb.WithConstProgress(0, "Initializing the cloud test"), + ) + printBar(progressBar) // The quiet option hides the progress bar and disallow aborting the test if quiet { @@ -209,19 +219,6 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud signal.Notify(sigC, os.Interrupt, syscall.SIGINT, syscall.SIGTERM) defer signal.Stop(sigC) - var progressErr error - testProgress := &cloud.TestProgressResponse{} - percentageFmt := "[" + pb.GetFixedLengthFloatFormat(100, 2) + "%%]" - progressBar.Modify( - pb.WithProgress(func() (float64, []string) { - if testProgress.RunStatus < lib.RunStatusRunning { - return 0, []string{testProgress.RunStatusText} - } - return testProgress.Progress, []string{ - fmt.Sprintf(percentageFmt, testProgress.Progress*100)} - }), - ) - ticker := time.NewTicker(time.Millisecond * 2000) shouldExitLoop := false @@ -239,6 +236,28 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud maxDuration, _ = lib.GetEndOffset(execReqs) } + var progressErr error + testProgress := &cloud.TestProgressResponse{} + progressBar.Modify( + pb.WithProgress(func() (float64, []string) { + statusText := testProgress.RunStatusText + + if testProgress.RunStatus == lib.RunStatusRunning { + if startTime.IsZero() { + startTime = time.Now() + } + spent := time.Since(startTime) + if spent > maxDuration { + statusText = maxDuration.String() + } else { + statusText = fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, maxDuration), maxDuration) + } + } + + return testProgress.Progress, []string{statusText} + }), + ) + runningLoop: for { select { @@ -248,19 +267,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud if (testProgress.RunStatus > lib.RunStatusRunning) || (exitOnRunning && testProgress.RunStatus == lib.RunStatusRunning) { shouldExitLoop = true } - statusText := testProgress.RunStatusText - if testProgress.RunStatus == lib.RunStatusRunning { - if startTime.IsZero() { - startTime = time.Now() - } - spent := time.Since(startTime) - if spent > maxDuration { - statusText = maxDuration.String() - } else { - statusText = fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, maxDuration), maxDuration) - } - } - printBar(progressBar, statusText) + printBar(progressBar) } else { logrus.WithError(progressErr).Error("Test progress error") } diff --git a/cmd/run.go b/cmd/run.go index 0bfc75cda30..3f36b12d6dc 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -99,10 +99,13 @@ a commandline interface for interacting with it.`, // TODO: disable in quiet mode? _, _ = BannerColor.Fprintf(stdout, "\n%s\n\n", consts.Banner) - initBar := pb.New(pb.WithConstLeft(" init")) + initBar := pb.New( + pb.WithConstLeft(" Init"), + pb.WithConstProgress(0, "runner"), + ) + printBar(initBar) // Create the Runner. - printBar(initBar, "runner") pwd, err := os.Getwd() if err != nil { return err @@ -124,7 +127,8 @@ a commandline interface for interacting with it.`, return err } - printBar(initBar, "options") + initBar.Modify(pb.WithConstProgress(0, "options")) + printBar(initBar) cliConf, err := getConfig(cmd.Flags()) if err != nil { @@ -166,7 +170,8 @@ a commandline interface for interacting with it.`, defer runCancel() // Create a local execution scheduler wrapping the runner. - printBar(initBar, "execution scheduler") + initBar.Modify(pb.WithConstProgress(0, "execution scheduler")) + printBar(initBar) execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err @@ -190,6 +195,7 @@ a commandline interface for interacting with it.`, // Create an engine. initBar.Modify(pb.WithConstProgress(0, "Init engine")) + printBar(initBar) engine, err := core.NewEngine(execScheduler, conf.Options, logger) if err != nil { return err @@ -209,6 +215,7 @@ a commandline interface for interacting with it.`, // Create a collector and assign it to the engine if requested. initBar.Modify(pb.WithConstProgress(0, "Init metric outputs")) + printBar(initBar) for _, out := range conf.Out { t, arg := parseCollector(out) collector, cerr := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) @@ -224,6 +231,7 @@ a commandline interface for interacting with it.`, // Spin up the REST API server, if not disabled. if address != "" { initBar.Modify(pb.WithConstProgress(0, "Init API server")) + printBar(initBar) go func() { logger.Debugf("Starting the REST API server on %s", address) if aerr := api.ListenAndServe(address, engine); aerr != nil { @@ -295,6 +303,7 @@ a commandline interface for interacting with it.`, // Initialize the engine initBar.Modify(pb.WithConstProgress(0, "Init VUs")) + printBar(initBar) engineRun, engineWait, err := engine.Init(globalCtx, runCtx) if err != nil { return getExitCodeFromEngine(err) @@ -319,6 +328,7 @@ a commandline interface for interacting with it.`, // Start the test run initBar.Modify(pb.WithConstProgress(0, "Start test")) + printBar(initBar) if err := engineRun(); err != nil { return getExitCodeFromEngine(err) } diff --git a/cmd/ui.go b/cmd/ui.go index 661bdbb48aa..966dff6945a 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -79,7 +79,7 @@ func (w *consoleWriter) Write(p []byte) (n int, err error) { return origLen, err } -func printBar(bar *pb.ProgressBar, rightText string) { +func printBar(bar *pb.ProgressBar) { end := "\n" if stdout.IsTTY { // If we're in a TTY, instead of printing the bar and going to the next @@ -91,7 +91,7 @@ func printBar(bar *pb.ProgressBar, rightText string) { } rendered := bar.Render(0, 0) // Only output the left and middle part of the progress bar - fprintf(stdout, "%s %s %s%s", rendered.Left, rendered.Progress(), rightText, end) + fprintf(stdout, "%s%s", rendered.String(), end) } //nolint: funlen diff --git a/core/local/local.go b/core/local/local.go index ed1d9d66232..a791ab04948 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -101,7 +101,7 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution logger: logger, options: options, - initProgress: pb.New(pb.WithConstLeft("Init")), + initProgress: pb.New(pb.WithConstLeft(" Init")), executors: executors, executorConfigs: executorConfigs, executionPlan: executionPlan, @@ -339,7 +339,7 @@ func (e *ExecutionScheduler) runExecutor( func (e *ExecutionScheduler) Run(globalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error { executorsCount := len(e.executors) logger := e.logger.WithField("phase", "local-execution-scheduler-run") - e.initProgress.Modify(pb.WithConstLeft("Run")) + e.initProgress.Modify(pb.WithConstLeft(" Run")) if e.state.IsPaused() { logger.Debug("Execution is paused, waiting for resume or interrupt...") From 20b988aa1332330d78dbd7433813daabdabc162e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:10:56 +0200 Subject: [PATCH 342/350] Revert progressbar status prefix whitespace in local execution Resolves: - https://github.com/loadimpact/k6/pull/1490#discussion_r449599581 - https://github.com/loadimpact/k6/pull/1490#discussion_r449599906 --- core/local/local.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/local/local.go b/core/local/local.go index a791ab04948..ed1d9d66232 100644 --- a/core/local/local.go +++ b/core/local/local.go @@ -101,7 +101,7 @@ func NewExecutionScheduler(runner lib.Runner, logger *logrus.Logger) (*Execution logger: logger, options: options, - initProgress: pb.New(pb.WithConstLeft(" Init")), + initProgress: pb.New(pb.WithConstLeft("Init")), executors: executors, executorConfigs: executorConfigs, executionPlan: executionPlan, @@ -339,7 +339,7 @@ func (e *ExecutionScheduler) runExecutor( func (e *ExecutionScheduler) Run(globalCtx, runCtx context.Context, engineOut chan<- stats.SampleContainer) error { executorsCount := len(e.executors) logger := e.logger.WithField("phase", "local-execution-scheduler-run") - e.initProgress.Modify(pb.WithConstLeft(" Run")) + e.initProgress.Modify(pb.WithConstLeft("Run")) if e.state.IsPaused() { logger.Debug("Execution is paused, waiting for resume or interrupt...") From 41f89e7b94beb82e44f77858c002bfa9e979d911 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:11:36 +0200 Subject: [PATCH 343/350] Add missing printBar call in cloud execution Resolves https://github.com/loadimpact/k6/pull/1490#discussion_r449593413 --- cmd/cloud.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/cloud.go b/cmd/cloud.go index d73838ba0a9..43295794ce6 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -71,6 +71,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud pb.WithConstLeft(" Init"), pb.WithConstProgress(0, "Parsing script"), ) + printBar(progressBar) // Runner pwd, err := os.Getwd() From 028c46ee99b0f5b147324412539b898858b35b17 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:11:16 +0200 Subject: [PATCH 344/350] Add helper function to modify and print the progressbar Resolves https://github.com/loadimpact/k6/pull/1490#discussion_r449594183 --- cmd/cloud.go | 19 +++++++------------ cmd/run.go | 21 +++++++-------------- cmd/ui.go | 5 +++++ 3 files changed, 19 insertions(+), 26 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 43295794ce6..4bac750a54f 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -91,15 +91,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } - progressBar.Modify(pb.WithConstProgress(0, "Getting script options")) - printBar(progressBar) + modifyAndPrintBar(progressBar, pb.WithConstProgress(0, "Getting script options")) r, err := newRunner(src, runType, filesystems, runtimeOptions) if err != nil { return err } - progressBar.Modify(pb.WithConstProgress(0, "Consolidating options")) - printBar(progressBar) + modifyAndPrintBar(progressBar, pb.WithConstProgress(0, "Consolidating options")) cliOpts, err := getOptions(cmd.Flags()) if err != nil { return err @@ -132,8 +130,7 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return errors.New("Not logged in, please use `k6 login cloud`.") } - progressBar.Modify(pb.WithConstProgress(0, "Building the archive")) - printBar(progressBar) + modifyAndPrintBar(progressBar, pb.WithConstProgress(0, "Building the archive")) arc := r.MakeArchive() // TODO: Fix this // We reuse cloud.Config for parsing options.ext.loadimpact, but this probably shouldn't be @@ -182,15 +179,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud } // Start cloud test run - progressBar.Modify(pb.WithConstProgress(0, "Validating script options")) - printBar(progressBar) + modifyAndPrintBar(progressBar, pb.WithConstProgress(0, "Validating script options")) client := cloud.NewClient(cloudConfig.Token.String, cloudConfig.Host.String, consts.Version) if err := client.ValidateOptions(arc.Options); err != nil { return err } - progressBar.Modify(pb.WithConstProgress(0, "Uploading archive")) - printBar(progressBar) + modifyAndPrintBar(progressBar, pb.WithConstProgress(0, "Uploading archive")) refID, err := client.StartCloudTestRun(name, cloudConfig.ProjectID.Int64, arc) if err != nil { return err @@ -204,11 +199,11 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud //TODO: print executors information fprintf(stdout, "\n") - progressBar.Modify( + modifyAndPrintBar( + progressBar, pb.WithConstLeft(" Run "), pb.WithConstProgress(0, "Initializing the cloud test"), ) - printBar(progressBar) // The quiet option hides the progress bar and disallow aborting the test if quiet { diff --git a/cmd/run.go b/cmd/run.go index 3f36b12d6dc..33dac005aa5 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -127,8 +127,7 @@ a commandline interface for interacting with it.`, return err } - initBar.Modify(pb.WithConstProgress(0, "options")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "options")) cliConf, err := getConfig(cmd.Flags()) if err != nil { @@ -170,8 +169,7 @@ a commandline interface for interacting with it.`, defer runCancel() // Create a local execution scheduler wrapping the runner. - initBar.Modify(pb.WithConstProgress(0, "execution scheduler")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "execution scheduler")) execScheduler, err := local.NewExecutionScheduler(r, logger) if err != nil { return err @@ -194,8 +192,7 @@ a commandline interface for interacting with it.`, }() // Create an engine. - initBar.Modify(pb.WithConstProgress(0, "Init engine")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Init engine")) engine, err := core.NewEngine(execScheduler, conf.Options, logger) if err != nil { return err @@ -214,8 +211,7 @@ a commandline interface for interacting with it.`, } // Create a collector and assign it to the engine if requested. - initBar.Modify(pb.WithConstProgress(0, "Init metric outputs")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Init metric outputs")) for _, out := range conf.Out { t, arg := parseCollector(out) collector, cerr := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) @@ -230,8 +226,7 @@ a commandline interface for interacting with it.`, // Spin up the REST API server, if not disabled. if address != "" { - initBar.Modify(pb.WithConstProgress(0, "Init API server")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Init API server")) go func() { logger.Debugf("Starting the REST API server on %s", address) if aerr := api.ListenAndServe(address, engine); aerr != nil { @@ -302,8 +297,7 @@ a commandline interface for interacting with it.`, }() // Initialize the engine - initBar.Modify(pb.WithConstProgress(0, "Init VUs")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Init VUs")) engineRun, engineWait, err := engine.Init(globalCtx, runCtx) if err != nil { return getExitCodeFromEngine(err) @@ -327,8 +321,7 @@ a commandline interface for interacting with it.`, } // Start the test run - initBar.Modify(pb.WithConstProgress(0, "Start test")) - printBar(initBar) + modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Start test")) if err := engineRun(); err != nil { return getExitCodeFromEngine(err) } diff --git a/cmd/ui.go b/cmd/ui.go index 966dff6945a..a5e470fbabb 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -94,6 +94,11 @@ func printBar(bar *pb.ProgressBar) { fprintf(stdout, "%s%s", rendered.String(), end) } +func modifyAndPrintBar(bar *pb.ProgressBar, options ...pb.ProgressBarOption) { + bar.Modify(options...) + printBar(bar) +} + //nolint: funlen func renderMultipleBars( isTTY, goBack bool, maxLeft, termWidth, widthDelta int, pbs []*pb.ProgressBar, From 1c7f2ce505166a488558310c3722f5ba6a270887 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:11:20 +0200 Subject: [PATCH 345/350] Unify execution description for cloud/local execution This now also prints a more detailed execution description for cloud tests. Resolves https://github.com/loadimpact/k6/pull/1490#discussion_r449606002 --- cmd/cloud.go | 23 ++++++++--------------- cmd/run.go | 39 ++------------------------------------- cmd/ui.go | 43 +++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 53 insertions(+), 52 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 4bac750a54f..3658934b8d9 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -191,13 +191,12 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } + et, err := lib.NewExecutionTuple(derivedConf.ExecutionSegment, derivedConf.ExecutionSegmentSequence) + if err != nil { + return err + } testURL := cloud.URLForResults(refID, cloudConfig) - fprintf(stdout, "\n\n") - fprintf(stdout, " executor: %s\n", ui.ValueColor.Sprint("cloud")) - fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) - fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(testURL)) - //TODO: print executors information - fprintf(stdout, "\n") + printExecutionDescription("cloud", filename, testURL, derivedConf, et, nil) modifyAndPrintBar( progressBar, @@ -222,15 +221,9 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud startTime time.Time maxDuration time.Duration ) - if derivedConf.Duration.Valid { - maxDuration = time.Duration(derivedConf.Duration.Duration) - } else if derivedConf.Stages != nil { - maxDuration = lib.SumStagesDuration(derivedConf.Stages) - } else if derivedConf.Scenarios != nil { - et, _ := lib.NewExecutionTuple(nil, nil) - execReqs := derivedConf.Scenarios.GetFullExecutionRequirements(et) - maxDuration, _ = lib.GetEndOffset(execReqs) - } + + executionPlan := derivedConf.Scenarios.GetFullExecutionRequirements(et) + maxDuration, _ = lib.GetEndOffset(executionPlan) var progressErr error testProgress := &cloud.TestProgressResponse{} diff --git a/cmd/run.go b/cmd/run.go index 33dac005aa5..a9cab1151aa 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -241,43 +241,8 @@ a commandline interface for interacting with it.`, }() } - // Write the big banner. - { // TODO: rewrite as Engine.GetTestRunDescription() and move out of here - out := "-" - link := "" - - for idx, collector := range engine.Collectors { - if out != "-" { - out = out + "; " + conf.Out[idx] - } else { - out = conf.Out[idx] - } - - if l := collector.Link(); l != "" { - link = link + " (" + l + ")" - } - } - - fprintf(stdout, " executor: %s\n", ui.ValueColor.Sprint("local")) - fprintf(stdout, " output: %s%s\n", ui.ValueColor.Sprint(out), ui.ExtraColor.Sprint(link)) - fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) - fprintf(stdout, "\n") - - plan := execScheduler.GetExecutionPlan() - executorConfigs := execScheduler.GetExecutorConfigs() - maxDuration, _ := lib.GetEndOffset(plan) - - fprintf(stdout, " scenarios: %s\n", ui.ValueColor.Sprintf( - "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", - conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), - lib.GetMaxPossibleVUs(plan), maxDuration), - ) - for _, ec := range executorConfigs { - fprintf(stdout, " * %s: %s\n", - ec.GetName(), ec.GetDescription(execScheduler.GetState().ExecutionTuple)) - } - fprintf(stdout, "\n") - } + printExecutionDescription("local", filename, "", conf, + execScheduler.GetState().ExecutionTuple, engine.Collectors) // Trap Interrupts, SIGINTs and SIGTERMs. sigC := make(chan os.Signal, 1) diff --git a/cmd/ui.go b/cmd/ui.go index a5e470fbabb..1947d85779f 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -37,6 +37,7 @@ import ( "github.com/loadimpact/k6/core/local" "github.com/loadimpact/k6/lib" + "github.com/loadimpact/k6/ui" "github.com/loadimpact/k6/ui/pb" ) @@ -99,6 +100,48 @@ func modifyAndPrintBar(bar *pb.ProgressBar, options ...pb.ProgressBarOption) { printBar(bar) } +func printExecutionDescription(execution, filename, output string, conf Config, + et *lib.ExecutionTuple, collectors []lib.Collector) { + fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprint(execution)) + fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) + + if execution == "local" { + out := "-" + link := "" + + for idx, collector := range collectors { + if out != "-" { + out = out + "; " + conf.Out[idx] + } else { + out = conf.Out[idx] + } + + if l := collector.Link(); l != "" { + link = link + " (" + l + ")" + } + } + fprintf(stdout, " output: %s%s\n", ui.ValueColor.Sprint(out), ui.ExtraColor.Sprint(link)) + } else { + fprintf(stdout, " output: %s\n", ui.ValueColor.Sprint(output)) + } + fprintf(stdout, "\n") + + executionPlan := conf.Scenarios.GetFullExecutionRequirements(et) + maxDuration, _ := lib.GetEndOffset(executionPlan) + executorConfigs := conf.Scenarios.GetSortedConfigs() + + fprintf(stdout, " scenarios: %s\n", ui.ValueColor.Sprintf( + "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", + conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), + lib.GetMaxPossibleVUs(executionPlan), maxDuration), + ) + for _, ec := range executorConfigs { + fprintf(stdout, " * %s: %s\n", + ec.GetName(), ec.GetDescription(et)) + } + fprintf(stdout, "\n") +} + //nolint: funlen func renderMultipleBars( isTTY, goBack bool, maxLeft, termWidth, widthDelta int, pbs []*pb.ProgressBar, From cc01c16eaf44a30fad43346882e9646b8ad97349 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:16:34 +0200 Subject: [PATCH 346/350] Revert executor.Stage / lib.Stage merge This reverts commit 74cf03144ea5e05d2359cb6de0ce9d138b5d4e68. Resolves https://github.com/loadimpact/k6/pull/1490#discussion_r449603598 --- cmd/config_consolidation_test.go | 8 ++--- lib/executor/execution_config_shortcuts.go | 2 +- lib/executor/executors_test.go | 4 +-- lib/executor/helpers.go | 11 ++++-- lib/executor/ramping_arrival_rate.go | 8 ++--- lib/executor/ramping_arrival_rate_test.go | 12 +++---- lib/executor/ramping_vus.go | 13 +++++-- lib/executor/ramping_vus_test.go | 42 +++++++++++----------- lib/helpers.go | 8 ----- 9 files changed, 57 insertions(+), 51 deletions(-) diff --git a/cmd/config_consolidation_test.go b/cmd/config_consolidation_test.go index fca0d91411c..b11763c7197 100644 --- a/cmd/config_consolidation_test.go +++ b/cmd/config_consolidation_test.go @@ -80,7 +80,7 @@ func verifyConstLoopingVUs(vus null.Int, duration time.Duration) func(t *testing } } -func verifyRampingVUs(startVus null.Int, stages []lib.Stage) func(t *testing.T, c Config) { +func verifyRampingVUs(startVus null.Int, stages []executor.Stage) func(t *testing.T, c Config) { return func(t *testing.T, c Config) { exec := c.Scenarios[lib.DefaultScenarioName] require.NotEmpty(t, exec) @@ -100,14 +100,14 @@ func verifyRampingVUs(startVus null.Int, stages []lib.Stage) func(t *testing.T, // A helper function that accepts (duration in second, VUs) pairs and returns // a valid slice of stage structs -func buildStages(durationsAndVUs ...int64) []lib.Stage { +func buildStages(durationsAndVUs ...int64) []executor.Stage { l := len(durationsAndVUs) if l%2 != 0 { panic("wrong len") } - result := make([]lib.Stage, 0, l/2) + result := make([]executor.Stage, 0, l/2) for i := 0; i < l; i += 2 { - result = append(result, lib.Stage{ + result = append(result, executor.Stage{ Duration: types.NullDurationFrom(time.Duration(durationsAndVUs[i]) * time.Second), Target: null.IntFrom(durationsAndVUs[i+1]), }) diff --git a/lib/executor/execution_config_shortcuts.go b/lib/executor/execution_config_shortcuts.go index 20d7de3afb6..418a0ec160a 100644 --- a/lib/executor/execution_config_shortcuts.go +++ b/lib/executor/execution_config_shortcuts.go @@ -50,7 +50,7 @@ func getRampingVUsScenario(stages []lib.Stage, startVUs null.Int) lib.ScenarioCo ds.StartVUs = startVUs for _, s := range stages { if s.Duration.Valid { - ds.Stages = append(ds.Stages, lib.Stage{Duration: s.Duration, Target: s.Target}) + ds.Stages = append(ds.Stages, Stage{Duration: s.Duration, Target: s.Target}) } } return lib.ScenarioConfigs{lib.DefaultScenarioName: ds} diff --git a/lib/executor/executors_test.go b/lib/executor/executors_test.go index 326d71c0fc8..222ccda8e02 100644 --- a/lib/executor/executors_test.go +++ b/lib/executor/executors_test.go @@ -128,7 +128,7 @@ var configMapTestCases = []configMapTestCase{ sched.GracefulRampDown = types.NullDurationFrom(10 * time.Second) sched.StartVUs = null.IntFrom(20) sched.StartTime = types.NullDurationFrom(23 * time.Second) - sched.Stages = []lib.Stage{ + sched.Stages = []Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(60 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(130 * time.Second)}, } @@ -369,7 +369,7 @@ var configMapTestCases = []configMapTestCase{ exp{custom: func(t *testing.T, cm lib.ScenarioConfigs) { sched := NewRampingArrivalRateConfig("varrival") sched.StartRate = null.IntFrom(10) - sched.Stages = []lib.Stage{ + sched.Stages = []Stage{ {Target: null.IntFrom(30), Duration: types.NullDurationFrom(180 * time.Second)}, {Target: null.IntFrom(10), Duration: types.NullDurationFrom(300 * time.Second)}, } diff --git a/lib/executor/helpers.go b/lib/executor/helpers.go index 02e44898066..46fdf3c432d 100644 --- a/lib/executor/helpers.go +++ b/lib/executor/helpers.go @@ -33,7 +33,14 @@ import ( "github.com/loadimpact/k6/ui/pb" ) -func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []lib.Stage) int64 { +func sumStagesDuration(stages []Stage) (result time.Duration) { + for _, s := range stages { + result += time.Duration(s.Duration.Duration) + } + return +} + +func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []Stage) int64 { max := unscaledStartValue for _, s := range stages { if s.Target.Int64 > max { @@ -44,7 +51,7 @@ func getStagesUnscaledMaxTarget(unscaledStartValue int64, stages []lib.Stage) in } // A helper function to avoid code duplication -func validateStages(stages []lib.Stage) []error { +func validateStages(stages []Stage) []error { var errors []error if len(stages) == 0 { errors = append(errors, fmt.Errorf("at least one stage has to be specified")) diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index e8c447799ce..98503ce944e 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -57,7 +57,7 @@ type RampingArrivalRateConfig struct { BaseConfig StartRate null.Int `json:"startRate"` TimeUnit types.NullDuration `json:"timeUnit"` - Stages []lib.Stage `json:"stages"` + Stages []Stage `json:"stages"` // Initialize `PreAllocatedVUs` number of VUs, and if more than that are needed, // they will be dynamically allocated, until `MaxVUs` is reached, which is an @@ -100,7 +100,7 @@ func (varc RampingArrivalRateConfig) GetDescription(et *lib.ExecutionTuple) stri ).Float64() return fmt.Sprintf("Up to %.2f iterations/s for %s over %d stages%s", - maxArrRatePerSec, lib.SumStagesDuration(varc.Stages), + maxArrRatePerSec, sumStagesDuration(varc.Stages), len(varc.Stages), varc.getBaseInfo(maxVUsRange)) } @@ -147,7 +147,7 @@ func (varc RampingArrivalRateConfig) GetExecutionRequirements(et *lib.ExecutionT MaxUnplannedVUs: uint64(et.Segment.Scale(varc.MaxVUs.Int64 - varc.PreAllocatedVUs.Int64)), }, { - TimeOffset: lib.SumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), + TimeOffset: sumStagesDuration(varc.Stages) + time.Duration(varc.GracefulStop.Duration), PlannedVUs: 0, MaxUnplannedVUs: 0, }, @@ -294,7 +294,7 @@ func (varc RampingArrivalRateConfig) cal(et *lib.ExecutionTuple, ch chan<- time. func (varr RampingArrivalRate) Run(parentCtx context.Context, out chan<- stats.SampleContainer) (err error) { segment := varr.executionState.ExecutionTuple.Segment gracefulStop := varr.config.GetGracefulStop() - duration := lib.SumStagesDuration(varr.config.Stages) + duration := sumStagesDuration(varr.config.Stages) preAllocatedVUs := varr.config.GetPreAllocatedVUs(varr.executionState.ExecutionTuple) maxVUs := varr.config.GetMaxVUs(varr.executionState.ExecutionTuple) diff --git a/lib/executor/ramping_arrival_rate_test.go b/lib/executor/ramping_arrival_rate_test.go index 173b464efbd..0b830ef62a1 100644 --- a/lib/executor/ramping_arrival_rate_test.go +++ b/lib/executor/ramping_arrival_rate_test.go @@ -45,7 +45,7 @@ func getTestRampingArrivalRateConfig() *RampingArrivalRateConfig { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(1 * time.Second)}, TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(10), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(time.Second * 1), Target: null.IntFrom(10), @@ -151,7 +151,7 @@ func TestRampingArrivalRateRunUnplannedVUs(t *testing.T) { var ctx, cancel, executor, logHook = setupExecutor( t, &RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), - Stages: []lib.Stage{ + Stages: []Stage{ { // the minus one makes it so only 9 iterations will be started instead of 10 // as the 10th happens to be just at the end and sometimes doesn't get executed :( @@ -258,7 +258,7 @@ func TestRampingArrivalRateCal(t *testing.T) { defaultTimeUnit = time.Second config = RampingArrivalRateConfig{ StartRate: null.IntFrom(0), - Stages: []lib.Stage{ // TODO make this even bigger and longer .. will need more time + Stages: []Stage{ // TODO make this even bigger and longer .. will need more time { Duration: types.NullDurationFrom(time.Second * 5), Target: null.IntFrom(1), @@ -358,7 +358,7 @@ func BenchmarkCal(b *testing.B) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(t), Target: null.IntFrom(49), @@ -394,7 +394,7 @@ func BenchmarkCalRat(b *testing.B) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(50), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(t), Target: null.IntFrom(49), @@ -430,7 +430,7 @@ func TestCompareCalImplementation(t *testing.T) { config := RampingArrivalRateConfig{ TimeUnit: types.NullDurationFrom(time.Second), StartRate: null.IntFrom(0), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(200), diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index 8777c834efc..3320b2095c6 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -49,11 +49,18 @@ func init() { ) } +// Stage contains +type Stage struct { + Duration types.NullDuration `json:"duration"` + Target null.Int `json:"target"` // TODO: maybe rename this to endVUs? something else? + // TODO: add a progression function? +} + // RampingVUsConfig stores the configuration for the stages executor type RampingVUsConfig struct { BaseConfig StartVUs null.Int `json:"startVUs"` - Stages []lib.Stage `json:"stages"` + Stages []Stage `json:"stages"` GracefulRampDown types.NullDuration `json:"gracefulRampDown"` } @@ -84,7 +91,7 @@ func (vlvc RampingVUsConfig) GetGracefulRampDown() time.Duration { func (vlvc RampingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { maxVUs := et.Segment.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", - maxVUs, lib.SumStagesDuration(vlvc.Stages), len(vlvc.Stages), + maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) } @@ -481,7 +488,7 @@ func (vlvc RampingVUsConfig) reserveVUsForGracefulRampDowns( //nolint:funlen func (vlvc RampingVUsConfig) GetExecutionRequirements(et *lib.ExecutionTuple) []lib.ExecutionStep { steps := vlvc.getRawExecutionSteps(et, false) - executorEndOffset := lib.SumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) + executorEndOffset := sumStagesDuration(vlvc.Stages) + time.Duration(vlvc.GracefulStop.Duration) // Handle graceful ramp-downs, if we have them if vlvc.GracefulRampDown.Duration > 0 { steps = vlvc.reserveVUsForGracefulRampDowns(steps, executorEndOffset) diff --git a/lib/executor/ramping_vus_test.go b/lib/executor/ramping_vus_test.go index e45c8b0f9a0..bdd2d710f4d 100644 --- a/lib/executor/ramping_vus_test.go +++ b/lib/executor/ramping_vus_test.go @@ -45,7 +45,7 @@ func TestRampingVUsRun(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(0), StartVUs: null.IntFrom(5), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(5), @@ -105,7 +105,7 @@ func TestRampingVUsGracefulStopWaits(t *testing.T) { config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, StartVUs: null.IntFrom(1), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1), @@ -154,7 +154,7 @@ func TestRampingVUsGracefulStopStops(t *testing.T) { config := RampingVUsConfig{ BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(time.Second)}, StartVUs: null.IntFrom(1), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(1), @@ -204,7 +204,7 @@ func TestRampingVUsGracefulRampDown(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(5 * time.Second)}, StartVUs: null.IntFrom(2), GracefulRampDown: types.NullDurationFrom(5 * time.Second), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(1 * time.Second), Target: null.IntFrom(2), @@ -269,7 +269,7 @@ func TestRampingVUsRampDownNoWobble(t *testing.T) { BaseConfig: BaseConfig{GracefulStop: types.NullDurationFrom(0)}, GracefulRampDown: types.NullDurationFrom(1 * time.Second), StartVUs: null.IntFrom(0), - Stages: []lib.Stage{ + Stages: []Stage{ { Duration: types.NullDurationFrom(3 * time.Second), Target: null.IntFrom(10), @@ -340,7 +340,7 @@ func TestRampingVUsConfigExecutionPlanExample(t *testing.T) { require.NoError(t, err) conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []lib.Stage{ + conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -440,7 +440,7 @@ func TestRampingVUsConfigExecutionPlanExampleOneThird(t *testing.T) { require.NoError(t, err) conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []lib.Stage{ + conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -518,7 +518,7 @@ func TestRampingVUsExecutionTupleTests(t *testing.T) { conf := NewRampingVUsConfig("test") conf.StartVUs = null.IntFrom(4) - conf.Stages = []lib.Stage{ + conf.Stages = []Stage{ {Target: null.IntFrom(6), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(1), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(5), Duration: types.NullDurationFrom(4 * time.Second)}, @@ -743,7 +743,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { name string expectedSteps []lib.ExecutionStep et *lib.ExecutionTuple - stages []lib.Stage + stages []Stage start int64 }{ { @@ -754,7 +754,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 4}, {TimeOffset: 2 * time.Second, PlannedVUs: 3}, }, - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(0 * time.Second)}, {Target: null.IntFrom(3), Duration: types.NullDurationFrom(2 * time.Second)}, }, @@ -767,7 +767,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 1 * time.Second, PlannedVUs: 4}, {TimeOffset: 2 * time.Second, PlannedVUs: 5}, }, - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(2 * time.Second)}, }, start: 3, @@ -785,7 +785,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 7 * time.Second, PlannedVUs: 1}, {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -802,7 +802,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/2"), nil), - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -819,7 +819,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 7 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("1/2:1"), nil), - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -832,7 +832,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 0 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("2/3:1"), newExecutionSegmentSequenceFromString("0,1/3,2/3,1")), - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -849,7 +849,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 8 * time.Second, PlannedVUs: 0}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:1/3"), newExecutionSegmentSequenceFromString("0,1/3,1/2,2/3,1")), - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(2 * time.Second)}, {Target: null.IntFrom(2), Duration: types.NullDurationFrom(2 * time.Second)}, @@ -871,7 +871,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 44 * time.Second, PlannedVUs: 9}, }, et: mustNewExecutionTuple(newExecutionSegmentFromString("0:0.3"), newExecutionSegmentSequenceFromString("0,0.3,0.6,0.9,1")), - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(20), Duration: types.NullDurationFrom(20 * time.Second)}, {Target: null.IntFrom(30), Duration: types.NullDurationFrom(30 * time.Second)}, }, @@ -891,7 +891,7 @@ func TestRampingVUsGetRawExecutionStepsCornerCases(t *testing.T) { {TimeOffset: 9 * time.Second, PlannedVUs: 1}, {TimeOffset: 10 * time.Second, PlannedVUs: 0}, }, - stages: []lib.Stage{ + stages: []Stage{ {Target: null.IntFrom(5), Duration: types.NullDurationFrom(5 * time.Second)}, {Target: null.IntFrom(0), Duration: types.NullDurationFrom(5 * time.Second)}, }, @@ -963,7 +963,7 @@ func BenchmarkRampingVUsGetRawExecutionSteps(b *testing.B) { et, err := lib.NewExecutionTuple(segment, &ess) require.NoError(b, err) for _, stageCase := range stageCases { - var st []lib.Stage + var st []Stage require.NoError(b, json.Unmarshal([]byte(stageCase.stages), &st)) vlvc := RampingVUsConfig{ Stages: st, @@ -1210,10 +1210,10 @@ func TestSumRandomSegmentSequenceMatchesNoSegment(t *testing.T) { ) getTestConfig := func(name string) RampingVUsConfig { stagesCount := 1 + r.Int31n(maxStages) - stages := make([]lib.Stage, stagesCount) + stages := make([]Stage, stagesCount) for s := int32(0); s < stagesCount; s++ { dur := (minStageDuration + time.Duration(r.Int63n(int64(maxStageDuration-minStageDuration)))).Round(time.Second) - stages[s] = lib.Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} + stages[s] = Stage{Duration: types.NullDurationFrom(dur), Target: null.IntFrom(r.Int63n(maxVUs))} } c := NewRampingVUsConfig(name) diff --git a/lib/helpers.go b/lib/helpers.go index 5a9b4d71465..8d2e133528a 100644 --- a/lib/helpers.go +++ b/lib/helpers.go @@ -112,11 +112,3 @@ func ConcatErrors(errors []error, separator string) string { } return strings.Join(errStrings, separator) } - -// SumStagesDuration returns the total duration across all stages. -func SumStagesDuration(stages []Stage) (result time.Duration) { - for _, s := range stages { - result += time.Duration(s.Duration.Duration) - } - return -} From 83ae1725b93e4ed1b8392e7084865f2c897a2608 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 12:54:42 +0200 Subject: [PATCH 347/350] Reorder some var declarations for better legibility --- cmd/cloud.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 3658934b8d9..897d47581e8 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -214,18 +214,13 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud signal.Notify(sigC, os.Interrupt, syscall.SIGINT, syscall.SIGTERM) defer signal.Stop(sigC) - ticker := time.NewTicker(time.Millisecond * 2000) - shouldExitLoop := false - var ( startTime time.Time maxDuration time.Duration ) - executionPlan := derivedConf.Scenarios.GetFullExecutionRequirements(et) maxDuration, _ = lib.GetEndOffset(executionPlan) - var progressErr error testProgress := &cloud.TestProgressResponse{} progressBar.Modify( pb.WithProgress(func() (float64, []string) { @@ -247,6 +242,10 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud }), ) + var progressErr error + ticker := time.NewTicker(time.Millisecond * 2000) + shouldExitLoop := false + runningLoop: for { select { From f57e189b607a1f278759bace5359ce98f38d71b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ivan=20Miri=C4=87?= Date: Mon, 6 Jul 2020 14:43:53 +0200 Subject: [PATCH 348/350] Remove duplicate execution plan calculations Resolves https://github.com/loadimpact/k6/pull/1490#pullrequestreview-443008350 --- cmd/cloud.go | 4 ++-- cmd/run.go | 8 +++++--- cmd/ui.go | 13 ++++++++----- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/cmd/cloud.go b/cmd/cloud.go index 897d47581e8..a0f1eb2bd3c 100644 --- a/cmd/cloud.go +++ b/cmd/cloud.go @@ -196,7 +196,8 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud return err } testURL := cloud.URLForResults(refID, cloudConfig) - printExecutionDescription("cloud", filename, testURL, derivedConf, et, nil) + executionPlan := derivedConf.Scenarios.GetFullExecutionRequirements(et) + printExecutionDescription("cloud", filename, testURL, derivedConf, et, executionPlan, nil) modifyAndPrintBar( progressBar, @@ -218,7 +219,6 @@ This will execute the test on the Load Impact cloud service. Use "k6 login cloud startTime time.Time maxDuration time.Duration ) - executionPlan := derivedConf.Scenarios.GetFullExecutionRequirements(et) maxDuration, _ = lib.GetEndOffset(executionPlan) testProgress := &cloud.TestProgressResponse{} diff --git a/cmd/run.go b/cmd/run.go index a9cab1151aa..d9032bc0b5f 100644 --- a/cmd/run.go +++ b/cmd/run.go @@ -210,11 +210,12 @@ a commandline interface for interacting with it.`, engine.SummaryExport = conf.SummaryExport.String != "" } + executionPlan := execScheduler.GetExecutionPlan() // Create a collector and assign it to the engine if requested. modifyAndPrintBar(initBar, pb.WithConstProgress(0, "Init metric outputs")) for _, out := range conf.Out { t, arg := parseCollector(out) - collector, cerr := newCollector(t, arg, src, conf, execScheduler.GetExecutionPlan()) + collector, cerr := newCollector(t, arg, src, conf, executionPlan) if cerr != nil { return cerr } @@ -241,8 +242,9 @@ a commandline interface for interacting with it.`, }() } - printExecutionDescription("local", filename, "", conf, - execScheduler.GetState().ExecutionTuple, engine.Collectors) + printExecutionDescription( + "local", filename, "", conf, execScheduler.GetState().ExecutionTuple, + executionPlan, engine.Collectors) // Trap Interrupts, SIGINTs and SIGTERMs. sigC := make(chan os.Signal, 1) diff --git a/cmd/ui.go b/cmd/ui.go index 1947d85779f..8b3f2000753 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -100,8 +100,12 @@ func modifyAndPrintBar(bar *pb.ProgressBar, options ...pb.ProgressBarOption) { printBar(bar) } -func printExecutionDescription(execution, filename, output string, conf Config, - et *lib.ExecutionTuple, collectors []lib.Collector) { +// Print execution description for both cloud and local execution. +// TODO: Clean this up as part of #1499 or #1427 +func printExecutionDescription( + execution, filename, output string, conf Config, et *lib.ExecutionTuple, + execPlan []lib.ExecutionStep, collectors []lib.Collector, +) { fprintf(stdout, " execution: %s\n", ui.ValueColor.Sprint(execution)) fprintf(stdout, " script: %s\n", ui.ValueColor.Sprint(filename)) @@ -126,14 +130,13 @@ func printExecutionDescription(execution, filename, output string, conf Config, } fprintf(stdout, "\n") - executionPlan := conf.Scenarios.GetFullExecutionRequirements(et) - maxDuration, _ := lib.GetEndOffset(executionPlan) + maxDuration, _ := lib.GetEndOffset(execPlan) executorConfigs := conf.Scenarios.GetSortedConfigs() fprintf(stdout, " scenarios: %s\n", ui.ValueColor.Sprintf( "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), - lib.GetMaxPossibleVUs(executionPlan), maxDuration), + lib.GetMaxPossibleVUs(execPlan), maxDuration), ) for _, ec := range executorConfigs { fprintf(stdout, " * %s: %s\n", From 182297b0b1614c054da5e3a5c306653a0b5d9e68 Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 6 Jul 2020 13:59:24 +0300 Subject: [PATCH 349/350] Display user-friendly times and correct VU numbers --- cmd/ui.go | 2 +- lib/executor/ramping_vus.go | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cmd/ui.go b/cmd/ui.go index 8b3f2000753..70b272a16a4 100644 --- a/cmd/ui.go +++ b/cmd/ui.go @@ -136,7 +136,7 @@ func printExecutionDescription( fprintf(stdout, " scenarios: %s\n", ui.ValueColor.Sprintf( "(%.2f%%) %d executors, %d max VUs, %s max duration (incl. graceful stop):", conf.ExecutionSegment.FloatLength()*100, len(executorConfigs), - lib.GetMaxPossibleVUs(execPlan), maxDuration), + lib.GetMaxPossibleVUs(execPlan), maxDuration.Round(100*time.Millisecond)), ) for _, ec := range executorConfigs { fprintf(stdout, " * %s: %s\n", diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index 3320b2095c6..50a85380ed4 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -78,7 +78,7 @@ var _ lib.ExecutorConfig = &RampingVUsConfig{} // GetStartVUs is just a helper method that returns the scaled starting VUs. func (vlvc RampingVUsConfig) GetStartVUs(et *lib.ExecutionTuple) int64 { - return et.Segment.Scale(vlvc.StartVUs.Int64) + return et.ScaleInt64(vlvc.StartVUs.Int64) } // GetGracefulRampDown is just a helper method that returns the graceful @@ -89,7 +89,7 @@ func (vlvc RampingVUsConfig) GetGracefulRampDown() time.Duration { // GetDescription returns a human-readable description of the executor options func (vlvc RampingVUsConfig) GetDescription(et *lib.ExecutionTuple) string { - maxVUs := et.Segment.Scale(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) + maxVUs := et.ScaleInt64(getStagesUnscaledMaxTarget(vlvc.StartVUs.Int64, vlvc.Stages)) return fmt.Sprintf("Up to %d looping VUs for %s over %d stages%s", maxVUs, sumStagesDuration(vlvc.Stages), len(vlvc.Stages), vlvc.getBaseInfo(fmt.Sprintf("gracefulRampDown: %s", vlvc.GetGracefulRampDown()))) @@ -563,6 +563,7 @@ func (vlv RampingVUs) Run(parentCtx context.Context, out chan<- stats.SampleCont activeVUsCount := new(int64) vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) + regularDurationStr := pb.GetFixedLengthDuration(regularDuration, regularDuration) progresFn := func() (float64, []string) { spent := time.Since(startTime) currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) @@ -571,7 +572,7 @@ func (vlv RampingVUs) Run(parentCtx context.Context, out chan<- stats.SampleCont return 1, []string{vus, regularDuration.String()} } progVUs := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentlyActiveVUs, maxVUs) - progDur := fmt.Sprintf("%s/%s", pb.GetFixedLengthDuration(spent, regularDuration), regularDuration) + progDur := pb.GetFixedLengthDuration(spent, regularDuration) + "/" + regularDurationStr return float64(spent) / float64(regularDuration), []string{progVUs, progDur} } vlv.progress.Modify(pb.WithProgress(progresFn)) From 488a1fbf1e7d3ab8bb6269e0e349e5f4e017cbab Mon Sep 17 00:00:00 2001 From: Nedyalko Andreev Date: Mon, 6 Jul 2020 14:00:27 +0300 Subject: [PATCH 350/350] Fix a variable name typo --- lib/executor/constant_arrival_rate.go | 6 +++--- lib/executor/constant_vus.go | 6 +++--- lib/executor/externally_controlled.go | 6 +++--- lib/executor/per_vu_iterations.go | 6 +++--- lib/executor/ramping_arrival_rate.go | 6 +++--- lib/executor/ramping_vus.go | 6 +++--- lib/executor/shared_iterations.go | 6 +++--- 7 files changed, 21 insertions(+), 21 deletions(-) diff --git a/lib/executor/constant_arrival_rate.go b/lib/executor/constant_arrival_rate.go index 751b424c39d..4c1a9e0d49f 100644 --- a/lib/executor/constant_arrival_rate.go +++ b/lib/executor/constant_arrival_rate.go @@ -291,7 +291,7 @@ func (car ConstantArrivalRate) Run(parentCtx context.Context, out chan<- stats.S vusFmt := pb.GetFixedLengthIntFormat(maxVUs) progIters := fmt.Sprintf( pb.GetFixedLengthFloatFormat(arrivalRatePerSec, 0)+" iters/s", arrivalRatePerSec) - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { spent := time.Since(startTime) currActiveVUs := atomic.LoadUint64(&activeVUsCount) vusInBuffer := uint64(len(activeVUs)) @@ -310,8 +310,8 @@ func (car ConstantArrivalRate) Run(parentCtx context.Context, out chan<- stats.S return math.Min(1, float64(spent)/float64(duration)), right } - car.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &car, progresFn) + car.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &car, progressFn) runIterationBasic := getIterationRunner(car.executionState, car.logger) runIteration := func(vu lib.ActiveVU) { diff --git a/lib/executor/constant_vus.go b/lib/executor/constant_vus.go index be21030b285..11a1c7a5685 100644 --- a/lib/executor/constant_vus.go +++ b/lib/executor/constant_vus.go @@ -155,7 +155,7 @@ func (clv ConstantVUs) Run(parentCtx context.Context, out chan<- stats.SampleCon logrus.Fields{"vus": numVUs, "duration": duration, "type": clv.config.GetType()}, ).Debug("Starting executor run...") - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { spent := time.Since(startTime) right := []string{fmt.Sprintf("%d VUs", numVUs)} if spent > duration { @@ -166,8 +166,8 @@ func (clv ConstantVUs) Run(parentCtx context.Context, out chan<- stats.SampleCon pb.GetFixedLengthDuration(spent, duration), duration)) return float64(spent) / float64(duration), right } - clv.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, clv, progresFn) + clv.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, clv, progressFn) // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} diff --git a/lib/executor/externally_controlled.go b/lib/executor/externally_controlled.go index d6e733bf035..643132ba2c4 100644 --- a/lib/executor/externally_controlled.go +++ b/lib/executor/externally_controlled.go @@ -401,7 +401,7 @@ func (rs *externallyControlledRunState) retrieveStartMaxVUs() error { return nil } -func (rs *externallyControlledRunState) progresFn() (float64, []string) { +func (rs *externallyControlledRunState) progressFn() (float64, []string) { // TODO: simulate spinner for the other case or cycle 0-100? currentActiveVUs := atomic.LoadInt64(rs.activeVUsCount) currentMaxVUs := atomic.LoadInt64(rs.maxVUs) @@ -536,8 +536,8 @@ func (mex *ExternallyControlled) Run(parentCtx context.Context, out chan<- stats return err } - mex.progress.Modify(pb.WithProgress(runState.progresFn)) // Keep track of the progress - go trackProgress(parentCtx, ctx, ctx, mex, runState.progresFn) + mex.progress.Modify(pb.WithProgress(runState.progressFn)) // Keep track of the progress + go trackProgress(parentCtx, ctx, ctx, mex, runState.progressFn) err = runState.handleConfigChange( // Start by setting MaxVUs to the starting MaxVUs ExternallyControlledConfigParams{MaxVUs: mex.config.MaxVUs}, currentControlConfig, diff --git a/lib/executor/per_vu_iterations.go b/lib/executor/per_vu_iterations.go index 15419fad6b6..302ded8799d 100644 --- a/lib/executor/per_vu_iterations.go +++ b/lib/executor/per_vu_iterations.go @@ -171,7 +171,7 @@ func (pvi PerVUIterations) Run(parentCtx context.Context, out chan<- stats.Sampl vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { spent := time.Since(startTime) progVUs := fmt.Sprintf(vusFmt+" VUs", numVUs) currentDoneIters := atomic.LoadUint64(doneIters) @@ -188,8 +188,8 @@ func (pvi PerVUIterations) Run(parentCtx context.Context, out chan<- stats.Sampl return float64(currentDoneIters) / float64(totalIters), right } - pvi.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, pvi, progresFn) + pvi.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, pvi, progressFn) // Actually schedule the VUs and iterations... activeVUs := &sync.WaitGroup{} diff --git a/lib/executor/ramping_arrival_rate.go b/lib/executor/ramping_arrival_rate.go index 98503ce944e..d996dc51161 100644 --- a/lib/executor/ramping_arrival_rate.go +++ b/lib/executor/ramping_arrival_rate.go @@ -378,7 +378,7 @@ func (varr RampingArrivalRate) Run(parentCtx context.Context, out chan<- stats.S vusFmt := pb.GetFixedLengthIntFormat(maxVUs) itersFmt := pb.GetFixedLengthFloatFormat(maxArrivalRatePerSec, 0) + " iters/s" - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { currActiveVUs := atomic.LoadUint64(&activeVUsCount) currentTickerPeriod := atomic.LoadInt64(&tickerPeriod) vusInBuffer := uint64(len(activeVUs)) @@ -405,8 +405,8 @@ func (varr RampingArrivalRate) Run(parentCtx context.Context, out chan<- stats.S return math.Min(1, float64(spent)/float64(duration)), right } - varr.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, varr, progresFn) + varr.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, varr, progressFn) regDurationDone := regDurationCtx.Done() runIterationBasic := getIterationRunner(varr.executionState, varr.logger) diff --git a/lib/executor/ramping_vus.go b/lib/executor/ramping_vus.go index 50a85380ed4..08d4a2e1bd7 100644 --- a/lib/executor/ramping_vus.go +++ b/lib/executor/ramping_vus.go @@ -564,7 +564,7 @@ func (vlv RampingVUs) Run(parentCtx context.Context, out chan<- stats.SampleCont activeVUsCount := new(int64) vusFmt := pb.GetFixedLengthIntFormat(int64(maxVUs)) regularDurationStr := pb.GetFixedLengthDuration(regularDuration, regularDuration) - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { spent := time.Since(startTime) currentlyActiveVUs := atomic.LoadInt64(activeVUsCount) vus := fmt.Sprintf(vusFmt+"/"+vusFmt+" VUs", currentlyActiveVUs, maxVUs) @@ -575,8 +575,8 @@ func (vlv RampingVUs) Run(parentCtx context.Context, out chan<- stats.SampleCont progDur := pb.GetFixedLengthDuration(spent, regularDuration) + "/" + regularDurationStr return float64(spent) / float64(regularDuration), []string{progVUs, progDur} } - vlv.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, vlv, progresFn) + vlv.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, vlv, progressFn) // Actually schedule the VUs and iterations, likely the most complicated // executor among all of them... diff --git a/lib/executor/shared_iterations.go b/lib/executor/shared_iterations.go index 10cfa164861..9367a25c9c3 100644 --- a/lib/executor/shared_iterations.go +++ b/lib/executor/shared_iterations.go @@ -200,7 +200,7 @@ func (si SharedIterations) Run(parentCtx context.Context, out chan<- stats.Sampl doneIters := new(uint64) vusFmt := pb.GetFixedLengthIntFormat(numVUs) itersFmt := pb.GetFixedLengthIntFormat(int64(totalIters)) - progresFn := func() (float64, []string) { + progressFn := func() (float64, []string) { spent := time.Since(startTime) progVUs := fmt.Sprintf(vusFmt+" VUs", numVUs) currentDoneIters := atomic.LoadUint64(doneIters) @@ -212,8 +212,8 @@ func (si SharedIterations) Run(parentCtx context.Context, out chan<- stats.Sampl return float64(currentDoneIters) / float64(totalIters), right } - si.progress.Modify(pb.WithProgress(progresFn)) - go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &si, progresFn) + si.progress.Modify(pb.WithProgress(progressFn)) + go trackProgress(parentCtx, maxDurationCtx, regDurationCtx, &si, progressFn) var attemptedIters uint64