diff --git a/cmd/simulator/cmd/root.go b/cmd/simulator/cmd/root.go new file mode 100644 index 00000000000..bfa94d2b541 --- /dev/null +++ b/cmd/simulator/cmd/root.go @@ -0,0 +1,134 @@ +package cmd + +import ( + "fmt" + "github.com/armadaproject/armada/internal/common/armadacontext" + "github.com/armadaproject/armada/internal/scheduler/simulator" + "github.com/armadaproject/armada/pkg/armadaevents" + "github.com/sirupsen/logrus" + "github.com/spf13/cobra" + log2 "log" + "os" + "strings" +) + +func RootCmd() *cobra.Command { + cmd := &cobra.Command{ + Use: "Simulate", + Short: "Simulate the scheduling of jobs on Armada", + RunE: runSimulations, + } + + cmd.Flags().BoolP("verbose", "v", false, "Logs detailed output to console when specified") + cmd.Flags().String("clusters", "", "Pattern specifying cluster configurations to simulate on") + cmd.Flags().String("workloads", "", "Pattern specifying workloads to simulate") + cmd.Flags().String("configs", "", "Pattern specifying scheduler configurations to use for simulation") + + return cmd +} + +func runSimulations(cmd *cobra.Command, args []string) error { + clusterPattern, err := cmd.Flags().GetString("clusters") + if err != nil { + return err + } + workloadPattern, err := cmd.Flags().GetString("workloads") + if err != nil { + return err + } + configPattern, err := cmd.Flags().GetString("configs") + if err != nil { + return err + } + verbose, err := cmd.Flags().GetBool("verbose") + if err != nil { + return err + } + + ctx := armadacontext.Background() + if !verbose { + logger := logrus.New() + logger.SetLevel(logrus.ErrorLevel) + ctx = armadacontext.New(ctx, logrus.NewEntry(logger)) + } + + clusterSpecs, err := simulator.ClusterSpecsFromPattern(clusterPattern) + if err != nil { + return err + } + workloadSpecs, err := simulator.WorkloadFromPattern(workloadPattern) + if err != nil { + return err + } + schedulingConfigsByFilePath, err := simulator.SchedulingConfigsByFilePathFromPattern(configPattern) + if err != nil { + return err + } + + simulators := make([]*simulator.Simulator, 0) + metricsCollectors := make([]*simulator.MetricsCollector, 0) + eventSequenceChannels := make([]<-chan *armadaevents.EventSequence, 0) + filePaths := make([]string, 0) + + for _, clusterSpec := range clusterSpecs { + for _, workloadSpec := range workloadSpecs { + for filePath, schedulingConfig := range schedulingConfigsByFilePath { + if s, err := simulator.NewSimulator(clusterSpec, workloadSpec, schedulingConfig); err != nil { + return err + } else { + simulators = append(simulators, s) + metricsCollectors = append(metricsCollectors, simulator.NewMetricsCollector(s.Output())) + filePaths = append(filePaths, filePath) + eventSequenceChannels = append(eventSequenceChannels, s.Output()) + } + } + } + } + + threadSafeLogger := log2.New(os.Stdout, "", 0) + g, ctx := armadacontext.ErrGroup(ctx) + for _, s := range simulators { + s := s + g.Go(func() error { + return s.Run(ctx) + }) + } + for _, c := range eventSequenceChannels { + c := c + g.Go(func() error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case eventSequence, ok := <-c: + if !ok { + return nil + } + ctx.Info(*eventSequence.Events[0].Created, simulator.EventSequenceSummary(eventSequence)) + } + } + }) + } + for i, mc := range metricsCollectors { + mc := mc + s := simulators[i] + fp := filePaths[i] + g.Go(func() error { + if err := mc.Run(ctx); err != nil { + return err + } + + var sb strings.Builder + sb.WriteString(fmt.Sprintf("\nRunning Simulation of workload %s on cluster %s with configuration %s\n", s.WorkloadSpec.Name, s.ClusterSpec.Name, fp)) + sb.WriteString(fmt.Sprint("Simulation Result: ", mc.String(), "\n")) + threadSafeLogger.Print(sb.String()) + return nil + }) + } + + if err := g.Wait(); err != nil { + return err + } + + return nil +} diff --git a/cmd/simulator/main.go b/cmd/simulator/main.go new file mode 100644 index 00000000000..3f7453dfca6 --- /dev/null +++ b/cmd/simulator/main.go @@ -0,0 +1,14 @@ +package main + +import ( + "fmt" + "github.com/armadaproject/armada/cmd/simulator/cmd" + "os" +) + +func main() { + if err := cmd.RootCmd().Execute(); err != nil { + fmt.Println(err.Error()) + os.Exit(1) + } +} diff --git a/cmd/testsuite/cmd/test.go b/cmd/testsuite/cmd/test.go index ccf8ecdf9fd..d6c931e94ad 100644 --- a/cmd/testsuite/cmd/test.go +++ b/cmd/testsuite/cmd/test.go @@ -79,7 +79,7 @@ func testCmdRunE(app *testsuite.App) func(cmd *cobra.Command, args []string) err app.Params.PrometheusPushGatewayJobName = prometheusPushgatewayJobName // Create a context that is cancelled on SIGINT/SIGTERM. - // Ensures test jobs are cancelled on ctrl-C. + // Ensures test jobs are cancelled on ctrl-c. ctx, cancel := context.WithCancel(context.Background()) defer cancel() stopSignal := make(chan os.Signal, 1) diff --git a/internal/scheduler/simulator/events.go b/internal/scheduler/simulator/events.go new file mode 100644 index 00000000000..c1f14578872 --- /dev/null +++ b/internal/scheduler/simulator/events.go @@ -0,0 +1,53 @@ +package simulator + +import "time" + +// Event is a simulator-internal event. +type Event struct { + // Time at which the event was submitted. + time time.Time + // Each event is assigned a sequence number. + // Events with equal time are ordered by their sequence number. + sequenceNumber int + // One of armadaevents.EventSequence or scheduleEvent.. + eventSequenceOrScheduleEvent any + // Maintained by the heap.Interface methods. + index int +} + +// scheduleEvent is an event indicating the scheduler should be run. +type scheduleEvent struct{} + +type EventLog []Event + +func (el EventLog) Len() int { return len(el) } + +func (el EventLog) Less(i, j int) bool { + if el[i].time == el[j].time { + return el[i].sequenceNumber < el[j].sequenceNumber + } + return el[j].time.After(el[i].time) +} + +func (el EventLog) Swap(i, j int) { + el[i], el[j] = el[j], el[i] + el[i].index = i + el[j].index = j +} + +func (el *EventLog) Push(x any) { + n := len(*el) + item := x.(Event) + item.index = n + *el = append(*el, item) +} + +func (el *EventLog) Pop() any { + old := *el + n := len(old) + item := old[n-1] + old[n-1] = Event{} // avoid memory leak + item.index = -1 // for safety + *el = old[0 : n-1] + return item +} diff --git a/internal/scheduler/simulator/metrics.go b/internal/scheduler/simulator/metrics.go new file mode 100644 index 00000000000..e213ac2f8e4 --- /dev/null +++ b/internal/scheduler/simulator/metrics.go @@ -0,0 +1,104 @@ +package simulator + +import ( + "context" + "fmt" + "github.com/armadaproject/armada/pkg/armadaevents" + "strings" + "time" +) + +type MetricsCollector struct { + c <-chan *armadaevents.EventSequence + Total Metrics + MetricsByQueue map[string]Metrics + LastSeenEvent *armadaevents.EventSequence_Event +} + +func NewMetricsCollector(c <-chan *armadaevents.EventSequence) *MetricsCollector { + return &MetricsCollector{ + c: c, + MetricsByQueue: make(map[string]Metrics), + } +} + +func (mc *MetricsCollector) String() string { + var sb strings.Builder + sb.WriteString("{") + sb.WriteString(fmt.Sprintf("Total: %s, Queues: {", mc.Total)) + + i := 0 + for queue, metrics := range mc.MetricsByQueue { + sb.WriteString(fmt.Sprintf("%s: %s", queue, metrics)) + i++ + if i != len(mc.MetricsByQueue) { + sb.WriteString(", ") + } + } + sb.WriteString("}}") + return sb.String() +} + +type Metrics struct { + LastJobSuccess time.Duration + NumEventsInTotal int + NumPreemptionEvents int + NumSchedulingEvents int + NumJobsSubmitted int + NumSchedules int + NumSuccesses int +} + +func (m Metrics) String() string { + return fmt.Sprintf( + "{LastJobSuccess: %s, NumEventsInTotal: %d, NumPreemptionEvents: %d, NumSchedulingEvents: %d}", + m.LastJobSuccess, m.NumEventsInTotal, m.NumPreemptionEvents, m.NumSchedulingEvents, + ) +} + +func (mc *MetricsCollector) Run(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + case eventSequence, ok := <-mc.c: + if !ok { + return nil + } + mc.addEventSequence(eventSequence) + mc.logLiveMetrics() + } + } +} + +func (mc *MetricsCollector) addEventSequence(eventSequence *armadaevents.EventSequence) { + queue := eventSequence.Queue + mc.Total.NumEventsInTotal += 1 + entry := mc.MetricsByQueue[queue] + entry.NumEventsInTotal += 1 + for _, event := range eventSequence.Events { + mc.LastSeenEvent = event + switch event.GetEvent().(type) { + case *armadaevents.EventSequence_Event_SubmitJob: + mc.Total.NumJobsSubmitted += 1 + entry.NumJobsSubmitted += 1 + case *armadaevents.EventSequence_Event_JobRunLeased: + mc.Total.NumSchedulingEvents += 1 + entry.NumSchedulingEvents += 1 + case *armadaevents.EventSequence_Event_JobRunPreempted: + mc.Total.NumPreemptionEvents += 1 + entry.NumPreemptionEvents += 1 + case *armadaevents.EventSequence_Event_JobSucceeded: + mc.Total.LastJobSuccess = event.Created.Sub(time.Time{}) + entry.LastJobSuccess = event.Created.Sub(time.Time{}) + mc.Total.NumSuccesses += 1 + } + } + mc.MetricsByQueue[queue] = entry +} + +func (mc *MetricsCollector) logLiveMetrics() { + if mc.Total.NumEventsInTotal%500 == 0 { + fmt.Printf("Total Events: %d, Total Successes: %d, Total Preemptions: %d, Total Submissions: %d, Total Queued: %d, Time Simulated %s, Last Seen Event %s\n", mc.Total.NumEventsInTotal, mc.Total.NumSuccesses, mc.Total.NumPreemptionEvents, mc.Total.NumJobsSubmitted, mc.Total.NumJobsSubmitted-mc.Total.NumSuccesses, mc.Total.LastJobSuccess, EventSummary(mc.LastSeenEvent)) + } +} diff --git a/internal/scheduler/simulator/runner.go b/internal/scheduler/simulator/runner.go new file mode 100644 index 00000000000..18ea1995393 --- /dev/null +++ b/internal/scheduler/simulator/runner.go @@ -0,0 +1,229 @@ +package simulator + +import ( + "bytes" + "fmt" + "github.com/armadaproject/armada/internal/common/armadacontext" + "os" + "path/filepath" + "strings" + + "github.com/mattn/go-zglob" + "github.com/pkg/errors" + "github.com/renstrom/shortuuid" + "github.com/spf13/viper" + "k8s.io/apimachinery/pkg/util/yaml" + + "github.com/armadaproject/armada/internal/armada/configuration" + commonconfig "github.com/armadaproject/armada/internal/common/config" +) + +func Simulate(ctx *armadacontext.Context, clusterSpecsPattern, workloadSpecsPattern, schedulingConfigsPattern string) error { + clusterSpecs, err := ClusterSpecsFromPattern(clusterSpecsPattern) + if err != nil { + return err + } + workloadSpecs, err := WorkloadFromPattern(workloadSpecsPattern) + if err != nil { + return err + } + schedulingConfigs, err := SchedulingConfigsFromPattern(schedulingConfigsPattern) + if err != nil { + return err + } + g, ctx := armadacontext.ErrGroup(ctx) + for _, clusterSpec := range clusterSpecs { + for _, workloadSpec := range workloadSpecs { + for _, schedulingConfig := range schedulingConfigs { + s, err := NewSimulator(clusterSpec, workloadSpec, schedulingConfig) + if err != nil { + return err + } + g.Go(func() error { + return s.Run(ctx) + }) + } + } + } + return g.Wait() +} + +func SchedulingConfigsByFilePathFromPattern(pattern string) (map[string]configuration.SchedulingConfig, error) { + filePaths, err := zglob.Glob(pattern) + if err != nil { + return nil, errors.WithStack(err) + } + + filePathConfigMap := make(map[string]configuration.SchedulingConfig) + for _, path := range filePaths { + config, err := SchedulingConfigsFromFilePaths(filePaths) + if err != nil { + return nil, err + } + filePathConfigMap[path] = config[0] + } + return filePathConfigMap, nil +} + +func SchedulingConfigsFromPattern(pattern string) ([]configuration.SchedulingConfig, error) { + filePaths, err := zglob.Glob(pattern) + if err != nil { + return nil, errors.WithStack(err) + } + return SchedulingConfigsFromFilePaths(filePaths) +} + +func SchedulingConfigsFromFilePaths(filePaths []string) ([]configuration.SchedulingConfig, error) { + rv := make([]configuration.SchedulingConfig, len(filePaths)) + for i, filePath := range filePaths { + config, err := SchedulingConfigFromFilePath(filePath) + if err != nil { + return nil, err + } + rv[i] = config + } + return rv, nil +} + +func SchedulingConfigFromFilePath(filePath string) (configuration.SchedulingConfig, error) { + config := configuration.SchedulingConfig{} + v := viper.NewWithOptions(viper.KeyDelimiter("::")) + v.SetConfigFile(filePath) + if err := v.ReadInConfig(); err != nil { + return config, errors.WithStack(err) + } + if err := v.Unmarshal(&config, commonconfig.CustomHooks...); err != nil { + return config, errors.WithStack(err) + } + return config, nil +} + +func ClusterSpecsFromPattern(pattern string) ([]*ClusterSpec, error) { + filePaths, err := zglob.Glob(pattern) + if err != nil { + return nil, errors.WithStack(err) + } + return ClusterSpecsFromFilePaths(filePaths) +} + +func WorkloadFromPattern(pattern string) ([]*WorkloadSpec, error) { + filePaths, err := zglob.Glob(pattern) + if err != nil { + return nil, errors.WithStack(err) + } + return WorkloadSpecsFromFilePaths(filePaths) +} + +func ClusterSpecsFromFilePaths(filePaths []string) ([]*ClusterSpec, error) { + rv := make([]*ClusterSpec, len(filePaths)) + for i, filePath := range filePaths { + clusterSpec, err := ClusterSpecFromFilePath(filePath) + if err != nil { + return nil, err + } + rv[i] = clusterSpec + } + return rv, nil +} + +func WorkloadSpecsFromFilePaths(filePaths []string) ([]*WorkloadSpec, error) { + rv := make([]*WorkloadSpec, len(filePaths)) + for i, filePath := range filePaths { + workloadSpec, err := WorkloadSpecFromFilePath(filePath) + if err != nil { + return nil, err + } + rv[i] = workloadSpec + } + return rv, nil +} + +func ClusterSpecFromFilePath(filePath string) (*ClusterSpec, error) { + yamlBytes, err := os.ReadFile(filePath) + if err != nil { + return nil, errors.WithStack(err) + } + if len(yamlBytes) == 0 { + return nil, errors.Errorf("%s does not exist or is empty", filePath) + } + var clusterSpec ClusterSpec + if err := unmarshalYamlBytes(yamlBytes, &clusterSpec); err != nil { + return nil, err + } + + // If no test name is provided, set it to be the filename. + if clusterSpec.Name == "" { + fileName := filepath.Base(filePath) + fileName = strings.TrimSuffix(fileName, filepath.Ext(fileName)) + clusterSpec.Name = fileName + } + initialiseClusterSpec(&clusterSpec) + + return &clusterSpec, nil +} + +func WorkloadSpecFromFilePath(filePath string) (*WorkloadSpec, error) { + yamlBytes, err := os.ReadFile(filePath) + if err != nil { + return nil, errors.WithStack(err) + } + if len(yamlBytes) == 0 { + return nil, errors.Errorf("%s does not exist or is empty", filePath) + } + var workloadSpec WorkloadSpec + if err := unmarshalYamlBytes(yamlBytes, &workloadSpec); err != nil { + return nil, err + } + + // If no test name is provided, set it to be the filename. + if workloadSpec.Name == "" { + fileName := filepath.Base(filePath) + fileName = strings.TrimSuffix(fileName, filepath.Ext(fileName)) + workloadSpec.Name = fileName + } + + // Generate random ids for any job templates without an explicitly set id. + for _, queue := range workloadSpec.Queues { + for j, jobTemplate := range queue.JobTemplates { + if jobTemplate.Id == "" { + jobTemplate.Id = shortuuid.New() + } + queue.JobTemplates[j] = jobTemplate + } + } + initialiseWorkloadSpec(&workloadSpec) + + return &workloadSpec, nil +} + +func initialiseClusterSpec(clusterSpec *ClusterSpec) { + // Assign names to executors with none specified. + for _, pool := range clusterSpec.Pools { + for i, executorGroup := range pool.ClusterGroups { + for j, executor := range executorGroup.Clusters { + if executor.Name == "" { + executor.Name = fmt.Sprintf("%s-%d-%d", pool.Name, i, j) + } + } + } + } +} + +func initialiseWorkloadSpec(workloadSpec *WorkloadSpec) { + // Assign names to jobTemplates with none specified. + for _, queue := range workloadSpec.Queues { + for i, jobTemplate := range queue.JobTemplates { + if jobTemplate.Id == "" { + jobTemplate.Id = fmt.Sprintf("%s-%d", queue.Name, i) + } + jobTemplate.Queue = queue.Name + } + } +} + +func unmarshalYamlBytes(yamlBytes []byte, dst any) error { + if err := yaml.NewYAMLOrJSONDecoder(bytes.NewReader(yamlBytes), 128).Decode(dst); err != nil { + return errors.WithStack(err) + } + return nil +} diff --git a/internal/scheduler/simulator/simulator.go b/internal/scheduler/simulator/simulator.go index 1c282e8c303..9b348915fe8 100644 --- a/internal/scheduler/simulator/simulator.go +++ b/internal/scheduler/simulator/simulator.go @@ -1,29 +1,23 @@ package simulator import ( - "bytes" "container/heap" - "fmt" - "os" - "path/filepath" - "strings" + fmt "fmt" + "github.com/armadaproject/armada/internal/common/armadacontext" + "github.com/golang/protobuf/proto" + log "github.com/sirupsen/logrus" + "math" + "math/rand" "time" - "github.com/caarlos0/log" - "github.com/mattn/go-zglob" "github.com/oklog/ulid" "github.com/pkg/errors" - "github.com/renstrom/shortuuid" - "github.com/spf13/viper" "golang.org/x/exp/maps" "golang.org/x/exp/slices" "golang.org/x/time/rate" v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/util/yaml" "github.com/armadaproject/armada/internal/armada/configuration" - "github.com/armadaproject/armada/internal/common/armadacontext" - commonconfig "github.com/armadaproject/armada/internal/common/config" armadaslices "github.com/armadaproject/armada/internal/common/slices" "github.com/armadaproject/armada/internal/common/util" "github.com/armadaproject/armada/internal/scheduler" @@ -32,14 +26,15 @@ import ( "github.com/armadaproject/armada/internal/scheduler/fairness" "github.com/armadaproject/armada/internal/scheduler/jobdb" "github.com/armadaproject/armada/internal/scheduler/nodedb" - "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" + schedulerobjects "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" "github.com/armadaproject/armada/internal/scheduleringester" "github.com/armadaproject/armada/pkg/armadaevents" ) // Simulator captures the parameters and state of the Armada simulator. type Simulator struct { - testCase *TestCase + ClusterSpec *ClusterSpec + WorkloadSpec *WorkloadSpec schedulingConfig configuration.SchedulingConfig // Map from jobId to the jobTemplate from which the job was created. jobTemplateByJobId map[string]*JobTemplate @@ -60,12 +55,17 @@ type Simulator struct { allocationByPoolAndQueueAndPriorityClass map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string] // Total resources across all executorGroups for each pool. totalResourcesByPool map[string]schedulerobjects.ResourceList + // Indicates whether a job has been submitted or terminated since the last scheduling round. + shouldSchedule bool // Current simulated time. time time.Time // Sequence number of the next event to be published. sequenceNumber int // Events stored in a priority queue ordered by submit time. eventLog EventLog + // Simulated events are emitted on these output channels. + // Create a channel by calling s.Output() before running the simulator. + outputs []chan *armadaevents.EventSequence // Simulated events are emitted on this channel in order. c chan *armadaevents.EventSequence @@ -75,39 +75,153 @@ type Simulator struct { limiterByQueue map[string]*rate.Limiter } -func NewSimulator(testCase *TestCase, schedulingConfig configuration.SchedulingConfig) (*Simulator, error) { - initialiseTestCase(testCase) - if err := validateTestCase(testCase); err != nil { +func NewSimulator(inputClusterSpec *ClusterSpec, inputWorkloadSpec *WorkloadSpec, schedulingConfig configuration.SchedulingConfig) (*Simulator, error) { + protoClusterSpec, err := proto.Marshal(inputClusterSpec) + if err != nil { + return nil, err + } + protoWorkloadSpec, err := proto.Marshal(inputWorkloadSpec) + if err != nil { + return nil, err + } + var clusterSpec ClusterSpec + err = proto.Unmarshal(protoClusterSpec, &clusterSpec) + if err != nil { + return nil, err + } + var workloadSpec WorkloadSpec + err = proto.Unmarshal(protoWorkloadSpec, &workloadSpec) + if err != nil { return nil, err } - // Setup nodes. - nodeDbByPoolAndExecutorGroup := make(map[string][]*nodedb.NodeDb) - totalResourcesByPool := make(map[string]schedulerobjects.ResourceList) - poolByNodeId := make(map[string]string) - // executorGroupByExecutor := make(map[string]string) - nodeDbByExecutorName := make(map[string]*nodedb.NodeDb) - for _, pool := range testCase.Pools { + initialiseClusterSpec(&clusterSpec) + initialiseWorkloadSpec(&workloadSpec) + if err := validateClusterSpec(&clusterSpec); err != nil { + return nil, err + } + if err := validateWorkloadSpec(&workloadSpec); err != nil { + return nil, err + } + s := &Simulator{ + ClusterSpec: &clusterSpec, + WorkloadSpec: &workloadSpec, + schedulingConfig: schedulingConfig, + jobTemplateByJobId: make(map[string]*JobTemplate), + jobTemplatesByDependencyIds: make(map[string]map[string]*JobTemplate), + activeJobTemplatesById: make(map[string]*JobTemplate), + jobDb: jobdb.NewJobDb(), + nodeDbByPoolAndExecutorGroup: make(map[string][]*nodedb.NodeDb), + poolByNodeId: make(map[string]string), + nodeDbByExecutorName: make(map[string]*nodedb.NodeDb), + allocationByPoolAndQueueAndPriorityClass: make(map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string]), + totalResourcesByPool: make(map[string]schedulerobjects.ResourceList), + limiter: rate.NewLimiter( + rate.Limit(schedulingConfig.MaximumSchedulingRate), + schedulingConfig.MaximumSchedulingBurst, + ), + limiterByQueue: make(map[string]*rate.Limiter), + } + if err := s.setupClusters(); err != nil { + return nil, err + } + if err := s.bootstrapWorkload(); err != nil { + return nil, err + } + return s, nil +} + +// Run runs the scheduler until all jobs have finished successfully. +func (s *Simulator) Run(ctx *armadacontext.Context) error { + // Bootstrap the simulator by triggering the scheduler to run. + // Then run the scheduler until all jobs have completed. + defer func() { + for _, c := range s.outputs { + close(c) + } + }() + s.pushScheduleEvent(s.time) + for s.eventLog.Len() > 0 { + select { + case <-ctx.Done(): + return ctx.Err() + default: + event := heap.Pop(&s.eventLog).(Event) + if err := s.handleSimulatorEvent(ctx, event); err != nil { + return err + } + } + } + return nil +} + +func (s *Simulator) Output() <-chan *armadaevents.EventSequence { + c := make(chan *armadaevents.EventSequence, 128) + s.outputs = append(s.outputs, c) + return c +} + +func validateClusterSpec(clusterSpec *ClusterSpec) error { + poolNames := util.Map(clusterSpec.Pools, func(pool *Pool) string { return pool.Name }) + if !slices.Equal(poolNames, armadaslices.Unique(poolNames)) { + return errors.Errorf("duplicate pool name: %v", poolNames) + } + + executorNames := make([]string, 0) + for _, pool := range clusterSpec.Pools { + for _, executorGroup := range pool.ClusterGroups { + for _, executor := range executorGroup.Clusters { + executorNames = append(executorNames, executor.Name) + } + } + } + if !slices.Equal(executorNames, armadaslices.Unique(executorNames)) { + return errors.Errorf("duplicate executor name: %v", executorNames) + } + return nil +} + +func validateWorkloadSpec(workloadSpec *WorkloadSpec) error { + queueNames := util.Map(workloadSpec.Queues, func(queue *Queue) string { return queue.Name }) + if !slices.Equal(queueNames, armadaslices.Unique(queueNames)) { + return errors.Errorf("duplicate queue name: %v", queueNames) + } + jobTemplateIdSlices := util.Map(workloadSpec.Queues, func(queue *Queue) []string { + return util.Map(queue.JobTemplates, func(template *JobTemplate) string { return template.Id }) + }) + jobTemplateIds := make([]string, 0) + for _, singleQueueTemplateIds := range jobTemplateIdSlices { + jobTemplateIds = append(jobTemplateIds, singleQueueTemplateIds...) + } + if !slices.Equal(jobTemplateIds, armadaslices.Unique(jobTemplateIds)) { + return errors.Errorf("duplicate job template ids: %v", jobTemplateIds) + } + + return nil +} + +func (s *Simulator) setupClusters() error { + for _, pool := range s.ClusterSpec.Pools { totalResourcesForPool := schedulerobjects.ResourceList{} - for executorGroupIndex, executorGroup := range pool.ExecutorGroups { + for executorGroupIndex, executorGroup := range pool.ClusterGroups { nodeDb, err := nodedb.NewNodeDb( - schedulingConfig.Preemption.PriorityClasses, - schedulingConfig.MaxExtraNodesToConsider, - schedulingConfig.IndexedResources, - schedulingConfig.IndexedTaints, - schedulingConfig.IndexedNodeLabels, + s.schedulingConfig.Preemption.PriorityClasses, + s.schedulingConfig.MaxExtraNodesToConsider, + s.schedulingConfig.IndexedResources, + s.schedulingConfig.IndexedTaints, + s.schedulingConfig.IndexedNodeLabels, ) if err != nil { - return nil, err + return err } - for executorIndex, executor := range executorGroup.Executors { + for executorIndex, executor := range executorGroup.Clusters { executorName := fmt.Sprintf("%s-%d-%d", pool.Name, executorGroupIndex, executorIndex) - nodeDbByExecutorName[executorName] = nodeDb + s.nodeDbByExecutorName[executorName] = nodeDb for nodeTemplateIndex, nodeTemplate := range executor.NodeTemplates { for i := 0; i < int(nodeTemplate.Number); i++ { nodeId := fmt.Sprintf("%s-%d-%d-%d-%d", pool.Name, executorGroupIndex, executorIndex, nodeTemplateIndex, i) allocatableByPriorityAndResource := make(map[int32]schedulerobjects.ResourceList) - for _, priorityClass := range schedulingConfig.Preemption.PriorityClasses { + for _, priorityClass := range s.schedulingConfig.Preemption.PriorityClasses { allocatableByPriorityAndResource[priorityClass.Priority] = nodeTemplate.TotalResources.DeepCopy() } node := &schedulerobjects.Node{ @@ -122,47 +236,31 @@ func NewSimulator(testCase *TestCase, schedulingConfig configuration.SchedulingC txn := nodeDb.Txn(true) if err := nodeDb.CreateAndInsertWithApiJobsWithTxn(txn, nil, node); err != nil { txn.Abort() - return nil, err + return err } txn.Commit() - poolByNodeId[nodeId] = pool.Name + s.poolByNodeId[nodeId] = pool.Name } } } - nodeDbByPoolAndExecutorGroup[pool.Name] = append(nodeDbByPoolAndExecutorGroup[pool.Name], nodeDb) + s.nodeDbByPoolAndExecutorGroup[pool.Name] = append(s.nodeDbByPoolAndExecutorGroup[pool.Name], nodeDb) totalResourcesForPool.Add(nodeDb.TotalResources()) } - totalResourcesByPool[pool.Name] = totalResourcesForPool - } - s := &Simulator{ - testCase: testCase, - schedulingConfig: schedulingConfig, - jobTemplateByJobId: make(map[string]*JobTemplate), - jobTemplatesByDependencyIds: make(map[string]map[string]*JobTemplate), - activeJobTemplatesById: make(map[string]*JobTemplate), - jobDb: jobdb.NewJobDb(), - poolByNodeId: poolByNodeId, - nodeDbByPoolAndExecutorGroup: nodeDbByPoolAndExecutorGroup, - nodeDbByExecutorName: nodeDbByExecutorName, - allocationByPoolAndQueueAndPriorityClass: make(map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string]), - totalResourcesByPool: totalResourcesByPool, - c: make(chan *armadaevents.EventSequence), - limiter: rate.NewLimiter( - rate.Limit(schedulingConfig.MaximumSchedulingRate), - schedulingConfig.MaximumSchedulingBurst, - ), - limiterByQueue: make(map[string]*rate.Limiter), + s.totalResourcesByPool[pool.Name] = totalResourcesForPool } + return nil +} +func (s *Simulator) bootstrapWorkload() error { // Mark all jobTemplates as active. - for _, queue := range testCase.Queues { + for _, queue := range s.WorkloadSpec.Queues { for _, jobTemplate := range queue.JobTemplates { s.activeJobTemplatesById[jobTemplate.Id] = jobTemplate } } // Publish submitJob messages for all jobTemplates without dependencies. - for _, queue := range testCase.Queues { + for _, queue := range s.WorkloadSpec.Queues { for _, jobTemplate := range queue.JobTemplates { if len(jobTemplate.Dependencies) > 0 { continue @@ -179,7 +277,7 @@ func NewSimulator(testCase *TestCase, schedulingConfig configuration.SchedulingC eventSequence.Events = append( eventSequence.Events, &armadaevents.EventSequence_Event{ - Created: pointer(maxTime(s.time, jobTemplate.MinSubmitTime)), + Created: pointer(s.time.Add(jobTemplate.EarliestSubmitTime)), Event: &armadaevents.EventSequence_Event_SubmitJob{ SubmitJob: submitJobFromJobTemplate(jobId, jobTemplate), }, @@ -194,12 +292,12 @@ func NewSimulator(testCase *TestCase, schedulingConfig configuration.SchedulingC } // Setup the jobTemplate dependency map. - for _, queue := range testCase.Queues { + for _, queue := range s.WorkloadSpec.Queues { for _, jobTemplate := range queue.JobTemplates { for _, dependencyJobTemplateId := range jobTemplate.Dependencies { dependencyJobTemplate, ok := s.activeJobTemplatesById[dependencyJobTemplateId] if !ok { - return nil, errors.Errorf( + return errors.Errorf( "jobTemplate %s depends on jobTemplate %s, which does not exist", jobTemplate.Id, dependencyJobTemplate.Id, ) @@ -213,64 +311,9 @@ func NewSimulator(testCase *TestCase, schedulingConfig configuration.SchedulingC } } } - - // Publish scheduleEvent. - s.pushScheduleEvent(s.time.Add(10 * time.Second)) - return s, nil -} - -func (s *Simulator) C() <-chan *armadaevents.EventSequence { - return s.c -} - -func validateTestCase(testCase *TestCase) error { - poolNames := util.Map(testCase.Pools, func(pool *Pool) string { return pool.Name }) - if !slices.Equal(poolNames, armadaslices.Unique(poolNames)) { - return errors.Errorf("duplicate pool name: %v", poolNames) - } - - executorNames := make([]string, 0) - for _, pool := range testCase.Pools { - for _, executorGroup := range pool.ExecutorGroups { - for _, executor := range executorGroup.Executors { - executorNames = append(executorNames, executor.Name) - } - } - } - if !slices.Equal(executorNames, armadaslices.Unique(executorNames)) { - return errors.Errorf("duplicate executor name: %v", executorNames) - } - - queueNames := util.Map(testCase.Queues, func(queue Queue) string { return queue.Name }) - if !slices.Equal(queueNames, armadaslices.Unique(queueNames)) { - return errors.Errorf("duplicate queue name: %v", queueNames) - } return nil } -func initialiseTestCase(testCase *TestCase) { - // Assign names to executors with none specified. - for _, pool := range testCase.Pools { - for i, executorGroup := range pool.ExecutorGroups { - for j, executor := range executorGroup.Executors { - if executor.Name == "" { - executor.Name = fmt.Sprintf("%s-%d-%d", pool.Name, i, j) - } - } - } - } - - // Assign names to jobTemplates with none specified. - for _, queue := range testCase.Queues { - for i, jobTemplate := range queue.JobTemplates { - if jobTemplate.Id == "" { - jobTemplate.Id = fmt.Sprintf("%s-%d", queue.Name, i) - } - jobTemplate.Queue = queue.Name - } - } -} - func submitJobFromJobTemplate(jobId ulid.ULID, jobTemplate *JobTemplate) *armadaevents.SubmitJob { return &armadaevents.SubmitJob{ JobId: armadaevents.ProtoUuidFromUlid(jobId), @@ -325,67 +368,7 @@ func (s *Simulator) pushScheduleEvent(time time.Time) { s.sequenceNumber++ } -type EventLog []Event - -type Event struct { - // Time at which the event was submitted. - time time.Time - // Each event is assigned a sequence number. - // Events with equal time are ordered by their sequence number. - sequenceNumber int - // One of armadaevents.EventSequence or scheduleEvent.. - eventSequenceOrScheduleEvent any - // Maintained by the heap.Interface methods. - index int -} - -func (el EventLog) Len() int { return len(el) } - -func (el EventLog) Less(i, j int) bool { - if el[i].time == el[j].time { - return el[i].sequenceNumber < el[j].sequenceNumber - } - return el[j].time.After(el[i].time) -} - -func (el EventLog) Swap(i, j int) { - el[i], el[j] = el[j], el[i] - el[i].index = i - el[j].index = j -} - -func (el *EventLog) Push(x any) { - n := len(*el) - item := x.(Event) - item.index = n - *el = append(*el, item) -} - -func (el *EventLog) Pop() any { - old := *el - n := len(old) - item := old[n-1] - old[n-1] = Event{} // avoid memory leak - item.index = -1 // for safety - *el = old[0 : n-1] - return item -} - -// scheduleEvent is an event indicating the scheduler should be run. -type scheduleEvent struct{} - -func (s *Simulator) Run() error { - defer close(s.c) - for s.eventLog.Len() > 0 { - event := heap.Pop(&s.eventLog).(Event) - if err := s.handleSimulatorEvent(event); err != nil { - return err - } - } - return nil -} - -func (s *Simulator) handleSimulatorEvent(event Event) error { +func (s *Simulator) handleSimulatorEvent(ctx *armadacontext.Context, event Event) error { s.time = event.time switch e := event.eventSequenceOrScheduleEvent.(type) { case *armadaevents.EventSequence: @@ -393,19 +376,28 @@ func (s *Simulator) handleSimulatorEvent(event Event) error { return err } case scheduleEvent: - if err := s.handleScheduleEvent(); err != nil { + if err := s.handleScheduleEvent(ctx); err != nil { return err } } return nil } -func (s *Simulator) handleScheduleEvent() error { +func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { + // Schedule the next run of the scheduler, unless there are no more active jobTemplates. + // TODO: Make timeout configurable. + if len(s.activeJobTemplatesById) > 0 { + s.pushScheduleEvent(s.time.Add(10 * time.Second)) + } + if !s.shouldSchedule { + return nil + } + var eventSequences []*armadaevents.EventSequence txn := s.jobDb.WriteTxn() defer txn.Abort() - for _, pool := range s.testCase.Pools { - for i := range pool.ExecutorGroups { + for _, pool := range s.ClusterSpec.Pools { + for i := range pool.ClusterGroups { nodeDb := s.nodeDbByPoolAndExecutorGroup[pool.Name][i] if err := nodeDb.Reset(); err != nil { return err @@ -427,8 +419,9 @@ func (s *Simulator) handleScheduleEvent() error { s.limiter, totalResources, ) + sctx.Started = s.time - for _, queue := range s.testCase.Queues { + for _, queue := range s.WorkloadSpec.Queues { limiter, ok := s.limiterByQueue[queue.Name] if !ok { limiter = rate.NewLimiter( @@ -450,7 +443,7 @@ func (s *Simulator) handleScheduleEvent() error { constraints := schedulerconstraints.SchedulingConstraintsFromSchedulingConfig( pool.Name, totalResources, - // Minimum job size not not used for simulation; use taints/tolerations instead. + // Minimum job size not used for simulation; use taints/tolerations instead. schedulerobjects.ResourceList{}, s.schedulingConfig, ) @@ -470,7 +463,6 @@ func (s *Simulator) handleScheduleEvent() error { if s.schedulingConfig.EnableNewPreemptionStrategy { sch.EnableNewPreemptionStrategy() } - ctx := armadacontext.Background() result, err := sch.Schedule(ctx) if err != nil { return err @@ -536,7 +528,11 @@ func (s *Simulator) handleScheduleEvent() error { } eventSequences, err = scheduler.AppendEventSequencesFromUnschedulableJobs(eventSequences, result.FailedJobs, s.time) if err != nil { - return err + return err + + // If nothing changed, we're in steady state and can safely skip scheduling until something external has changed. + if len(result.ScheduledJobs) == 0 && len(result.PreemptedJobs) == 0 && len(result.FailedJobs) == 0 { + s.shouldSchedule = false } } } @@ -546,12 +542,6 @@ func (s *Simulator) handleScheduleEvent() error { for _, eventSequence := range eventSequences { s.pushEventSequence(eventSequence) } - - // Schedule the next run of the scheduler, unless there are no more active jobTemplates. - // TODO: Make timeout configurable. - if len(s.activeJobTemplatesById) > 0 { - s.pushScheduleEvent(s.time.Add(10 * time.Second)) - } return nil } @@ -565,12 +555,15 @@ func (s *Simulator) handleEventSequence(es *armadaevents.EventSequence) error { var err error = nil switch eventType := event.GetEvent().(type) { case *armadaevents.EventSequence_Event_SubmitJob: + s.shouldSchedule = true ok, err = s.handleSubmitJob(txn, event.GetSubmitJob(), *event.Created, es) case *armadaevents.EventSequence_Event_JobRunLeased: ok, err = s.handleJobRunLeased(txn, event.GetJobRunLeased()) case *armadaevents.EventSequence_Event_JobSucceeded: + s.shouldSchedule = true ok, err = s.handleJobSucceeded(txn, event.GetJobSucceeded()) case *armadaevents.EventSequence_Event_JobRunPreempted: + s.shouldSchedule = true ok, err = s.handleJobRunPreempted(txn, event.GetJobRunPreempted()) case *armadaevents.EventSequence_Event_ReprioritisedJob, *armadaevents.EventSequence_Event_JobDuplicateDetected, @@ -604,7 +597,9 @@ func (s *Simulator) handleEventSequence(es *armadaevents.EventSequence) error { txn.Commit() es.Events = eventsToPublish if len(es.Events) > 0 { - s.c <- es + for _, c := range s.outputs { + c <- es + } } return nil } @@ -636,12 +631,15 @@ func (s *Simulator) handleSubmitJob(txn *jobdb.Txn, e *armadaevents.SubmitJob, t func (s *Simulator) handleJobRunLeased(txn *jobdb.Txn, e *armadaevents.JobRunLeased) (bool, error) { jobId := armadaevents.UlidFromProtoUuid(e.JobId).String() job := s.jobDb.GetById(txn, jobId) - // TODO: Randomise runtime. jobTemplate := s.jobTemplateByJobId[jobId] if jobTemplate == nil { return false, errors.Errorf("no jobTemplate associated with job %s", jobId) } - jobSuccessTime := s.time.Add(time.Duration(jobTemplate.RuntimeMean) * time.Second) + trueVarianceSeconds := float64(jobTemplate.RuntimeVariance.Nanoseconds()) / 1e9 + minRuntimeSeconds := float64(jobTemplate.RuntimeMin.Nanoseconds()) / 1e9 + rateParam := 1.0 / math.Sqrt(trueVarianceSeconds) + sampledFromExponential := rand.ExpFloat64() / rateParam + jobSuccessTime := s.time.Add(time.Duration(1e9 * (minRuntimeSeconds + sampledFromExponential))) s.pushEventSequence( &armadaevents.EventSequence{ Queue: job.Queue(), @@ -704,7 +702,8 @@ func (s *Simulator) handleJobSucceeded(txn *jobdb.Txn, e *armadaevents.JobSuccee eventSequence.Events = append( eventSequence.Events, &armadaevents.EventSequence_Event{ - Created: pointer(maxTime(s.time, dependentJobTemplate.MinSubmitTime)), + // EarliestSubmitTimeFromDependencyCompletion must be positive + Created: pointer(maxTime(time.Time{}.Add(dependentJobTemplate.EarliestSubmitTime), s.time.Add(dependentJobTemplate.EarliestSubmitTimeFromDependencyCompletion))), Event: &armadaevents.EventSequence_Event_SubmitJob{ SubmitJob: submitJobFromJobTemplate(jobId, dependentJobTemplate), }, @@ -777,110 +776,6 @@ func (s *Simulator) handleJobRunPreempted(txn *jobdb.Txn, e *armadaevents.JobRun return true, nil } -// func (a *App) TestPattern(ctx *context.Context, pattern string) (*TestSuiteReport, error) { -// testSpecs, err := TestSpecsFromPattern(pattern) -// if err != nil { -// return nil, err -// } -// return a.RunTests(ctx, testSpecs) -// } - -func SchedulingConfigsFromPattern(pattern string) ([]configuration.SchedulingConfig, error) { - filePaths, err := zglob.Glob(pattern) - if err != nil { - return nil, errors.WithStack(err) - } - return SchedulingConfigsFromFilePaths(filePaths) -} - -func SchedulingConfigsFromFilePaths(filePaths []string) ([]configuration.SchedulingConfig, error) { - rv := make([]configuration.SchedulingConfig, len(filePaths)) - for i, filePath := range filePaths { - config, err := SchedulingConfigFromFilePath(filePath) - if err != nil { - return nil, err - } - rv[i] = config - } - return rv, nil -} - -func SchedulingConfigFromFilePath(filePath string) (configuration.SchedulingConfig, error) { - config := configuration.SchedulingConfig{} - v := viper.New() - v.SetConfigFile(filePath) - if err := v.ReadInConfig(); err != nil { - return config, errors.WithStack(err) - } - if err := v.Unmarshal(&config, commonconfig.CustomHooks...); err != nil { - return config, errors.WithStack(err) - } - return config, nil -} - -func TestCasesFromPattern(pattern string) ([]*TestCase, error) { - filePaths, err := zglob.Glob(pattern) - if err != nil { - return nil, errors.WithStack(err) - } - return TestCasesFromFilePaths(filePaths) -} - -func TestCasesFromFilePaths(filePaths []string) ([]*TestCase, error) { - rv := make([]*TestCase, len(filePaths)) - for i, filePath := range filePaths { - testCase, err := TestCaseFromFilePath(filePath) - if err != nil { - return nil, err - } - rv[i] = testCase - } - return rv, nil -} - -func TestCaseFromFilePath(filePath string) (*TestCase, error) { - yamlBytes, err := os.ReadFile(filePath) - if err != nil { - return nil, errors.WithStack(err) - } - if len(yamlBytes) == 0 { - return nil, errors.Errorf("%s does not exist or is empty", filePath) - } - testCase, err := TestCaseFromBytes(yamlBytes) - if err != nil { - return nil, err - } - - // If no test name is provided, set it to be the filename. - if testCase.Name == "" { - fileName := filepath.Base(filePath) - fileName = strings.TrimSuffix(fileName, filepath.Ext(fileName)) - testCase.Name = fileName - } - - // Generate random ids for any job templates without an explicitly set id. - for i, queue := range testCase.Queues { - for j, jobTemplate := range queue.JobTemplates { - if jobTemplate.Id == "" { - jobTemplate.Id = shortuuid.New() - } - queue.JobTemplates[j] = jobTemplate - } - testCase.Queues[i] = queue - } - - return testCase, nil -} - -// TestCaseFromBytes unmarshalls bytes into a TestCase. -func TestCaseFromBytes(yamlBytes []byte) (*TestCase, error) { - var testCase TestCase - if err := yaml.NewYAMLOrJSONDecoder(bytes.NewReader(yamlBytes), 128).Decode(&testCase); err != nil { - return nil, errors.WithStack(err) - } - return &testCase, nil -} - func maxTime(a, b time.Time) time.Time { if a.Before(b) { return b diff --git a/internal/scheduler/simulator/simulator.pb.go b/internal/scheduler/simulator/simulator.pb.go index fdf830cd63c..507c1586abb 100644 --- a/internal/scheduler/simulator/simulator.pb.go +++ b/internal/scheduler/simulator/simulator.pb.go @@ -7,6 +7,7 @@ import ( encoding_binary "encoding/binary" fmt "fmt" schedulerobjects "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" + armadaevents "github.com/armadaproject/armada/pkg/armadaevents" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" _ "github.com/gogo/protobuf/types" @@ -30,29 +31,25 @@ var _ = time.Kitchen // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package -// TODO: -// Runtime family. -// Workflow manager delay. -// Job pending delay. -type TestCase struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - RandomSeed int64 `protobuf:"varint,2,opt,name=random_seed,json=randomSeed,proto3" json:"randomSeed,omitempty"` - Pools []*Pool `protobuf:"bytes,3,rep,name=pools,proto3" json:"pools,omitempty"` - Queues []Queue `protobuf:"bytes,4,rep,name=queues,proto3" json:"queues"` -} - -func (m *TestCase) Reset() { *m = TestCase{} } -func (m *TestCase) String() string { return proto.CompactTextString(m) } -func (*TestCase) ProtoMessage() {} -func (*TestCase) Descriptor() ([]byte, []int) { +type SimulationResult struct { + Events []*armadaevents.EventSequence `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` + ClusterSpec *ClusterSpec `protobuf:"bytes,2,opt,name=clusterSpec,proto3" json:"clusterSpec,omitempty"` + WorkloadSpec *WorkloadSpec `protobuf:"bytes,3,opt,name=workloadSpec,proto3" json:"workloadSpec,omitempty"` + SchedulingConfig string `protobuf:"bytes,4,opt,name=schedulingConfig,proto3" json:"schedulingConfig,omitempty"` +} + +func (m *SimulationResult) Reset() { *m = SimulationResult{} } +func (m *SimulationResult) String() string { return proto.CompactTextString(m) } +func (*SimulationResult) ProtoMessage() {} +func (*SimulationResult) Descriptor() ([]byte, []int) { return fileDescriptor_63baccdfe9127510, []int{0} } -func (m *TestCase) XXX_Unmarshal(b []byte) error { +func (m *SimulationResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *TestCase) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *SimulationResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_TestCase.Marshal(b, m, deterministic) + return xxx_messageInfo_SimulationResult.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -62,40 +59,152 @@ func (m *TestCase) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return b[:n], nil } } -func (m *TestCase) XXX_Merge(src proto.Message) { - xxx_messageInfo_TestCase.Merge(m, src) +func (m *SimulationResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_SimulationResult.Merge(m, src) } -func (m *TestCase) XXX_Size() int { +func (m *SimulationResult) XXX_Size() int { return m.Size() } -func (m *TestCase) XXX_DiscardUnknown() { - xxx_messageInfo_TestCase.DiscardUnknown(m) +func (m *SimulationResult) XXX_DiscardUnknown() { + xxx_messageInfo_SimulationResult.DiscardUnknown(m) } -var xxx_messageInfo_TestCase proto.InternalMessageInfo +var xxx_messageInfo_SimulationResult proto.InternalMessageInfo -func (m *TestCase) GetName() string { +func (m *SimulationResult) GetEvents() []*armadaevents.EventSequence { if m != nil { - return m.Name + return m.Events + } + return nil +} + +func (m *SimulationResult) GetClusterSpec() *ClusterSpec { + if m != nil { + return m.ClusterSpec + } + return nil +} + +func (m *SimulationResult) GetWorkloadSpec() *WorkloadSpec { + if m != nil { + return m.WorkloadSpec + } + return nil +} + +func (m *SimulationResult) GetSchedulingConfig() string { + if m != nil { + return m.SchedulingConfig } return "" } -func (m *TestCase) GetRandomSeed() int64 { +type ClusterSpec struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Pools []*Pool `protobuf:"bytes,2,rep,name=pools,proto3" json:"pools,omitempty"` +} + +func (m *ClusterSpec) Reset() { *m = ClusterSpec{} } +func (m *ClusterSpec) String() string { return proto.CompactTextString(m) } +func (*ClusterSpec) ProtoMessage() {} +func (*ClusterSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_63baccdfe9127510, []int{1} +} +func (m *ClusterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClusterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ClusterSpec.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ClusterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClusterSpec.Merge(m, src) +} +func (m *ClusterSpec) XXX_Size() int { + return m.Size() +} +func (m *ClusterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ClusterSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_ClusterSpec proto.InternalMessageInfo + +func (m *ClusterSpec) GetName() string { if m != nil { - return m.RandomSeed + return m.Name } - return 0 + return "" } -func (m *TestCase) GetPools() []*Pool { +func (m *ClusterSpec) GetPools() []*Pool { if m != nil { return m.Pools } return nil } -func (m *TestCase) GetQueues() []Queue { +type WorkloadSpec struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + RandomSeed int64 `protobuf:"varint,2,opt,name=random_seed,json=randomSeed,proto3" json:"randomSeed,omitempty"` + Queues []*Queue `protobuf:"bytes,3,rep,name=queues,proto3" json:"queues,omitempty"` +} + +func (m *WorkloadSpec) Reset() { *m = WorkloadSpec{} } +func (m *WorkloadSpec) String() string { return proto.CompactTextString(m) } +func (*WorkloadSpec) ProtoMessage() {} +func (*WorkloadSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_63baccdfe9127510, []int{2} +} +func (m *WorkloadSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WorkloadSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_WorkloadSpec.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *WorkloadSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_WorkloadSpec.Merge(m, src) +} +func (m *WorkloadSpec) XXX_Size() int { + return m.Size() +} +func (m *WorkloadSpec) XXX_DiscardUnknown() { + xxx_messageInfo_WorkloadSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_WorkloadSpec proto.InternalMessageInfo + +func (m *WorkloadSpec) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *WorkloadSpec) GetRandomSeed() int64 { + if m != nil { + return m.RandomSeed + } + return 0 +} + +func (m *WorkloadSpec) GetQueues() []*Queue { if m != nil { return m.Queues } @@ -103,15 +212,15 @@ func (m *TestCase) GetQueues() []Queue { } type Pool struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - ExecutorGroups []*ExecutorGroup `protobuf:"bytes,2,rep,name=executor_groups,json=executorGroups,proto3" json:"executorGroups,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + ClusterGroups []*ClusterGroup `protobuf:"bytes,2,rep,name=cluster_groups,json=clusterGroups,proto3" json:"clusterGroups,omitempty"` } func (m *Pool) Reset() { *m = Pool{} } func (m *Pool) String() string { return proto.CompactTextString(m) } func (*Pool) ProtoMessage() {} func (*Pool) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{1} + return fileDescriptor_63baccdfe9127510, []int{3} } func (m *Pool) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -147,29 +256,29 @@ func (m *Pool) GetName() string { return "" } -func (m *Pool) GetExecutorGroups() []*ExecutorGroup { +func (m *Pool) GetClusterGroups() []*ClusterGroup { if m != nil { - return m.ExecutorGroups + return m.ClusterGroups } return nil } -type ExecutorGroup struct { - Executors []*Executor `protobuf:"bytes,1,rep,name=executors,proto3" json:"executors,omitempty"` +type ClusterGroup struct { + Clusters []*Cluster `protobuf:"bytes,1,rep,name=clusters,proto3" json:"clusters,omitempty"` } -func (m *ExecutorGroup) Reset() { *m = ExecutorGroup{} } -func (m *ExecutorGroup) String() string { return proto.CompactTextString(m) } -func (*ExecutorGroup) ProtoMessage() {} -func (*ExecutorGroup) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{2} +func (m *ClusterGroup) Reset() { *m = ClusterGroup{} } +func (m *ClusterGroup) String() string { return proto.CompactTextString(m) } +func (*ClusterGroup) ProtoMessage() {} +func (*ClusterGroup) Descriptor() ([]byte, []int) { + return fileDescriptor_63baccdfe9127510, []int{4} } -func (m *ExecutorGroup) XXX_Unmarshal(b []byte) error { +func (m *ClusterGroup) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *ExecutorGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *ClusterGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_ExecutorGroup.Marshal(b, m, deterministic) + return xxx_messageInfo_ClusterGroup.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -179,42 +288,42 @@ func (m *ExecutorGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error return b[:n], nil } } -func (m *ExecutorGroup) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecutorGroup.Merge(m, src) +func (m *ClusterGroup) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClusterGroup.Merge(m, src) } -func (m *ExecutorGroup) XXX_Size() int { +func (m *ClusterGroup) XXX_Size() int { return m.Size() } -func (m *ExecutorGroup) XXX_DiscardUnknown() { - xxx_messageInfo_ExecutorGroup.DiscardUnknown(m) +func (m *ClusterGroup) XXX_DiscardUnknown() { + xxx_messageInfo_ClusterGroup.DiscardUnknown(m) } -var xxx_messageInfo_ExecutorGroup proto.InternalMessageInfo +var xxx_messageInfo_ClusterGroup proto.InternalMessageInfo -func (m *ExecutorGroup) GetExecutors() []*Executor { +func (m *ClusterGroup) GetClusters() []*Cluster { if m != nil { - return m.Executors + return m.Clusters } return nil } -type Executor struct { +type Cluster struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` NodeTemplates []*NodeTemplate `protobuf:"bytes,2,rep,name=node_templates,json=nodeTemplates,proto3" json:"nodeTemplates,omitempty"` } -func (m *Executor) Reset() { *m = Executor{} } -func (m *Executor) String() string { return proto.CompactTextString(m) } -func (*Executor) ProtoMessage() {} -func (*Executor) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{3} +func (m *Cluster) Reset() { *m = Cluster{} } +func (m *Cluster) String() string { return proto.CompactTextString(m) } +func (*Cluster) ProtoMessage() {} +func (*Cluster) Descriptor() ([]byte, []int) { + return fileDescriptor_63baccdfe9127510, []int{5} } -func (m *Executor) XXX_Unmarshal(b []byte) error { +func (m *Cluster) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *Executor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *Cluster) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_Executor.Marshal(b, m, deterministic) + return xxx_messageInfo_Cluster.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -224,26 +333,26 @@ func (m *Executor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { return b[:n], nil } } -func (m *Executor) XXX_Merge(src proto.Message) { - xxx_messageInfo_Executor.Merge(m, src) +func (m *Cluster) XXX_Merge(src proto.Message) { + xxx_messageInfo_Cluster.Merge(m, src) } -func (m *Executor) XXX_Size() int { +func (m *Cluster) XXX_Size() int { return m.Size() } -func (m *Executor) XXX_DiscardUnknown() { - xxx_messageInfo_Executor.DiscardUnknown(m) +func (m *Cluster) XXX_DiscardUnknown() { + xxx_messageInfo_Cluster.DiscardUnknown(m) } -var xxx_messageInfo_Executor proto.InternalMessageInfo +var xxx_messageInfo_Cluster proto.InternalMessageInfo -func (m *Executor) GetName() string { +func (m *Cluster) GetName() string { if m != nil { return m.Name } return "" } -func (m *Executor) GetNodeTemplates() []*NodeTemplate { +func (m *Cluster) GetNodeTemplates() []*NodeTemplate { if m != nil { return m.NodeTemplates } @@ -261,7 +370,7 @@ func (m *NodeTemplate) Reset() { *m = NodeTemplate{} } func (m *NodeTemplate) String() string { return proto.CompactTextString(m) } func (*NodeTemplate) ProtoMessage() {} func (*NodeTemplate) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{4} + return fileDescriptor_63baccdfe9127510, []int{6} } func (m *NodeTemplate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -328,7 +437,7 @@ func (m *Queue) Reset() { *m = Queue{} } func (m *Queue) String() string { return proto.CompactTextString(m) } func (*Queue) ProtoMessage() {} func (*Queue) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{5} + return fileDescriptor_63baccdfe9127510, []int{7} } func (m *Queue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -387,27 +496,30 @@ type JobTemplate struct { // Queue to which this template belongs. Populated automatically. Queue string `protobuf:"bytes,3,opt,name=queue,proto3" json:"queue,omitempty"` // Unique id for this template. An id is generated if empty. - Id string `protobuf:"bytes,4,opt,name=id,proto3" json:"id,omitempty"` - JobSet string `protobuf:"bytes,5,opt,name=job_set,json=jobSet,proto3" json:"jobSet,omitempty"` - QueuePriority uint32 `protobuf:"varint,6,opt,name=queue_priority,json=queuePriority,proto3" json:"queuePriority,omitempty"` - PriorityClassName string `protobuf:"bytes,7,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priorityClassName,omitempty"` - Requirements schedulerobjects.PodRequirements `protobuf:"bytes,8,opt,name=requirements,proto3" json:"requirements"` + Id string `protobuf:"bytes,4,opt,name=id,proto3" json:"id,omitempty"` + JobSet string `protobuf:"bytes,5,opt,name=job_set,json=jobSet,proto3" json:"jobSet,omitempty"` + QueuePriority uint32 `protobuf:"varint,6,opt,name=queue_priority,json=queuePriority,proto3" json:"queuePriority,omitempty"` + PriorityClassName string `protobuf:"bytes,7,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priorityClassName,omitempty"` + // Scheduling requirements for the pod embedded in the job. + Requirements schedulerobjects.PodRequirements `protobuf:"bytes,8,opt,name=requirements,proto3" json:"requirements"` // List of template ids that must be completed before this template is submitted. Dependencies []string `protobuf:"bytes,9,rep,name=dependencies,proto3" json:"dependencies,omitempty"` - // Minimum time from which jobs are created from this template. - MinSubmitTime time.Time `protobuf:"bytes,10,opt,name=min_submit_time,json=minSubmitTime,proto3,stdtime" json:"minSubmitTime"` - // Job runtime mean in seconds. - RuntimeMean int64 `protobuf:"varint,11,opt,name=runtime_mean,json=runtimeMean,proto3" json:"runtimeMean,omitempty"` - // Job runtime variance in seconds squared. - // If zero, runtime is deterministic. - RuntimeVariance int64 `protobuf:"varint,12,opt,name=runtime_variance,json=runtimeVariance,proto3" json:"runtimeVariance,omitempty"` + // Earliest time at which jobs from this template are submitted. + // Measured from the start of the simulation. + EarliestSubmitTime time.Duration `protobuf:"bytes,10,opt,name=earliest_submit_time,json=earliestSubmitTime,proto3,stdduration" json:"earliestSubmitTime"` + // Minimum job runtime when sampling from shifted exponential distribution. + RuntimeMin time.Duration `protobuf:"bytes,11,opt,name=runtime_min,json=runtimeMin,proto3,stdduration" json:"runtimeMin"` + // Job runtime variance. If zero, runtime is deterministic. + RuntimeVariance time.Duration `protobuf:"bytes,12,opt,name=runtime_variance,json=runtimeVariance,proto3,stdduration" json:"runtimeVariance"` + // Earliest time job can be submitted from when all dependencies have completed + EarliestSubmitTimeFromDependencyCompletion time.Duration `protobuf:"bytes,13,opt,name=earliest_submit_time_from_dependency_completion,json=earliestSubmitTimeFromDependencyCompletion,proto3,stdduration" json:"earliestSubmitTimeFromDependencyCompletion"` } func (m *JobTemplate) Reset() { *m = JobTemplate{} } func (m *JobTemplate) String() string { return proto.CompactTextString(m) } func (*JobTemplate) ProtoMessage() {} func (*JobTemplate) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{6} + return fileDescriptor_63baccdfe9127510, []int{8} } func (m *JobTemplate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -499,32 +611,41 @@ func (m *JobTemplate) GetDependencies() []string { return nil } -func (m *JobTemplate) GetMinSubmitTime() time.Time { +func (m *JobTemplate) GetEarliestSubmitTime() time.Duration { if m != nil { - return m.MinSubmitTime + return m.EarliestSubmitTime } - return time.Time{} + return 0 } -func (m *JobTemplate) GetRuntimeMean() int64 { +func (m *JobTemplate) GetRuntimeMin() time.Duration { if m != nil { - return m.RuntimeMean + return m.RuntimeMin } return 0 } -func (m *JobTemplate) GetRuntimeVariance() int64 { +func (m *JobTemplate) GetRuntimeVariance() time.Duration { if m != nil { return m.RuntimeVariance } return 0 } +func (m *JobTemplate) GetEarliestSubmitTimeFromDependencyCompletion() time.Duration { + if m != nil { + return m.EarliestSubmitTimeFromDependencyCompletion + } + return 0 +} + func init() { - proto.RegisterType((*TestCase)(nil), "simulator.TestCase") + proto.RegisterType((*SimulationResult)(nil), "simulator.SimulationResult") + proto.RegisterType((*ClusterSpec)(nil), "simulator.ClusterSpec") + proto.RegisterType((*WorkloadSpec)(nil), "simulator.WorkloadSpec") proto.RegisterType((*Pool)(nil), "simulator.Pool") - proto.RegisterType((*ExecutorGroup)(nil), "simulator.ExecutorGroup") - proto.RegisterType((*Executor)(nil), "simulator.Executor") + proto.RegisterType((*ClusterGroup)(nil), "simulator.ClusterGroup") + proto.RegisterType((*Cluster)(nil), "simulator.Cluster") proto.RegisterType((*NodeTemplate)(nil), "simulator.NodeTemplate") proto.RegisterMapType((map[string]string)(nil), "simulator.NodeTemplate.LabelsEntry") proto.RegisterType((*Queue)(nil), "simulator.Queue") @@ -536,75 +657,87 @@ func init() { } var fileDescriptor_63baccdfe9127510 = []byte{ - // 1025 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0x4f, 0x6f, 0xdb, 0x36, - 0x14, 0x8f, 0xe2, 0xc6, 0x8d, 0xe9, 0x3f, 0x49, 0x99, 0x2c, 0x51, 0xdc, 0xcd, 0xf2, 0x5c, 0x60, - 0xf0, 0x80, 0x54, 0x46, 0xbb, 0x4b, 0x16, 0x14, 0x03, 0xa6, 0xa2, 0xd8, 0x10, 0x74, 0x5d, 0xea, - 0x04, 0x1d, 0xb0, 0x60, 0x10, 0x68, 0xe9, 0xd5, 0x61, 0x22, 0x89, 0xaa, 0x48, 0x65, 0xcb, 0xa7, - 0x58, 0x4f, 0x3b, 0xed, 0x73, 0xec, 0x33, 0xf4, 0xd8, 0xe3, 0x4e, 0xda, 0x90, 0xdc, 0xf4, 0x29, - 0x06, 0x51, 0x54, 0x4c, 0x27, 0xdd, 0x90, 0x9d, 0x2c, 0xfe, 0xfe, 0x3c, 0x3e, 0x3e, 0xbf, 0x27, - 0x0a, 0x6d, 0xd3, 0x48, 0x40, 0x12, 0x91, 0x60, 0xc4, 0xbd, 0x63, 0xf0, 0xd3, 0x00, 0x92, 0x11, - 0xa7, 0x61, 0x1a, 0x10, 0xc1, 0xb4, 0x27, 0x3b, 0x4e, 0x98, 0x60, 0xb8, 0x71, 0x05, 0x74, 0xad, - 0x29, 0x63, 0xd3, 0x00, 0x46, 0x92, 0x98, 0xa4, 0xaf, 0x47, 0x82, 0x86, 0xc0, 0x05, 0x09, 0xe3, - 0x52, 0xdb, 0x1d, 0x9c, 0xee, 0x70, 0x9b, 0xb2, 0x11, 0x89, 0xe9, 0xc8, 0x63, 0x09, 0x8c, 0xce, - 0x1e, 0x8d, 0xa6, 0x10, 0x41, 0x42, 0x04, 0xf8, 0x4a, 0xf3, 0x70, 0x4a, 0xc5, 0x71, 0x3a, 0xb1, - 0x3d, 0x16, 0x8e, 0xa6, 0x6c, 0xca, 0x66, 0xd1, 0x8a, 0x95, 0x5c, 0xc8, 0x27, 0x25, 0xdf, 0xfd, - 0x50, 0xb2, 0xd5, 0x13, 0x9b, 0x9c, 0x80, 0x27, 0xf8, 0x0d, 0xa0, 0xf4, 0x0e, 0x2e, 0x0d, 0xb4, - 0x7c, 0x08, 0x5c, 0x3c, 0x25, 0x1c, 0xf0, 0x67, 0xe8, 0x4e, 0x44, 0x42, 0x30, 0x8d, 0xbe, 0x31, - 0x6c, 0x38, 0x38, 0xcf, 0xac, 0x4e, 0xb1, 0xde, 0x66, 0x21, 0x15, 0x10, 0xc6, 0xe2, 0x7c, 0x2c, - 0x79, 0xfc, 0x25, 0x6a, 0x26, 0x24, 0xf2, 0x59, 0xe8, 0x72, 0x00, 0xdf, 0x5c, 0xec, 0x1b, 0xc3, - 0x9a, 0x63, 0xe6, 0x99, 0xb5, 0x5e, 0xc2, 0x07, 0x00, 0xbe, 0x66, 0x42, 0x33, 0x14, 0xef, 0xa2, - 0xa5, 0x98, 0xb1, 0x80, 0x9b, 0xb5, 0x7e, 0x6d, 0xd8, 0x7c, 0xbc, 0x62, 0xcf, 0x6a, 0xb9, 0xcf, - 0x58, 0xe0, 0xac, 0xe5, 0x99, 0xb5, 0x22, 0x15, 0x5a, 0x80, 0xd2, 0x82, 0x77, 0x50, 0xfd, 0x4d, - 0x0a, 0x29, 0x70, 0xf3, 0x8e, 0x34, 0xaf, 0x6a, 0xe6, 0x97, 0x05, 0xe1, 0x74, 0xde, 0x65, 0xd6, - 0x42, 0x9e, 0x59, 0x4a, 0x37, 0x56, 0xbf, 0x83, 0x5f, 0x0d, 0x74, 0xa7, 0x08, 0x7f, 0xeb, 0x13, - 0xba, 0x68, 0x05, 0x7e, 0x01, 0x2f, 0x15, 0x2c, 0x71, 0xa7, 0x09, 0x4b, 0x63, 0x6e, 0x2e, 0xca, - 0x3d, 0x4d, 0x6d, 0xcf, 0x67, 0x4a, 0xf1, 0x4d, 0x21, 0x70, 0x3e, 0xce, 0x33, 0xcb, 0x04, 0x1d, - 0xd2, 0x8f, 0xd0, 0x99, 0x67, 0x06, 0x47, 0xa8, 0x3d, 0x67, 0xc7, 0x7b, 0xa8, 0x51, 0x49, 0xb8, - 0x69, 0xc8, 0xbd, 0xd6, 0x3e, 0xb0, 0x97, 0xb3, 0x99, 0x67, 0xd6, 0xda, 0x95, 0x52, 0xdb, 0x61, - 0x66, 0x2f, 0x8e, 0xbb, 0x5c, 0x19, 0x6e, 0x7d, 0xe4, 0x23, 0xd4, 0x89, 0x98, 0x0f, 0x6e, 0x01, - 0x06, 0x44, 0x40, 0x75, 0xe2, 0x4d, 0x2d, 0x8b, 0x17, 0xcc, 0x87, 0x43, 0xc5, 0x3b, 0xf7, 0xf3, - 0xcc, 0xda, 0x8c, 0x34, 0x44, 0xcf, 0xa6, 0x3d, 0x47, 0x0c, 0x7e, 0xab, 0xa1, 0x96, 0x6e, 0xc6, - 0xdb, 0xa8, 0x1e, 0xa5, 0xe1, 0x04, 0x12, 0x99, 0x57, 0xcd, 0x59, 0xcf, 0x33, 0x6b, 0xb5, 0x44, - 0xb4, 0x28, 0x4a, 0x83, 0xbf, 0x46, 0x75, 0x41, 0x68, 0x24, 0xaa, 0x9c, 0xb6, 0xec, 0x72, 0x8a, - 0x6c, 0x12, 0x53, 0xbb, 0x98, 0x22, 0xfb, 0xec, 0x91, 0x7d, 0x58, 0x28, 0x66, 0x2d, 0x50, 0x1a, - 0xc6, 0xea, 0x17, 0xbf, 0x44, 0xf5, 0x80, 0x4c, 0xe0, 0xaa, 0xf3, 0x1e, 0xfc, 0xcb, 0xb1, 0xec, - 0xe7, 0x52, 0xf5, 0x2c, 0x12, 0xc9, 0x79, 0x99, 0x55, 0x69, 0xd3, 0xb3, 0x2a, 0x91, 0xa2, 0x49, - 0x04, 0x13, 0x24, 0x70, 0x13, 0xe0, 0x2c, 0x4d, 0x3c, 0xd9, 0x98, 0xc6, 0xb0, 0xf9, 0xb8, 0x67, - 0xdf, 0x98, 0xb6, 0xb1, 0x92, 0x3c, 0xa7, 0x5c, 0x38, 0x1b, 0x2a, 0xc7, 0x8e, 0xb4, 0x57, 0x14, - 0x1f, 0x5f, 0x5b, 0x77, 0x09, 0x6a, 0x6a, 0xd9, 0xe0, 0x07, 0xa8, 0x76, 0x0a, 0xe7, 0xea, 0x8f, - 0xbc, 0x97, 0x67, 0x56, 0xfb, 0x14, 0xce, 0xb5, 0xbc, 0x0a, 0x16, 0x7f, 0x8e, 0x96, 0xce, 0x48, - 0x90, 0x82, 0x9c, 0xca, 0x46, 0x39, 0x4f, 0x12, 0xd0, 0xe7, 0x49, 0x02, 0xbb, 0x8b, 0x3b, 0xc6, - 0xe0, 0x0f, 0x03, 0x2d, 0xc9, 0xd9, 0xb9, 0x75, 0x9f, 0x6c, 0xa3, 0xfa, 0xcf, 0x40, 0xa7, 0xc7, - 0x42, 0xee, 0x60, 0x94, 0x35, 0x2a, 0x11, 0xbd, 0x46, 0x25, 0x82, 0x7f, 0x40, 0xed, 0x13, 0x36, - 0xd1, 0x9a, 0xaa, 0xac, 0xfe, 0x86, 0x56, 0xfd, 0x3d, 0x36, 0xb9, 0xea, 0xa9, 0x6e, 0x9e, 0x59, - 0x1b, 0x27, 0x33, 0x40, 0x2f, 0x7b, 0x4b, 0xc7, 0x07, 0xbf, 0xd7, 0x51, 0x53, 0x73, 0xfe, 0xcf, - 0x86, 0xda, 0x43, 0x8a, 0x3b, 0x48, 0x3d, 0x0f, 0x38, 0x7f, 0x9d, 0x06, 0xea, 0x35, 0xd6, 0xcb, - 0x33, 0xab, 0x7b, 0x9d, 0xd3, 0x22, 0xdc, 0xf0, 0x15, 0x15, 0x97, 0xaf, 0x19, 0xb3, 0x36, 0xab, - 0xb8, 0x04, 0xf4, 0x8a, 0x4b, 0x00, 0xf7, 0xd1, 0x22, 0xf5, 0x65, 0x93, 0x34, 0x9c, 0xd5, 0x3c, - 0xb3, 0x5a, 0x54, 0x7f, 0x4f, 0x2e, 0x52, 0x1f, 0x3f, 0x44, 0x77, 0x8b, 0x7a, 0x71, 0x10, 0xe6, - 0x92, 0x94, 0xc9, 0x73, 0x9c, 0xb0, 0xc9, 0x01, 0xcc, 0x95, 0xb7, 0x44, 0xb0, 0x83, 0x3a, 0x32, - 0xb2, 0x1b, 0x27, 0x94, 0x25, 0x54, 0x9c, 0x9b, 0xf5, 0xbe, 0x31, 0x6c, 0x97, 0xb3, 0x29, 0x99, - 0x7d, 0x45, 0xe8, 0xb3, 0x39, 0x47, 0xe0, 0xef, 0xd1, 0x5a, 0xe5, 0x76, 0xbd, 0x80, 0x70, 0xee, - 0xca, 0x3e, 0xb8, 0x2b, 0xb7, 0xb7, 0xf2, 0xcc, 0xba, 0x5f, 0xd1, 0x4f, 0x0b, 0xf6, 0xc5, 0x7c, - 0x53, 0xdc, 0xbb, 0x41, 0xe2, 0x23, 0xd4, 0x4a, 0xe0, 0x4d, 0x4a, 0x13, 0x08, 0xa1, 0x98, 0xd9, - 0x65, 0x39, 0x14, 0x9f, 0xde, 0x1c, 0x8a, 0x7d, 0xe6, 0x8f, 0x35, 0xa1, 0xb3, 0xae, 0xe6, 0x62, - 0xce, 0x3e, 0x9e, 0x5b, 0xe1, 0xaf, 0x50, 0xcb, 0x87, 0x18, 0x22, 0x1f, 0x22, 0x8f, 0x02, 0x37, - 0x1b, 0xfd, 0xda, 0xb0, 0x51, 0xf6, 0x8d, 0x8e, 0xeb, 0x7d, 0xa3, 0xe3, 0xf8, 0x27, 0xb4, 0x12, - 0xd2, 0xc8, 0xe5, 0xe9, 0x24, 0xa4, 0xc2, 0x2d, 0x6e, 0x67, 0x13, 0xc9, 0xfc, 0xba, 0x76, 0x79, - 0x75, 0xdb, 0xd5, 0x65, 0x6b, 0x1f, 0x56, 0x57, 0xb7, 0xb3, 0xa5, 0x12, 0x6b, 0x87, 0x34, 0x3a, - 0x90, 0xce, 0x82, 0x7b, 0xfb, 0x97, 0x65, 0x8c, 0xe7, 0x21, 0xfc, 0x04, 0xb5, 0x92, 0x34, 0x2a, - 0xc2, 0xba, 0x21, 0x90, 0xc8, 0x6c, 0xca, 0xa6, 0xda, 0xca, 0x33, 0xeb, 0x23, 0x85, 0x7f, 0x07, - 0x24, 0xd2, 0xb2, 0x6b, 0x6a, 0x30, 0xfe, 0x16, 0xad, 0x56, 0xee, 0x33, 0x92, 0x50, 0x12, 0x79, - 0x60, 0xb6, 0x64, 0x84, 0x4f, 0xf2, 0xcc, 0xda, 0x52, 0xdc, 0x2b, 0x45, 0x69, 0x51, 0x56, 0xae, - 0x51, 0xce, 0xab, 0x77, 0x17, 0x3d, 0xe3, 0xfd, 0x45, 0xcf, 0xf8, 0xfb, 0xa2, 0x67, 0xbc, 0xbd, - 0xec, 0x2d, 0xbc, 0xbf, 0xec, 0x2d, 0xfc, 0x79, 0xd9, 0x5b, 0xf8, 0xf1, 0x89, 0xf6, 0x71, 0x41, - 0x92, 0x90, 0xf8, 0x24, 0x4e, 0x58, 0xf1, 0x7f, 0xa8, 0xd5, 0xe8, 0xbf, 0x3e, 0x7d, 0x26, 0x75, - 0x59, 0x9d, 0x2f, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff, 0x90, 0x45, 0x44, 0xb8, 0x21, 0x09, 0x00, + // 1217 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0x5f, 0x6f, 0xd4, 0x46, + 0x10, 0x8f, 0x73, 0xe4, 0x20, 0x7b, 0x97, 0x3f, 0x2c, 0x69, 0x30, 0x89, 0x7a, 0x4e, 0x0f, 0xa9, + 0x4a, 0xab, 0x60, 0x0b, 0x2a, 0x55, 0x14, 0x55, 0x48, 0x75, 0x80, 0xaa, 0x88, 0x52, 0xb8, 0x43, + 0x20, 0x95, 0x07, 0x6b, 0xcf, 0x9e, 0x1c, 0x9b, 0xd8, 0x5e, 0x67, 0xbd, 0x0e, 0xca, 0x67, 0xa8, + 0x54, 0x55, 0x7d, 0xa8, 0xfa, 0x35, 0xfa, 0xc2, 0x67, 0xe0, 0x91, 0xb7, 0xf6, 0xc9, 0xad, 0xe0, + 0xcd, 0x9f, 0xa2, 0xf2, 0xee, 0x3a, 0xd9, 0xcb, 0xa5, 0x85, 0x3c, 0x9d, 0xf7, 0xf7, 0x9b, 0xf9, + 0x79, 0x66, 0x3c, 0x33, 0xb7, 0x68, 0x8b, 0xa6, 0x02, 0x78, 0x4a, 0x62, 0x2f, 0x0f, 0x5f, 0x40, + 0x54, 0xc4, 0xc0, 0xbd, 0x9c, 0x26, 0x45, 0x4c, 0x04, 0x33, 0x9e, 0xdc, 0x8c, 0x33, 0xc1, 0xf0, + 0xfc, 0x11, 0xb0, 0xd6, 0x1b, 0x33, 0x36, 0x8e, 0xc1, 0x93, 0xc4, 0xa8, 0xd8, 0xf1, 0xa2, 0x82, + 0x13, 0x41, 0x59, 0xaa, 0x4c, 0xd7, 0x9c, 0x93, 0xbc, 0xa0, 0x09, 0xe4, 0x82, 0x24, 0x99, 0x36, + 0xe8, 0xef, 0xdd, 0xcc, 0x5d, 0xca, 0x3c, 0x92, 0x51, 0x2f, 0x64, 0x1c, 0xbc, 0x83, 0xeb, 0xde, + 0x18, 0x52, 0xe0, 0x44, 0x40, 0xa4, 0x6d, 0xae, 0x8d, 0xa9, 0x78, 0x51, 0x8c, 0xdc, 0x90, 0x25, + 0xde, 0x98, 0x8d, 0xd9, 0xb1, 0x5a, 0x7d, 0x92, 0x07, 0xf9, 0xa4, 0xcd, 0x6f, 0x9d, 0x96, 0x4c, + 0xf3, 0xc4, 0x46, 0xbb, 0x10, 0x8a, 0x7c, 0x0a, 0xd0, 0xbe, 0x1f, 0x67, 0x7b, 0x63, 0x8f, 0xf0, + 0x84, 0x44, 0x04, 0x0e, 0x20, 0x15, 0xb9, 0xa7, 0x7e, 0x14, 0xdd, 0xff, 0x73, 0x16, 0x2d, 0x0f, + 0x55, 0xf2, 0x94, 0xa5, 0x03, 0xc8, 0x8b, 0x58, 0xe0, 0xef, 0x50, 0x5b, 0x19, 0xd9, 0xd6, 0x46, + 0x6b, 0xb3, 0x73, 0x63, 0xdd, 0x35, 0x05, 0xdc, 0xbb, 0xf5, 0xcf, 0x10, 0xf6, 0x0b, 0x48, 0x43, + 0xf0, 0x57, 0xaa, 0xd2, 0x59, 0x56, 0xcc, 0x16, 0x4b, 0xa8, 0x80, 0x24, 0x13, 0x87, 0x03, 0x2d, + 0x80, 0x87, 0xa8, 0x13, 0xc6, 0x45, 0x2e, 0x80, 0x0f, 0x33, 0x08, 0xed, 0xd9, 0x0d, 0x6b, 0xb3, + 0x73, 0x63, 0xd5, 0x3d, 0xfe, 0x00, 0xdb, 0xc7, 0xac, 0x7f, 0xa5, 0x2a, 0x9d, 0x8f, 0x0c, 0x73, + 0x43, 0xcf, 0x54, 0xc1, 0xcf, 0x50, 0xf7, 0x25, 0xe3, 0x7b, 0x31, 0x23, 0x91, 0x54, 0x6d, 0x49, + 0xd5, 0xcb, 0x86, 0xea, 0x33, 0x83, 0xf6, 0xd7, 0xaa, 0xd2, 0x59, 0x35, 0x1d, 0x0c, 0xdd, 0x09, + 0x21, 0x7c, 0x1f, 0x2d, 0xeb, 0x32, 0xd2, 0x74, 0xbc, 0xcd, 0xd2, 0x1d, 0x3a, 0xb6, 0xcf, 0x6d, + 0x58, 0x9b, 0xf3, 0x7e, 0xaf, 0x2a, 0x9d, 0xb5, 0x93, 0x9c, 0xa1, 0x33, 0xe5, 0xd7, 0xdf, 0x47, + 0x1d, 0x23, 0x37, 0xfc, 0x29, 0x3a, 0x97, 0x92, 0x04, 0x6c, 0x4b, 0xca, 0xe1, 0xaa, 0x74, 0x16, + 0xeb, 0xb3, 0x21, 0x21, 0x79, 0x7c, 0x0b, 0xcd, 0x65, 0x8c, 0xc5, 0xb9, 0x3d, 0x2b, 0x4b, 0xbf, + 0x64, 0x24, 0xf5, 0x88, 0xb1, 0xd8, 0xbf, 0x54, 0x95, 0xce, 0x92, 0xb4, 0x30, 0x5c, 0x95, 0x4b, + 0xff, 0x0f, 0x0b, 0x75, 0xcd, 0xcc, 0x3f, 0xf8, 0xa5, 0x5f, 0xa1, 0x0e, 0x27, 0x69, 0xc4, 0x92, + 0x20, 0x07, 0x88, 0xe4, 0x57, 0x6a, 0xf9, 0x76, 0x55, 0x3a, 0x2b, 0x0a, 0x1e, 0x02, 0x44, 0x86, + 0x13, 0x3a, 0x46, 0xf1, 0x6d, 0xd4, 0xde, 0x2f, 0xa0, 0x80, 0xdc, 0x6e, 0xc9, 0x80, 0x97, 0x8d, + 0x80, 0x1f, 0xd7, 0x84, 0x6a, 0x10, 0x65, 0x63, 0x36, 0x88, 0x42, 0xfa, 0x3f, 0x59, 0xe8, 0x5c, + 0x9d, 0xd8, 0x07, 0xc7, 0xfa, 0x1c, 0x2d, 0xea, 0x5e, 0x08, 0xc6, 0x9c, 0x15, 0x59, 0x53, 0xa9, + 0xcb, 0xd3, 0x4d, 0xf5, 0x6d, 0xcd, 0xfb, 0xeb, 0x55, 0xe9, 0x5c, 0x0e, 0x0d, 0xc4, 0x0c, 0x63, + 0x61, 0x82, 0xe8, 0x3f, 0x45, 0x5d, 0xd3, 0x17, 0xdf, 0x43, 0x17, 0xb4, 0x41, 0x33, 0x0b, 0x78, + 0xfa, 0x35, 0xfe, 0x6a, 0x55, 0x3a, 0xb8, 0xb1, 0x33, 0xc4, 0x8f, 0x7c, 0xfb, 0x3f, 0x5b, 0xe8, + 0xbc, 0xb6, 0x3e, 0x4b, 0xa2, 0x29, 0x8b, 0x20, 0xa8, 0xc1, 0x98, 0x08, 0x38, 0x2d, 0xd1, 0x87, + 0x2c, 0x82, 0x27, 0x9a, 0x57, 0x89, 0xa6, 0x06, 0x32, 0x91, 0xe8, 0x04, 0xd1, 0xff, 0xad, 0x85, + 0xba, 0xa6, 0x33, 0xde, 0x42, 0xed, 0xb4, 0x48, 0x46, 0xc0, 0x65, 0x5c, 0x2d, 0xf5, 0xd5, 0x14, + 0x62, 0x7e, 0x35, 0x85, 0xe0, 0x6f, 0x50, 0x5b, 0x10, 0x5a, 0x6f, 0x08, 0x15, 0xd3, 0x15, 0x57, + 0x6d, 0x3d, 0x97, 0x64, 0xd4, 0xad, 0xb7, 0x9e, 0x7b, 0x70, 0xdd, 0x7d, 0x52, 0x5b, 0xf8, 0x8b, + 0xaf, 0x4b, 0x67, 0xa6, 0x2a, 0x1d, 0xed, 0x30, 0xd0, 0xbf, 0xf8, 0x31, 0x6a, 0xc7, 0x64, 0x04, + 0x71, 0xd3, 0x38, 0x57, 0xff, 0x23, 0x2d, 0xf7, 0x81, 0xb4, 0xba, 0x9b, 0x0a, 0x7e, 0xa8, 0xa2, + 0x52, 0x6e, 0x66, 0x54, 0x0a, 0xc1, 0x01, 0x5a, 0x12, 0x4c, 0x90, 0x38, 0xe0, 0x90, 0xb3, 0x82, + 0x87, 0x90, 0xcb, 0xe9, 0xed, 0xdc, 0xe8, 0xb9, 0x53, 0xdb, 0x71, 0xa0, 0x4d, 0x1e, 0xd0, 0x5c, + 0xf8, 0xab, 0x3a, 0xc6, 0x45, 0xe9, 0xde, 0x50, 0xf9, 0xe0, 0xc4, 0x79, 0x8d, 0xa0, 0x8e, 0x11, + 0x0d, 0xbe, 0x8a, 0x5a, 0x7b, 0x70, 0xa8, 0x3f, 0xe4, 0xc5, 0xaa, 0x74, 0x16, 0xf6, 0xe0, 0xd0, + 0x88, 0xab, 0x66, 0xf1, 0x67, 0x68, 0xee, 0x80, 0xc4, 0x05, 0xc8, 0xa9, 0x9a, 0x57, 0xf3, 0x2b, + 0x01, 0x73, 0x7e, 0x25, 0x70, 0x6b, 0xf6, 0xa6, 0xd5, 0x7f, 0x65, 0xa1, 0x39, 0x39, 0x37, 0x1f, + 0xdc, 0x27, 0x5b, 0xa8, 0xfd, 0x12, 0xe8, 0xf8, 0x85, 0x90, 0x6f, 0xb0, 0x54, 0x8d, 0x14, 0x62, + 0xd6, 0x48, 0x21, 0xf8, 0x19, 0x5a, 0xd8, 0x65, 0x23, 0xa3, 0xa9, 0x54, 0xf5, 0xcd, 0x95, 0x7c, + 0x9f, 0x8d, 0x8e, 0x7a, 0x4a, 0xee, 0xce, 0xdd, 0x63, 0xc0, 0x2c, 0x7b, 0xd7, 0xc4, 0xfb, 0xbf, + 0x5e, 0x40, 0x1d, 0xc3, 0xf3, 0x8c, 0x0d, 0x75, 0x1f, 0x69, 0x6e, 0x58, 0x84, 0x21, 0xe4, 0xf9, + 0x4e, 0x11, 0xeb, 0x35, 0x24, 0x37, 0xef, 0x49, 0xce, 0xdc, 0xbc, 0x27, 0xb9, 0xba, 0xe2, 0x72, + 0xb9, 0xc8, 0xff, 0x05, 0x5d, 0x71, 0x09, 0x98, 0x15, 0x97, 0x00, 0xde, 0x40, 0xb3, 0x34, 0xd2, + 0x2b, 0x7e, 0xb9, 0x2a, 0x9d, 0x2e, 0x35, 0xf7, 0xdc, 0x2c, 0x8d, 0xf0, 0x35, 0x74, 0xbe, 0xae, + 0x57, 0x0e, 0xc2, 0x9e, 0x93, 0x66, 0x32, 0x8f, 0x5d, 0x36, 0x1a, 0xc2, 0x44, 0x79, 0x15, 0x82, + 0x7d, 0xb4, 0x28, 0x95, 0x83, 0x8c, 0x53, 0xc6, 0xa9, 0x38, 0xb4, 0xdb, 0x1b, 0xd6, 0xe6, 0x82, + 0x9a, 0x4d, 0xc9, 0x3c, 0xd2, 0x84, 0x39, 0x9b, 0x13, 0x04, 0xfe, 0x01, 0x5d, 0x6a, 0xbc, 0x83, + 0x30, 0x26, 0x79, 0x1e, 0xc8, 0x3e, 0x38, 0x2f, 0x5f, 0xef, 0x54, 0xa5, 0xb3, 0xde, 0xd0, 0xdb, + 0x35, 0xfb, 0x70, 0xb2, 0x29, 0x2e, 0x4e, 0x91, 0xf8, 0x39, 0xea, 0x72, 0xd8, 0x2f, 0x28, 0x87, + 0x44, 0xfe, 0xab, 0x5f, 0x90, 0x43, 0xf1, 0xc9, 0xf4, 0x50, 0x3c, 0x62, 0xd1, 0xc0, 0x30, 0xf4, + 0x57, 0xf4, 0x5c, 0x4c, 0xb8, 0x0f, 0x26, 0x4e, 0xf8, 0x36, 0xea, 0x46, 0x90, 0x41, 0x1a, 0x41, + 0x1a, 0x52, 0xc8, 0xed, 0xf9, 0x8d, 0xd6, 0xe6, 0xbc, 0xea, 0x1b, 0x13, 0x37, 0xfb, 0xc6, 0xc4, + 0xf1, 0x1e, 0x5a, 0x01, 0xc2, 0x63, 0x0a, 0xb9, 0x08, 0xf2, 0x62, 0x94, 0x50, 0x11, 0xd4, 0x57, + 0x2a, 0x1b, 0xc9, 0x20, 0xaf, 0xb8, 0xea, 0xbe, 0xe5, 0x36, 0x37, 0x24, 0xf7, 0x8e, 0xbe, 0x8f, + 0xf9, 0x3d, 0x1d, 0x1c, 0x6e, 0xdc, 0x87, 0xd2, 0xfb, 0x09, 0x4d, 0xe0, 0xf7, 0xbf, 0x1d, 0x6b, + 0x70, 0x0a, 0x8e, 0x07, 0xa8, 0xc3, 0x8b, 0xb4, 0xd6, 0x0f, 0x12, 0x9a, 0xda, 0x9d, 0xf7, 0xbd, + 0xa3, 0x59, 0x0c, 0x48, 0x7b, 0x7d, 0x4f, 0x53, 0xa9, 0x6d, 0x9c, 0x31, 0x41, 0xcb, 0x8d, 0xe6, + 0x01, 0xe1, 0x94, 0xa4, 0x21, 0xd8, 0xdd, 0xf7, 0x09, 0xaf, 0x6b, 0xe1, 0x25, 0xed, 0xfa, 0x54, + 0x7b, 0x4a, 0xf5, 0x93, 0x20, 0x7e, 0x65, 0x21, 0xef, 0xb4, 0x22, 0x05, 0x3b, 0x9c, 0x25, 0xc1, + 0x51, 0x39, 0x0f, 0x83, 0x90, 0x25, 0x59, 0x0c, 0xf5, 0x1b, 0xec, 0x85, 0xf7, 0x85, 0xf0, 0xa5, + 0x0e, 0xe1, 0xf3, 0xe9, 0x3a, 0xdd, 0xe3, 0x2c, 0xb9, 0x73, 0xa4, 0xba, 0x7d, 0x24, 0x2a, 0xa3, + 0x3b, 0x83, 0xbd, 0xff, 0xf4, 0xf5, 0xdb, 0x9e, 0xf5, 0xe6, 0x6d, 0xcf, 0xfa, 0xe7, 0x6d, 0xcf, + 0xfa, 0xe5, 0x5d, 0x6f, 0xe6, 0xcd, 0xbb, 0xde, 0xcc, 0x5f, 0xef, 0x7a, 0x33, 0x3f, 0x7e, 0x6d, + 0x5c, 0x81, 0xd5, 0xed, 0x32, 0xe3, 0xac, 0xee, 0x42, 0x7d, 0xf2, 0xfe, 0xef, 0x02, 0x3f, 0x6a, + 0xcb, 0x74, 0xbe, 0xf8, 0x37, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x70, 0xff, 0x6b, 0xe7, 0x0b, 0x00, 0x00, } -func (m *TestCase) Marshal() (dAtA []byte, err error) { +func (m *SimulationResult) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -614,20 +747,51 @@ func (m *TestCase) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *TestCase) MarshalTo(dAtA []byte) (int, error) { +func (m *SimulationResult) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *TestCase) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *SimulationResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Queues) > 0 { - for iNdEx := len(m.Queues) - 1; iNdEx >= 0; iNdEx-- { + if len(m.SchedulingConfig) > 0 { + i -= len(m.SchedulingConfig) + copy(dAtA[i:], m.SchedulingConfig) + i = encodeVarintSimulator(dAtA, i, uint64(len(m.SchedulingConfig))) + i-- + dAtA[i] = 0x22 + } + if m.WorkloadSpec != nil { + { + size, err := m.WorkloadSpec.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSimulator(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.ClusterSpec != nil { + { + size, err := m.ClusterSpec.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSimulator(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.Events) > 0 { + for iNdEx := len(m.Events) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Queues[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Events[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -635,9 +799,32 @@ func (m *TestCase) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintSimulator(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x22 + dAtA[i] = 0xa } } + return len(dAtA) - i, nil +} + +func (m *ClusterSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClusterSpec) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClusterSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l if len(m.Pools) > 0 { for iNdEx := len(m.Pools) - 1; iNdEx >= 0; iNdEx-- { { @@ -649,6 +836,50 @@ func (m *TestCase) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintSimulator(dAtA, i, uint64(size)) } i-- + dAtA[i] = 0x12 + } + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintSimulator(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *WorkloadSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *WorkloadSpec) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *WorkloadSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Queues) > 0 { + for iNdEx := len(m.Queues) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Queues[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSimulator(dAtA, i, uint64(size)) + } + i-- dAtA[i] = 0x1a } } @@ -687,10 +918,10 @@ func (m *Pool) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.ExecutorGroups) > 0 { - for iNdEx := len(m.ExecutorGroups) - 1; iNdEx >= 0; iNdEx-- { + if len(m.ClusterGroups) > 0 { + for iNdEx := len(m.ClusterGroups) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.ExecutorGroups[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.ClusterGroups[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -711,7 +942,7 @@ func (m *Pool) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *ExecutorGroup) Marshal() (dAtA []byte, err error) { +func (m *ClusterGroup) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -721,20 +952,20 @@ func (m *ExecutorGroup) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ExecutorGroup) MarshalTo(dAtA []byte) (int, error) { +func (m *ClusterGroup) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *ExecutorGroup) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *ClusterGroup) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Executors) > 0 { - for iNdEx := len(m.Executors) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Clusters) > 0 { + for iNdEx := len(m.Clusters) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Executors[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Clusters[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -748,7 +979,7 @@ func (m *ExecutorGroup) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *Executor) Marshal() (dAtA []byte, err error) { +func (m *Cluster) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -758,12 +989,12 @@ func (m *Executor) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *Executor) MarshalTo(dAtA []byte) (int, error) { +func (m *Cluster) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *Executor) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *Cluster) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -933,22 +1164,36 @@ func (m *JobTemplate) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.RuntimeVariance != 0 { - i = encodeVarintSimulator(dAtA, i, uint64(m.RuntimeVariance)) - i-- - dAtA[i] = 0x60 + n4, err4 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.EarliestSubmitTimeFromDependencyCompletion, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTimeFromDependencyCompletion):]) + if err4 != nil { + return 0, err4 } - if m.RuntimeMean != 0 { - i = encodeVarintSimulator(dAtA, i, uint64(m.RuntimeMean)) - i-- - dAtA[i] = 0x58 + i -= n4 + i = encodeVarintSimulator(dAtA, i, uint64(n4)) + i-- + dAtA[i] = 0x6a + n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.RuntimeVariance, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.RuntimeVariance):]) + if err5 != nil { + return 0, err5 } - n2, err2 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.MinSubmitTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.MinSubmitTime):]) - if err2 != nil { - return 0, err2 + i -= n5 + i = encodeVarintSimulator(dAtA, i, uint64(n5)) + i-- + dAtA[i] = 0x62 + n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.RuntimeMin, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.RuntimeMin):]) + if err6 != nil { + return 0, err6 + } + i -= n6 + i = encodeVarintSimulator(dAtA, i, uint64(n6)) + i-- + dAtA[i] = 0x5a + n7, err7 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.EarliestSubmitTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTime):]) + if err7 != nil { + return 0, err7 } - i -= n2 - i = encodeVarintSimulator(dAtA, i, uint64(n2)) + i -= n7 + i = encodeVarintSimulator(dAtA, i, uint64(n7)) i-- dAtA[i] = 0x52 if len(m.Dependencies) > 0 { @@ -1027,18 +1272,42 @@ func encodeVarintSimulator(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } -func (m *TestCase) Size() (n int) { +func (m *SimulationResult) Size() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Name) + if len(m.Events) > 0 { + for _, e := range m.Events { + l = e.Size() + n += 1 + l + sovSimulator(uint64(l)) + } + } + if m.ClusterSpec != nil { + l = m.ClusterSpec.Size() + n += 1 + l + sovSimulator(uint64(l)) + } + if m.WorkloadSpec != nil { + l = m.WorkloadSpec.Size() + n += 1 + l + sovSimulator(uint64(l)) + } + l = len(m.SchedulingConfig) if l > 0 { n += 1 + l + sovSimulator(uint64(l)) } - if m.RandomSeed != 0 { - n += 1 + sovSimulator(uint64(m.RandomSeed)) + return n +} + +func (m *ClusterSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) } if len(m.Pools) > 0 { for _, e := range m.Pools { @@ -1046,6 +1315,22 @@ func (m *TestCase) Size() (n int) { n += 1 + l + sovSimulator(uint64(l)) } } + return n +} + +func (m *WorkloadSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } + if m.RandomSeed != 0 { + n += 1 + sovSimulator(uint64(m.RandomSeed)) + } if len(m.Queues) > 0 { for _, e := range m.Queues { l = e.Size() @@ -1065,8 +1350,8 @@ func (m *Pool) Size() (n int) { if l > 0 { n += 1 + l + sovSimulator(uint64(l)) } - if len(m.ExecutorGroups) > 0 { - for _, e := range m.ExecutorGroups { + if len(m.ClusterGroups) > 0 { + for _, e := range m.ClusterGroups { l = e.Size() n += 1 + l + sovSimulator(uint64(l)) } @@ -1074,14 +1359,14 @@ func (m *Pool) Size() (n int) { return n } -func (m *ExecutorGroup) Size() (n int) { +func (m *ClusterGroup) Size() (n int) { if m == nil { return 0 } var l int _ = l - if len(m.Executors) > 0 { - for _, e := range m.Executors { + if len(m.Clusters) > 0 { + for _, e := range m.Clusters { l = e.Size() n += 1 + l + sovSimulator(uint64(l)) } @@ -1089,7 +1374,7 @@ func (m *ExecutorGroup) Size() (n int) { return n } -func (m *Executor) Size() (n int) { +func (m *Cluster) Size() (n int) { if m == nil { return 0 } @@ -1155,66 +1440,254 @@ func (m *Queue) Size() (n int) { n += 1 + l + sovSimulator(uint64(l)) } } - return n -} + return n +} + +func (m *JobTemplate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Number != 0 { + n += 1 + sovSimulator(uint64(m.Number)) + } + if m.NumberSuccessful != 0 { + n += 1 + sovSimulator(uint64(m.NumberSuccessful)) + } + l = len(m.Queue) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } + l = len(m.Id) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } + l = len(m.JobSet) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } + if m.QueuePriority != 0 { + n += 1 + sovSimulator(uint64(m.QueuePriority)) + } + l = len(m.PriorityClassName) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } + l = m.Requirements.Size() + n += 1 + l + sovSimulator(uint64(l)) + if len(m.Dependencies) > 0 { + for _, s := range m.Dependencies { + l = len(s) + n += 1 + l + sovSimulator(uint64(l)) + } + } + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTime) + n += 1 + l + sovSimulator(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.RuntimeMin) + n += 1 + l + sovSimulator(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.RuntimeVariance) + n += 1 + l + sovSimulator(uint64(l)) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTimeFromDependencyCompletion) + n += 1 + l + sovSimulator(uint64(l)) + return n +} + +func sovSimulator(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozSimulator(x uint64) (n int) { + return sovSimulator(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *SimulationResult) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SimulationResult: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SimulationResult: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Events", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Events = append(m.Events, &armadaevents.EventSequence{}) + if err := m.Events[len(m.Events)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClusterSpec", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ClusterSpec == nil { + m.ClusterSpec = &ClusterSpec{} + } + if err := m.ClusterSpec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkloadSpec", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WorkloadSpec == nil { + m.WorkloadSpec = &WorkloadSpec{} + } + if err := m.WorkloadSpec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SchedulingConfig", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SchedulingConfig = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSimulator(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSimulator + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } -func (m *JobTemplate) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Number != 0 { - n += 1 + sovSimulator(uint64(m.Number)) - } - if m.NumberSuccessful != 0 { - n += 1 + sovSimulator(uint64(m.NumberSuccessful)) - } - l = len(m.Queue) - if l > 0 { - n += 1 + l + sovSimulator(uint64(l)) - } - l = len(m.Id) - if l > 0 { - n += 1 + l + sovSimulator(uint64(l)) - } - l = len(m.JobSet) - if l > 0 { - n += 1 + l + sovSimulator(uint64(l)) - } - if m.QueuePriority != 0 { - n += 1 + sovSimulator(uint64(m.QueuePriority)) - } - l = len(m.PriorityClassName) - if l > 0 { - n += 1 + l + sovSimulator(uint64(l)) - } - l = m.Requirements.Size() - n += 1 + l + sovSimulator(uint64(l)) - if len(m.Dependencies) > 0 { - for _, s := range m.Dependencies { - l = len(s) - n += 1 + l + sovSimulator(uint64(l)) - } - } - l = github_com_gogo_protobuf_types.SizeOfStdTime(m.MinSubmitTime) - n += 1 + l + sovSimulator(uint64(l)) - if m.RuntimeMean != 0 { - n += 1 + sovSimulator(uint64(m.RuntimeMean)) - } - if m.RuntimeVariance != 0 { - n += 1 + sovSimulator(uint64(m.RuntimeVariance)) + if iNdEx > l { + return io.ErrUnexpectedEOF } - return n -} - -func sovSimulator(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozSimulator(x uint64) (n int) { - return sovSimulator(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + return nil } -func (m *TestCase) Unmarshal(dAtA []byte) error { +func (m *ClusterSpec) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1237,10 +1710,10 @@ func (m *TestCase) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TestCase: wiretype end group for non-group") + return fmt.Errorf("proto: ClusterSpec: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TestCase: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ClusterSpec: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -1276,10 +1749,10 @@ func (m *TestCase) Unmarshal(dAtA []byte) error { m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RandomSeed", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pools", wireType) } - m.RandomSeed = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSimulator @@ -1289,16 +1762,81 @@ func (m *TestCase) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RandomSeed |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 3: + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pools = append(m.Pools, &Pool{}) + if err := m.Pools[len(m.Pools)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSimulator(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSimulator + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *WorkloadSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: WorkloadSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WorkloadSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Pools", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSimulator @@ -1308,27 +1846,44 @@ func (m *TestCase) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthSimulator } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthSimulator } if postIndex > l { return io.ErrUnexpectedEOF } - m.Pools = append(m.Pools, &Pool{}) - if err := m.Pools[len(m.Pools)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Name = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RandomSeed", wireType) + } + m.RandomSeed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RandomSeed |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Queues", wireType) } @@ -1357,7 +1912,7 @@ func (m *TestCase) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Queues = append(m.Queues, Queue{}) + m.Queues = append(m.Queues, &Queue{}) if err := m.Queues[len(m.Queues)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1446,7 +2001,7 @@ func (m *Pool) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ExecutorGroups", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ClusterGroups", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1473,8 +2028,8 @@ func (m *Pool) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ExecutorGroups = append(m.ExecutorGroups, &ExecutorGroup{}) - if err := m.ExecutorGroups[len(m.ExecutorGroups)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.ClusterGroups = append(m.ClusterGroups, &ClusterGroup{}) + if err := m.ClusterGroups[len(m.ClusterGroups)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1499,7 +2054,7 @@ func (m *Pool) Unmarshal(dAtA []byte) error { } return nil } -func (m *ExecutorGroup) Unmarshal(dAtA []byte) error { +func (m *ClusterGroup) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1522,15 +2077,15 @@ func (m *ExecutorGroup) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecutorGroup: wiretype end group for non-group") + return fmt.Errorf("proto: ClusterGroup: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecutorGroup: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ClusterGroup: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Executors", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Clusters", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1557,8 +2112,8 @@ func (m *ExecutorGroup) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Executors = append(m.Executors, &Executor{}) - if err := m.Executors[len(m.Executors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Clusters = append(m.Clusters, &Cluster{}) + if err := m.Clusters[len(m.Clusters)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1583,7 +2138,7 @@ func (m *ExecutorGroup) Unmarshal(dAtA []byte) error { } return nil } -func (m *Executor) Unmarshal(dAtA []byte) error { +func (m *Cluster) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1606,10 +2161,10 @@ func (m *Executor) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Executor: wiretype end group for non-group") + return fmt.Errorf("proto: Cluster: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Executor: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Cluster: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -2370,7 +2925,7 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MinSubmitTime", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field EarliestSubmitTime", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -2397,15 +2952,15 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.MinSubmitTime, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.EarliestSubmitTime, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RuntimeMean", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeMin", wireType) } - m.RuntimeMean = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSimulator @@ -2415,16 +2970,63 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RuntimeMean |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.RuntimeMin, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 12: - if wireType != 0 { + if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field RuntimeVariance", wireType) } - m.RuntimeVariance = 0 + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.RuntimeVariance, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EarliestSubmitTimeFromDependencyCompletion", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSimulator @@ -2434,11 +3036,25 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RuntimeVariance |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthSimulator + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSimulator + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.EarliestSubmitTimeFromDependencyCompletion, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSimulator(dAtA[iNdEx:]) diff --git a/internal/scheduler/simulator/simulator.proto b/internal/scheduler/simulator/simulator.proto index dbc02fb1b6b..13459858db2 100644 --- a/internal/scheduler/simulator/simulator.proto +++ b/internal/scheduler/simulator/simulator.proto @@ -2,32 +2,46 @@ syntax = 'proto3'; package simulator; option go_package = "github.com/armadaproject/armada/internal/scheduler/simulator"; +import "google/protobuf/duration.proto"; import "google/protobuf/timestamp.proto"; import "k8s.io/api/core/v1/generated.proto"; import "github.com/gogo/protobuf/gogoproto/gogo.proto"; import "internal/scheduler/schedulerobjects/schedulerobjects.proto"; +import "pkg/armadaevents/events.proto"; + +message SimulationResult { + repeated armadaevents.EventSequence events = 1; + ClusterSpec clusterSpec = 2; + WorkloadSpec workloadSpec = 3; + string schedulingConfig = 4; +} + +message ClusterSpec { + string name = 1; + repeated Pool pools = 2; +} + +message WorkloadSpec { + string name = 1; + int64 random_seed = 2; + repeated Queue queues = 3; +} // TODO: // Runtime family. // Workflow manager delay. // Job pending delay. -message TestCase { - string name = 1; - int64 random_seed = 2; - repeated Pool pools = 3; - repeated Queue queues = 4 [(gogoproto.nullable) = false]; -} message Pool { string name = 1; - repeated ExecutorGroup executor_groups = 2; + repeated ClusterGroup cluster_groups = 2; } -message ExecutorGroup { - repeated Executor executors = 1; +message ClusterGroup { + repeated Cluster clusters = 1; } -message Executor { +message Cluster { string name = 1; repeated NodeTemplate node_templates = 2; } @@ -58,14 +72,17 @@ message JobTemplate { string job_set = 5; uint32 queue_priority = 6; string priority_class_name = 7; + // Scheduling requirements for the pod embedded in the job. schedulerobjects.PodRequirements requirements = 8 [(gogoproto.nullable) = false]; // List of template ids that must be completed before this template is submitted. repeated string dependencies = 9; - // Minimum time from which jobs are created from this template. - google.protobuf.Timestamp min_submit_time = 10 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; - // Job runtime mean in seconds. - int64 runtime_mean = 11; - // Job runtime variance in seconds squared. - // If zero, runtime is deterministic. - int64 runtime_variance = 12; -} \ No newline at end of file + // Earliest time at which jobs from this template are submitted. + // Measured from the start of the simulation. + google.protobuf.Duration earliest_submit_time = 10 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; + // Minimum job runtime when sampling from shifted exponential distribution. + google.protobuf.Duration runtime_min = 11 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; + // Job runtime variance. If zero, runtime is deterministic. + google.protobuf.Duration runtime_variance = 12 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; + // Earliest time job can be submitted from when all dependencies have completed + google.protobuf.Duration earliest_submit_time_from_dependency_completion = 13 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; +} diff --git a/internal/scheduler/simulator/simulator_test.go b/internal/scheduler/simulator/simulator_test.go index 0d891d3cfff..9646c2e2c8e 100644 --- a/internal/scheduler/simulator/simulator_test.go +++ b/internal/scheduler/simulator/simulator_test.go @@ -1,36 +1,37 @@ package simulator import ( - fmt "fmt" - "strings" + "github.com/armadaproject/armada/internal/common/armadacontext" + "reflect" "testing" "time" - "github.com/stretchr/testify/require" - v1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/api/resource" - "github.com/armadaproject/armada/internal/armada/configuration" armadaslices "github.com/armadaproject/armada/internal/common/slices" "github.com/armadaproject/armada/internal/common/util" - "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" "github.com/armadaproject/armada/internal/scheduler/testfixtures" "github.com/armadaproject/armada/pkg/armadaevents" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestSimulator(t *testing.T) { tests := map[string]struct { - testCase *TestCase + clusterSpec *ClusterSpec + workloadSpec *WorkloadSpec schedulingConfig configuration.SchedulingConfig expectedEventSequences []*armadaevents.EventSequence + simulatedTimeLimit time.Duration }{ "Two jobs in parallel": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "basic", Pools: []*Pool{Pool32Cpu("Pool", 1, 1, 2)}, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, JobTemplate32Cpu(2, "foo", testfixtures.TestDefaultPriorityClass), ), }, @@ -43,14 +44,17 @@ func TestSimulator(t *testing.T) { {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, "Two jobs in sequence": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "basic", Pools: []*Pool{Pool32Cpu("Pool", 1, 1, 1)}, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, JobTemplate32Cpu(2, "foo", testfixtures.TestDefaultPriorityClass), ), }, @@ -63,14 +67,17 @@ func TestSimulator(t *testing.T) { {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobRunLeased()}}, {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, "10 jobs in sequence": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "basic", Pools: []*Pool{Pool32Cpu("Pool", 1, 1, 1)}, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, JobTemplate32Cpu(10, "foo", testfixtures.TestDefaultPriorityClass), ), }, @@ -86,14 +93,17 @@ func TestSimulator(t *testing.T) { &armadaevents.EventSequence{Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, )..., ), + simulatedTimeLimit: 20 * time.Minute, }, "JobTemplate dependencies": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "basic", Pools: []*Pool{Pool32Cpu("Pool", 1, 1, 3)}, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, WithIdJobTemplate( JobTemplate32Cpu(2, "foo", testfixtures.TestDefaultPriorityClass), "jobTemplate", @@ -116,21 +126,24 @@ func TestSimulator(t *testing.T) { {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobRunLeased()}}, {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, "Preemption": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "basic", Pools: []*Pool{Pool32Cpu("Pool", 1, 1, 2)}, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, JobTemplate32Cpu(2, "foo", testfixtures.PriorityClass0), ), WithJobTemplatesQueue( - Queue{Name: "B", Weight: 1}, + &Queue{Name: "B", Weight: 1}, WithMinSubmitTimeJobTemplate( JobTemplate32Cpu(1, "bar", testfixtures.PriorityClass0), - time.Time{}.Add(30*time.Second), + 30*time.Second, ), ), }, @@ -149,9 +162,10 @@ func TestSimulator(t *testing.T) { {Queue: "B", JobSetName: "bar", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, "Preemption cascade": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "test", Pools: []*Pool{ WithExecutorGroupsPool( @@ -161,20 +175,22 @@ func TestSimulator(t *testing.T) { ExecutorGroup32Cpu(1, 1), ), }, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "B", Weight: 1}, + &Queue{Name: "B", Weight: 1}, JobTemplate32Cpu(1, "foo", testfixtures.PriorityClass0), ), WithJobTemplatesQueue( - Queue{Name: "C", Weight: 1}, + &Queue{Name: "C", Weight: 1}, JobTemplate32Cpu(2, "foo", testfixtures.PriorityClass0), ), WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, WithMinSubmitTimeJobTemplate( JobTemplate32Cpu(1, "foo", testfixtures.PriorityClass0), - time.Time{}.Add(30*time.Second), + 30*time.Second, ), ), }, @@ -199,9 +215,10 @@ func TestSimulator(t *testing.T) { {Queue: "B", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, {Queue: "C", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, "No preemption cascade with unified scheduling": { - testCase: &TestCase{ + clusterSpec: &ClusterSpec{ Name: "test", Pools: []*Pool{ WithExecutorGroupsPool( @@ -209,20 +226,22 @@ func TestSimulator(t *testing.T) { ExecutorGroup32Cpu(3, 1), ), }, - Queues: []Queue{ + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ WithJobTemplatesQueue( - Queue{Name: "B", Weight: 1}, + &Queue{Name: "B", Weight: 1}, JobTemplate32Cpu(1, "foo", testfixtures.PriorityClass0), ), WithJobTemplatesQueue( - Queue{Name: "C", Weight: 1}, + &Queue{Name: "C", Weight: 1}, JobTemplate32Cpu(2, "foo", testfixtures.PriorityClass0), ), WithJobTemplatesQueue( - Queue{Name: "A", Weight: 1}, + &Queue{Name: "A", Weight: 1}, WithMinSubmitTimeJobTemplate( JobTemplate32Cpu(1, "foo", testfixtures.PriorityClass0), - time.Time{}.Add(30*time.Second), + 30*time.Second, ), ), }, @@ -244,200 +263,58 @@ func TestSimulator(t *testing.T) { {Queue: "A", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, {Queue: "C", JobSetName: "foo", Events: []*armadaevents.EventSequence_Event{JobSucceeded()}}, }, + simulatedTimeLimit: 5 * time.Minute, }, } for name, tc := range tests { t.Run(name, func(t *testing.T) { - s, err := NewSimulator(tc.testCase, tc.schedulingConfig) - require.NoError(t, err) - go func() { err = s.Run() }() - actualEventSequences := make([]*armadaevents.EventSequence, 0, len(tc.expectedEventSequences)) - for eventSequence := range s.C() { - t.Log(*eventSequence.Events[0].Created, eventSequenceSummary(eventSequence)) - actualEventSequences = append(actualEventSequences, eventSequence) - } + simResult, metricsCollector, err := SimulationRun(armadacontext.Background(), tc.clusterSpec, tc.workloadSpec, tc.schedulingConfig) require.NoError(t, err) - t.Logf("Simulated time: %s", s.time.Sub(time.Time{})) + + t.Logf("Simulation Results: %s", metricsCollector.String()) if tc.expectedEventSequences != nil { require.Equal( t, - util.Map(tc.expectedEventSequences, func(eventSequence *armadaevents.EventSequence) string { return eventSequenceSummary(eventSequence) }), - util.Map(actualEventSequences, func(eventSequence *armadaevents.EventSequence) string { return eventSequenceSummary(eventSequence) }), + util.Map(tc.expectedEventSequences, func(eventSequence *armadaevents.EventSequence) string { return EventSequenceSummary(eventSequence) }), + util.Map(simResult.Events, func(eventSequence *armadaevents.EventSequence) string { return EventSequenceSummary(eventSequence) }), "Expected:\n%s\nReceived:\n%s", - eventSequencesSummary(tc.expectedEventSequences), - eventSequencesSummary(actualEventSequences), + EventSequencesSummary(tc.expectedEventSequences), + EventSequencesSummary(simResult.Events), ) } + require.LessOrEqual(t, metricsCollector.Total.LastJobSuccess, tc.simulatedTimeLimit) }) } } -func WithExecutorGroupsPool(pool *Pool, executorGroups ...*ExecutorGroup) *Pool { - pool.ExecutorGroups = append(pool.ExecutorGroups, executorGroups...) - return pool -} - -func WithExecutorsExecutorGroup(executorGroup *ExecutorGroup, executors ...*Executor) *ExecutorGroup { - executorGroup.Executors = append(executorGroup.Executors, executors...) - return executorGroup -} - -func WithNodeTemplatesExecutor(executor *Executor, nodeTemplates ...*NodeTemplate) *Executor { - executor.NodeTemplates = append(executor.NodeTemplates, nodeTemplates...) - return executor -} - -func Pool32Cpu(name string, numExecutorGroups, numExecutorsPerGroup, numNodesPerExecutor int64) *Pool { - executorGroups := make([]*ExecutorGroup, numExecutorGroups) - for i := 0; i < int(numExecutorGroups); i++ { - executorGroups[i] = ExecutorGroup32Cpu(numExecutorsPerGroup, numNodesPerExecutor) - } - return &Pool{ - Name: name, - ExecutorGroups: executorGroups, - } -} - -func ExecutorGroup32Cpu(numExecutors, numNodesPerExecutor int64) *ExecutorGroup { - executors := make([]*Executor, numExecutors) - for i := 0; i < int(numExecutors); i++ { - executors[i] = Executor32Cpu(numNodesPerExecutor) - } - return &ExecutorGroup{ - Executors: executors, - } -} - -func Executor32Cpu(numNodes int64) *Executor { - return &Executor{ - NodeTemplates: []*NodeTemplate{ - NodeTemplate32Cpu(numNodes), - }, - } -} - -func NodeTemplate32Cpu(n int64) *NodeTemplate { - return &NodeTemplate{ - Number: n, - TotalResources: schedulerobjects.ResourceList{ - Resources: map[string]resource.Quantity{ - "cpu": resource.MustParse("32"), - "memory": resource.MustParse("256Gi"), - }, - }, - } -} - -func WithJobTemplatesQueue(queue Queue, jobTemplate ...*JobTemplate) Queue { - queue.JobTemplates = append(queue.JobTemplates, jobTemplate...) - return queue -} - -func WithIdJobTemplate(jobTemplate *JobTemplate, id string) *JobTemplate { - jobTemplate.Id = id - return jobTemplate -} - -func WithDependenciesJobTemplate(jobTemplate *JobTemplate, dependencyIds ...string) *JobTemplate { - jobTemplate.Dependencies = append(jobTemplate.Dependencies, dependencyIds...) - return jobTemplate +func TestSchedulingConfigsFromPattern(t *testing.T) { + actual, err := SchedulingConfigsFromPattern("./testdata/schedulingConfigs/basicSchedulingConfig.yaml") + require.NoError(t, err) + expected := GetBasicSchedulingConfig() + assert.Equal(t, expected, actual[0]) } -func WithMinSubmitTimeJobTemplate(jobTemplate *JobTemplate, minSubmitTime time.Time) *JobTemplate { - jobTemplate.MinSubmitTime = minSubmitTime - return jobTemplate -} - -func JobTemplate32Cpu(n int64, jobSet, priorityClassName string) *JobTemplate { - return &JobTemplate{ - Number: n, - JobSet: jobSet, - PriorityClassName: priorityClassName, - Requirements: schedulerobjects.PodRequirements{ - ResourceRequirements: v1.ResourceRequirements{ - Requests: v1.ResourceList{ - "cpu": resource.MustParse("32"), - "memory": resource.MustParse("256Gi"), - }, - }, - }, - RuntimeMean: 60, - } +func TestClusterSpecsFromPattern(t *testing.T) { + clusterSpecs, err := ClusterSpecsFromPattern("./testdata/clusters/tinyCluster.yaml") + assert.Equal(t, clusterSpecs[0], GetTwoPoolTwoNodeCluster()) + require.NoError(t, err) } -func JobTemplate1Cpu(n int64, jobSet, priorityClassName string) *JobTemplate { - return &JobTemplate{ - Number: n, - JobSet: jobSet, - PriorityClassName: priorityClassName, - Requirements: schedulerobjects.PodRequirements{ - ResourceRequirements: v1.ResourceRequirements{ - Requests: v1.ResourceList{ - "cpu": resource.MustParse("1"), - "memory": resource.MustParse("8Gi"), - }, - }, - }, - RuntimeMean: 60, - } +func TestWorkloadFromPattern(t *testing.T) { + workloadSpecs, err := WorkloadFromPattern("./testdata/workloads/basicWorkload.yaml") + assert.Equal(t, workloadSpecs[0], GetOneQueue10JobWorkload()) + require.NoError(t, err) } -func SubmitJob() *armadaevents.EventSequence_Event { - return &armadaevents.EventSequence_Event{ - Event: &armadaevents.EventSequence_Event_SubmitJob{ - SubmitJob: &armadaevents.SubmitJob{}, - }, - } -} +func TestAggregateClusterResourcesEquivalent(t *testing.T) { + clusterSpecsSegregated, err := ClusterSpecsFromPattern("./testdata/clusters/tinyCluster.yaml") + require.NoError(t, err) + clusterSpecsUnified, err := ClusterSpecsFromPattern("./testdata/clusters/tinyClusterAlt.yaml") + require.NoError(t, err) -func JobRunLeased() *armadaevents.EventSequence_Event { - return &armadaevents.EventSequence_Event{ - Event: &armadaevents.EventSequence_Event_JobRunLeased{ - JobRunLeased: &armadaevents.JobRunLeased{}, - }, - } -} - -func JobRunPreempted() *armadaevents.EventSequence_Event { - return &armadaevents.EventSequence_Event{ - Event: &armadaevents.EventSequence_Event_JobRunPreempted{ - JobRunPreempted: &armadaevents.JobRunPreempted{}, - }, - } -} - -func JobSucceeded() *armadaevents.EventSequence_Event { - return &armadaevents.EventSequence_Event{ - Event: &armadaevents.EventSequence_Event_JobSucceeded{ - JobSucceeded: &armadaevents.JobSucceeded{}, - }, - } -} - -func eventSequencesSummary(eventSequences []*armadaevents.EventSequence) string { - var sb strings.Builder - for i, eventSequence := range eventSequences { - sb.WriteString(eventSequenceSummary(eventSequence)) - if i != len(eventSequences)-1 { - sb.WriteString("\n") - } - } - return sb.String() -} - -func eventSequenceSummary(eventSequence *armadaevents.EventSequence) string { - var sb strings.Builder - sb.WriteString(fmt.Sprintf("EventSequence{Queue: %s, JobSetName: %s, Events: [", eventSequence.Queue, eventSequence.JobSetName)) - for i, event := range eventSequence.Events { - sb.WriteString(eventSummary(event)) - if i != len(eventSequence.Events)-1 { - sb.WriteString(", ") - } - } - sb.WriteString("]}") - return sb.String() -} + srm := CalculateClusterAggregateResources(clusterSpecsSegregated[0]) + urm := CalculateClusterAggregateResources(clusterSpecsUnified[0]) -func eventSummary(event *armadaevents.EventSequence_Event) string { - return strings.ReplaceAll(fmt.Sprintf("%T", event.Event), "*armadaevents.EventSequence_Event_", "") + require.True(t, reflect.DeepEqual(srm, urm)) + t.Logf("seg, unified | cpu %d %d, gpu %d %d, mem %d %d", srm["cpu"], urm["cpu"], srm["gpu"], urm["gpu"], srm["memory"], urm["memory"]) } diff --git a/internal/scheduler/simulator/test_utils.go b/internal/scheduler/simulator/test_utils.go new file mode 100644 index 00000000000..e1f0a0f1ebe --- /dev/null +++ b/internal/scheduler/simulator/test_utils.go @@ -0,0 +1,369 @@ +package simulator + +import ( + "fmt" + "github.com/armadaproject/armada/internal/armada/configuration" + "github.com/armadaproject/armada/internal/common/armadacontext" + "github.com/armadaproject/armada/internal/common/types" + "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" + "github.com/armadaproject/armada/pkg/armadaevents" + v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + "math" + "strings" + "time" +) + +func SimulationRun(ctx *armadacontext.Context, clusterSpec *ClusterSpec, workloadSpec *WorkloadSpec, schedulingConfig configuration.SchedulingConfig) (*SimulationResult, *MetricsCollector, error) { + s, err := NewSimulator(clusterSpec, workloadSpec, schedulingConfig) + if err != nil { + return &SimulationResult{}, &MetricsCollector{}, err + } + + metricsCollector := NewMetricsCollector(s.Output()) + actualEventSequences := make([]*armadaevents.EventSequence, 0, 128) + c := s.Output() + + g, threadCtx := armadacontext.ErrGroup(ctx) + g.Go(func() error { + return metricsCollector.Run(threadCtx) + }) + g.Go(func() error { + for { + select { + case <-threadCtx.Done(): + return threadCtx.Err() + case eventSequence, ok := <-c: + if !ok { + return nil + } + ctx.Info(*eventSequence.Events[0].Created, EventSequenceSummary(eventSequence)) + actualEventSequences = append(actualEventSequences, eventSequence) + } + } + }) + g.Go(func() error { + return s.Run(threadCtx) + }) + if err := g.Wait(); err != nil { + return &SimulationResult{}, metricsCollector, err + } + + simResult := &SimulationResult{ + Events: actualEventSequences, + ClusterSpec: clusterSpec, + WorkloadSpec: workloadSpec, + } + + return simResult, metricsCollector, nil +} + +func GetTwoPoolTwoNodeCluster() *ClusterSpec { + cs := &ClusterSpec{ + Name: "Tiny Cluster", + Pools: []*Pool{ + Pool32Cpu("pool1", 1, 1, 1), + PoolGpu("pool2", 1, 1, 1), + }, + } + initialiseClusterSpec(cs) + return cs +} + +func GetOneQueue10JobWorkload() *WorkloadSpec { + ws := &WorkloadSpec{ + Name: "Basic Workload", + Queues: []*Queue{ + WithJobTemplatesQueue( + &Queue{Name: "A", Weight: 1}, + JobTemplate1Cpu(10, "", "armada-default", "myFirstJobTemplate"), + ), + }, + } + initialiseWorkloadSpec(ws) + return ws +} + +func GetBasicSchedulingConfig() configuration.SchedulingConfig { + return configuration.SchedulingConfig{ + Preemption: configuration.PreemptionConfig{ + NodeEvictionProbability: 1.0, + PriorityClasses: map[string]types.PriorityClass{ + "armada-default": { + Priority: 30000, + Preemptible: false, + }, + "armada-preemptible": { + Priority: 30000, + Preemptible: true, + }, + }, + }, + MaximumResourceFractionToSchedule: map[string]float64{ + "memory": 0.025, + "cpu": 0.025, + }, + FairnessModel: "DominantResourceFairness", + DominantResourceFairnessResourcesToConsider: []string{"cpu", "memory", "nvidia.com/gpu", "ephemeral-storage"}, + IndexedResources: []configuration.IndexedResource{ + { + Name: "cpu", + Resolution: resource.MustParse("1"), + }, + { + Name: "memory", + Resolution: resource.MustParse("1Mi"), + }, + { + Name: "nvidia.com/gpu", + Resolution: resource.MustParse("1"), + }, + }, + MaximumSchedulingRate: math.Inf(1), + MaximumSchedulingBurst: math.MaxInt, + MaximumPerQueueSchedulingRate: math.Inf(1), + MaximumPerQueueSchedulingBurst: math.MaxInt, + } +} + +// Calculates the aggregate number/amount of CPU cores, GPUs and Memory (GB) within a cluster. +func CalculateClusterAggregateResources(clusterSpec *ClusterSpec) map[string]int64 { + resourceMapping := map[string]int64{ + "cpus": 0, + "gpus": 0, + "memory": 0, + } + + for _, pool := range clusterSpec.Pools { + for _, clusterGroup := range pool.ClusterGroups { + for _, cluster := range clusterGroup.Clusters { + for _, nt := range cluster.NodeTemplates { + count := nt.Number + thisNodeCores := nt.TotalResources.Resources["cpu"] + resourceMapping["cpus"] += thisNodeCores.Value() * count + thisNodeGPUs := nt.TotalResources.Resources["nvidia.com/gpu"] + resourceMapping["gpus"] += thisNodeGPUs.Value() * count + thisNodeMemory := nt.TotalResources.Resources["memory"] + resourceMapping["memory"] += thisNodeMemory.Value() * count + } + } + } + } + + return resourceMapping +} + +func WithExecutorGroupsPool(pool *Pool, executorGroups ...*ClusterGroup) *Pool { + pool.ClusterGroups = append(pool.ClusterGroups, executorGroups...) + return pool +} + +func WithExecutorsExecutorGroup(executorGroup *ClusterGroup, executors ...*Cluster) *ClusterGroup { + executorGroup.Clusters = append(executorGroup.Clusters, executors...) + return executorGroup +} + +func WithNodeTemplatesExecutor(executor *Cluster, nodeTemplates ...*NodeTemplate) *Cluster { + executor.NodeTemplates = append(executor.NodeTemplates, nodeTemplates...) + return executor +} + +func Pool32Cpu(name string, numExecutorGroups, numExecutorsPerGroup, numNodesPerExecutor int64) *Pool { + executorGroups := make([]*ClusterGroup, numExecutorGroups) + for i := 0; i < int(numExecutorGroups); i++ { + executorGroups[i] = ExecutorGroup32Cpu(numExecutorsPerGroup, numNodesPerExecutor) + } + return &Pool{ + Name: name, + ClusterGroups: executorGroups, + } +} + +func PoolGpu(name string, numExecutorGroups, numExecutorsPerGroup, numNodesPerExecutor int64) *Pool { + executorGroups := make([]*ClusterGroup, numExecutorGroups) + for i := 0; i < int(numExecutorGroups); i++ { + executorGroups[i] = ExecutorGroupGpu(numExecutorsPerGroup, numNodesPerExecutor) + } + return &Pool{ + Name: name, + ClusterGroups: executorGroups, + } +} + +func ExecutorGroup32Cpu(numExecutors, numNodesPerExecutor int64) *ClusterGroup { + executors := make([]*Cluster, numExecutors) + for i := 0; i < int(numExecutors); i++ { + executors[i] = Executor32Cpu(numNodesPerExecutor) + } + return &ClusterGroup{ + Clusters: executors, + } +} + +func ExecutorGroupGpu(numExecutors, numNodesPerExecutor int64) *ClusterGroup { + executors := make([]*Cluster, numExecutors) + for i := 0; i < int(numExecutors); i++ { + executors[i] = ExecutorGpu(numNodesPerExecutor) + } + return &ClusterGroup{ + Clusters: executors, + } +} + +func Executor32Cpu(numNodes int64) *Cluster { + return &Cluster{ + NodeTemplates: []*NodeTemplate{ + NodeTemplate32Cpu(numNodes), + }, + } +} + +func ExecutorGpu(numNodes int64) *Cluster { + return &Cluster{ + NodeTemplates: []*NodeTemplate{ + NodeTemplateGpu(numNodes), + }, + } +} + +func NodeTemplate32Cpu(n int64) *NodeTemplate { + return &NodeTemplate{ + Number: n, + TotalResources: schedulerobjects.ResourceList{ + Resources: map[string]resource.Quantity{ + "cpu": resource.MustParse("32"), + "memory": resource.MustParse("256Gi"), + }, + }, + } +} + +func NodeTemplateGpu(n int64) *NodeTemplate { + return &NodeTemplate{ + Number: n, + TotalResources: schedulerobjects.ResourceList{ + Resources: map[string]resource.Quantity{ + "cpu": resource.MustParse("128"), + "memory": resource.MustParse("4096Gi"), + "nvidia.com/gpu": resource.MustParse("8"), + }, + }, + } +} + +func WithJobTemplatesQueue(queue *Queue, jobTemplate ...*JobTemplate) *Queue { + queue.JobTemplates = append(queue.JobTemplates, jobTemplate...) + return queue +} + +func WithIdJobTemplate(jobTemplate *JobTemplate, id string) *JobTemplate { + jobTemplate.Id = id + return jobTemplate +} + +func WithDependenciesJobTemplate(jobTemplate *JobTemplate, dependencyIds ...string) *JobTemplate { + jobTemplate.Dependencies = append(jobTemplate.Dependencies, dependencyIds...) + return jobTemplate +} + +func WithMinSubmitTimeJobTemplate(jobTemplate *JobTemplate, minSubmitTime time.Duration) *JobTemplate { + jobTemplate.EarliestSubmitTime = minSubmitTime + return jobTemplate +} + +func JobTemplate32Cpu(n int64, jobSet, priorityClassName string) *JobTemplate { + return &JobTemplate{ + Number: n, + JobSet: jobSet, + PriorityClassName: priorityClassName, + Requirements: schedulerobjects.PodRequirements{ + ResourceRequirements: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + "cpu": resource.MustParse("32"), + "memory": resource.MustParse("256Gi"), + }, + }, + }, + RuntimeMin: 60 * time.Second, + } +} + +func JobTemplate1Cpu(n int64, jobSet, priorityClassName string, id string) *JobTemplate { + return &JobTemplate{ + Number: n, + JobSet: jobSet, + Id: id, + PriorityClassName: priorityClassName, + Requirements: schedulerobjects.PodRequirements{ + ResourceRequirements: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + "cpu": resource.MustParse("1"), + "memory": resource.MustParse("10Gi"), + }, + }, + }, + + RuntimeMin: 10 * time.Minute, + RuntimeVariance: 1 * time.Minute, + } +} + +func SubmitJob() *armadaevents.EventSequence_Event { + return &armadaevents.EventSequence_Event{ + Event: &armadaevents.EventSequence_Event_SubmitJob{ + SubmitJob: &armadaevents.SubmitJob{}, + }, + } +} + +func JobRunLeased() *armadaevents.EventSequence_Event { + return &armadaevents.EventSequence_Event{ + Event: &armadaevents.EventSequence_Event_JobRunLeased{ + JobRunLeased: &armadaevents.JobRunLeased{}, + }, + } +} + +func JobRunPreempted() *armadaevents.EventSequence_Event { + return &armadaevents.EventSequence_Event{ + Event: &armadaevents.EventSequence_Event_JobRunPreempted{ + JobRunPreempted: &armadaevents.JobRunPreempted{}, + }, + } +} + +func JobSucceeded() *armadaevents.EventSequence_Event { + return &armadaevents.EventSequence_Event{ + Event: &armadaevents.EventSequence_Event_JobSucceeded{ + JobSucceeded: &armadaevents.JobSucceeded{}, + }, + } +} + +func EventSequencesSummary(eventSequences []*armadaevents.EventSequence) string { + var sb strings.Builder + for i, eventSequence := range eventSequences { + sb.WriteString(EventSequenceSummary(eventSequence)) + if i != len(eventSequences)-1 { + sb.WriteString("\n") + } + } + return sb.String() +} + +func EventSequenceSummary(eventSequence *armadaevents.EventSequence) string { + var sb strings.Builder + sb.WriteString(fmt.Sprintf("EventSequence{Queue: %s, JobSetName: %s, Events: [", eventSequence.Queue, eventSequence.JobSetName)) + for i, event := range eventSequence.Events { + sb.WriteString(EventSummary(event)) + if i != len(eventSequence.Events)-1 { + sb.WriteString(", ") + } + } + sb.WriteString("]}") + return sb.String() +} + +func EventSummary(event *armadaevents.EventSequence_Event) string { + return strings.ReplaceAll(fmt.Sprintf("%T", event.Event), "*armadaevents.EventSequence_Event_", "") +} diff --git a/internal/scheduler/simulator/testdata/clusters/tinyCluster.yaml b/internal/scheduler/simulator/testdata/clusters/tinyCluster.yaml new file mode 100644 index 00000000000..02269a49542 --- /dev/null +++ b/internal/scheduler/simulator/testdata/clusters/tinyCluster.yaml @@ -0,0 +1,23 @@ +name: "Tiny Cluster" +pools: + - name: "pool1" + clusterGroups: + - clusters: + - name: "pool1-0-0" + nodeTemplates: + - number: 1 + totalResources: + resources: + cpu: "32" + memory: "256Gi" + - name: "pool2" + clusterGroups: + - clusters: + - name: "pool2-0-0" + nodeTemplates: + - number: 1 + totalResources: + resources: + cpu: "128" + memory: "4096Gi" + nvidia.com/gpu: "8" diff --git a/internal/scheduler/simulator/testdata/clusters/tinyClusterAlt.yaml b/internal/scheduler/simulator/testdata/clusters/tinyClusterAlt.yaml new file mode 100644 index 00000000000..74858933101 --- /dev/null +++ b/internal/scheduler/simulator/testdata/clusters/tinyClusterAlt.yaml @@ -0,0 +1,21 @@ +name: "Tiny Cluster" +pools: + - name: "pool1" + clusterGroups: + - clusters: + - name: "" + nodeTemplates: + - number: 1 + totalResources: + resources: + cpu: "32" + memory: "256Gi" + - clusters: + - name: "" + nodeTemplates: + - number: 1 + totalResources: + resources: + cpu: "128" + memory: "4096Gi" + nvidia.com/gpu: "8" diff --git a/internal/scheduler/simulator/testdata/diva-plat.yaml b/internal/scheduler/simulator/testdata/diva-plat.yaml deleted file mode 100644 index a4106287879..00000000000 --- a/internal/scheduler/simulator/testdata/diva-plat.yaml +++ /dev/null @@ -1,30 +0,0 @@ -name: "DIVA-plat" -pools: - - name: "CPU" - executorGroups: - - executors: - - name: "Executor-CPU-1" - nodeTemplates: - - number: 1 - totalResources: - resources: - cpu: "1" - memory: "1Gi" - - name: "Executor-CPU-2" - nodeTemplates: - - number: 2 - totalResources: - resources: - cpu: "1" - memory: "1Gi" - - name: "GPU" - executorGroups: - - executors: - - name: "Executor-GPU" - nodeTemplates: - - number: 2 - totalResources: - resources: - cpu: "1" - memory: "1Gi" - ndivia.com/gpu: "1" \ No newline at end of file diff --git a/internal/scheduler/simulator/testdata/schedulingconfigs/basicSchedulingConfig.yaml b/internal/scheduler/simulator/testdata/schedulingconfigs/basicSchedulingConfig.yaml new file mode 100644 index 00000000000..6153509c001 --- /dev/null +++ b/internal/scheduler/simulator/testdata/schedulingconfigs/basicSchedulingConfig.yaml @@ -0,0 +1,29 @@ +maximumSchedulingRate: "+inf" +maximumSchedulingBurst: 9223372036854775807 +maximumPerQueueSchedulingRate: "+Inf" +maximumPerQueueSchedulingBurst: 9223372036854775807 +fairnessModel: "DominantResourceFairness" +dominantResourceFairnessResourcesToConsider: + - "cpu" + - "memory" + - "nvidia.com/gpu" + - "ephemeral-storage" +maximumResourceFractionToSchedule: + memory: 0.025 + cpu: 0.025 +indexedResources: + - name: "cpu" + resolution: "1" + - name: "memory" + resolution: "1Mi" + - name: "nvidia.com/gpu" + resolution: "1" +preemption: + nodeEvictionProbability: 1.0 + priorityClasses: + armada-default: + priority: 30000 + preemptible: false + armada-preemptible: + priority: 30000 + preemptible: true diff --git a/internal/scheduler/simulator/testdata/workloads/basicWorkload.yaml b/internal/scheduler/simulator/testdata/workloads/basicWorkload.yaml new file mode 100644 index 00000000000..e34e66ecf61 --- /dev/null +++ b/internal/scheduler/simulator/testdata/workloads/basicWorkload.yaml @@ -0,0 +1,15 @@ +name: "Basic Workload" +queues: + - name: "A" + weight: 1 # = 1 / priorityFactor + jobTemplates: + - number: 10 + id: "myFirstJobTemplate" + priorityClassName: "armada-default" + requirements: + resourceRequirements: + requests: + cpu: 1 + memory: 10Gi + runtimeMin: 600e9 + runtimeVariance: 60e9