From a8cdc224f4511797fad175ca9cfbf2edc4a2e201 Mon Sep 17 00:00:00 2001 From: Martynas Asipauskas Date: Wed, 23 Oct 2024 11:26:18 +0100 Subject: [PATCH 1/9] More scheduler metrics Co-authored-by: Gary Edwards --- internal/scheduler/metrics/cycle_metrics.go | 148 ++++++++++++++++-- .../scheduler/metrics/cycle_metrics_test.go | 18 +++ internal/scheduler/scheduling/eviction.go | 28 ++-- .../scheduling/preempting_queue_scheduler.go | 29 ++-- .../scheduler/scheduling/queue_scheduler.go | 84 +++++----- internal/scheduler/scheduling/result.go | 29 ++++ .../scheduler/scheduling/scheduling_algo.go | 7 +- 7 files changed, 266 insertions(+), 77 deletions(-) diff --git a/internal/scheduler/metrics/cycle_metrics.go b/internal/scheduler/metrics/cycle_metrics.go index f289c6cc35f..b6fdfa22a91 100644 --- a/internal/scheduler/metrics/cycle_metrics.go +++ b/internal/scheduler/metrics/cycle_metrics.go @@ -9,8 +9,10 @@ import ( ) var ( + poolLabels = []string{poolLabel} poolAndQueueLabels = []string{poolLabel, queueLabel} queueAndPriorityClassLabels = []string{queueLabel, priorityClassLabel} + poolQueueAndResourceLabels = []string{poolLabel, queueLabel, resourceLabel} ) type cycleMetrics struct { @@ -27,6 +29,14 @@ type cycleMetrics struct { cappedDemand *prometheus.GaugeVec scheduleCycleTime prometheus.Histogram reconciliationCycleTime prometheus.Histogram + gangsConsidered *prometheus.GaugeVec + gangsScheduled *prometheus.GaugeVec + firstGangQueuePosition *prometheus.GaugeVec + lastGangQueuePosition *prometheus.GaugeVec + perQueueCycleTime *prometheus.GaugeVec + loopNumber *prometheus.GaugeVec + evictedJobs *prometheus.GaugeVec + evictedResources *prometheus.GaugeVec allResettableMetrics []resettableMetric } @@ -119,18 +129,90 @@ func newCycleMetrics() *cycleMetrics { }, ) + gangsConsidered := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "gangs_considered", + Help: "Number of gangs considered in this scheduling cycle", + }, + poolAndQueueLabels, + ) + + gangsScheduled := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "gangs_scheduled", + Help: "Number of gangs scheduled in this scheduling cycle", + }, + poolAndQueueLabels, + ) + + firstGangQueuePosition := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "first_gang_queue_position", + Help: "First position in the scheduling loop where a gang was considered", + }, + poolAndQueueLabels, + ) + + lastGangQueuePosition := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "last_gang_queue_position", + Help: "Last position in the scheduling loop where a gang was considered", + }, + poolAndQueueLabels, + ) + + perQueueCycleTime := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "per_queue_schedule_cycle_times", + Help: "Per queue cycle time when in a scheduling round.", + }, + poolAndQueueLabels, + ) + + loopNumber := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "loop_number", + Help: "Number of scheduling loops in this cycle", + }, + poolLabels, + ) + + evictedJobs := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "evicted_jobs", + Help: "Number of jobs evicted in this cycle", + }, + poolAndQueueLabels, + ) + + evictedResources := prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: prefix + "evicted_resources", + Help: "Resources evicted in this cycle", + }, + poolQueueAndResourceLabels, + ) + return &cycleMetrics{ - leaderMetricsEnabled: true, - scheduledJobs: scheduledJobs, - premptedJobs: premptedJobs, - consideredJobs: consideredJobs, - fairShare: fairShare, - adjustedFairShare: adjustedFairShare, - actualShare: actualShare, - demand: demand, - cappedDemand: cappedDemand, - fairnessError: fairnessError, - scheduleCycleTime: scheduleCycleTime, + leaderMetricsEnabled: true, + scheduledJobs: scheduledJobs, + premptedJobs: premptedJobs, + consideredJobs: consideredJobs, + fairShare: fairShare, + adjustedFairShare: adjustedFairShare, + actualShare: actualShare, + demand: demand, + cappedDemand: cappedDemand, + fairnessError: fairnessError, + scheduleCycleTime: scheduleCycleTime, + gangsConsidered: gangsConsidered, + gangsScheduled: gangsScheduled, + firstGangQueuePosition: firstGangQueuePosition, + lastGangQueuePosition: lastGangQueuePosition, + perQueueCycleTime: perQueueCycleTime, + loopNumber: loopNumber, + evictedJobs: evictedJobs, + evictedResources: evictedResources, allResettableMetrics: []resettableMetric{ scheduledJobs, premptedJobs, @@ -141,6 +223,14 @@ func newCycleMetrics() *cycleMetrics { demand, cappedDemand, fairnessError, + gangsConsidered, + gangsScheduled, + firstGangQueuePosition, + lastGangQueuePosition, + perQueueCycleTime, + loopNumber, + evictedJobs, + evictedResources, }, reconciliationCycleTime: reconciliationCycleTime, } @@ -196,6 +286,26 @@ func (m *cycleMetrics) ReportSchedulerResult(result scheduling.SchedulerResult) for _, jobCtx := range result.PreemptedJobs { m.premptedJobs.WithLabelValues(jobCtx.Job.Queue(), jobCtx.PriorityClassName).Inc() } + + for pool, schedulingStats := range result.PerPoolSchedulingStats { + for queue, s := range schedulingStats.StatsPerQueue { + m.gangsConsidered.WithLabelValues(pool, queue).Set(float64(s.GangsConsidered)) + m.gangsScheduled.WithLabelValues(pool, queue).Set(float64(s.GangsScheduled)) + m.firstGangQueuePosition.WithLabelValues(pool, queue).Set(float64(s.FirstGangConsideredQueuePosition)) + m.lastGangQueuePosition.WithLabelValues(pool, queue).Set(float64(s.LastGangScheduledQueuePosition)) + m.perQueueCycleTime.WithLabelValues(pool, queue).Set(float64(s.Time.Milliseconds())) + } + + m.loopNumber.WithLabelValues(pool).Set(float64(schedulingStats.LoopNumber)) + + for queue, s := range schedulingStats.EvictorResult.GetStatsPerQueue() { + m.evictedJobs.WithLabelValues(pool, queue).Set(float64(s.EvictedJobCount)) + + for _, r := range s.EvictedResources.GetResources() { + m.evictedResources.WithLabelValues(pool, queue, r.Name).Set(float64(r.Value)) + } + } + } } func (m *cycleMetrics) describe(ch chan<- *prometheus.Desc) { @@ -210,6 +320,14 @@ func (m *cycleMetrics) describe(ch chan<- *prometheus.Desc) { m.demand.Describe(ch) m.cappedDemand.Describe(ch) m.scheduleCycleTime.Describe(ch) + m.gangsConsidered.Describe(ch) + m.gangsScheduled.Describe(ch) + m.firstGangQueuePosition.Describe(ch) + m.lastGangQueuePosition.Describe(ch) + m.perQueueCycleTime.Describe(ch) + m.loopNumber.Describe(ch) + m.evictedJobs.Describe(ch) + m.evictedResources.Describe(ch) } m.reconciliationCycleTime.Describe(ch) @@ -227,6 +345,14 @@ func (m *cycleMetrics) collect(ch chan<- prometheus.Metric) { m.demand.Collect(ch) m.cappedDemand.Collect(ch) m.scheduleCycleTime.Collect(ch) + m.gangsConsidered.Collect(ch) + m.gangsScheduled.Collect(ch) + m.firstGangQueuePosition.Collect(ch) + m.lastGangQueuePosition.Collect(ch) + m.perQueueCycleTime.Collect(ch) + m.loopNumber.Collect(ch) + m.evictedJobs.Collect(ch) + m.evictedResources.Collect(ch) } m.reconciliationCycleTime.Collect(ch) diff --git a/internal/scheduler/metrics/cycle_metrics_test.go b/internal/scheduler/metrics/cycle_metrics_test.go index c18efde9549..86c810b6fa2 100644 --- a/internal/scheduler/metrics/cycle_metrics_test.go +++ b/internal/scheduler/metrics/cycle_metrics_test.go @@ -82,7 +82,9 @@ func TestReportStateTransitions(t *testing.T) { func TestResetLeaderMetrics(t *testing.T) { m := newCycleMetrics() + poolLabelValues := []string{"pool1"} poolQueueLabelValues := []string{"pool1", "queue1"} + poolQueueResouceLabelValues := []string{"pool1", "queue1", "cpu"} queuePriorityClassLabelValues := []string{"pool1", "priorityClass1"} testResetCounter := func(vec *prometheus.CounterVec, labelValues []string) { @@ -111,6 +113,14 @@ func TestResetLeaderMetrics(t *testing.T) { testResetGauge(m.fairnessError, []string{"pool1"}) testResetGauge(m.demand, poolQueueLabelValues) testResetGauge(m.cappedDemand, poolQueueLabelValues) + testResetGauge(m.gangsConsidered, poolQueueLabelValues) + testResetGauge(m.gangsScheduled, poolQueueLabelValues) + testResetGauge(m.firstGangQueuePosition, poolQueueLabelValues) + testResetGauge(m.lastGangQueuePosition, poolQueueLabelValues) + testResetGauge(m.perQueueCycleTime, poolQueueLabelValues) + testResetGauge(m.loopNumber, poolLabelValues) + testResetGauge(m.evictedJobs, poolQueueLabelValues) + testResetGauge(m.evictedResources, poolQueueResouceLabelValues) } func TestDisableLeaderMetrics(t *testing.T) { @@ -131,6 +141,14 @@ func TestDisableLeaderMetrics(t *testing.T) { m.cappedDemand.WithLabelValues(poolQueueLabelValues...).Inc() m.scheduleCycleTime.Observe(float64(1000)) m.reconciliationCycleTime.Observe(float64(1000)) + m.gangsConsidered.WithLabelValues("pool1", "queue1").Inc() + m.gangsScheduled.WithLabelValues("pool1", "queue1").Inc() + m.firstGangQueuePosition.WithLabelValues("pool1", "queue1").Inc() + m.lastGangQueuePosition.WithLabelValues("pool1", "queue1").Inc() + m.perQueueCycleTime.WithLabelValues("pool1", "queue1").Inc() + m.loopNumber.WithLabelValues("pool1").Inc() + m.evictedJobs.WithLabelValues("pool1", "queue1").Inc() + m.evictedResources.WithLabelValues("pool1", "queue1", "cpu").Inc() ch := make(chan prometheus.Metric, 1000) m.collect(ch) diff --git a/internal/scheduler/scheduling/eviction.go b/internal/scheduler/scheduling/eviction.go index f77e389de62..eefbda16f2b 100644 --- a/internal/scheduler/scheduling/eviction.go +++ b/internal/scheduler/scheduling/eviction.go @@ -33,21 +33,29 @@ type queueChecker interface { QueueContextExists(job *jobdb.Job) bool } -func (er *EvictorResult) SummaryString() string { - type queueStats struct { - evictedJobCount int - evictedResources internaltypes.ResourceList - } - statsPerQueue := map[string]queueStats{} +type EvictorPerQueueStats struct { + EvictedJobCount int + EvictedResources internaltypes.ResourceList +} + +func (er *EvictorResult) GetStatsPerQueue() map[string]EvictorPerQueueStats { + statsPerQueue := map[string]EvictorPerQueueStats{} for _, jctx := range er.EvictedJctxsByJobId { queue := jctx.Job.Queue() stats := statsPerQueue[queue] - stats.evictedJobCount++ - stats.evictedResources = stats.evictedResources.Add(jctx.Job.KubernetesResourceRequirements()) + stats.EvictedJobCount++ + stats.EvictedResources = stats.EvictedResources.Add(jctx.Job.KubernetesResourceRequirements()) statsPerQueue[queue] = stats } - return fmt.Sprintf("%v", armadamaps.MapValues(statsPerQueue, func(s queueStats) string { - return fmt.Sprintf("{evictedJobCount=%d, evictedResources={%s}}", s.evictedJobCount, s.evictedResources.String()) + + return statsPerQueue +} + +func (er *EvictorResult) SummaryString() string { + statsPerQueue := er.GetStatsPerQueue() + + return fmt.Sprintf("%v", armadamaps.MapValues(statsPerQueue, func(s EvictorPerQueueStats) string { + return fmt.Sprintf("{evictedJobCount=%d, evictedResources={%s}}", s.EvictedJobCount, s.EvictedResources.String()) })) } diff --git a/internal/scheduler/scheduling/preempting_queue_scheduler.go b/internal/scheduler/scheduling/preempting_queue_scheduler.go index 60b75e5bef7..84c88cb0518 100644 --- a/internal/scheduler/scheduling/preempting_queue_scheduler.go +++ b/internal/scheduler/scheduling/preempting_queue_scheduler.go @@ -161,7 +161,7 @@ func (sch *PreemptingQueueScheduler) Schedule(ctx *armadacontext.Context) (*Sche // Evict jobs on oversubscribed nodes. ctx.WithField("stage", "scheduling-algo").Info("Evicting jobs from oversubscribed nodes") - evictorResult, inMemoryJobRepo, err = sch.evict( + reevictResult, inMemoryJobRepo, err := sch.evict( armadacontext.WithLogField(ctx, "stage", "evict oversubscribed"), NewOversubscribedEvictor( sch.schedulingContext, @@ -176,24 +176,24 @@ func (sch *PreemptingQueueScheduler) Schedule(ctx *armadacontext.Context) (*Sche scheduledAndEvictedJobsById := armadamaps.FilterKeys( scheduledJobsById, func(jobId string) bool { - _, ok := evictorResult.EvictedJctxsByJobId[jobId] + _, ok := reevictResult.EvictedJctxsByJobId[jobId] return ok }, ) - for jobId, jctx := range evictorResult.EvictedJctxsByJobId { + for jobId, jctx := range reevictResult.EvictedJctxsByJobId { if _, ok := scheduledJobsById[jobId]; ok { delete(scheduledJobsById, jobId) } else { preemptedJobsById[jobId] = jctx } } - maps.Copy(sch.nodeIdByJobId, evictorResult.NodeIdByJobId) + maps.Copy(sch.nodeIdByJobId, reevictResult.NodeIdByJobId) // Re-schedule evicted jobs/schedule new jobs. // Only necessary if a non-zero number of jobs were evicted. - if len(evictorResult.EvictedJctxsByJobId) > 0 { + if len(reevictResult.EvictedJctxsByJobId) > 0 { ctx.WithField("stage", "scheduling-algo").Info("Performing second scheduling ") - schedulerResult, err = sch.schedule( + rescheduleSchedulerResult, rescheduleErr := sch.schedule( armadacontext.WithLogField(ctx, "stage", "schedule after oversubscribed eviction"), inMemoryJobRepo, // Only evicted jobs should be scheduled in this round. @@ -201,11 +201,11 @@ func (sch *PreemptingQueueScheduler) Schedule(ctx *armadacontext.Context) (*Sche true, // Since no new jobs are considered in this round, the scheduling key check brings no benefit. true, // when choosing which queue to consider use the priority class of the next job ) - if err != nil { - return nil, err + if rescheduleErr != nil { + return nil, rescheduleErr } ctx.WithField("stage", "scheduling-algo").Info("Finished second scheduling pass") - for _, jctx := range schedulerResult.ScheduledJobs { + for _, jctx := range rescheduleSchedulerResult.ScheduledJobs { if _, ok := preemptedJobsById[jctx.JobId]; ok { delete(preemptedJobsById, jctx.JobId) } else { @@ -213,7 +213,7 @@ func (sch *PreemptingQueueScheduler) Schedule(ctx *armadacontext.Context) (*Sche } delete(scheduledAndEvictedJobsById, jctx.JobId) } - maps.Copy(sch.nodeIdByJobId, schedulerResult.NodeIdByJobId) + maps.Copy(sch.nodeIdByJobId, rescheduleSchedulerResult.NodeIdByJobId) } preemptedJobs := maps.Values(preemptedJobsById) @@ -231,11 +231,20 @@ func (sch *PreemptingQueueScheduler) Schedule(ctx *armadacontext.Context) (*Sche schedulercontext.PrintJobSummary(ctx, "Scheduling new jobs;", scheduledJobs) // TODO: Show failed jobs. + schedulingStats := PerPoolSchedulingStats{ + StatsPerQueue: schedulerResult.PerPoolSchedulingStats[sch.schedulingContext.Pool].StatsPerQueue, + LoopNumber: schedulerResult.PerPoolSchedulingStats[sch.schedulingContext.Pool].LoopNumber, + EvictorResult: evictorResult, + } + return &SchedulerResult{ PreemptedJobs: preemptedJobs, ScheduledJobs: scheduledJobs, NodeIdByJobId: sch.nodeIdByJobId, SchedulingContexts: []*schedulercontext.SchedulingContext{sch.schedulingContext}, + PerPoolSchedulingStats: map[string]PerPoolSchedulingStats{ + sch.schedulingContext.Pool: schedulingStats, + }, }, nil } diff --git a/internal/scheduler/scheduling/queue_scheduler.go b/internal/scheduler/scheduling/queue_scheduler.go index 86c68495305..2ee793e028c 100644 --- a/internal/scheduler/scheduling/queue_scheduler.go +++ b/internal/scheduler/scheduling/queue_scheduler.go @@ -66,22 +66,7 @@ func (sch *QueueScheduler) Schedule(ctx *armadacontext.Context) (*SchedulerResul nodeIdByJobId := make(map[string]string) ctx.Infof("Looping through candidate gangs for pool %s...", sch.schedulingContext.Pool) - type queueStats struct { - gangsConsidered int - jobsConsidered int - gangsScheduled int - firstGangConsideredSampleJobId string - firstGangConsideredResult string - firstGangConsideredQueuePosition int - lastGangScheduledSampleJobId string - lastGangScheduledQueuePosition int - lastGangScheduledQueueCost float64 - lastGangScheduledResources schedulerobjects.ResourceList - lastGangScheduledQueueResources schedulerobjects.ResourceList - time time.Duration - } - - statsPerQueue := map[string]queueStats{} + statsPerQueue := map[string]QueueStats{} loopNumber := 0 for { // Peek() returns the next gang to try to schedule. Call Clear() before calling Peek() again. @@ -134,37 +119,37 @@ func (sch *QueueScheduler) Schedule(ctx *armadacontext.Context) (*SchedulerResul duration := time.Now().Sub(start) stats := statsPerQueue[gctx.Queue] - stats.gangsConsidered++ - stats.jobsConsidered += gctx.Cardinality() + stats.GangsConsidered++ + stats.JobsConsidered += gctx.Cardinality() if scheduledOk { - stats.gangsScheduled++ + stats.GangsScheduled++ } - if stats.firstGangConsideredSampleJobId == "" { - stats.firstGangConsideredSampleJobId = gctx.JobIds()[0] - stats.firstGangConsideredQueuePosition = loopNumber + if stats.FirstGangConsideredSampleJobId == "" { + stats.FirstGangConsideredSampleJobId = gctx.JobIds()[0] + stats.FirstGangConsideredQueuePosition = loopNumber if scheduledOk { - stats.firstGangConsideredResult = "scheduled" + stats.FirstGangConsideredResult = "scheduled" } else { - stats.firstGangConsideredResult = unschedulableReason + stats.FirstGangConsideredResult = unschedulableReason } } if scheduledOk { - stats.lastGangScheduledSampleJobId = gctx.JobIds()[0] - stats.lastGangScheduledQueueCost = queueCostInclGang - stats.lastGangScheduledQueuePosition = loopNumber + stats.LastGangScheduledSampleJobId = gctx.JobIds()[0] + stats.LastGangScheduledQueueCost = queueCostInclGang + stats.LastGangScheduledQueuePosition = loopNumber queue, queueOK := sch.candidateGangIterator.queueRepository.GetQueue(gctx.Queue) if queueOK { - stats.lastGangScheduledResources = gctx.TotalResourceRequests.DeepCopy() - stats.lastGangScheduledQueueResources = queue.GetAllocation().DeepCopy() + stats.LastGangScheduledResources = gctx.TotalResourceRequests.DeepCopy() + stats.LastGangScheduledQueueResources = queue.GetAllocation().DeepCopy() } else { - stats.lastGangScheduledResources = schedulerobjects.NewResourceListWithDefaultSize() - stats.lastGangScheduledQueueResources = schedulerobjects.NewResourceListWithDefaultSize() + stats.LastGangScheduledResources = schedulerobjects.NewResourceListWithDefaultSize() + stats.LastGangScheduledQueueResources = schedulerobjects.NewResourceListWithDefaultSize() } } - stats.time += duration + stats.Time += duration statsPerQueue[gctx.Queue] = stats if duration.Seconds() > 1 { ctx.Infof("Slow schedule: queue %s, gang cardinality %d, sample job id %s, time %fs", gctx.Queue, gctx.Cardinality(), gctx.JobIds()[0], duration.Seconds()) @@ -179,23 +164,23 @@ func (sch *QueueScheduler) Schedule(ctx *armadacontext.Context) (*SchedulerResul loopNumber++ } - ctx.Infof("Finished %d loops through candidate gangs for pool %s: details %v", loopNumber, sch.schedulingContext.Pool, armadamaps.MapValues(statsPerQueue, func(s queueStats) string { + ctx.Infof("Finished %d loops through candidate gangs for pool %s: details %v", loopNumber, sch.schedulingContext.Pool, armadamaps.MapValues(statsPerQueue, func(s QueueStats) string { return fmt.Sprintf("{gangsConsidered=%d, jobsConsidered=%d, gangsScheduled=%d, "+ "firstGangConsideredSampleJobId=%s, firstGangConsideredResult=%s, firstGangConsideredQueuePosition=%d, "+ "lastGangScheduledSampleJobId=%s, lastGangScheduledQueuePosition=%d, lastGangScheduledQueueCost=%f,"+ "lastGangScheduledResources=%s, lastGangScheduledQueueResources=%s, time=%fs}", - s.gangsConsidered, - s.jobsConsidered, - s.gangsScheduled, - s.firstGangConsideredSampleJobId, - s.firstGangConsideredResult, - s.firstGangConsideredQueuePosition, - s.lastGangScheduledSampleJobId, - s.lastGangScheduledQueuePosition, - s.lastGangScheduledQueueCost, - s.lastGangScheduledResources.CompactString(), - s.lastGangScheduledQueueResources.CompactString(), - s.time.Seconds()) + s.GangsConsidered, + s.JobsConsidered, + s.GangsScheduled, + s.FirstGangConsideredSampleJobId, + s.FirstGangConsideredResult, + s.FirstGangConsideredQueuePosition, + s.LastGangScheduledSampleJobId, + s.LastGangScheduledQueuePosition, + s.LastGangScheduledQueueCost, + s.LastGangScheduledResources.CompactString(), + s.LastGangScheduledQueueResources.CompactString(), + s.Time.Seconds()) })) if sch.schedulingContext.TerminationReason == "" { @@ -204,11 +189,20 @@ func (sch *QueueScheduler) Schedule(ctx *armadacontext.Context) (*SchedulerResul if len(scheduledJobs) != len(nodeIdByJobId) { return nil, errors.Errorf("only %d out of %d jobs mapped to a node", len(nodeIdByJobId), len(scheduledJobs)) } + + schedulingStats := PerPoolSchedulingStats{ + StatsPerQueue: statsPerQueue, + LoopNumber: loopNumber, + } + return &SchedulerResult{ PreemptedJobs: nil, ScheduledJobs: scheduledJobs, NodeIdByJobId: nodeIdByJobId, SchedulingContexts: []*schedulercontext.SchedulingContext{sch.schedulingContext}, + PerPoolSchedulingStats: map[string]PerPoolSchedulingStats{ + sch.schedulingContext.Pool: schedulingStats, + }, }, nil } diff --git a/internal/scheduler/scheduling/result.go b/internal/scheduler/scheduling/result.go index f6518ce7797..a8669a6d6a2 100644 --- a/internal/scheduler/scheduling/result.go +++ b/internal/scheduler/scheduling/result.go @@ -1,10 +1,37 @@ package scheduling import ( + "time" + "github.com/armadaproject/armada/internal/scheduler/jobdb" + "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" "github.com/armadaproject/armada/internal/scheduler/scheduling/context" ) +type QueueStats struct { + GangsConsidered int + JobsConsidered int + GangsScheduled int + FirstGangConsideredSampleJobId string + FirstGangConsideredResult string + FirstGangConsideredQueuePosition int + LastGangScheduledSampleJobId string + LastGangScheduledQueuePosition int + LastGangScheduledQueueCost float64 + LastGangScheduledResources schedulerobjects.ResourceList + LastGangScheduledQueueResources schedulerobjects.ResourceList + Time time.Duration +} + +type PerPoolSchedulingStats struct { + // scheduling stats per queue + StatsPerQueue map[string]QueueStats + // number of loops executed in this cycle + LoopNumber int + // Result of any eviction in this cycle + EvictorResult *EvictorResult +} + // SchedulerResult is returned by Rescheduler.Schedule(). type SchedulerResult struct { // Running jobs that should be preempted. @@ -18,6 +45,8 @@ type SchedulerResult struct { // These are the corresponding scheduling contexts. // TODO: This doesn't seem like the right approach. SchedulingContexts []*context.SchedulingContext + // scheduling stats + PerPoolSchedulingStats map[string]PerPoolSchedulingStats } // PreemptedJobsFromSchedulerResult returns the slice of preempted jobs in the result. diff --git a/internal/scheduler/scheduling/scheduling_algo.go b/internal/scheduler/scheduling/scheduling_algo.go index c360e53a93c..bb9aa5b4814 100644 --- a/internal/scheduler/scheduling/scheduling_algo.go +++ b/internal/scheduler/scheduling/scheduling_algo.go @@ -97,7 +97,8 @@ func (l *FairSchedulingAlgo) Schedule( defer cancel() } overallSchedulerResult := &SchedulerResult{ - NodeIdByJobId: make(map[string]string), + NodeIdByJobId: make(map[string]string), + PerPoolSchedulingStats: make(map[string]PerPoolSchedulingStats), } // Exit immediately if scheduling is disabled. @@ -162,6 +163,10 @@ func (l *FairSchedulingAlgo) Schedule( overallSchedulerResult.ScheduledJobs = append(overallSchedulerResult.ScheduledJobs, schedulerResult.ScheduledJobs...) overallSchedulerResult.SchedulingContexts = append(overallSchedulerResult.SchedulingContexts, schedulerResult.SchedulingContexts...) maps.Copy(overallSchedulerResult.NodeIdByJobId, schedulerResult.NodeIdByJobId) + + for p, s := range schedulerResult.PerPoolSchedulingStats { + overallSchedulerResult.PerPoolSchedulingStats[p] = s + } } return overallSchedulerResult, nil } From 741f897a42ad94578a9a4e986f4779f376aaf798 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 24 Oct 2024 10:20:27 +0100 Subject: [PATCH 2/9] Improvements to simulation following testing (#260) (#4023) * fix test * lint Co-authored-by: Christopher Martin --- cmd/simulator/cmd/root.go | 2 +- internal/scheduler/simulator/simulator.go | 5 +- .../simulator/sink/fair_share_writer.go | 64 ----------- .../simulator/sink/queue_stats_writer.go | 101 ++++++++++++++++++ internal/scheduler/simulator/sink/sink.go | 14 +-- 5 files changed, 113 insertions(+), 73 deletions(-) delete mode 100644 internal/scheduler/simulator/sink/fair_share_writer.go create mode 100644 internal/scheduler/simulator/sink/queue_stats_writer.go diff --git a/cmd/simulator/cmd/root.go b/cmd/simulator/cmd/root.go index 6f3fcc97375..0e5951d5e4a 100644 --- a/cmd/simulator/cmd/root.go +++ b/cmd/simulator/cmd/root.go @@ -84,7 +84,7 @@ func runSimulations(cmd *cobra.Command, args []string) error { } if pathExists(outputDirPath) && overwriteDirIfExists { - err := os.Remove(outputDirPath) + err := os.RemoveAll(outputDirPath) if err != nil { return err } diff --git a/internal/scheduler/simulator/simulator.go b/internal/scheduler/simulator/simulator.go index 9b4da445c1f..f600329c187 100644 --- a/internal/scheduler/simulator/simulator.go +++ b/internal/scheduler/simulator/simulator.go @@ -210,7 +210,7 @@ func (s *Simulator) Run(ctx *armadacontext.Context) error { return err } } - if time.Now().Unix()-lastLogTime.Unix() >= 15 { + if time.Now().Unix()-lastLogTime.Unix() >= 5 { ctx.Infof("Simulator time %s", s.time) lastLogTime = s.time } @@ -518,6 +518,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { return err } } + sctx.UpdateFairShares() constraints := schedulerconstraints.NewSchedulingConstraints(pool, totalResources, s.schedulingConfig, nil) sch := scheduling.NewPreemptingQueueScheduler( sctx, @@ -544,7 +545,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { return err } - err = s.sink.OnCycleEnd(result) + err = s.sink.OnCycleEnd(s.time, result) if err != nil { return err } diff --git a/internal/scheduler/simulator/sink/fair_share_writer.go b/internal/scheduler/simulator/sink/fair_share_writer.go deleted file mode 100644 index 43ead572a66..00000000000 --- a/internal/scheduler/simulator/sink/fair_share_writer.go +++ /dev/null @@ -1,64 +0,0 @@ -package sink - -import ( - "os" - - parquetWriter "github.com/xitongsys/parquet-go/writer" - - "github.com/armadaproject/armada/internal/common/armadacontext" - "github.com/armadaproject/armada/internal/scheduler/scheduling" -) - -type FairShareRow struct { - Ts int64 `parquet:"name=ts, type=INT64"` - Queue string `parquet:"name=queue, type=BYTE_ARRAY, convertedtype=UTF8, encoding=PLAIN_DICTIONARY"` - Pool string `parquet:"name=pool, type=BYTE_ARRAY, convertedtype=UTF8"` - FairShare float64 `parquet:"name=fair_share, type=DOUBLE"` - AdjustedFairShare float64 `parquet:"name=adjusted_fair_share, type=DOUBLE"` - ActualShare float64 `parquet:"name=actual_share, type=DOUBLE"` -} - -type FairShareWriter struct { - writer *parquetWriter.ParquetWriter -} - -func NewFairShareWriter(path string) (*FairShareWriter, error) { - fileWriter, err := os.Create(path + "/fair_share.parquet") - if err != nil { - return nil, err - } - pw, err := parquetWriter.NewParquetWriterFromWriter(fileWriter, new(FairShareRow), 1) - if err != nil { - return nil, err - } - return &FairShareWriter{ - writer: pw, - }, nil -} - -func (j *FairShareWriter) Update(result *scheduling.SchedulerResult) error { - for _, sctx := range result.SchedulingContexts { - for _, qctx := range sctx.QueueSchedulingContexts { - row := FairShareRow{ - Ts: 0, - Queue: qctx.Queue, - Pool: sctx.Pool, - FairShare: qctx.FairShare, - AdjustedFairShare: qctx.AdjustedFairShare, - ActualShare: sctx.FairnessCostProvider.UnweightedCostFromQueue(qctx), - } - err := j.writer.Write(row) - if err != nil { - return err - } - } - } - return nil -} - -func (j *FairShareWriter) Close(ctx *armadacontext.Context) { - err := j.writer.WriteStop() - if err != nil { - ctx.Warnf("Could not clearnly close fair share parquet file: %s", err) - } -} diff --git a/internal/scheduler/simulator/sink/queue_stats_writer.go b/internal/scheduler/simulator/sink/queue_stats_writer.go new file mode 100644 index 00000000000..07b575e6509 --- /dev/null +++ b/internal/scheduler/simulator/sink/queue_stats_writer.go @@ -0,0 +1,101 @@ +package sink + +import ( + "os" + "time" + + parquetWriter "github.com/xitongsys/parquet-go/writer" + + "github.com/armadaproject/armada/internal/common/armadacontext" + "github.com/armadaproject/armada/internal/scheduler/scheduling" + "github.com/armadaproject/armada/internal/scheduler/scheduling/context" +) + +type QueueStatsRow struct { + Ts int64 `parquet:"name=ts, type=INT64"` + Queue string `parquet:"name=queue, type=BYTE_ARRAY, convertedtype=UTF8, encoding=PLAIN_DICTIONARY"` + Pool string `parquet:"name=pool, type=BYTE_ARRAY, convertedtype=UTF8"` + FairShare float64 `parquet:"name=fair_share, type=DOUBLE"` + AdjustedFairShare float64 `parquet:"name=adjusted_fair_share, type=DOUBLE"` + ActualShare float64 `parquet:"name=actual_share, type=DOUBLE"` + CpuShare float64 `parquet:"name=cpu_share, type=DOUBLE"` + MemoryShare float64 `parquet:"name=memory_share, type=DOUBLE"` + GpuShare float64 `parquet:"name=gpu_share, type=DOUBLE"` + AllocatedCPU int `parquet:"name=allocated_cpu, type=INT64"` + AllocatedMemory int `parquet:"name=allocated_memory, type=INT64"` + AllocatedGPU int `parquet:"name=allocated_gpu, type=INT64"` + NumScheduled int `parquet:"name=num_scheduled, type=INT32"` + NumPreempted int `parquet:"name=num_preempted, type=INT32"` + NumEvicted int `parquet:"name=num_evicted, type=INT32"` +} + +type QueueStatsWriter struct { + writer *parquetWriter.ParquetWriter +} + +func NewQueueStatsWriter(path string) (*QueueStatsWriter, error) { + fileWriter, err := os.Create(path + "/queue_stats.parquet") + if err != nil { + return nil, err + } + pw, err := parquetWriter.NewParquetWriterFromWriter(fileWriter, new(QueueStatsRow), 1) + if err != nil { + return nil, err + } + return &QueueStatsWriter{ + writer: pw, + }, nil +} + +func (j *QueueStatsWriter) Update(time time.Time, result *scheduling.SchedulerResult) error { + // Work out number of preemptions per queue + preemptedJobsByQueue := map[string]int{} + for _, job := range result.PreemptedJobs { + preemptedJobsByQueue[job.Job.Queue()] = preemptedJobsByQueue[job.Job.Queue()] + 1 + } + + for _, sctx := range result.SchedulingContexts { + for _, qctx := range sctx.QueueSchedulingContexts { + row := QueueStatsRow{ + Ts: time.Unix(), + Queue: qctx.Queue, + Pool: sctx.Pool, + FairShare: qctx.FairShare, + AdjustedFairShare: qctx.AdjustedFairShare, + ActualShare: sctx.FairnessCostProvider.UnweightedCostFromQueue(qctx), + CpuShare: calculateResourceShare(sctx, qctx, "cpu"), + MemoryShare: calculateResourceShare(sctx, qctx, "memory"), + GpuShare: calculateResourceShare(sctx, qctx, "nvidia.com/gpu"), + AllocatedCPU: allocatedResources(qctx, "cpu"), + AllocatedMemory: allocatedResources(qctx, "memory") / (1024 * 1024), // in MB + AllocatedGPU: allocatedResources(qctx, "nvidia.com/gpu"), + NumScheduled: len(qctx.SuccessfulJobSchedulingContexts), + NumPreempted: preemptedJobsByQueue[qctx.Queue], + NumEvicted: len(qctx.EvictedJobsById), + } + err := j.writer.Write(row) + if err != nil { + return err + } + } + } + return nil +} + +func (j *QueueStatsWriter) Close(ctx *armadacontext.Context) { + err := j.writer.WriteStop() + if err != nil { + ctx.Warnf("Could not cleanly close queue_stats parquet file: %s", err) + } +} + +func calculateResourceShare(sctx *context.SchedulingContext, qctx *context.QueueSchedulingContext, resource string) float64 { + total := sctx.Allocated.Resources[resource] + allocated := qctx.Allocated.Resources[resource] + return allocated.AsApproximateFloat64() / total.AsApproximateFloat64() +} + +func allocatedResources(qctx *context.QueueSchedulingContext, resource string) int { + allocated := qctx.Allocated.Resources[resource] + return int(allocated.AsApproximateFloat64()) +} diff --git a/internal/scheduler/simulator/sink/sink.go b/internal/scheduler/simulator/sink/sink.go index 2a65cc26711..b2c599dc07d 100644 --- a/internal/scheduler/simulator/sink/sink.go +++ b/internal/scheduler/simulator/sink/sink.go @@ -1,6 +1,8 @@ package sink import ( + "time" + "github.com/armadaproject/armada/internal/common/armadacontext" "github.com/armadaproject/armada/internal/scheduler/scheduling" "github.com/armadaproject/armada/internal/scheduler/simulator/model" @@ -8,13 +10,13 @@ import ( type Sink interface { OnNewStateTransitions(transitions []*model.StateTransition) error - OnCycleEnd(result *scheduling.SchedulerResult) error + OnCycleEnd(time time.Time, result *scheduling.SchedulerResult) error Close(ctx *armadacontext.Context) } type ParquetSink struct { jobWriter *JobWriter - fairShareWriter *FairShareWriter + fairShareWriter *QueueStatsWriter } func NewParquetSink(outputDir string) (*ParquetSink, error) { @@ -22,7 +24,7 @@ func NewParquetSink(outputDir string) (*ParquetSink, error) { if err != nil { return nil, err } - fairShareWriter, err := NewFairShareWriter(outputDir) + fairShareWriter, err := NewQueueStatsWriter(outputDir) if err != nil { return nil, err } @@ -42,8 +44,8 @@ func (s *ParquetSink) OnNewStateTransitions(transitions []*model.StateTransition return nil } -func (s *ParquetSink) OnCycleEnd(result *scheduling.SchedulerResult) error { - err := s.fairShareWriter.Update(result) +func (s *ParquetSink) OnCycleEnd(time time.Time, result *scheduling.SchedulerResult) error { + err := s.fairShareWriter.Update(time, result) if err != nil { return err } @@ -61,7 +63,7 @@ func (s NullSink) OnNewStateTransitions(_ []*model.StateTransition) error { return nil } -func (s NullSink) OnCycleEnd(_ *scheduling.SchedulerResult) error { +func (s NullSink) OnCycleEnd(_ time.Time, _ *scheduling.SchedulerResult) error { return nil } From 95fa3b22e4858bf4dac8b01eecc0d933032de547 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Fri, 25 Oct 2024 10:41:34 +0100 Subject: [PATCH 3/9] Make Gang Jobs Work in The simulator (#4024) * basic add gang Signed-off-by: Chris Martin * wip Signed-off-by: Chris Martin * wip Signed-off-by: Chris Martin * wip Signed-off-by: Chris Martin * wip Signed-off-by: Chris Martin * added test for gang preemption Signed-off-by: Chris Martin * lint Signed-off-by: Chris Martin * exct number of gang jobs Signed-off-by: Chris Martin * const label Signed-off-by: Chris Martin * const label Signed-off-by: Chris Martin --------- Signed-off-by: Chris Martin --- internal/scheduler/simulator/simulator.go | 231 ++++++++++++----- internal/scheduler/simulator/simulator.pb.go | 240 ++++++++++++------ internal/scheduler/simulator/simulator.proto | 4 + .../scheduler/simulator/simulator_test.go | 75 ++++++ internal/scheduler/simulator/test_utils.go | 8 +- 5 files changed, 421 insertions(+), 137 deletions(-) diff --git a/internal/scheduler/simulator/simulator.go b/internal/scheduler/simulator/simulator.go index f600329c187..2dde5298010 100644 --- a/internal/scheduler/simulator/simulator.go +++ b/internal/scheduler/simulator/simulator.go @@ -5,6 +5,8 @@ import ( "fmt" "math" "math/rand" + "strconv" + "strings" "sync/atomic" "time" @@ -35,11 +37,34 @@ import ( "github.com/armadaproject/armada/internal/scheduler/simulator/model" "github.com/armadaproject/armada/internal/scheduler/simulator/sink" "github.com/armadaproject/armada/internal/scheduleringester" + serverconfig "github.com/armadaproject/armada/internal/server/configuration" "github.com/armadaproject/armada/pkg/armadaevents" ) +const clusterLabel = "armadaproject.io/clusterName" + var epochStart = time.Unix(0, 0).UTC() +type accounting struct { + // Map from node id to the pool to which the node belongs. + poolByNodeId map[string]string + // Separate nodeDb per pool + nodeDbByPool map[string]*nodedb.NodeDb + // Allocation by pool for each queue and priority class. + // Stored across invocations of the scheduler. + allocationByPoolAndQueueAndPriorityClass map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string] + // Demand for each queue + demandByQueue map[string]schedulerobjects.ResourceList + // Total resources across all executorGroups for each pool. + totalResourcesByPool map[string]schedulerobjects.ResourceList + // Mapping of job Id -> nodeId. Needed by preemptingqueuescheduler for gang preemption. + nodeIdByJobId map[string]string + // Mapping of gangId -> jobsINGang. Needed by preemptingqueuescheduler for gang preemption. + jobIdsByGangId map[string]map[string]bool + // Mapping of jobId -> gangId. Needed by preemptingqueuescheduler for gang preemption. + gangIdByJobId map[string]string +} + // Simulator captures the parameters and state of the Armada simulator. type Simulator struct { ClusterSpec *ClusterSpec @@ -52,18 +77,7 @@ type Simulator struct { // Map from job template id to jobTemplate for templates for which all jobs have not yet succeeded. activeJobTemplatesById map[string]*JobTemplate // The JobDb stores all jobs that have yet to terminate. - jobDb *jobdb.JobDb - // Map from node id to the pool to which the node belongs. - poolByNodeId map[string]string - // Separate nodeDb per pool - nodeDbByPool map[string]*nodedb.NodeDb - // Allocation by pool for each queue and priority class. - // Stored across invocations of the scheduler. - allocationByPoolAndQueueAndPriorityClass map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string] - demandByQueue map[string]schedulerobjects.ResourceList - // 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. + jobDb *jobdb.JobDb shouldSchedule bool // Current simulated time. time time.Time @@ -96,6 +110,8 @@ type Simulator struct { sink sink.Sink // Floating resource info floatingResourceTypes *floatingresources.FloatingResourceTypes + // Keeps track of what's allocated where + accounting accounting } func NewSimulator( @@ -139,27 +155,32 @@ func NewSimulator( randomSeed = time.Now().Unix() } 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, - nodeDbByPool: make(map[string]*nodedb.NodeDb), - poolByNodeId: make(map[string]string), - allocationByPoolAndQueueAndPriorityClass: make(map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string]), - demandByQueue: make(map[string]schedulerobjects.ResourceList), - totalResourcesByPool: make(map[string]schedulerobjects.ResourceList), - limiter: rate.NewLimiter(rate.Inf, math.MaxInt), // Unlimited - rand: rand.New(rand.NewSource(randomSeed)), - resourceListFactory: resourceListFactory, - enableFastForward: enableFastForward, - hardTerminationMinutes: hardTerminationMinutes, - schedulerCyclePeriodSeconds: schedulerCyclePeriodSeconds, - floatingResourceTypes: floatingResourceTypes, - time: epochStart, - sink: sink, + 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, + limiter: rate.NewLimiter(rate.Inf, math.MaxInt), // Unlimited + rand: rand.New(rand.NewSource(randomSeed)), + resourceListFactory: resourceListFactory, + enableFastForward: enableFastForward, + hardTerminationMinutes: hardTerminationMinutes, + schedulerCyclePeriodSeconds: schedulerCyclePeriodSeconds, + floatingResourceTypes: floatingResourceTypes, + time: epochStart, + sink: sink, + accounting: accounting{ + nodeDbByPool: make(map[string]*nodedb.NodeDb), + poolByNodeId: make(map[string]string), + allocationByPoolAndQueueAndPriorityClass: make(map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string]), + demandByQueue: make(map[string]schedulerobjects.ResourceList), + totalResourcesByPool: make(map[string]schedulerobjects.ResourceList), + nodeIdByJobId: make(map[string]string), + jobIdsByGangId: make(map[string]map[string]bool), + gangIdByJobId: make(map[string]string), + }, } jobDb.SetClock(s) if err := s.setupClusters(); err != nil { @@ -265,23 +286,38 @@ func validateWorkloadSpec(workloadSpec *WorkloadSpec) error { if !slices.Equal(jobTemplateIds, armadaslices.Unique(jobTemplateIds)) { return errors.Errorf("duplicate job template ids: %v", jobTemplateIds) } - + for _, queue := range workloadSpec.Queues { + for _, template := range queue.JobTemplates { + // Confirm that we can create an exact number of gang jobs + if template.GangCardinality != 0 && int(template.Number)%int(template.GangCardinality) != 0 { + return errors.Errorf("template.Number [%d] is not exactly divisible by template.GangCardinality [%d]", template.Number, template.GangCardinality) + } + } + } return nil } func (s *Simulator) setupClusters() error { + indexedNodeLabels := s.schedulingConfig.IndexedNodeLabels + if indexedNodeLabels == nil { + indexedNodeLabels = []string{} + } + if !slices.Contains(indexedNodeLabels, clusterLabel) { + indexedNodeLabels = append(indexedNodeLabels, clusterLabel) + } + nodeFactory := internaltypes.NewNodeFactory(s.schedulingConfig.IndexedTaints, - s.schedulingConfig.IndexedNodeLabels, + indexedNodeLabels, s.resourceListFactory) for _, cluster := range s.ClusterSpec.Clusters { - nodeDb, ok := s.nodeDbByPool[cluster.Pool] + nodeDb, ok := s.accounting.nodeDbByPool[cluster.Pool] if !ok { newNodeDb, err := nodedb.NewNodeDb( s.schedulingConfig.PriorityClasses, s.schedulingConfig.IndexedResources, s.schedulingConfig.IndexedTaints, - s.schedulingConfig.IndexedNodeLabels, + indexedNodeLabels, s.schedulingConfig.WellKnownNodeTypes, s.resourceListFactory, ) @@ -289,15 +325,20 @@ func (s *Simulator) setupClusters() error { return err } nodeDb = newNodeDb - s.nodeDbByPool[cluster.Pool] = nodeDb + s.accounting.nodeDbByPool[cluster.Pool] = nodeDb } - totalResourcesForPool, ok := s.totalResourcesByPool[cluster.Pool] + totalResourcesForPool, ok := s.accounting.totalResourcesByPool[cluster.Pool] if !ok { totalResourcesForPool = schedulerobjects.ResourceList{} } for nodeTemplateIndex, nodeTemplate := range cluster.NodeTemplates { + labels := map[string]string{} + if nodeTemplate.Labels != nil { + labels = maps.Clone(nodeTemplate.Labels) + } + labels[clusterLabel] = cluster.Name for i := 0; i < int(nodeTemplate.Number); i++ { nodeId := fmt.Sprintf("%s-%d-%d", cluster.Name, nodeTemplateIndex, i) node := &schedulerobjects.Node{ @@ -306,7 +347,7 @@ func (s *Simulator) setupClusters() error { Executor: cluster.Name, Pool: cluster.Pool, Taints: slices.Clone(nodeTemplate.Taints), - Labels: maps.Clone(nodeTemplate.Labels), + Labels: labels, TotalResources: nodeTemplate.TotalResources.DeepCopy(), AllocatableByPriorityAndResource: schedulerobjects.NewAllocatableByPriorityAndResourceType( types.AllowedPriorities(s.schedulingConfig.PriorityClasses), @@ -324,11 +365,11 @@ func (s *Simulator) setupClusters() error { return err } txn.Commit() - s.poolByNodeId[nodeId] = cluster.Pool + s.accounting.poolByNodeId[nodeId] = cluster.Pool } } totalResourcesForPool.Add(nodeDb.TotalKubernetesResources()) - s.totalResourcesByPool[cluster.Pool] = totalResourcesForPool + s.accounting.totalResourcesByPool[cluster.Pool] = totalResourcesForPool } return nil } @@ -351,17 +392,22 @@ func (s *Simulator) bootstrapWorkload() error { Queue: queue.Name, JobSetName: jobTemplate.JobSet, } + gangId := "" + gangCardinality := int(jobTemplate.GangCardinality) for k := 0; k < int(jobTemplate.Number); k++ { if len(jobTemplate.Dependencies) > 0 { continue } jobId := util.NewULID() + if gangCardinality != 0 && k%gangCardinality == 0 { + gangId = fmt.Sprintf("%s-0", util.NewULID()) + } eventSequence.Events = append( eventSequence.Events, &armadaevents.EventSequence_Event{ Created: protoutil.ToTimestamp(s.time.Add(jobTemplate.EarliestSubmitTime)), Event: &armadaevents.EventSequence_Event_SubmitJob{ - SubmitJob: submitJobFromJobTemplate(jobId, jobTemplate), + SubmitJob: submitJobFromJobTemplate(jobId, jobTemplate, gangId), }, }, ) @@ -396,13 +442,28 @@ func (s *Simulator) bootstrapWorkload() error { return nil } -func submitJobFromJobTemplate(jobId string, jobTemplate *JobTemplate) *armadaevents.SubmitJob { +func submitJobFromJobTemplate(jobId string, jobTemplate *JobTemplate, gangId string) *armadaevents.SubmitJob { + annotations := jobTemplate.Requirements.Annotations + if annotations == nil { + annotations = map[string]string{} + } + + if jobTemplate.GangCardinality > 0 { + annotations[serverconfig.GangIdAnnotation] = gangId + annotations[serverconfig.GangCardinalityAnnotation] = strconv.Itoa(int(jobTemplate.GangCardinality)) + if jobTemplate.GangNodeUniformityLabel != "" { + annotations[serverconfig.GangNodeUniformityLabelAnnotation] = jobTemplate.GangNodeUniformityLabel + } else { + annotations[serverconfig.GangNodeUniformityLabelAnnotation] = "armadaproject.io/clusterName" + } + } + return &armadaevents.SubmitJob{ JobId: jobId, Priority: jobTemplate.QueuePriority, MainObject: &armadaevents.KubernetesMainObject{ ObjectMeta: &armadaevents.ObjectMeta{ - Annotations: jobTemplate.Requirements.Annotations, + Annotations: annotations, }, Object: &armadaevents.KubernetesMainObject_PodSpec{ PodSpec: &armadaevents.PodSpecWithAvoidList{ @@ -480,11 +541,11 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { var eventSequences []*armadaevents.EventSequence txn := s.jobDb.WriteTxn() defer txn.Abort() - for pool, nodeDb := range s.nodeDbByPool { + for pool, nodeDb := range s.accounting.nodeDbByPool { if err := nodeDb.Reset(); err != nil { return err } - totalResources := s.totalResourcesByPool[pool] + totalResources := s.accounting.totalResourcesByPool[pool] fairnessCostProvider, err := fairness.NewDominantResourceFairness( totalResources, s.schedulingConfig, @@ -501,7 +562,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { sctx.Started = s.time for _, queue := range s.WorkloadSpec.Queues { - demand, hasDemand := s.demandByQueue[queue.Name] + demand, hasDemand := s.accounting.demandByQueue[queue.Name] if !hasDemand { // To ensure fair share is computed only from active queues, i.e., queues with jobs queued or running. continue @@ -509,7 +570,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { err := sctx.AddQueueSchedulingContext( queue.Name, queue.Weight, - s.allocationByPoolAndQueueAndPriorityClass[pool][queue.Name], + s.accounting.allocationByPoolAndQueueAndPriorityClass[pool][queue.Name], demand, demand, s.limiter, @@ -527,10 +588,9 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { s.schedulingConfig.ProtectedFractionOfFairShare, txn, nodeDb, - // TODO: Necessary to support partial eviction. - nil, - nil, - nil, + maps.Clone(s.accounting.nodeIdByJobId), + maps.Clone(s.accounting.jobIdsByGangId), + maps.Clone(s.accounting.gangIdByJobId), ) schedulerCtx := ctx @@ -572,6 +632,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { return lessJob(a.Job, b.Job) }) for i, job := range preemptedJobs { + delete(s.accounting.nodeIdByJobId, job.Id()) if run := job.LatestRun(); run != nil { job = job.WithUpdatedRun(run.WithFailed(true)) } else { @@ -588,6 +649,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { if node, err := nodeDb.GetNode(nodeId); err != nil { return err } else { + s.accounting.nodeIdByJobId[job.Id()] = nodeId priority, ok := nodeDb.GetScheduledAtPriority(job.Id()) if !ok { return errors.Errorf("job %s not mapped to a priority", job.Id()) @@ -603,7 +665,7 @@ func (s *Simulator) handleScheduleEvent(ctx *armadacontext.Context) error { } // Update allocation. - s.allocationByPoolAndQueueAndPriorityClass[pool] = sctx.AllocatedByQueueAndPriority() + s.accounting.allocationByPoolAndQueueAndPriorityClass[pool] = sctx.AllocatedByQueueAndPriority() // Generate eventSequences. eventSequences, err = scheduler.AppendEventSequencesFromPreemptedJobs(eventSequences, preemptedJobs, s.time) @@ -724,10 +786,23 @@ func (s *Simulator) handleSubmitJob(txn *jobdb.Txn, e *armadaevents.SubmitJob, t false, poolNames, ) + if err != nil { + return nil, false, err + } s.addJobToDemand(job) + gangInfo, err := schedulercontext.GangInfoFromLegacySchedulerJob(job) if err != nil { return nil, false, err } + if gangInfo.Cardinality > 1 { + gangIds := s.accounting.jobIdsByGangId[gangInfo.Id] + if gangIds == nil { + gangIds = make(map[string]bool, gangInfo.Cardinality) + s.accounting.jobIdsByGangId[gangInfo.Id] = gangIds + } + gangIds[job.Id()] = true + s.accounting.gangIdByJobId[job.Id()] = gangInfo.Id + } if err := txn.Upsert([]*jobdb.Job{job}); err != nil { return nil, false, err } @@ -787,14 +862,28 @@ func (s *Simulator) handleJobSucceeded(txn *jobdb.Txn, e *armadaevents.JobSuccee // Job already terminated; nothing more to do. return nil, false, nil } + + delete(s.accounting.nodeIdByJobId, job.Id()) + delete(s.accounting.gangIdByJobId, job.Id()) + gangInfo, err := schedulercontext.GangInfoFromLegacySchedulerJob(job) + if err != nil { + return nil, false, err + } + if gangInfo.Cardinality > 1 { + gangIds := s.accounting.jobIdsByGangId[gangInfo.Id] + if gangIds != nil { + delete(s.accounting.jobIdsByGangId[gangInfo.Id], jobId) + } + s.accounting.gangIdByJobId[job.Id()] = gangInfo.Id + } if err := txn.BatchDelete([]string{jobId}); err != nil { return nil, false, err } // Subtract the allocation of this job from the queue allocation. run := job.LatestRun() - pool := s.poolByNodeId[run.NodeId()] - s.allocationByPoolAndQueueAndPriorityClass[pool][job.Queue()].SubV1ResourceList( + pool := s.accounting.poolByNodeId[run.NodeId()] + s.accounting.allocationByPoolAndQueueAndPriorityClass[pool][job.Queue()].SubV1ResourceList( job.PriorityClassName(), job.ResourceRequirements().Requests, ) @@ -822,15 +911,20 @@ func (s *Simulator) handleJobSucceeded(txn *jobdb.Txn, e *armadaevents.JobSuccee Queue: dependentJobTemplate.Queue, JobSetName: dependentJobTemplate.JobSet, } + gangId := "" + gangCardinality := int(dependentJobTemplate.GangCardinality) for k := 0; k < int(dependentJobTemplate.Number); k++ { jobId := util.NewULID() + if gangCardinality != 0 && k%gangCardinality == 0 { + gangId = fmt.Sprintf("%s-0", util.NewULID()) + } eventSequence.Events = append( eventSequence.Events, &armadaevents.EventSequence_Event{ // EarliestSubmitTimeFromDependencyCompletion must be positive Created: protoutil.ToTimestamp(maxTime(time.Time{}.Add(dependentJobTemplate.EarliestSubmitTime), s.time.Add(dependentJobTemplate.EarliestSubmitTimeFromDependencyCompletion))), Event: &armadaevents.EventSequence_Event_SubmitJob{ - SubmitJob: submitJobFromJobTemplate(jobId, dependentJobTemplate), + SubmitJob: submitJobFromJobTemplate(jobId, dependentJobTemplate, gangId), }, }, ) @@ -859,7 +953,7 @@ func (s *Simulator) unbindRunningJob(job *jobdb.Job) error { if run.NodeId() == "" { return errors.Errorf("empty nodeId for run %s of job %s", run.Id(), job.Id()) } - nodeDb := s.nodeDbByPool[run.Pool()] + nodeDb := s.accounting.nodeDbByPool[run.Pool()] node, err := nodeDb.GetNode(run.NodeId()) if err != nil { return err @@ -884,6 +978,19 @@ func (s *Simulator) handleJobRunPreempted(txn *jobdb.Txn, e *armadaevents.JobRun jobTemplate := s.jobTemplateByJobId[job.Id()] retryJobId := util.NewULID() resubmitTime := s.time.Add(s.generateRandomShiftedExponentialDuration(s.ClusterSpec.WorkflowManagerDelayDistribution)) + gangInfo, err := schedulercontext.GangInfoFromLegacySchedulerJob(job) + if err != nil { + return nil, false, err + } + gangId := "" + if gangInfo.Cardinality > 1 { + toks := strings.Split(gangInfo.Id, "-") + attempt, err := strconv.Atoi(toks[1]) + if err != nil { + return nil, false, err + } + gangId = fmt.Sprintf("%s-%d", gangInfo.Id, attempt+1) + } s.pushEventSequence( &armadaevents.EventSequence{ Queue: job.Queue(), @@ -892,7 +999,7 @@ func (s *Simulator) handleJobRunPreempted(txn *jobdb.Txn, e *armadaevents.JobRun { Created: protoutil.ToTimestamp(resubmitTime), Event: &armadaevents.EventSequence_Event_SubmitJob{ - SubmitJob: submitJobFromJobTemplate(retryJobId, jobTemplate), + SubmitJob: submitJobFromJobTemplate(retryJobId, jobTemplate, gangId), }, }, }, @@ -914,16 +1021,16 @@ func maxTime(a, b time.Time) time.Time { } func (s *Simulator) addJobToDemand(job *jobdb.Job) { - r, ok := s.demandByQueue[job.Queue()] + r, ok := s.accounting.demandByQueue[job.Queue()] if !ok { r = schedulerobjects.NewResourceList(len(job.PodRequirements().ResourceRequirements.Requests)) - s.demandByQueue[job.Queue()] = r + s.accounting.demandByQueue[job.Queue()] = r } r.AddV1ResourceList(job.PodRequirements().ResourceRequirements.Requests) } func (s *Simulator) removeJobFromDemand(job *jobdb.Job) { - r, ok := s.demandByQueue[job.Queue()] + r, ok := s.accounting.demandByQueue[job.Queue()] if ok { r.SubV1ResourceList(job.PodRequirements().ResourceRequirements.Requests) } diff --git a/internal/scheduler/simulator/simulator.pb.go b/internal/scheduler/simulator/simulator.pb.go index 684588df401..472ca2cf906 100644 --- a/internal/scheduler/simulator/simulator.pb.go +++ b/internal/scheduler/simulator/simulator.pb.go @@ -380,6 +380,10 @@ type JobTemplate struct { // see https://bora.uib.no/bora-xmlui/bitstream/handle/11250/3014726/drthesis_2022_severinson.pdf?sequence=2 // for a discussion on the topic. RuntimeDistribution ShiftedExponential `protobuf:"bytes,12,opt,name=runtime_distribution,json=runtimeDistribution,proto3" json:"runtimeDistribution"` + // If set, jobs will be assigned to gangs with the given size. In this case `number` must be exactly divisible by the gang size + GangCardinality uint32 `protobuf:"varint,13,opt,name=gang_cardinality,json=gangCardinality,proto3" json:"gangCardinality,omitempty"` + // Node Uniformity label when scheduling gangs. Only applies if gang_cardinality is non-zero. If unset it defaults to armadaproject.io/clusterName + GangNodeUniformityLabel string `protobuf:"bytes,14,opt,name=gang_node_uniformity_label,json=gangNodeUniformityLabel,proto3" json:"gangNodeUniformityLabel,omitempty"` } func (m *JobTemplate) Reset() { *m = JobTemplate{} } @@ -499,6 +503,20 @@ func (m *JobTemplate) GetRuntimeDistribution() ShiftedExponential { return ShiftedExponential{} } +func (m *JobTemplate) GetGangCardinality() uint32 { + if m != nil { + return m.GangCardinality + } + return 0 +} + +func (m *JobTemplate) GetGangNodeUniformityLabel() string { + if m != nil { + return m.GangNodeUniformityLabel + } + return "" +} + type ShiftedExponential struct { Minimum time.Duration `protobuf:"bytes,1,opt,name=minimum,proto3,stdduration" json:"minimum"` TailMean time.Duration `protobuf:"bytes,2,opt,name=tail_mean,json=tailMean,proto3,stdduration" json:"tailMean"` @@ -567,80 +585,84 @@ func init() { } var fileDescriptor_63baccdfe9127510 = []byte{ - // 1157 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xc1, 0x6e, 0xdb, 0x46, - 0x13, 0x36, 0xad, 0x44, 0x8e, 0x56, 0xb2, 0x93, 0xac, 0x0d, 0x47, 0x51, 0xf0, 0x8b, 0xfa, 0x15, - 0x20, 0x50, 0x0b, 0x87, 0x42, 0x52, 0xa0, 0x48, 0x83, 0x22, 0x40, 0x69, 0x27, 0x28, 0x02, 0x27, - 0x4d, 0x24, 0xa3, 0x01, 0x9a, 0x03, 0xb1, 0x22, 0xc7, 0xf2, 0x5a, 0x24, 0x57, 0x59, 0x2e, 0xed, - 0xea, 0xd0, 0x77, 0xe8, 0xa5, 0x41, 0x1f, 0xa2, 0x97, 0x5e, 0xf2, 0x0c, 0x41, 0x4f, 0x39, 0xf6, - 0xc4, 0x16, 0xf6, 0x8d, 0x4f, 0x51, 0x70, 0xb9, 0x94, 0x57, 0x91, 0xe3, 0xd8, 0x27, 0x69, 0xbe, - 0x99, 0x6f, 0xf6, 0xdb, 0xd9, 0x99, 0xe5, 0xa2, 0x0d, 0x1a, 0x0a, 0xe0, 0x21, 0xf1, 0xbb, 0x91, - 0xbb, 0x07, 0x5e, 0xec, 0x03, 0xef, 0x46, 0x34, 0x88, 0x7d, 0x22, 0x98, 0xf6, 0xcf, 0x1a, 0x73, - 0x26, 0x18, 0xae, 0x4c, 0x81, 0x46, 0x73, 0xc8, 0xd8, 0xd0, 0x87, 0xae, 0x74, 0x0c, 0xe2, 0xdd, - 0xae, 0x17, 0x73, 0x22, 0x28, 0x0b, 0xf3, 0xd0, 0x46, 0x7b, 0xf4, 0x20, 0xb2, 0x28, 0xeb, 0x92, - 0x31, 0xed, 0xba, 0x8c, 0x43, 0xf7, 0xe0, 0x5e, 0x77, 0x08, 0x21, 0x70, 0x22, 0xc0, 0x53, 0x31, - 0x77, 0x87, 0x54, 0xec, 0xc5, 0x03, 0xcb, 0x65, 0x41, 0x77, 0xc8, 0x86, 0xec, 0x24, 0x59, 0x66, - 0x49, 0x43, 0xfe, 0x53, 0xe1, 0x0f, 0x4f, 0xd3, 0x5a, 0xfc, 0x63, 0x83, 0x7d, 0x70, 0x45, 0x34, - 0x07, 0xe4, 0xdc, 0xf6, 0x6f, 0x25, 0x54, 0xdd, 0xf4, 0xe3, 0x48, 0x00, 0xef, 0x8f, 0xc1, 0xc5, - 0x77, 0xd0, 0xa5, 0x90, 0x04, 0x50, 0x37, 0x5a, 0x46, 0xa7, 0x62, 0xe3, 0x34, 0x31, 0x57, 0x32, - 0x7b, 0x83, 0x05, 0x54, 0x40, 0x30, 0x16, 0x93, 0x9e, 0xf4, 0xe3, 0x27, 0xe8, 0x8a, 0x9b, 0xd3, - 0xa2, 0xfa, 0x62, 0xab, 0xd4, 0xa9, 0xde, 0xc7, 0xd6, 0x49, 0x55, 0x54, 0x46, 0x7b, 0x3d, 0x4d, - 0x4c, 0x5c, 0xc4, 0x69, 0x39, 0xa6, 0x5c, 0xfc, 0xd6, 0x40, 0xb7, 0x0f, 0x19, 0x1f, 0xed, 0xfa, - 0xec, 0xd0, 0x09, 0x48, 0x48, 0x86, 0xc0, 0x1d, 0x0f, 0x7c, 0x32, 0x71, 0x3c, 0x1a, 0x09, 0x4e, - 0x07, 0x71, 0x56, 0xbc, 0x7a, 0xa9, 0x65, 0x74, 0xaa, 0xf7, 0xff, 0xa7, 0xad, 0xd1, 0xdf, 0xa3, - 0xbb, 0x02, 0xbc, 0xc7, 0x3f, 0x8f, 0x59, 0x08, 0xa1, 0xa0, 0xc4, 0xb7, 0x3b, 0xef, 0x13, 0x73, - 0x21, 0x4d, 0xcc, 0x56, 0x91, 0xf1, 0x59, 0x9e, 0x70, 0x2b, 0xcb, 0xb7, 0xa5, 0xa5, 0xeb, 0x7d, - 0x36, 0x02, 0xff, 0x82, 0x1a, 0x63, 0x08, 0x3d, 0x1a, 0x0e, 0x4f, 0x93, 0x73, 0xe9, 0x3c, 0x72, - 0x5a, 0x4a, 0x4e, 0x5d, 0x25, 0x9a, 0x97, 0xf1, 0x49, 0x4f, 0xfb, 0x9d, 0x81, 0x96, 0x54, 0x15, - 0xcf, 0x7d, 0x26, 0x77, 0xd0, 0xa5, 0x31, 0x63, 0xbe, 0xac, 0x95, 0x8a, 0xcb, 0x6c, 0x3d, 0x2e, - 0xb3, 0xf1, 0x6b, 0xb4, 0x12, 0x32, 0x0f, 0x9c, 0x0c, 0xf4, 0x89, 0x80, 0xe2, 0x04, 0x6f, 0x68, - 0xdb, 0x79, 0xce, 0x3c, 0xd8, 0x51, 0x7e, 0xfb, 0x56, 0x9a, 0x98, 0x37, 0x42, 0x0d, 0xd1, 0xcf, - 0x72, 0x79, 0xc6, 0xd1, 0xfe, 0xd3, 0x40, 0xb5, 0x57, 0x8c, 0x8f, 0x7c, 0x46, 0xbc, 0x0b, 0x75, - 0xd4, 0x37, 0xa8, 0xca, 0x49, 0xe8, 0xb1, 0xc0, 0x89, 0x00, 0xbc, 0xfa, 0x62, 0xcb, 0xe8, 0x94, - 0xec, 0x7a, 0x9a, 0x98, 0x6b, 0x39, 0xdc, 0x07, 0xf0, 0x34, 0x12, 0x3a, 0x41, 0xf1, 0x23, 0x54, - 0x7e, 0x13, 0x43, 0x0c, 0x51, 0xbd, 0x24, 0x37, 0x72, 0x4d, 0xdb, 0xc8, 0xcb, 0xcc, 0x61, 0xaf, - 0xa5, 0x89, 0x79, 0x2d, 0x8f, 0xd1, 0x72, 0x28, 0x56, 0xfb, 0x6d, 0x09, 0xd5, 0xf4, 0x0d, 0xe3, - 0x0d, 0x54, 0x0e, 0xe3, 0x60, 0x00, 0x5c, 0xaa, 0x2e, 0xe5, 0xf4, 0x1c, 0xd1, 0xe9, 0x39, 0x82, - 0xbf, 0x43, 0x65, 0x41, 0x68, 0x28, 0x8a, 0x3a, 0xde, 0xb4, 0xf2, 0x19, 0xb7, 0xc8, 0x98, 0x5a, - 0xd9, 0x8c, 0x5b, 0x07, 0xf7, 0xac, 0x9d, 0x2c, 0xc2, 0x5e, 0x51, 0x2d, 0xa1, 0x08, 0x3d, 0xf5, - 0x8b, 0x5f, 0xa2, 0xb2, 0x4f, 0x06, 0xe0, 0x17, 0x3b, 0xb8, 0xfd, 0x89, 0xa3, 0xb0, 0xb6, 0x65, - 0xd4, 0xe3, 0x50, 0xf0, 0x49, 0xae, 0x2a, 0xa7, 0xe9, 0xaa, 0x72, 0x04, 0x3b, 0xe8, 0xaa, 0x60, - 0x82, 0xf8, 0x0e, 0x87, 0x88, 0xc5, 0xdc, 0x85, 0x48, 0x75, 0x6d, 0xd3, 0x9a, 0xbb, 0x0b, 0x7a, - 0x2a, 0x64, 0x9b, 0x46, 0xc2, 0x5e, 0x57, 0x1a, 0x57, 0x24, 0xbd, 0x70, 0x45, 0xbd, 0x8f, 0xec, - 0x06, 0x41, 0x55, 0x4d, 0x0d, 0xbe, 0x8d, 0x4a, 0x23, 0x98, 0xa8, 0x63, 0xbe, 0x9e, 0x26, 0xe6, - 0xf2, 0x08, 0x26, 0x9a, 0xae, 0xcc, 0x8b, 0xbf, 0x40, 0x97, 0x0f, 0x88, 0x1f, 0x83, 0x3c, 0xde, - 0x8a, 0xbd, 0x9a, 0x26, 0xe6, 0x55, 0x09, 0x68, 0x81, 0x79, 0xc4, 0xc3, 0xc5, 0x07, 0x46, 0x36, - 0x05, 0x97, 0xe5, 0x01, 0x9e, 0xbb, 0x8b, 0x36, 0x50, 0xf9, 0x10, 0xe8, 0x70, 0x4f, 0xc8, 0x15, - 0x8c, 0xbc, 0x46, 0x39, 0xa2, 0xd7, 0x28, 0x47, 0xf0, 0x2b, 0xb4, 0xbc, 0xcf, 0x06, 0xda, 0x20, - 0xe4, 0xd5, 0x5f, 0xd7, 0xaa, 0xff, 0x94, 0x0d, 0xa6, 0x73, 0xd0, 0x48, 0x13, 0x73, 0x7d, 0xff, - 0x04, 0xd0, 0xcb, 0x5e, 0xd3, 0xf1, 0xf6, 0x5f, 0x4b, 0xa8, 0xaa, 0x31, 0x2f, 0xd8, 0x50, 0x4f, - 0x91, 0xf2, 0xf5, 0x63, 0xd7, 0x85, 0x28, 0xda, 0x8d, 0x7d, 0x35, 0x0f, 0xcd, 0x34, 0x31, 0x1b, - 0x1f, 0xfb, 0xb4, 0x0c, 0x73, 0xbc, 0xac, 0xe2, 0xb2, 0xcb, 0xd5, 0xad, 0x20, 0x2b, 0x2e, 0x01, - 0xbd, 0xe2, 0x12, 0xc0, 0x2d, 0xb4, 0x48, 0x3d, 0xd9, 0x24, 0x15, 0xfb, 0x5a, 0x9a, 0x98, 0x35, - 0xaa, 0x0f, 0xdc, 0x22, 0xf5, 0xf0, 0x5d, 0xb4, 0x94, 0xd5, 0x2b, 0x02, 0x51, 0xbf, 0x2c, 0xc3, - 0xe4, 0x3e, 0xf6, 0xd9, 0xa0, 0x0f, 0x33, 0xe5, 0xcd, 0x11, 0x6c, 0xa3, 0x15, 0x99, 0xd9, 0x19, - 0x73, 0xca, 0x38, 0x15, 0x93, 0x7a, 0xb9, 0x65, 0x74, 0x96, 0xf3, 0xfb, 0x44, 0x7a, 0x5e, 0x28, - 0x87, 0x7e, 0x9f, 0xcc, 0x38, 0xf0, 0x0f, 0x68, 0xb5, 0x60, 0x3b, 0xae, 0x4f, 0xa2, 0xc8, 0x91, - 0x7d, 0xb0, 0x24, 0x97, 0x37, 0xd3, 0xc4, 0xbc, 0x55, 0xb8, 0x37, 0x33, 0xef, 0xf3, 0xd9, 0xa6, - 0xb8, 0x3e, 0xe7, 0xc4, 0xaf, 0x51, 0x8d, 0xc3, 0x9b, 0x98, 0x72, 0x08, 0x20, 0x9b, 0xd9, 0x2b, - 0x72, 0x28, 0xfe, 0x3f, 0x3f, 0x14, 0x2f, 0x98, 0xd7, 0xd3, 0x02, 0xed, 0x35, 0x35, 0x17, 0x33, - 0xf4, 0xde, 0x8c, 0x85, 0x1f, 0xa1, 0x9a, 0x07, 0xd9, 0xa5, 0x0e, 0xa1, 0x4b, 0x21, 0xaa, 0x57, - 0x5a, 0xa5, 0x4e, 0x25, 0xef, 0x1b, 0x1d, 0xd7, 0xfb, 0x46, 0xc7, 0xf1, 0x08, 0xad, 0x01, 0xe1, - 0x3e, 0x85, 0x48, 0x38, 0x51, 0x3c, 0x08, 0xa8, 0x70, 0x04, 0x0d, 0xa0, 0x8e, 0xa4, 0xc8, 0x9b, - 0x56, 0xfe, 0xb8, 0xb0, 0x8a, 0xf7, 0x80, 0xb5, 0xa5, 0x1e, 0x17, 0x76, 0x53, 0x89, 0xc3, 0x05, - 0xbd, 0x2f, 0xd9, 0x3b, 0x34, 0x80, 0xdf, 0xff, 0x31, 0x8d, 0xde, 0x29, 0x38, 0x7e, 0x67, 0xa0, - 0xee, 0x69, 0xab, 0x39, 0xbb, 0x9c, 0x05, 0xce, 0x54, 0xd7, 0xc4, 0x71, 0x59, 0x30, 0xf6, 0x41, - 0x7e, 0xf8, 0xaa, 0x9f, 0x13, 0xf2, 0xb5, 0x12, 0xf2, 0xe5, 0xfc, 0x82, 0x4f, 0x38, 0x0b, 0xb6, - 0xa6, 0x59, 0x37, 0xa7, 0x49, 0xa5, 0xc0, 0x0b, 0xc4, 0xe3, 0x00, 0xad, 0xf1, 0x38, 0x94, 0x52, - 0x67, 0xbe, 0xca, 0xb5, 0xf3, 0x7c, 0x95, 0x6f, 0x29, 0x81, 0xab, 0x2a, 0xc5, 0xcc, 0x07, 0xf9, - 0x34, 0xb0, 0xfd, 0x87, 0x81, 0xf0, 0x7c, 0x22, 0xfc, 0x3d, 0x5a, 0x0a, 0x68, 0x48, 0x83, 0x38, - 0x90, 0x43, 0x7d, 0x66, 0x55, 0x56, 0xd5, 0xa2, 0x05, 0x43, 0x6e, 0xb9, 0x30, 0xf0, 0x36, 0xaa, - 0x08, 0x42, 0x7d, 0x27, 0x00, 0x12, 0xca, 0x41, 0x3f, 0x33, 0x57, 0xd1, 0x87, 0x57, 0x32, 0xce, - 0x33, 0x20, 0x79, 0xfd, 0xa6, 0x96, 0xfd, 0xe3, 0xfb, 0xa3, 0xa6, 0xf1, 0xe1, 0xa8, 0x69, 0xfc, - 0x7b, 0xd4, 0x34, 0x7e, 0x3d, 0x6e, 0x2e, 0x7c, 0x38, 0x6e, 0x2e, 0xfc, 0x7d, 0xdc, 0x5c, 0xf8, - 0xe9, 0x5b, 0xed, 0x5d, 0x49, 0x78, 0x40, 0x3c, 0x32, 0xe6, 0x2c, 0x6b, 0x76, 0x65, 0x75, 0xcf, - 0x7a, 0xf4, 0x0e, 0xca, 0x52, 0xca, 0x57, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x99, 0x04, 0xb7, - 0x97, 0x1b, 0x0b, 0x00, 0x00, + // 1228 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x6e, 0x13, 0x47, + 0x18, 0xcf, 0xc6, 0xe0, 0xc4, 0x63, 0x27, 0x84, 0x49, 0x14, 0x8c, 0x11, 0x5e, 0x63, 0x54, 0xe4, + 0x56, 0x61, 0x2d, 0xa8, 0x54, 0x51, 0x54, 0x21, 0x75, 0x13, 0x10, 0x42, 0x40, 0xc1, 0xa1, 0x45, + 0x2a, 0x87, 0xd5, 0xec, 0xee, 0x17, 0x67, 0xe2, 0xdd, 0x1d, 0x33, 0x3b, 0x0b, 0xf5, 0xa1, 0xef, + 0xd0, 0x4b, 0x51, 0x1f, 0xa2, 0x97, 0x5e, 0x78, 0x06, 0x8e, 0x1c, 0x7b, 0xda, 0x56, 0x70, 0xdb, + 0x27, 0xe8, 0xb1, 0xda, 0xd9, 0x59, 0x67, 0x8c, 0x03, 0x24, 0x27, 0xfb, 0xfb, 0x7d, 0x7f, 0xf6, + 0x37, 0xdf, 0xbf, 0x19, 0xb4, 0x45, 0x23, 0x01, 0x3c, 0x22, 0x41, 0x3f, 0xf6, 0xf6, 0xc1, 0x4f, + 0x02, 0xe0, 0xfd, 0x98, 0x86, 0x49, 0x40, 0x04, 0xd3, 0xfe, 0x59, 0x63, 0xce, 0x04, 0xc3, 0xb5, + 0x29, 0xd0, 0x6a, 0x0f, 0x19, 0x1b, 0x06, 0xd0, 0x97, 0x0a, 0x37, 0xd9, 0xeb, 0xfb, 0x09, 0x27, + 0x82, 0xb2, 0xa8, 0x30, 0x6d, 0x75, 0x47, 0x37, 0x62, 0x8b, 0xb2, 0x3e, 0x19, 0xd3, 0xbe, 0xc7, + 0x38, 0xf4, 0x5f, 0x5c, 0xeb, 0x0f, 0x21, 0x02, 0x4e, 0x04, 0xf8, 0xca, 0xe6, 0xea, 0x90, 0x8a, + 0xfd, 0xc4, 0xb5, 0x3c, 0x16, 0xf6, 0x87, 0x6c, 0xc8, 0x0e, 0x83, 0xe5, 0x92, 0x14, 0xe4, 0x3f, + 0x65, 0x7e, 0xf3, 0x28, 0xae, 0xe5, 0x3f, 0xe6, 0x1e, 0x80, 0x27, 0xe2, 0x39, 0xa0, 0xf0, 0xed, + 0xfe, 0x5e, 0x41, 0xf5, 0xed, 0x20, 0x89, 0x05, 0xf0, 0xdd, 0x31, 0x78, 0xf8, 0x0a, 0x3a, 0x15, + 0x91, 0x10, 0x9a, 0x46, 0xc7, 0xe8, 0xd5, 0x6c, 0x9c, 0xa5, 0xe6, 0x6a, 0x2e, 0x6f, 0xb1, 0x90, + 0x0a, 0x08, 0xc7, 0x62, 0x32, 0x90, 0x7a, 0x7c, 0x07, 0x2d, 0x7b, 0x85, 0x5b, 0xdc, 0x5c, 0xec, + 0x54, 0x7a, 0xf5, 0xeb, 0xd8, 0x3a, 0xcc, 0x8a, 0x8a, 0x68, 0x6f, 0x66, 0xa9, 0x89, 0x4b, 0x3b, + 0x2d, 0xc6, 0xd4, 0x17, 0xbf, 0x32, 0xd0, 0xe5, 0x97, 0x8c, 0x8f, 0xf6, 0x02, 0xf6, 0xd2, 0x09, + 0x49, 0x44, 0x86, 0xc0, 0x1d, 0x1f, 0x02, 0x32, 0x71, 0x7c, 0x1a, 0x0b, 0x4e, 0xdd, 0x24, 0x4f, + 0x5e, 0xb3, 0xd2, 0x31, 0x7a, 0xf5, 0xeb, 0x17, 0xb5, 0x6f, 0xec, 0xee, 0xd3, 0x3d, 0x01, 0xfe, + 0xed, 0x5f, 0xc6, 0x2c, 0x82, 0x48, 0x50, 0x12, 0xd8, 0xbd, 0x37, 0xa9, 0xb9, 0x90, 0xa5, 0x66, + 0xa7, 0x8c, 0xf8, 0xa0, 0x08, 0xb8, 0x93, 0xc7, 0xdb, 0xd1, 0xc2, 0x0d, 0x3e, 0x6b, 0x81, 0x7f, + 0x45, 0xad, 0x31, 0x44, 0x3e, 0x8d, 0x86, 0x47, 0xd1, 0x39, 0x75, 0x1c, 0x3a, 0x1d, 0x45, 0xa7, + 0xa9, 0x02, 0xcd, 0xd3, 0xf8, 0xa8, 0xa6, 0xfb, 0xda, 0x40, 0x4b, 0x2a, 0x8b, 0xc7, 0xae, 0xc9, + 0x15, 0x74, 0x6a, 0xcc, 0x58, 0x20, 0x73, 0xa5, 0xec, 0x72, 0x59, 0xb7, 0xcb, 0x65, 0xfc, 0x0c, + 0xad, 0x46, 0xcc, 0x07, 0x27, 0x07, 0x03, 0x22, 0xa0, 0xac, 0xe0, 0x39, 0xed, 0x38, 0x0f, 0x99, + 0x0f, 0x4f, 0x94, 0xde, 0xbe, 0x90, 0xa5, 0xe6, 0xb9, 0x48, 0x43, 0xf4, 0x5a, 0xae, 0xcc, 0x28, + 0xba, 0x7f, 0x19, 0xa8, 0xf1, 0x94, 0xf1, 0x51, 0xc0, 0x88, 0x7f, 0xa2, 0x8e, 0xfa, 0x16, 0xd5, + 0x39, 0x89, 0x7c, 0x16, 0x3a, 0x31, 0x80, 0xdf, 0x5c, 0xec, 0x18, 0xbd, 0x8a, 0xdd, 0xcc, 0x52, + 0x73, 0xa3, 0x80, 0x77, 0x01, 0x7c, 0xcd, 0x09, 0x1d, 0xa2, 0xf8, 0x16, 0xaa, 0x3e, 0x4f, 0x20, + 0x81, 0xb8, 0x59, 0x91, 0x07, 0x59, 0xd3, 0x0e, 0xf2, 0x38, 0x57, 0xd8, 0x1b, 0x59, 0x6a, 0xae, + 0x15, 0x36, 0x5a, 0x0c, 0xe5, 0xd5, 0x7d, 0x55, 0x41, 0x0d, 0xfd, 0xc0, 0x78, 0x0b, 0x55, 0xa3, + 0x24, 0x74, 0x81, 0x4b, 0xd6, 0x95, 0xc2, 0xbd, 0x40, 0x74, 0xf7, 0x02, 0xc1, 0xdf, 0xa3, 0xaa, + 0x20, 0x34, 0x12, 0x65, 0x1e, 0xcf, 0x5b, 0xc5, 0x8c, 0x5b, 0x64, 0x4c, 0xad, 0x7c, 0xc6, 0xad, + 0x17, 0xd7, 0xac, 0x27, 0xb9, 0x85, 0xbd, 0xaa, 0x5a, 0x42, 0x39, 0x0c, 0xd4, 0x2f, 0x7e, 0x8c, + 0xaa, 0x01, 0x71, 0x21, 0x28, 0x4f, 0x70, 0xf9, 0x23, 0xa5, 0xb0, 0xee, 0x4b, 0xab, 0xdb, 0x91, + 0xe0, 0x93, 0x82, 0x55, 0xe1, 0xa6, 0xb3, 0x2a, 0x10, 0xec, 0xa0, 0x33, 0x82, 0x09, 0x12, 0x38, + 0x1c, 0x62, 0x96, 0x70, 0x0f, 0x62, 0xd5, 0xb5, 0x6d, 0x6b, 0x6e, 0x17, 0x0c, 0x94, 0xc9, 0x7d, + 0x1a, 0x0b, 0x7b, 0x53, 0x71, 0x5c, 0x95, 0xee, 0xa5, 0x2a, 0x1e, 0x7c, 0x20, 0xb7, 0x08, 0xaa, + 0x6b, 0x6c, 0xf0, 0x65, 0x54, 0x19, 0xc1, 0x44, 0x95, 0xf9, 0x6c, 0x96, 0x9a, 0x2b, 0x23, 0x98, + 0x68, 0xbc, 0x72, 0x2d, 0xfe, 0x12, 0x9d, 0x7e, 0x41, 0x82, 0x04, 0x64, 0x79, 0x6b, 0xf6, 0x7a, + 0x96, 0x9a, 0x67, 0x24, 0xa0, 0x19, 0x16, 0x16, 0x37, 0x17, 0x6f, 0x18, 0xf9, 0x14, 0x9c, 0x96, + 0x05, 0x3c, 0x76, 0x17, 0x6d, 0xa1, 0xea, 0x4b, 0xa0, 0xc3, 0x7d, 0x21, 0xbf, 0x60, 0x14, 0x39, + 0x2a, 0x10, 0x3d, 0x47, 0x05, 0x82, 0x9f, 0xa2, 0x95, 0x03, 0xe6, 0x6a, 0x83, 0x50, 0x64, 0x7f, + 0x53, 0xcb, 0xfe, 0x3d, 0xe6, 0x4e, 0xe7, 0xa0, 0x95, 0xa5, 0xe6, 0xe6, 0xc1, 0x21, 0xa0, 0xa7, + 0xbd, 0xa1, 0xe3, 0xdd, 0xff, 0x96, 0x51, 0x5d, 0xf3, 0x3c, 0x61, 0x43, 0xdd, 0x43, 0x4a, 0xb7, + 0x9b, 0x78, 0x1e, 0xc4, 0xf1, 0x5e, 0x12, 0xa8, 0x79, 0x68, 0x67, 0xa9, 0xd9, 0xfa, 0x50, 0xa7, + 0x45, 0x98, 0xf3, 0xcb, 0x33, 0x2e, 0xbb, 0x5c, 0x6d, 0x05, 0x99, 0x71, 0x09, 0xe8, 0x19, 0x97, + 0x00, 0xee, 0xa0, 0x45, 0xea, 0xcb, 0x26, 0xa9, 0xd9, 0x6b, 0x59, 0x6a, 0x36, 0xa8, 0x3e, 0x70, + 0x8b, 0xd4, 0xc7, 0x57, 0xd1, 0x52, 0x9e, 0xaf, 0x18, 0x44, 0xf3, 0xb4, 0x34, 0x93, 0xe7, 0x38, + 0x60, 0xee, 0x2e, 0xcc, 0xa4, 0xb7, 0x40, 0xb0, 0x8d, 0x56, 0x65, 0x64, 0x67, 0xcc, 0x29, 0xe3, + 0x54, 0x4c, 0x9a, 0xd5, 0x8e, 0xd1, 0x5b, 0x29, 0xf6, 0x89, 0xd4, 0x3c, 0x52, 0x0a, 0x7d, 0x9f, + 0xcc, 0x28, 0xf0, 0x0f, 0x68, 0xbd, 0xf4, 0x76, 0xbc, 0x80, 0xc4, 0xb1, 0x23, 0xfb, 0x60, 0x49, + 0x7e, 0xde, 0xcc, 0x52, 0xf3, 0x42, 0xa9, 0xde, 0xce, 0xb5, 0x0f, 0x67, 0x9b, 0xe2, 0xec, 0x9c, + 0x12, 0x3f, 0x43, 0x0d, 0x0e, 0xcf, 0x13, 0xca, 0x21, 0x84, 0x7c, 0x66, 0x97, 0xe5, 0x50, 0x5c, + 0x9a, 0x1f, 0x8a, 0x47, 0xcc, 0x1f, 0x68, 0x86, 0xf6, 0x86, 0x9a, 0x8b, 0x19, 0xf7, 0xc1, 0x8c, + 0x84, 0x6f, 0xa1, 0x86, 0x0f, 0xf9, 0x52, 0x87, 0xc8, 0xa3, 0x10, 0x37, 0x6b, 0x9d, 0x4a, 0xaf, + 0x56, 0xf4, 0x8d, 0x8e, 0xeb, 0x7d, 0xa3, 0xe3, 0x78, 0x84, 0x36, 0x80, 0xf0, 0x80, 0x42, 0x2c, + 0x9c, 0x38, 0x71, 0x43, 0x2a, 0x1c, 0x41, 0x43, 0x68, 0x22, 0x49, 0xf2, 0xbc, 0x55, 0x3c, 0x2e, + 0xac, 0xf2, 0x3d, 0x60, 0xed, 0xa8, 0xc7, 0x85, 0xdd, 0x56, 0xe4, 0x70, 0xe9, 0xbe, 0x2b, 0xbd, + 0x9f, 0xd0, 0x10, 0xfe, 0xf8, 0xc7, 0x34, 0x06, 0x47, 0xe0, 0xf8, 0xb5, 0x81, 0xfa, 0x47, 0x7d, + 0xcd, 0xd9, 0xe3, 0x2c, 0x74, 0xa6, 0xbc, 0x26, 0x8e, 0xc7, 0xc2, 0x71, 0x00, 0xf2, 0xe2, 0xab, + 0x7f, 0x8e, 0xc8, 0x37, 0x8a, 0xc8, 0x57, 0xf3, 0x1f, 0xbc, 0xc3, 0x59, 0xb8, 0x33, 0x8d, 0xba, + 0x3d, 0x0d, 0x2a, 0x09, 0x9e, 0xc0, 0x1e, 0x87, 0x68, 0x83, 0x27, 0x91, 0xa4, 0x3a, 0x73, 0x2b, + 0x37, 0x8e, 0x73, 0x2b, 0x5f, 0x50, 0x04, 0xd7, 0x55, 0x88, 0x99, 0x0b, 0xf9, 0x28, 0x10, 0xdf, + 0x45, 0x6b, 0x43, 0x12, 0x0d, 0x1d, 0x8f, 0x70, 0x9f, 0x46, 0x24, 0xc8, 0x1b, 0x79, 0x45, 0x36, + 0xf2, 0xc5, 0x2c, 0x35, 0xcf, 0xe7, 0xba, 0xed, 0x43, 0x95, 0x56, 0xdb, 0x33, 0x1f, 0xa8, 0xb0, + 0x8b, 0x5a, 0x32, 0x92, 0xbc, 0x7e, 0x93, 0x88, 0xee, 0x31, 0x1e, 0xe6, 0x8d, 0x2d, 0x57, 0x76, + 0x73, 0x55, 0xf6, 0xf4, 0x17, 0x59, 0x6a, 0x5e, 0xca, 0xad, 0xf2, 0xbd, 0xff, 0xe3, 0xd4, 0x46, + 0xae, 0x5a, 0x2d, 0xf6, 0xb9, 0x8f, 0x98, 0x74, 0xff, 0x34, 0x10, 0x9e, 0x3f, 0x36, 0xbe, 0x8b, + 0x96, 0x42, 0x1a, 0xd1, 0x30, 0x09, 0xe5, 0x0a, 0xfa, 0x64, 0x0d, 0xd7, 0x55, 0x8a, 0x4a, 0x0f, + 0x59, 0xa0, 0x52, 0xc0, 0xf7, 0x51, 0x4d, 0x10, 0x1a, 0x38, 0x21, 0x90, 0x48, 0xae, 0xa5, 0x4f, + 0xc6, 0x2a, 0xa7, 0x66, 0x39, 0xf7, 0x79, 0x00, 0xa4, 0xa8, 0xf6, 0x54, 0xb2, 0x7f, 0x7a, 0xf3, + 0xae, 0x6d, 0xbc, 0x7d, 0xd7, 0x36, 0xfe, 0x7d, 0xd7, 0x36, 0x7e, 0x7b, 0xdf, 0x5e, 0x78, 0xfb, + 0xbe, 0xbd, 0xf0, 0xf7, 0xfb, 0xf6, 0xc2, 0xcf, 0xdf, 0x69, 0xaf, 0x60, 0xc2, 0x43, 0xe2, 0x93, + 0x31, 0x67, 0xf9, 0x68, 0x2a, 0xa9, 0xff, 0xa9, 0x27, 0xba, 0x5b, 0x95, 0x54, 0xbe, 0xfe, 0x3f, + 0x00, 0x00, 0xff, 0xff, 0xb2, 0x34, 0xbe, 0x7a, 0xc9, 0x0b, 0x00, 0x00, } func (m *ClusterSpec) Marshal() (dAtA []byte, err error) { @@ -948,6 +970,18 @@ func (m *JobTemplate) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.GangNodeUniformityLabel) > 0 { + i -= len(m.GangNodeUniformityLabel) + copy(dAtA[i:], m.GangNodeUniformityLabel) + i = encodeVarintSimulator(dAtA, i, uint64(len(m.GangNodeUniformityLabel))) + i-- + dAtA[i] = 0x72 + } + if m.GangCardinality != 0 { + i = encodeVarintSimulator(dAtA, i, uint64(m.GangCardinality)) + i-- + dAtA[i] = 0x68 + } { size, err := m.RuntimeDistribution.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1252,6 +1286,13 @@ func (m *JobTemplate) Size() (n int) { n += 1 + l + sovSimulator(uint64(l)) l = m.RuntimeDistribution.Size() n += 1 + l + sovSimulator(uint64(l)) + if m.GangCardinality != 0 { + n += 1 + sovSimulator(uint64(m.GangCardinality)) + } + l = len(m.GangNodeUniformityLabel) + if l > 0 { + n += 1 + l + sovSimulator(uint64(l)) + } return n } @@ -2507,6 +2548,57 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GangCardinality", wireType) + } + m.GangCardinality = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GangCardinality |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GangNodeUniformityLabel", 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.GangNodeUniformityLabel = string(dAtA[iNdEx:postIndex]) + 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 4183e348e44..7aea2c46321 100644 --- a/internal/scheduler/simulator/simulator.proto +++ b/internal/scheduler/simulator/simulator.proto @@ -75,6 +75,10 @@ message JobTemplate { // see https://bora.uib.no/bora-xmlui/bitstream/handle/11250/3014726/drthesis_2022_severinson.pdf?sequence=2 // for a discussion on the topic. ShiftedExponential runtime_distribution = 12 [(gogoproto.nullable) = false]; + // If set, jobs will be assigned to gangs with the given size. In this case `number` must be exactly divisible by the gang size + uint32 gang_cardinality = 13; + // Node Uniformity label when scheduling gangs. Only applies if gang_cardinality is non-zero. If unset it defaults to armadaproject.io/clusterName + string gang_node_uniformity_label = 14; } message ShiftedExponential { diff --git a/internal/scheduler/simulator/simulator_test.go b/internal/scheduler/simulator/simulator_test.go index 55f3674bdd3..55116184548 100644 --- a/internal/scheduler/simulator/simulator_test.go +++ b/internal/scheduler/simulator/simulator_test.go @@ -439,6 +439,81 @@ func TestSimulator(t *testing.T) { ), simulatedTimeLimit: 24 * time.Hour, }, + "Gang Job": { + clusterSpec: &ClusterSpec{ + Name: "basic", + Clusters: []*Cluster{ + { + Name: "Cluster1", + Pool: "TestPool", + NodeTemplates: []*NodeTemplate{NodeTemplate32Cpu(8)}, + }, + { + // This cluster should be too small to run gangs + Name: "Cluster2", + Pool: "TestPool", + NodeTemplates: []*NodeTemplate{NodeTemplate32Cpu(1)}, + }, + }, + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ + WithJobTemplatesQueue( + &Queue{Name: "A", Weight: 1}, + GangJobTemplate32Cpu(16, 8, "foo", testfixtures.TestDefaultPriorityClass), + ), + }, + }, + schedulingConfig: testfixtures.TestSchedulingConfig(), + expectedEventSequences: armadaslices.Concatenate( + armadaslices.Repeat(1, SubmitJob(16, "A", "foo")), + armadaslices.Repeat(8, JobRunLeased(1, "A", "foo")), + armadaslices.Repeat(8, JobSucceeded(1, "A", "foo")), + armadaslices.Repeat(8, JobRunLeased(1, "A", "foo")), + armadaslices.Repeat(8, JobSucceeded(1, "A", "foo")), + ), + simulatedTimeLimit: 5 * time.Minute, + }, + "Preempted Gang Job": { + clusterSpec: &ClusterSpec{ + Name: "basic", + Clusters: []*Cluster{ + { + Name: "Cluster1", + Pool: "TestPool", + NodeTemplates: []*NodeTemplate{NodeTemplate32Cpu(8)}, + }, + }, + }, + workloadSpec: &WorkloadSpec{ + Queues: []*Queue{ + WithJobTemplatesQueue( + &Queue{Name: "A", Weight: 1}, + GangJobTemplate32Cpu(8, 8, "foo", testfixtures.PriorityClass2), + ), + WithJobTemplatesQueue( + &Queue{Name: "B", Weight: 1}, + WithMinSubmitTimeJobTemplate( + JobTemplate32Cpu(1, "bar", testfixtures.PriorityClass3), + 30*time.Second, + ), + ), + }, + }, + schedulingConfig: testfixtures.TestSchedulingConfig(), + expectedEventSequences: armadaslices.Concatenate( + armadaslices.Repeat(1, SubmitJob(8, "A", "foo")), + armadaslices.Repeat(8, JobRunLeased(1, "A", "foo")), + armadaslices.Repeat(1, SubmitJob(1, "B", "bar")), + armadaslices.Repeat(8, JobRunPreempted(1, "A", "foo")), + armadaslices.Repeat(1, JobRunLeased(1, "B", "bar")), + armadaslices.Repeat(8, SubmitJob(1, "A", "foo")), + armadaslices.Repeat(1, JobSucceeded(1, "B", "bar")), + armadaslices.Repeat(8, JobRunLeased(1, "A", "foo")), + armadaslices.Repeat(8, JobSucceeded(1, "A", "foo")), + ), + simulatedTimeLimit: 5 * time.Minute, + }, } for name, tc := range tests { t.Run(name, func(t *testing.T) { diff --git a/internal/scheduler/simulator/test_utils.go b/internal/scheduler/simulator/test_utils.go index eb842af7b28..38e7a3d976f 100644 --- a/internal/scheduler/simulator/test_utils.go +++ b/internal/scheduler/simulator/test_utils.go @@ -159,10 +159,16 @@ func JobTemplate32Cpu(n int64, jobSet, priorityClassName string) *JobTemplate { }, }, }, - RuntimeDistribution: ShiftedExponential{Minimum: time.Minute}, + RuntimeDistribution: ShiftedExponential{Minimum: 1 * time.Minute}, } } +func GangJobTemplate32Cpu(numJobs int64, gangCardinality uint32, jobSet, priorityClassName string) *JobTemplate { + jobTemplate := JobTemplate32Cpu(numJobs, jobSet, priorityClassName) + jobTemplate.GangCardinality = gangCardinality + return jobTemplate +} + func JobTemplate1Cpu(n int64, jobSet, priorityClassName string) *JobTemplate { return &JobTemplate{ Number: n, From aa07e003d2717292922a68e00128be59422913aa Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Fri, 25 Oct 2024 10:54:10 +0100 Subject: [PATCH 4/9] update a bunch of libraries (#4017) Signed-off-by: Chris Martin Co-authored-by: Dejan Zele Pejchev --- go.mod | 66 +++++++++++++------------- go.sum | 144 ++++++++++++++++++++++++++++----------------------------- 2 files changed, 105 insertions(+), 105 deletions(-) diff --git a/go.mod b/go.mod index c5ee29b1f5c..c01b5c50f49 100644 --- a/go.mod +++ b/go.mod @@ -28,27 +28,27 @@ require ( github.com/hashicorp/go-multierror v1.1.1 github.com/hashicorp/golang-lru v1.0.2 github.com/jolestar/go-commons-pool v2.0.0+incompatible - github.com/jstemmer/go-junit-report/v2 v2.0.0 - github.com/mattn/go-zglob v0.0.4 + github.com/jstemmer/go-junit-report/v2 v2.1.0 + github.com/mattn/go-zglob v0.0.6 github.com/mitchellh/go-homedir v1.1.0 github.com/mitchellh/mapstructure v1.5.0 github.com/oklog/ulid v1.3.1 github.com/patrickmn/go-cache v2.1.0+incompatible github.com/pkg/errors v0.9.1 - github.com/prometheus/client_golang v1.19.1 + github.com/prometheus/client_golang v1.20.5 github.com/renstrom/shortuuid v3.0.0+incompatible github.com/sirupsen/logrus v1.9.3 - github.com/spf13/cobra v1.8.0 + github.com/spf13/cobra v1.8.1 github.com/spf13/pflag v1.0.5 - github.com/spf13/viper v1.18.2 - github.com/stretchr/testify v1.8.4 + github.com/spf13/viper v1.19.0 + github.com/stretchr/testify v1.9.0 github.com/weaveworks/promrus v1.2.0 - golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 - golang.org/x/net v0.22.0 - golang.org/x/oauth2 v0.18.0 - golang.org/x/sync v0.6.0 - google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f // indirect - google.golang.org/grpc v1.59.0 + golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c + golang.org/x/net v0.30.0 + golang.org/x/oauth2 v0.23.0 + golang.org/x/sync v0.8.0 + google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/grpc v1.67.1 gopkg.in/yaml.v2 v2.4.0 k8s.io/api v0.26.15 k8s.io/apimachinery v0.26.15 @@ -78,14 +78,14 @@ require ( github.com/magefile/mage v1.14.0 github.com/minio/highwayhash v1.0.2 github.com/openconfig/goyang v1.2.0 - github.com/prometheus/common v0.48.0 + github.com/prometheus/common v0.60.0 github.com/redis/go-redis/extra/redisprometheus/v9 v9.0.5 - github.com/redis/go-redis/v9 v9.5.1 + github.com/redis/go-redis/v9 v9.7.0 github.com/segmentio/fasthash v1.0.3 github.com/xitongsys/parquet-go v1.6.2 - golang.org/x/term v0.18.0 + golang.org/x/term v0.25.0 golang.org/x/time v0.5.0 - google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f + google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142 gopkg.in/inf.v0 v0.9.1 ) @@ -106,7 +106,7 @@ require ( github.com/bits-and-blooms/bitset v1.4.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/caarlos0/log v0.4.4 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/charmbracelet/lipgloss v0.9.1 // indirect github.com/danieljoos/wincred v1.1.2 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect @@ -122,7 +122,7 @@ require ( github.com/fsnotify/fsnotify v1.7.0 // indirect github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/go-errors/errors v1.0.1 // indirect - github.com/go-logr/logr v1.2.3 // indirect + github.com/go-logr/logr v1.4.1 // indirect github.com/go-openapi/jsonpointer v0.20.2 // indirect github.com/go-playground/locales v0.14.1 // indirect github.com/go-playground/universal-translator v0.18.1 // indirect @@ -152,7 +152,8 @@ require ( github.com/jackc/puddle/v2 v2.2.1 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.5 // indirect + github.com/klauspost/compress v1.17.9 // indirect + github.com/kylelemons/godebug v1.1.0 // indirect github.com/leodido/go-urn v1.2.4 // indirect github.com/liggitt/tabwriter v0.0.0-20181228230101-89fcab3d43de // indirect github.com/linkedin/goavro/v2 v2.9.8 // indirect @@ -170,21 +171,21 @@ require ( github.com/muesli/termenv v0.15.2 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect - github.com/pelletier/go-toml/v2 v2.1.0 // indirect + github.com/pelletier/go-toml/v2 v2.2.3 // indirect github.com/peterbourgon/diskv v2.0.1+incompatible // indirect github.com/pierrec/lz4 v2.0.5+incompatible // indirect github.com/pierrec/lz4/v4 v4.1.8 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect - github.com/pquerna/cachecontrol v0.1.0 // indirect - github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/procfs v0.12.0 // indirect + github.com/pquerna/cachecontrol v0.2.0 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.2 // indirect - github.com/sagikazarmark/locafero v0.4.0 // indirect + github.com/sagikazarmark/locafero v0.6.0 // indirect github.com/sagikazarmark/slog-shim v0.1.0 // indirect github.com/sourcegraph/conc v0.3.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/spf13/afero v1.11.0 // indirect - github.com/spf13/cast v1.6.0 // indirect + github.com/spf13/cast v1.7.0 // indirect github.com/subosito/gotenv v1.6.0 // indirect github.com/wk8/go-ordered-map/v2 v2.1.8 // indirect github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0 // indirect @@ -195,16 +196,15 @@ require ( go.starlark.net v0.0.0-20200306205701-8dd3e2ee1dd5 // indirect go.uber.org/atomic v1.9.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/crypto v0.21.0 // indirect - golang.org/x/mod v0.15.0 // indirect - golang.org/x/sys v0.18.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/crypto v0.28.0 // indirect + golang.org/x/mod v0.21.0 // indirect + golang.org/x/sys v0.26.0 // indirect + golang.org/x/text v0.19.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect + google.golang.org/protobuf v1.35.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect - gopkg.in/square/go-jose.v2 v2.4.1 // indirect + gopkg.in/square/go-jose.v2 v2.6.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/cli-runtime v0.26.15 // indirect k8s.io/klog/v2 v2.100.1 // indirect diff --git a/go.sum b/go.sum index 88c3d7a4b89..f63c61de172 100644 --- a/go.sum +++ b/go.sum @@ -8,14 +8,13 @@ cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.110.10 h1:LXy9GEO+timppncPIAZoOj3l58LIU9k+kn48AN7IO3Y= +cloud.google.com/go v0.112.1 h1:uJSeirPke5UNZHIb4SxfZklVSiWWVqW4oXlETwZziwM= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= -cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute v1.24.0 h1:phWcR2eWzRJaL/kOiJwfFsPs4BaKq1j6vnpZrc1YlVg= +cloud.google.com/go/compute/metadata v0.5.0 h1:Zr0eK8JbFv6+Wi4ilXAR8FJ3wyNdpxHKJNPos6LTZOY= +cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= @@ -86,8 +85,8 @@ github.com/caarlos0/testfs v0.4.4 h1:3PHvzHi5Lt+g332CiShwS8ogTgS3HjrmzZxCm6JCDr8 github.com/caarlos0/testfs v0.4.4/go.mod h1:bRN55zgG4XCUVVHZCeU+/Tz1Q6AxEJOEJTliBy+1DMk= github.com/cenkalti/backoff/v4 v4.0.0/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/charmbracelet/glamour v0.7.0 h1:2BtKGZ4iVJCDfMF229EzbeR1QRKLWztO9dMtjmqZSng= github.com/charmbracelet/glamour v0.7.0/go.mod h1:jUMh5MeihljJPQbJ/wf4ldw2+yBP59+ctV36jASy7ps= github.com/charmbracelet/lipgloss v0.9.1 h1:PNyd3jvaJbg4jRHKWXnCj1akQm4rh8dbEzN1p/u1KWg= @@ -100,7 +99,7 @@ github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGX github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/coreos/go-oidc v2.2.1+incompatible h1:mh48q/BqXqgjVHpy2ZY7WnWAbenxRjsz9N1i1YxjHAk= github.com/coreos/go-oidc v2.2.1+incompatible/go.mod h1:CgnwVTmzoESiwO9qyAFEMiHoZ1nMCKZlZ9V6mm3/LKc= -github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/danieljoos/wincred v1.1.2 h1:QLdCxFs1/Yl4zduvBdcHB8goaYk9RARS2SgLLRuAyr0= github.com/danieljoos/wincred v1.1.2/go.mod h1:GijpziifJoIBfYh+S7BbkdUTU4LfM+QnGqR5Vl2tAx0= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -148,8 +147,8 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= +github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-openapi/analysis v0.22.2 h1:ZBmNoP2h5omLKr/srIC9bfqrUGzT6g6gNv03HE9Vpj0= github.com/go-openapi/analysis v0.22.2/go.mod h1:pDF4UbZsQTo/oNuRfAWWd4dAh4yuYf//LYorPTjrpvo= github.com/go-openapi/errors v0.21.0 h1:FhChC/duCnfoLj1gZ0BgaBmzhJC2SL/sJr8a2vAobSY= @@ -330,15 +329,15 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jstemmer/go-junit-report/v2 v2.0.0 h1:bMZNO9B16VFn07tKyi4YJFIbZtVmJaa5Xakv9dcwK58= -github.com/jstemmer/go-junit-report/v2 v2.0.0/go.mod h1:mgHVr7VUo5Tn8OLVr1cKnLuEy0M92wdRntM99h7RkgQ= +github.com/jstemmer/go-junit-report/v2 v2.1.0 h1:X3+hPYlSczH9IMIpSC9CQSZA0L+BipYafciZUWHEmsc= +github.com/jstemmer/go-junit-report/v2 v2.1.0/go.mod h1:mgHVr7VUo5Tn8OLVr1cKnLuEy0M92wdRntM99h7RkgQ= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.5 h1:d4vBd+7CHydUqpFBgUEKkSdtSugf9YFmSkvUYPquI5E= -github.com/klauspost/compress v1.17.5/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= @@ -348,6 +347,7 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= @@ -371,8 +371,8 @@ github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mattn/go-zglob v0.0.4 h1:LQi2iOm0/fGgu80AioIJ/1j9w9Oh+9DZ39J4VAGzHQM= -github.com/mattn/go-zglob v0.0.4/go.mod h1:MxxjyoXXnMxfIpxTK2GAkw1w8glPsQILx3N5wrKakiY= +github.com/mattn/go-zglob v0.0.6 h1:mP8RnmCgho4oaUYDIDn6GNxYk+qJGUs8fJLn+twYj2A= +github.com/mattn/go-zglob v0.0.6/go.mod h1:MxxjyoXXnMxfIpxTK2GAkw1w8glPsQILx3N5wrKakiY= github.com/microcosm-cc/bluemonday v1.0.25 h1:4NEwSfiJ+Wva0VxN5B8OwMicaJvD8r9tlJWm9rtloEg= github.com/microcosm-cc/bluemonday v1.0.25/go.mod h1:ZIOjCQp1OrzBBPIJmfX4qDYFuhU02nx4bn030ixfHLE= github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= @@ -420,8 +420,8 @@ github.com/patrickmn/go-cache v2.1.0+incompatible h1:HRMgzkcYKYpi3C8ajMPV8OFXaaR github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pborman/getopt v0.0.0-20190409184431-ee0cd42419d3/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= -github.com/pelletier/go-toml/v2 v2.1.0 h1:FnwAJ4oYMvbT/34k9zzHuZNrhlz48GB3/s6at6/MHO4= -github.com/pelletier/go-toml/v2 v2.1.0/go.mod h1:tJU2Z3ZkXwnxa4DPO899bsyIoywizdUvyaeZurnPPDc= +github.com/pelletier/go-toml/v2 v2.2.3 h1:YmeHyLY8mFWbdkNWwpr+qIL2bEqT0o95WSdkNHvL12M= +github.com/pelletier/go-toml/v2 v2.2.3/go.mod h1:MfCQTFTvCcUyyvvwm1+G6H/jORL20Xlb6rzQu9GuUkc= github.com/peterbourgon/diskv v2.0.1+incompatible h1:UBdAOUP5p4RWqPBg048CAvpKN+vxiaj6gdUUzhl4XmI= github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU= github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= @@ -434,21 +434,21 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/pquerna/cachecontrol v0.1.0 h1:yJMy84ti9h/+OEWa752kBTKv4XC30OtVVHYv/8cTqKc= -github.com/pquerna/cachecontrol v0.1.0/go.mod h1:NrUG3Z7Rdu85UNR3vm7SOsl1nFIeSiQnrHV5K9mBcUI= -github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE= -github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho= +github.com/pquerna/cachecontrol v0.2.0 h1:vBXSNuE5MYP9IJ5kjsdo8uq+w41jSPgvba2DEnkRx9k= +github.com/pquerna/cachecontrol v0.2.0/go.mod h1:NrUG3Z7Rdu85UNR3vm7SOsl1nFIeSiQnrHV5K9mBcUI= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= -github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.48.0 h1:QO8U2CdOzSn1BBsmXJXduaaW+dY/5QLjfB8svtSzKKE= -github.com/prometheus/common v0.48.0/go.mod h1:0/KsvlIEfPQCQ5I2iNSAWKPZziNCvRs5EC6ILDTlAPc= -github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= -github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.60.0 h1:+V9PAREWNvJMAuJ1x1BaWl9dewMW4YrHZQbx0sJNllA= +github.com/prometheus/common v0.60.0/go.mod h1:h0LYf1R1deLSKtD4Vdg8gy4RuOvENW2J/h19V5NADQw= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/redis/go-redis/extra/redisprometheus/v9 v9.0.5 h1:kvl0LOTQD23VR1R7A9vDti9msfV6mOE2+j6ngYkFsfg= github.com/redis/go-redis/extra/redisprometheus/v9 v9.0.5/go.mod h1:VhyLk7MdSTKbJCx6+wXlj3/ebh49gTq3yBiXymYrG7w= -github.com/redis/go-redis/v9 v9.5.1 h1:H1X4D3yHPaYrkL5X06Wh6xNVM/pX0Ft4RV0vMGvLBh8= -github.com/redis/go-redis/v9 v9.5.1/go.mod h1:hdY0cQFCN4fnSYT6TkisLufl/4W5UIXyv0b/CLO2V2M= +github.com/redis/go-redis/v9 v9.7.0 h1:HhLSs+B6O021gwzl+locl0zEDnyNkxMtf/Z3NNBMa9E= +github.com/redis/go-redis/v9 v9.7.0/go.mod h1:f6zhXITC7JUJIlPEiBOTXxJgPLdZcA93GewI7inzyWw= github.com/renstrom/shortuuid v3.0.0+incompatible h1:F6T1U7bWlI3FTV+JE8HyeR7bkTeYZJntqQLA9ST4HOQ= github.com/renstrom/shortuuid v3.0.0+incompatible/go.mod h1:n18Ycpn8DijG+h/lLBQVnGKv1BCtTeXo8KKSbBOrQ8c= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= @@ -460,8 +460,8 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/sagikazarmark/locafero v0.4.0 h1:HApY1R9zGo4DBgr7dqsTH/JJxLTTsOt7u6keLGt6kNQ= -github.com/sagikazarmark/locafero v0.4.0/go.mod h1:Pe1W6UlPYUk/+wc/6KFhbORCfqzgYEpgQ3O5fPuL3H4= +github.com/sagikazarmark/locafero v0.6.0 h1:ON7AQg37yzcRPU69mt7gwhFEBwxI6P9T4Qu3N51bwOk= +github.com/sagikazarmark/locafero v0.6.0/go.mod h1:77OmuIc6VTraTXKXIs/uvUxKGUXjE1GbemJYHqdNjX0= github.com/sagikazarmark/slog-shim v0.1.0 h1:diDBnUNK9N/354PgrxMywXnAwEr1QZcOr6gto+ugjYE= github.com/sagikazarmark/slog-shim v0.1.0/go.mod h1:SrcSrq8aKtyuqEI1uvTDTK1arOWRIczQRv+GVI1AkeQ= github.com/segmentio/fasthash v1.0.3 h1:EI9+KE1EwvMLBWwjpRDc+fEM+prwxDYbslddQGtrmhM= @@ -478,21 +478,22 @@ github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2 github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= github.com/spf13/afero v1.11.0 h1:WJQKhtpdm3v2IzqG8VMqrr6Rf3UYpEF239Jy9wNepM8= github.com/spf13/afero v1.11.0/go.mod h1:GH9Y3pIexgf1MTIWtNGyogA5MwRIDXGUr+hbWNoBjkY= -github.com/spf13/cast v1.6.0 h1:GEiTHELF+vaR5dhz3VqZfFSzZjYbgeKDpBxQVS4GYJ0= -github.com/spf13/cast v1.6.0/go.mod h1:ancEpBxwJDODSW/UG4rDrAqiKolqNNh2DX3mk86cAdo= -github.com/spf13/cobra v1.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= -github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= +github.com/spf13/cast v1.7.0 h1:ntdiHjuueXFgm5nzDRdOS4yfT43P5Fnud6DH50rz/7w= +github.com/spf13/cast v1.7.0/go.mod h1:ancEpBxwJDODSW/UG4rDrAqiKolqNNh2DX3mk86cAdo= +github.com/spf13/cobra v1.8.1 h1:e5/vxKd/rZsfSJMUX1agtjeTDf+qv1/JdBF8gg5k9ZM= +github.com/spf13/cobra v1.8.1/go.mod h1:wHxEcudfqmLYa8iTfL+OuZPbBZkmvliBWKIezN3kD9Y= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.18.2 h1:LUXCnvUvSM6FXAsj6nnfc8Q2tp1dIgUfY9Kc8GsSOiQ= -github.com/spf13/viper v1.18.2/go.mod h1:EKmWIqdnk5lOcmR72yw6hS+8OPYcwD0jteitLMVB+yk= +github.com/spf13/viper v1.19.0 h1:RWq5SEjt8o25SROyN3z2OrDB9l7RPd3lwTWU8EcEdcI= +github.com/spf13/viper v1.19.0/go.mod h1:GQUN9bilAbhU/jgc1bKs99f/suXKeUMct8Adx5+Ntkg= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.0/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -503,8 +504,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= -github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= github.com/weaveworks/promrus v1.2.0 h1:jOLf6pe6/vss4qGHjXmGz4oDJQA+AOCqEL3FvvZGz7M= @@ -558,8 +559,8 @@ golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= -golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= +golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -570,8 +571,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUFH7PGP+OQ6mgDYo3yuQ= -golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= +golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c h1:7dEasQXItcW1xKJ2+gg5VOiBnqWrJc+rq0DPKyvvdbY= +golang.org/x/exp v0.0.0-20241009180824-f66d83c29e7c/go.mod h1:NQtJDoLvd6faHhE7m4T/1IY708gDefGGjR/iUW8yQQ8= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -593,8 +594,8 @@ golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.15.0 h1:SernR4v+D55NyBH2QiEQrlBAnj1ECL6AGrA5+dPaMY8= -golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= +golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -619,15 +620,15 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= -golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= +golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= -golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= +golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= +golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -637,8 +638,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= -golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -673,12 +674,12 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= +golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= -golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= +golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -688,8 +689,8 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= +golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -749,8 +750,6 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= -google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -773,12 +772,12 @@ google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20201019141844-1ed22bb0c154/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f h1:Vn+VyHU5guc9KjB5KrjI2q0wCOWEOIh0OEsleqakHJg= -google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f/go.mod h1:nWSwAFPb+qfNJXsoeO3Io7zf4tMSfN8EA8RlDA04GhY= -google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f h1:2yNACc1O40tTnrsbk9Cv6oxiW8pxI/pXj0wRtdlYmgY= -google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f/go.mod h1:Uy9bTZJqmfrw2rIBxgGLnamc78euZULUBrLZ9XTITKI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f h1:ultW7fxlIvee4HYrtnaRPon9HpEgFk5zYpmfMgtKB5I= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f/go.mod h1:L9KNLi232K1/xB6f7AlSX692koaRnKaWSR0stBki0Yc= +google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9 h1:9+tzLLstTlPTRyJTh+ah5wIMsBW5c4tQwGTN3thOW9Y= +google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:mqHbVIp48Muh7Ywss/AD6I5kNVKZMmAa/QEW58Gxp2s= +google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142 h1:wKguEg1hsxI2/L3hUYrpo1RVi48K+uTyzKqprwLXsb8= +google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142/go.mod h1:d6be+8HhtEtucleCbxpPW9PA9XwISACu8nvpPqF0BVo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= @@ -790,8 +789,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= -google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -802,8 +801,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= +google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -821,8 +820,9 @@ gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mN gopkg.in/jcmturner/gokrb5.v7 v7.3.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/square/go-jose.v2 v2.4.1 h1:H0TmLt7/KmzlrDOpa1F+zr0Tk90PbJYBfsVUmRLrf9Y= gopkg.in/square/go-jose.v2 v2.4.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= +gopkg.in/square/go-jose.v2 v2.6.0 h1:NGk74WTnPKBNUhNzQX7PYcTLUjoq7mzKk2OKbvwk2iI= +gopkg.in/square/go-jose.v2 v2.6.0/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= From b9a0eb432afbcfee075fbcc0e9a4ce66bd656535 Mon Sep 17 00:00:00 2001 From: Eleanor Pratt <101330560+eleanorpratt@users.noreply.github.com> Date: Tue, 29 Oct 2024 17:28:14 +0000 Subject: [PATCH 5/9] Lookout Ingester DB Load Tester (#264) (#4028) Creates a load test that which using the Lookout Ingester message conversion and database insertion code, allowing us to insert realistic event patterns into the database and measure comparative performance of changes to the database code. Co-authored-by: Eleanor Pratt --- cmd/lookoutingesterv2/dbloadtester/main.go | 91 +++++ internal/common/ingest/batch.go | 3 + .../lookoutingesterv2/dbloadtester/queue.go | 323 ++++++++++++++++++ .../dbloadtester/simulator.go | 229 +++++++++++++ .../dbloadtester/test_data.yaml | 42 +++ 5 files changed, 688 insertions(+) create mode 100644 cmd/lookoutingesterv2/dbloadtester/main.go create mode 100644 internal/lookoutingesterv2/dbloadtester/queue.go create mode 100644 internal/lookoutingesterv2/dbloadtester/simulator.go create mode 100644 internal/lookoutingesterv2/dbloadtester/test_data.yaml diff --git a/cmd/lookoutingesterv2/dbloadtester/main.go b/cmd/lookoutingesterv2/dbloadtester/main.go new file mode 100644 index 00000000000..8daf0960f2f --- /dev/null +++ b/cmd/lookoutingesterv2/dbloadtester/main.go @@ -0,0 +1,91 @@ +package main + +import ( + "fmt" + "time" + + log "github.com/sirupsen/logrus" + "github.com/spf13/pflag" + "github.com/spf13/viper" + "sigs.k8s.io/yaml" + + "github.com/armadaproject/armada/internal/common" + "github.com/armadaproject/armada/internal/common/app" + "github.com/armadaproject/armada/internal/lookoutingesterv2/configuration" + "github.com/armadaproject/armada/internal/lookoutingesterv2/dbloadtester" +) + +func init() { + pflag.StringSlice( + "lookoutIngesterConfig", + []string{}, + "Fully qualified path to application configuration file (for multiple config files repeat this arg or separate paths with commas)", + ) + pflag.Parse() +} + +const ReportTemplate string = ` + Load Test on LookoutIngester at %s + + Configuration: + Total Jobs Simulated: %d + Total Concurrent Jobs Simulated: %d + Maximum Batch of Jobs Per Queue: %d + Queues in Use: %s + LookoutIngester Config: + +%s + + Results: + Total Load Test Duration: %s + Total DB Insertion Duration: %s + Number of Events Processed: %d + Average DB Insertion Time Per Event: %f milliseconds + Events Processed By DB Per Second: %f events +` + +func main() { + common.ConfigureLogging() + common.BindCommandlineArguments() + + var config configuration.LookoutIngesterV2Configuration + userSpecifiedConfigs := viper.GetStringSlice("lookoutIngesterConfig") + common.LoadConfig(&config, "./config/lookoutingesterv2", userSpecifiedConfigs) + + loadtesterConfig := dbloadtester.Config{ + TotalJobs: 500000, + TotalConcurrentJobs: 50000, + QueueSubmitBatchSize: 300, + QueueNames: []string{"queue1", "queue2", "queue3"}, + JobTemplateFile: "internal/lookoutingesterv2/dbloadtester/test_data.yaml", + } + + loadtester := dbloadtester.Setup( + config, + loadtesterConfig, + ) + + results, err := loadtester.Run(app.CreateContextWithShutdown()) + if err != nil { + log.Errorf("Ingestion simulator failed: %v", err) + } + + LIConfig, err := yaml.Marshal(config) + if err != nil { + log.Warn("Failed to marshal lookout ingester config for report output") + } + fmt.Printf( + ReportTemplate, + time.Now().Format("2006-01-02"), + loadtesterConfig.TotalJobs, + loadtesterConfig.TotalConcurrentJobs, + loadtesterConfig.QueueSubmitBatchSize, + loadtesterConfig.QueueNames, + string(LIConfig), + results.TotalTestDuration, + results.TotalDBInsertionDuration, + results.TotalEventsProcessed, + float64(results.TotalDBInsertionDuration.Milliseconds())/float64(results.TotalEventsProcessed), + float64(results.TotalEventsProcessed)/float64(results.TotalDBInsertionDuration.Seconds()), + ) +} diff --git a/internal/common/ingest/batch.go b/internal/common/ingest/batch.go index 2e7c239f3af..02a35f7490c 100644 --- a/internal/common/ingest/batch.go +++ b/internal/common/ingest/batch.go @@ -53,6 +53,9 @@ func (b *Batcher[T]) Run(ctx *armadacontext.Context) { case value, ok := <-b.input: if !ok { // input channel has closed + if totalNumberOfItems > 0 { + b.publish <- b.buffer + } return } b.mutex.Lock() diff --git a/internal/lookoutingesterv2/dbloadtester/queue.go b/internal/lookoutingesterv2/dbloadtester/queue.go new file mode 100644 index 00000000000..6ad9775488f --- /dev/null +++ b/internal/lookoutingesterv2/dbloadtester/queue.go @@ -0,0 +1,323 @@ +package dbloadtester + +import ( + "fmt" + "math/rand" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/google/uuid" + "github.com/pkg/errors" + log "github.com/sirupsen/logrus" + v1 "k8s.io/api/core/v1" + + "github.com/armadaproject/armada/internal/common/database/lookout" + "github.com/armadaproject/armada/internal/common/ingest/utils" + protoutil "github.com/armadaproject/armada/internal/common/proto" + "github.com/armadaproject/armada/internal/common/util" + "github.com/armadaproject/armada/pkg/armadaevents" +) + +var templateAnnotations = map[string]string{ + "armadaproject.io/attempt": "0", + "armadaproject.io/custom_name": "jobcustomnameforreference", + "armadaproject.io/stage": "first_stage_of_complex_workflow", + "armadaproject.io/grade": "Dev", + "armadaproject.io/tracking_guid": "Loremipsumdolorsitametconsecteturadi", + "armadaproject.io/tracking_name": "00000000", + "armadaproject.io/job-request-application": "orchestrate", + "armadaproject.io/namespace": "queueName", + "armadaproject.io/task": "loremipsumdolorsitametconsecteturadipiscingelitseddoeiusmodte", + "armadaproject.io/task_name": "Loremipsumdolors", + "armadaproject.io/workflow": "workflow", + "armadaproject.io/log-path": "loremipsumdolorsi/ametconsecteturadipiscin/elitseddoeiusmodtemporinc/diduntu", + "armadaproject.io/version": "0.00.00", +} + +type QueueEventGenerator struct { + queueName string + jobSetName string + totalJobs int + creationBatchSize int + + jobToJobState map[string]int + jobIdToJobRun map[string]string + newEventTimeToJobs map[int64][]string + + activeJobs int + maxActiveJobs int + completedJobs int + + stepSeconds int64 + startTime int64 + + jobTemplate *v1.PodSpec +} + +func NewQueueEventGenerator(queueName, jobSetName string, totalJobs, maxActiveJobs, creationBatchSize int, timeStep int64, jobTemplate *v1.PodSpec) *QueueEventGenerator { + return &QueueEventGenerator{ + queueName: queueName, + jobSetName: jobSetName, + totalJobs: totalJobs, + creationBatchSize: creationBatchSize, + maxActiveJobs: maxActiveJobs, + jobToJobState: map[string]int{}, + jobIdToJobRun: map[string]string{}, + newEventTimeToJobs: map[int64][]string{}, + activeJobs: 0, + completedJobs: 0, + stepSeconds: timeStep, + jobTemplate: jobTemplate, + } +} + +func (q *QueueEventGenerator) Generate(eventsCh chan<- *utils.EventsWithIds[*armadaevents.EventSequence]) { + totalEventsGenerated := 0 + for i := q.startTime; ; i += q.stepSeconds { + if q.completedJobs >= q.totalJobs { + return + } + events, err := q.generateEventsAtTime(i) + if err != nil { + log.Panicf("failed to generate events %s", err) + } + if len(events) == 0 { + continue + } + eventSequenceWithIds := &utils.EventsWithIds[*armadaevents.EventSequence]{ + Events: []*armadaevents.EventSequence{ + { + Queue: q.queueName, + JobSetName: q.jobSetName, + UserId: q.queueName, + Events: events, + }, + }, + MessageIds: make([]pulsar.MessageID, len(events)), + } + totalEventsGenerated += len(events) + // log.Infof("Queue %s generated %d events so far", q.queueName, totalEventsGenerated) + eventsCh <- eventSequenceWithIds + } +} + +func (q *QueueEventGenerator) generateEventsAtTime(t int64) ([]*armadaevents.EventSequence_Event, error) { + newEventJobs := q.newEventTimeToJobs[t] + var eventsToPublish []*armadaevents.EventSequence_Event + + submitEvents, err := q.generateSubmitEvents(t) + if err != nil { + return nil, err + } + + eventsToPublish = append(eventsToPublish, submitEvents...) + for _, jobId := range newEventJobs { + jobState := q.jobToJobState[jobId] + newJobState := q.getNextState(jobState) + + jobEvents, err := q.getEventsFromTargetState(t, jobId, q.jobIdToJobRun[jobId], newJobState) + if err != nil { + return nil, errors.WithMessage(err, "failed to create events from target state") + } + + eventsToPublish = append(eventsToPublish, jobEvents...) + if isStateTerminal(newJobState) { + delete(q.jobToJobState, jobId) + delete(q.jobIdToJobRun, jobId) + q.activeJobs -= 1 + q.completedJobs += 1 + } else { + timeAtNextEvent := t + q.getStateDuration(newJobState) + q.jobToJobState[jobId] = newJobState + q.newEventTimeToJobs[timeAtNextEvent] = append(q.newEventTimeToJobs[timeAtNextEvent], jobId) + } + } + + // clean up the event generation tracker at time t, as they have all now been processed + delete(q.newEventTimeToJobs, t) + + return eventsToPublish, nil +} + +func (q *QueueEventGenerator) generateSubmitEvents(t int64) ([]*armadaevents.EventSequence_Event, error) { + available := q.maxActiveJobs - q.activeJobs + remainingJobs := q.totalJobs - q.completedJobs - q.activeJobs + if available == 0 || remainingJobs == 0 { + return nil, nil + } + submitEvents := make([]*armadaevents.EventSequence_Event, min(q.creationBatchSize, remainingJobs, available)) + for i := range submitEvents { + jobId := util.NewULID() + jobRunId := uuid.NewString() + + q.jobToJobState[jobId] = lookout.JobQueuedOrdinal + q.jobIdToJobRun[jobId] = jobRunId + nextEventTime := t + q.getStateDuration(lookout.JobQueuedOrdinal) + + q.newEventTimeToJobs[nextEventTime] = append(q.newEventTimeToJobs[nextEventTime], jobId) + + events, err := q.getEventsFromTargetState(t, jobId, jobRunId, lookout.JobQueuedOrdinal) + if err != nil { + return nil, errors.WithMessage(err, "failed to create submit event") + } + submitEvents[i] = events[0] + } + q.activeJobs += len(submitEvents) + + return submitEvents, nil +} + +func (q *QueueEventGenerator) getEventsFromTargetState( + t int64, + jobId string, + jobRunId string, + targetState int, +) ([]*armadaevents.EventSequence_Event, error) { + createdTime := protoutil.ToTimestamp(time.Unix(t, 0)) + switch targetState { + case lookout.JobQueuedOrdinal: + return []*armadaevents.EventSequence_Event{ + { + Event: &armadaevents.EventSequence_Event_SubmitJob{ + SubmitJob: q.getJob(jobId), + }, + Created: createdTime, + }, + }, nil + case lookout.JobLeasedOrdinal: + return []*armadaevents.EventSequence_Event{ + { + Event: &armadaevents.EventSequence_Event_JobRunLeased{ + JobRunLeased: &armadaevents.JobRunLeased{ + JobId: jobId, + RunId: jobRunId, + NodeId: "one_true_node", + }, + }, + Created: createdTime, + }, + }, nil + case lookout.JobPendingOrdinal: + return []*armadaevents.EventSequence_Event{ + { + Event: &armadaevents.EventSequence_Event_JobRunAssigned{ + JobRunAssigned: &armadaevents.JobRunAssigned{ + JobId: jobId, + RunId: jobRunId, + }, + }, + Created: createdTime, + }, + }, nil + case lookout.JobRunningOrdinal: + return []*armadaevents.EventSequence_Event{ + { + Event: &armadaevents.EventSequence_Event_JobRunRunning{ + JobRunRunning: &armadaevents.JobRunRunning{ + JobId: jobId, + RunId: jobRunId, + }, + }, + Created: createdTime, + }, + }, nil + case lookout.JobSucceededOrdinal: + return []*armadaevents.EventSequence_Event{ + { + Event: &armadaevents.EventSequence_Event_JobRunSucceeded{ + JobRunSucceeded: &armadaevents.JobRunSucceeded{ + JobId: jobId, + RunId: jobRunId, + }, + }, + Created: createdTime, + }, + { + Event: &armadaevents.EventSequence_Event_JobSucceeded{ + JobSucceeded: &armadaevents.JobSucceeded{ + JobId: jobId, + }, + }, + Created: createdTime, + }, + }, nil + } + + return nil, fmt.Errorf("unknown target state %d %s", targetState, lookout.JobStateMap[targetState]) +} + +func (q *QueueEventGenerator) getNextState(state int) int { + // submitted to leased, leased to pending, pending to running, + switch state { + case 0: + return lookout.JobQueuedOrdinal + case lookout.JobQueuedOrdinal: + return lookout.JobLeasedOrdinal + case lookout.JobLeasedOrdinal: + return lookout.JobPendingOrdinal + case lookout.JobPendingOrdinal: + return lookout.JobRunningOrdinal + case lookout.JobRunningOrdinal: + return lookout.JobSucceededOrdinal + default: + return lookout.JobSucceededOrdinal + } +} + +func (q *QueueEventGenerator) getStateDuration(state int) int64 { + switch state { + case lookout.JobQueuedOrdinal: + return q.getEventDuration(60 * q.stepSeconds) + case lookout.JobLeasedOrdinal: + return q.getEventDuration(10 * q.stepSeconds) + case lookout.JobPendingOrdinal: + return q.getEventDuration(10 * q.stepSeconds) + case lookout.JobRunningOrdinal: + return q.getEventDuration(120 * q.stepSeconds) + default: + return q.stepSeconds + } +} + +// getEventDuration returns the time before the next event, calculated as a random number of steps between the +// minDuration and max duration of one stepSeconds larger than two times the minDuration. +func (q *QueueEventGenerator) getEventDuration(minDuration int64) int64 { + return minDuration + rand.Int63n((minDuration+q.stepSeconds)/q.stepSeconds+1)*q.stepSeconds +} + +func (q *QueueEventGenerator) getJob(jobId string) *armadaevents.SubmitJob { + submitJob := &armadaevents.SubmitJob{ + JobId: jobId, + Priority: 1000, + ObjectMeta: &armadaevents.ObjectMeta{ + Namespace: fmt.Sprintf("gold-%s", q.queueName), + Annotations: templateAnnotations, + }, + MainObject: &armadaevents.KubernetesMainObject{ + Object: &armadaevents.KubernetesMainObject_PodSpec{ + PodSpec: &armadaevents.PodSpecWithAvoidList{ + PodSpec: q.jobTemplate, + }, + }, + }, + } + + return submitJob +} + +func isStateTerminal(state int) bool { + switch state { + case lookout.JobQueuedOrdinal: + return false + case lookout.JobLeasedOrdinal: + return false + case lookout.JobPendingOrdinal: + return false + case lookout.JobRunningOrdinal: + return false + case lookout.JobSucceededOrdinal: + return true + default: + return true + } +} diff --git a/internal/lookoutingesterv2/dbloadtester/simulator.go b/internal/lookoutingesterv2/dbloadtester/simulator.go new file mode 100644 index 00000000000..c15e31825c9 --- /dev/null +++ b/internal/lookoutingesterv2/dbloadtester/simulator.go @@ -0,0 +1,229 @@ +package dbloadtester + +import ( + "context" + "math" + "regexp" + "sync" + "time" + + "github.com/pkg/errors" + log "github.com/sirupsen/logrus" + v1 "k8s.io/api/core/v1" + + "github.com/armadaproject/armada/internal/common/armadacontext" + "github.com/armadaproject/armada/internal/common/compress" + "github.com/armadaproject/armada/internal/common/database" + "github.com/armadaproject/armada/internal/common/ingest" + "github.com/armadaproject/armada/internal/common/ingest/utils" + "github.com/armadaproject/armada/internal/lookoutingesterv2/configuration" + "github.com/armadaproject/armada/internal/lookoutingesterv2/instructions" + "github.com/armadaproject/armada/internal/lookoutingesterv2/lookoutdb" + "github.com/armadaproject/armada/internal/lookoutingesterv2/metrics" + "github.com/armadaproject/armada/internal/lookoutingesterv2/model" + "github.com/armadaproject/armada/pkg/armadaevents" + clientUtil "github.com/armadaproject/armada/pkg/client/util" +) + +type LoadTester struct { + totalJobs int + queueNames []string + totalConcurrentJobs int + useExistingQueues bool + jobTemplate *v1.PodSpec + batchSize int + batchDuration time.Duration + queueSubmitBatchSize int + + db *lookoutdb.LookoutDb + converter *instructions.InstructionConverter +} + +type Config struct { + TotalJobs int + QueueNames []string + TotalConcurrentJobs int + JobTemplateFile string + QueueSubmitBatchSize int +} + +type Results struct { + TotalTestDuration time.Duration + TotalDBInsertionDuration time.Duration + TotalEventsProcessed int +} + +func Setup(lookoutIngesterConfig configuration.LookoutIngesterV2Configuration, testConfig Config) *LoadTester { + m := metrics.Get() + + db, err := database.OpenPgxPool(lookoutIngesterConfig.Postgres) + if err != nil { + panic(errors.WithMessage(err, "Error opening connection to postgres")) + } + + fatalRegexes := make([]*regexp.Regexp, len(lookoutIngesterConfig.FatalInsertionErrors)) + for i, str := range lookoutIngesterConfig.FatalInsertionErrors { + rgx, err := regexp.Compile(str) + if err != nil { + log.Errorf("Error compiling regex %s", str) + panic(err) + } + fatalRegexes[i] = rgx + } + + lookoutDb := lookoutdb.NewLookoutDb(db, fatalRegexes, m, lookoutIngesterConfig.MaxBackoff) + + // To avoid load testing the compression algorithm, the compressor is configured not to compress. + compressor, err := compress.NewZlibCompressor(math.MaxInt) + if err != nil { + panic(errors.WithMessage(err, "Error creating compressor")) + } + + converter := instructions.NewInstructionConverter(m.Metrics, lookoutIngesterConfig.UserAnnotationPrefix, compressor) + + submitJobTemplate := &v1.PodSpec{} + err = clientUtil.BindJsonOrYaml(testConfig.JobTemplateFile, submitJobTemplate) + if err != nil { + panic(errors.WithMessage(err, "Error reading job template yaml")) + } + + if len(testConfig.QueueNames) > testConfig.TotalConcurrentJobs { + panic("Performance simulator currently requires a minimum of one concurrent job per queue") + } + + return &LoadTester{ + testConfig.TotalJobs, + testConfig.QueueNames, + testConfig.TotalConcurrentJobs, + false, + submitJobTemplate, + lookoutIngesterConfig.BatchSize, + lookoutIngesterConfig.BatchDuration, + testConfig.QueueSubmitBatchSize, + lookoutDb, + converter, + } +} + +// Run performs the load test with the configuration and database provided by the loadtester +func (l *LoadTester) Run(ctx *armadacontext.Context) (*Results, error) { + loadTestStart := time.Now() + + // generates events, simulated to have a realistic pattern + simulatedEvents := make(chan *utils.EventsWithIds[*armadaevents.EventSequence]) + go func() { + l.GenerateEvents(simulatedEvents) + close(simulatedEvents) + }() + + // set up batching, to match expected batching behaviour in non-test code + batchedEventSequences := make(chan []*utils.EventsWithIds[*armadaevents.EventSequence]) + eventCounterFunc := func(seq *utils.EventsWithIds[*armadaevents.EventSequence]) int { + totalEvents := 0 + for _, sequence := range seq.Events { + totalEvents += len(sequence.Events) + } + return totalEvents + } + + // batch the generated events + batcher := ingest.NewBatcher[*utils.EventsWithIds[*armadaevents.EventSequence]](simulatedEvents, l.batchSize, l.batchDuration, eventCounterFunc, batchedEventSequences) + go func() { + batcher.Run(ctx) + close(batchedEventSequences) + }() + + // Merge intermediate event batches + mergedEventBatches := make(chan *utils.EventsWithIds[*armadaevents.EventSequence]) + go func() { + for batch := range batchedEventSequences { + allEvents := &utils.EventsWithIds[*armadaevents.EventSequence]{} + for _, eventsWithIds := range batch { + allEvents.Events = append(allEvents.Events, eventsWithIds.Events...) + allEvents.MessageIds = append(allEvents.MessageIds, eventsWithIds.MessageIds...) + } + mergedEventBatches <- allEvents + } + close(mergedEventBatches) + }() + + // convert the events into the instructionSet taken by the db + instructionSets := make(chan *model.InstructionSet) + go func() { + for msg := range mergedEventBatches { + start := time.Now() + converted := l.converter.Convert(ctx, msg) + taken := time.Now().Sub(start) + log.Infof("Processed %d pulsar messages in %dms", len(msg.MessageIds), taken.Milliseconds()) + instructionSets <- converted + } + close(instructionSets) + }() + + // benchmark the insertion into the db + var totalDBTime time.Duration + var totalMessages int + for msg := range instructionSets { + start := time.Now() + err := l.db.Store(ctx, msg) + totalDBTime += time.Now().Sub(start) + if err != nil { + log.WithError(err).Warn("Error inserting messages") + log.Panic("db err") + } else { + log.Infof("Inserted %d pulsar messages in %dms", len(msg.GetMessageIDs()), totalDBTime.Milliseconds()) + totalMessages += len(msg.GetMessageIDs()) + } + if errors.Is(err, context.DeadlineExceeded) { + // This occurs when we're shutting down- it's a signal to stop processing immediately + break + } + } + loadTestDuration := time.Now().Sub(loadTestStart) + + return &Results{ + TotalTestDuration: loadTestDuration, + TotalDBInsertionDuration: totalDBTime, + TotalEventsProcessed: totalMessages, + }, nil +} + +// GenerateEvents generates EventSequencesWithIds consisting of job and job run events onto the given channel. +func (l *LoadTester) GenerateEvents(eventsCh chan<- *utils.EventsWithIds[*armadaevents.EventSequence]) { + totalPerQueue := l.totalJobs / len(l.queueNames) + additionalJobs := l.totalJobs % len(l.queueNames) + + totalConcurrentPerQueue := l.totalConcurrentJobs / len(l.queueNames) + additionalConcurrentJobs := l.totalConcurrentJobs % len(l.queueNames) + + // create queues + queues := make([]*QueueEventGenerator, len(l.queueNames)) + for i, queueName := range l.queueNames { + if i == len(l.queueNames)-1 { + totalPerQueue += additionalJobs + totalConcurrentPerQueue += additionalConcurrentJobs + } + queue := NewQueueEventGenerator( + queueName, + queueName, + totalPerQueue, + totalConcurrentPerQueue, + l.queueSubmitBatchSize, + 1, + l.jobTemplate, + ) + queues[i] = queue + } + + wg := sync.WaitGroup{} + wg.Add(len(queues)) + for _, queue := range queues { + go func() { + defer wg.Done() + queue.Generate(eventsCh) + }() + } + wg.Wait() + + return +} diff --git a/internal/lookoutingesterv2/dbloadtester/test_data.yaml b/internal/lookoutingesterv2/dbloadtester/test_data.yaml new file mode 100644 index 00000000000..84d29dccb6c --- /dev/null +++ b/internal/lookoutingesterv2/dbloadtester/test_data.yaml @@ -0,0 +1,42 @@ +containers: + - name: maincontainer + image: loremipsumdolo.rsitametconsectetur.adipiscingel/itseddoeius/modtempor.incididu.ntu:LATEST-00000 + command: + - loremip/sumdolo/rsitamet/consectetur/adipisc/ingel.sh + args: + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + - 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' + resources: + limits: + cpu: '2' + ephemeral-storage: 64Gi + memory: 24Gi + requests: + cpu: '2' + ephemeral-storage: 64Gi + memory: 24Gi + imagePullPolicy: IfNotPresent +restartPolicy: Never +terminationGracePeriodSeconds: 1 +activeDeadlineSeconds: 259200 +serviceAccount: default +shareProcessNamespace: false +priorityClassName: armada-default From e096d6a60eafa285f5698ca5e55bf67187cd3934 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Wed, 30 Oct 2024 09:12:05 +0000 Subject: [PATCH 6/9] Add post processor that will add a gang id label (#265) (#4029) * Add post processor that will add a gang id label * lint * whitespace Co-authored-by: Christopher Martin --- internal/server/configuration/types.go | 2 + .../server/submit/conversion/post_process.go | 19 +++++++ .../submit/conversion/post_process_test.go | 50 +++++++++++++++++++ 3 files changed, 71 insertions(+) diff --git a/internal/server/configuration/types.go b/internal/server/configuration/types.go index 57c5dde4489..91c26f424cd 100644 --- a/internal/server/configuration/types.go +++ b/internal/server/configuration/types.go @@ -93,6 +93,8 @@ type SubmissionConfig struct { MaxOversubscriptionByResourceRequest map[string]float64 // Enforce that an init containers requestion non-integer cpu. This is due to https://github.com/kubernetes/kubernetes/issues/112228 AssertInitContainersRequestFractionalCpu bool + // Controls whether we add the gang id annotation as a label. + AddGangIdLabel bool } // TODO: we can probably just typedef this to map[string]string diff --git a/internal/server/submit/conversion/post_process.go b/internal/server/submit/conversion/post_process.go index 1c71c8664a5..dfe57ccc3d4 100644 --- a/internal/server/submit/conversion/post_process.go +++ b/internal/server/submit/conversion/post_process.go @@ -23,6 +23,7 @@ var ( msgLevelProcessors = []msgProcessor{ templateMeta, defaultGangNodeUniformityLabel, + addGangIdLabel, } podLevelProcessors = []podProcessor{ defaultActiveDeadlineSeconds, @@ -166,6 +167,24 @@ func defaultGangNodeUniformityLabel(msg *armadaevents.SubmitJob, config configur } } +// Add a gangId label if the gangId annotation is set. We do this because labels are much faster to search on than +// annotations and a gang may want to hit the kubeapi to find its other gang members. +func addGangIdLabel(msg *armadaevents.SubmitJob, config configuration.SubmissionConfig) { + if !config.AddGangIdLabel { + return + } + + gangId := msg.GetObjectMeta().GetAnnotations()[configuration.GangIdAnnotation] + if gangId != "" { + labels := msg.GetObjectMeta().GetLabels() + if labels == nil { + labels = map[string]string{} + } + labels[configuration.GangIdAnnotation] = gangId + msg.GetObjectMeta().Labels = labels + } +} + // Templates the JobId in labels and annotations. This allows users to define labels and annotations containing the string // {JobId} and have it populated with the actual id of the job. func templateMeta(msg *armadaevents.SubmitJob, _ configuration.SubmissionConfig) { diff --git a/internal/server/submit/conversion/post_process_test.go b/internal/server/submit/conversion/post_process_test.go index 2573960daa6..1374bb8eb44 100644 --- a/internal/server/submit/conversion/post_process_test.go +++ b/internal/server/submit/conversion/post_process_test.go @@ -584,6 +584,56 @@ func TestDefaultTerminationGracePeriod(t *testing.T) { } } +func TestAddGangIdLabel(t *testing.T) { + tests := map[string]struct { + annotations map[string]string + initialLabels map[string]string + expectedLabels map[string]string + enabled bool + }{ + "Unchanged if no gang id set": { + annotations: map[string]string{}, + enabled: true, + }, + "Label added if gang id set": { + annotations: map[string]string{ + configuration.GangIdAnnotation: "foo", + }, + expectedLabels: map[string]string{ + configuration.GangIdAnnotation: "foo", + }, + enabled: true, + }, + "Doesn't modify existing labels": { + annotations: map[string]string{ + configuration.GangIdAnnotation: "foo", + }, + initialLabels: map[string]string{ + "fish": "chips", + }, + expectedLabels: map[string]string{ + "fish": "chips", + configuration.GangIdAnnotation: "foo", + }, + enabled: true, + }, + "Unchanged if disabled": { + annotations: map[string]string{ + configuration.GangIdAnnotation: "foo", + }, + enabled: false, + }, + } + for name, tc := range tests { + t.Run(name, func(t *testing.T) { + submitMsg := submitMsgFromAnnotations(tc.annotations) + submitMsg.ObjectMeta.Labels = tc.initialLabels + addGangIdLabel(submitMsg, configuration.SubmissionConfig{AddGangIdLabel: tc.enabled}) + assert.Equal(t, tc.expectedLabels, submitMsg.ObjectMeta.Labels) + }) + } +} + func submitMsgFromAnnotations(annotations map[string]string) *armadaevents.SubmitJob { return &armadaevents.SubmitJob{ ObjectMeta: &armadaevents.ObjectMeta{ From af6554b2190908100f497a0b7cb469a7ffe9bdd8 Mon Sep 17 00:00:00 2001 From: Jason Parraga Date: Wed, 30 Oct 2024 16:06:41 -0700 Subject: [PATCH 7/9] Update scheduler initialization behavior to avoid OOM (#4026) * Update scheduler initialization behavior to avoid OOM Signed-off-by: Jason Parraga * lint Signed-off-by: Jason Parraga * Add unit tests to scheduler Signed-off-by: Jason Parraga * Address some comments Signed-off-by: Jason Parraga * Add assertions for serials Signed-off-by: Jason Parraga * Address comments Signed-off-by: Jason Parraga --------- Signed-off-by: Jason Parraga --- internal/scheduler/database/job_repository.go | 68 +++++++ .../scheduler/database/job_repository_test.go | 174 ++++++++++++++++++ internal/scheduler/database/modelsext.go | 5 + internal/scheduler/database/query.sql.go | 146 +++++++++++++++ internal/scheduler/database/query/query.sql | 6 + internal/scheduler/mocks/job_repository.go | 16 ++ internal/scheduler/scheduler.go | 19 +- internal/scheduler/scheduler_test.go | 148 ++++++++++++++- magefiles/tests.go | 19 +- 9 files changed, 593 insertions(+), 8 deletions(-) diff --git a/internal/scheduler/database/job_repository.go b/internal/scheduler/database/job_repository.go index fabb791d92d..9e84614d329 100644 --- a/internal/scheduler/database/job_repository.go +++ b/internal/scheduler/database/job_repository.go @@ -36,6 +36,9 @@ type JobRunLease struct { // JobRepository is an interface to be implemented by structs which provide job and run information type JobRepository interface { + // FetchInitialJobs returns all non-terminal jobs and their associated job runs. + FetchInitialJobs(ctx *armadacontext.Context) ([]Job, []Run, error) + // FetchJobUpdates returns all jobs and job dbRuns that have been updated after jobSerial and jobRunSerial respectively // These updates are guaranteed to be consistent with each other FetchJobUpdates(ctx *armadacontext.Context, jobSerial int64, jobRunSerial int64) ([]Job, []Run, error) @@ -73,6 +76,71 @@ func NewPostgresJobRepository(db *pgxpool.Pool, batchSize int32) *PostgresJobRep } } +func (r *PostgresJobRepository) FetchInitialJobs(ctx *armadacontext.Context) ([]Job, []Run, error) { + var updatedJobs []Job + var initialRuns []Run + + start := time.Now() + defer func() { + ctx.Infof( + "received %d initial jobs and %d initial job runs from postgres in %s", + len(updatedJobs), len(initialRuns), time.Since(start), + ) + }() + + // Use a RepeatableRead transaction here so that we get consistency between jobs and dbRuns + err := pgx.BeginTxFunc(ctx, r.db, pgx.TxOptions{ + IsoLevel: pgx.RepeatableRead, + AccessMode: pgx.ReadOnly, + DeferrableMode: pgx.Deferrable, + }, func(tx pgx.Tx) error { + var err error + queries := New(tx) + + // Fetch jobs + initialJobRows, err := fetch(0, r.batchSize, func(from int64) ([]SelectInitialJobsRow, error) { + return queries.SelectInitialJobs(ctx, SelectInitialJobsParams{Serial: from, Limit: r.batchSize}) + }) + if err != nil { + return err + } + + updatedJobs = make([]Job, len(initialJobRows)) + updatedJobIds := make([]string, len(initialJobRows)) + for i, row := range initialJobRows { + updatedJobIds[i] = row.JobID + updatedJobs[i] = Job{ + JobID: row.JobID, + JobSet: row.JobSet, + Queue: row.Queue, + Priority: row.Priority, + Submitted: row.Submitted, + Validated: row.Validated, + Queued: row.Queued, + QueuedVersion: row.QueuedVersion, + CancelRequested: row.CancelRequested, + Cancelled: row.Cancelled, + CancelByJobsetRequested: row.CancelByJobsetRequested, + Succeeded: row.Succeeded, + Failed: row.Failed, + SchedulingInfo: row.SchedulingInfo, + SchedulingInfoVersion: row.SchedulingInfoVersion, + Serial: row.Serial, + Pools: row.Pools, + } + } + + // Fetch dbRuns + initialRuns, err = fetch(0, r.batchSize, func(from int64) ([]Run, error) { + return queries.SelectInitialRuns(ctx, SelectInitialRunsParams{Serial: from, Limit: r.batchSize, JobIds: updatedJobIds}) + }) + + return err + }) + + return updatedJobs, initialRuns, err +} + // FetchJobRunErrors returns all armadaevents.JobRunErrors for the provided job run ids. The returned map is // keyed by job run id. Any dbRuns which don't have errors wil be absent from the map. func (r *PostgresJobRepository) FetchJobRunErrors(ctx *armadacontext.Context, runIds []string) (map[string]*armadaevents.Error, error) { diff --git a/internal/scheduler/database/job_repository_test.go b/internal/scheduler/database/job_repository_test.go index 7e3706da736..4a29c508337 100644 --- a/internal/scheduler/database/job_repository_test.go +++ b/internal/scheduler/database/job_repository_test.go @@ -24,6 +24,180 @@ import ( const defaultBatchSize = 1 +func TestFetchInitialJobs(t *testing.T) { + leasedJob := Job{ + JobID: util.NewULID(), + JobSet: "test-jobset", + Queue: "test-queue", + QueuedVersion: 1, + SchedulingInfo: []byte{byte(0)}, + SubmitMessage: []byte{}, + } + + expectedLeasedJob := Job{ + JobID: leasedJob.JobID, + JobSet: "test-jobset", + Queue: "test-queue", + QueuedVersion: 1, + SchedulingInfo: []byte{byte(0)}, + Serial: 1, + } + + leasedJobRun := Run{ + RunID: util.NewULID(), + JobID: leasedJob.JobID, + JobSet: "test-jobset", + Executor: "test-executor", + Node: fmt.Sprintf("test-node-%d", 0), + Running: true, + } + + expectedLeasedJobRun := Run{ + RunID: leasedJobRun.RunID, + JobID: leasedJob.JobID, + JobSet: "test-jobset", + Executor: "test-executor", + Node: fmt.Sprintf("test-node-%d", 0), + Running: true, + Serial: 1, + } + + queuedJob := Job{ + JobID: util.NewULID(), + JobSet: "test-jobset", + Queue: "test-queue", + Queued: true, + QueuedVersion: 1, + SchedulingInfo: []byte{byte(0)}, + SubmitMessage: []byte{}, + } + + expectedQueuedJob := Job{ + JobID: queuedJob.JobID, + JobSet: "test-jobset", + Queue: "test-queue", + Queued: true, + QueuedVersion: 1, + SchedulingInfo: []byte{byte(0)}, + Serial: 2, + } + + cancelledJob := Job{ + JobID: util.NewULID(), + JobSet: "test-jobset", + Queue: "test-queue", + QueuedVersion: 1, + Cancelled: true, + SchedulingInfo: []byte{byte(0)}, + SubmitMessage: []byte{}, + } + + cancelledJobRun := Run{ + RunID: util.NewULID(), + JobID: cancelledJob.JobID, + JobSet: "test-jobset", + Executor: "test-executor", + Node: fmt.Sprintf("test-node-%d", 0), + Cancelled: true, + } + + failedJob := Job{ + JobID: util.NewULID(), + JobSet: "test-jobset", + Queue: "test-queue", + QueuedVersion: 1, + Failed: true, + SchedulingInfo: []byte{byte(0)}, + SubmitMessage: []byte{}, + } + + failedJobRun := Run{ + RunID: util.NewULID(), + JobID: failedJob.JobID, + JobSet: "test-jobset", + Executor: "test-executor", + Node: fmt.Sprintf("test-node-%d", 0), + Failed: true, + } + + succeededJob := Job{ + JobID: util.NewULID(), + JobSet: "test-jobset", + Queue: "test-queue", + Queued: false, + QueuedVersion: 1, + Succeeded: true, + SchedulingInfo: []byte{byte(0)}, + SubmitMessage: []byte{}, + } + + succeededJobRun := Run{ + RunID: util.NewULID(), + JobID: succeededJob.JobID, + JobSet: "test-jobset", + Executor: "test-executor", + Node: fmt.Sprintf("test-node-%d", 0), + Succeeded: true, + } + + tests := map[string]struct { + dbJobs []Job + dbRuns []Run + expectedJobs []Job + expectedRuns []Run + }{ + "all jobs and runs": { + dbJobs: []Job{leasedJob, queuedJob, cancelledJob, failedJob, succeededJob}, + expectedJobs: []Job{expectedLeasedJob, expectedQueuedJob}, + dbRuns: []Run{leasedJobRun, cancelledJobRun, failedJobRun, succeededJobRun}, + expectedRuns: []Run{expectedLeasedJobRun}, + }, + "only jobs": { + dbJobs: []Job{leasedJob, queuedJob, cancelledJob, failedJob, succeededJob}, + expectedJobs: []Job{expectedLeasedJob, expectedQueuedJob}, + expectedRuns: []Run{}, + }, + "only runs": { + dbRuns: []Run{leasedJobRun, cancelledJobRun, failedJobRun, succeededJobRun}, + expectedJobs: []Job{}, + expectedRuns: []Run{}, + }, + "empty db": { + expectedJobs: []Job{}, + expectedRuns: []Run{}, + }, + } + for name, tc := range tests { + t.Run(name, func(t *testing.T) { + err := withJobRepository(func(repo *PostgresJobRepository) error { + ctx, cancel := armadacontext.WithTimeout(armadacontext.Background(), 5*time.Second) + + // Set up db + err := database.UpsertWithTransaction(ctx, repo.db, "jobs", tc.dbJobs) + require.NoError(t, err) + err = database.UpsertWithTransaction(ctx, repo.db, "runs", tc.dbRuns) + require.NoError(t, err) + + // Fetch updates + jobs, runs, err := repo.FetchInitialJobs(ctx) + require.NoError(t, err) + + // Runs will have LastModified filled in- we don't want to compare this + for i := range runs { + runs[i].LastModified = time.Time{} + } + + // Assert results + assert.Equal(t, tc.expectedJobs, jobs) + assert.Equal(t, tc.expectedRuns, runs) + cancel() + return nil + }) + require.NoError(t, err) + }) + } +} + func TestFetchJobUpdates(t *testing.T) { dbJobs, expectedJobs := createTestJobs(10) dbRuns, expectedRuns := createTestRuns(10) diff --git a/internal/scheduler/database/modelsext.go b/internal/scheduler/database/modelsext.go index f5d918b276b..8ccd80aae24 100644 --- a/internal/scheduler/database/modelsext.go +++ b/internal/scheduler/database/modelsext.go @@ -19,3 +19,8 @@ func (run Run) GetSerial() int64 { func (row SelectUpdatedJobsRow) GetSerial() int64 { return row.Serial } + +// GetSerial is needed for the HasSerial interface +func (row SelectInitialJobsRow) GetSerial() int64 { + return row.Serial +} diff --git a/internal/scheduler/database/query.sql.go b/internal/scheduler/database/query.sql.go index af77c1716c6..f4e8f9a95a4 100644 --- a/internal/scheduler/database/query.sql.go +++ b/internal/scheduler/database/query.sql.go @@ -398,6 +398,73 @@ func (q *Queries) SelectJobsForExecutor(ctx context.Context, arg SelectJobsForEx return items, nil } +const selectInitialJobs = `-- name: SelectInitialJobs :many +SELECT job_id, job_set, queue, priority, submitted, queued, queued_version, validated, cancel_requested, cancel_by_jobset_requested, cancelled, succeeded, failed, scheduling_info, scheduling_info_version, pools, serial FROM jobs WHERE serial > $1 AND cancelled = 'false' AND succeeded = 'false' and failed = 'false' ORDER BY serial LIMIT $2 +` + +type SelectInitialJobsParams struct { + Serial int64 `db:"serial"` + Limit int32 `db:"limit"` +} + +type SelectInitialJobsRow struct { + JobID string `db:"job_id"` + JobSet string `db:"job_set"` + Queue string `db:"queue"` + Priority int64 `db:"priority"` + Submitted int64 `db:"submitted"` + Queued bool `db:"queued"` + QueuedVersion int32 `db:"queued_version"` + Validated bool `db:"validated"` + CancelRequested bool `db:"cancel_requested"` + CancelByJobsetRequested bool `db:"cancel_by_jobset_requested"` + Cancelled bool `db:"cancelled"` + Succeeded bool `db:"succeeded"` + Failed bool `db:"failed"` + SchedulingInfo []byte `db:"scheduling_info"` + SchedulingInfoVersion int32 `db:"scheduling_info_version"` + Pools []string `db:"pools"` + Serial int64 `db:"serial"` +} + +func (q *Queries) SelectInitialJobs(ctx context.Context, arg SelectInitialJobsParams) ([]SelectInitialJobsRow, error) { + rows, err := q.db.Query(ctx, selectInitialJobs, arg.Serial, arg.Limit) + if err != nil { + return nil, err + } + defer rows.Close() + var items []SelectInitialJobsRow + for rows.Next() { + var i SelectInitialJobsRow + if err := rows.Scan( + &i.JobID, + &i.JobSet, + &i.Queue, + &i.Priority, + &i.Submitted, + &i.Queued, + &i.QueuedVersion, + &i.Validated, + &i.CancelRequested, + &i.CancelByJobsetRequested, + &i.Cancelled, + &i.Succeeded, + &i.Failed, + &i.SchedulingInfo, + &i.SchedulingInfoVersion, + &i.Pools, + &i.Serial, + ); err != nil { + return nil, err + } + items = append(items, i) + } + if err := rows.Err(); err != nil { + return nil, err + } + return items, nil +} + const selectNewJobs = `-- name: SelectNewJobs :many SELECT job_id, job_set, queue, user_id, submitted, groups, priority, queued, queued_version, cancel_requested, cancelled, cancel_by_jobset_requested, succeeded, failed, submit_message, scheduling_info, scheduling_info_version, serial, last_modified, validated, pools FROM jobs WHERE serial > $1 ORDER BY serial LIMIT $2 ` @@ -449,6 +516,63 @@ func (q *Queries) SelectNewJobs(ctx context.Context, arg SelectNewJobsParams) ([ return items, nil } +const selectInitialRuns = `-- name: SelectInitialRuns :many +SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool FROM runs WHERE serial > $1 AND job_id = ANY($3::text[]) ORDER BY serial LIMIT $2 +` + +type SelectInitialRunsParams struct { + Serial int64 `db:"serial"` + Limit int32 `db:"limit"` + JobIds []string `db:"job_ids"` +} + +func (q *Queries) SelectInitialRuns(ctx context.Context, arg SelectInitialRunsParams) ([]Run, error) { + rows, err := q.db.Query(ctx, selectInitialRuns, arg.Serial, arg.Limit, arg.JobIds) + if err != nil { + return nil, err + } + defer rows.Close() + var items []Run + for rows.Next() { + var i Run + if err := rows.Scan( + &i.RunID, + &i.JobID, + &i.Created, + &i.JobSet, + &i.Executor, + &i.Node, + &i.Cancelled, + &i.Running, + &i.Succeeded, + &i.Failed, + &i.Returned, + &i.RunAttempted, + &i.Serial, + &i.LastModified, + &i.LeasedTimestamp, + &i.PendingTimestamp, + &i.RunningTimestamp, + &i.TerminatedTimestamp, + &i.ScheduledAtPriority, + &i.Preempted, + &i.Pending, + &i.PreemptedTimestamp, + &i.PodRequirementsOverlay, + &i.PreemptRequested, + &i.Queue, + &i.Pool, + ); err != nil { + return nil, err + } + items = append(items, i) + } + if err := rows.Err(); err != nil { + return nil, err + } + return items, nil +} + const selectNewRuns = `-- name: SelectNewRuns :many SELECT run_id, job_id, created, job_set, executor, node, cancelled, running, succeeded, failed, returned, run_attempted, serial, last_modified, leased_timestamp, pending_timestamp, running_timestamp, terminated_timestamp, scheduled_at_priority, preempted, pending, preempted_timestamp, pod_requirements_overlay, preempt_requested, queue, pool FROM runs WHERE serial > $1 ORDER BY serial LIMIT $2 ` @@ -816,3 +940,25 @@ func (q *Queries) SelectAllExecutorSettings(ctx context.Context) ([]ExecutorSett } return items, nil } + +const selectLatestJobSerial = `-- name: SelectLatestJobSerial :one +SELECT serial FROM jobs ORDER BY serial DESC LIMIT 1 +` + +func (q *Queries) SelectLatestJobSerial(ctx context.Context) (int64, error) { + row := q.db.QueryRow(ctx, selectLatestJobSerial) + var serial int64 + err := row.Scan(&serial) + return serial, err +} + +const selectLatestJobRunSerial = `-- name: SelectLatestJobRunSerial :one +SELECT serial FROM runs ORDER BY serial DESC LIMIT 1 +` + +func (q *Queries) SelectLatestJobRunSerial(ctx context.Context) (int64, error) { + row := q.db.QueryRow(ctx, selectLatestJobRunSerial) + var serial int64 + err := row.Scan(&serial) + return serial, err +} diff --git a/internal/scheduler/database/query/query.sql b/internal/scheduler/database/query/query.sql index 48d70b8b630..eb943fd5e61 100644 --- a/internal/scheduler/database/query/query.sql +++ b/internal/scheduler/database/query/query.sql @@ -4,6 +4,9 @@ SELECT * FROM jobs WHERE serial > $1 ORDER BY serial LIMIT $2; -- name: SelectAllJobIds :many SELECT job_id FROM jobs; +-- name: SelectInitialJobs :many +SELECT job_id, job_set, queue, priority, submitted, queued, queued_version, validated, cancel_requested, cancel_by_jobset_requested, cancelled, succeeded, failed, scheduling_info, scheduling_info_version, pools, serial FROM jobs WHERE serial > $1 AND cancelled = 'false' AND succeeded = 'false' and failed = 'false' ORDER BY serial LIMIT $2; + -- name: SelectUpdatedJobs :many SELECT job_id, job_set, queue, priority, submitted, queued, queued_version, validated, cancel_requested, cancel_by_jobset_requested, cancelled, succeeded, failed, scheduling_info, scheduling_info_version, pools, serial FROM jobs WHERE serial > $1 ORDER BY serial LIMIT $2; @@ -28,6 +31,9 @@ UPDATE jobs SET failed = true WHERE job_id = ANY(sqlc.arg(job_ids)::text[]); -- name: UpdateJobPriorityById :exec UPDATE jobs SET priority = $1 WHERE queue = sqlc.arg(queue) and job_set = sqlc.arg(job_set) and job_id = ANY(sqlc.arg(job_ids)::text[]); +-- name: SelectInitialRuns :many +SELECT * FROM runs WHERE serial > $1 AND job_id = ANY(sqlc.arg(job_ids)::text[]) ORDER BY serial LIMIT $2; + -- name: SelectNewRuns :many SELECT * FROM runs WHERE serial > $1 ORDER BY serial LIMIT $2; diff --git a/internal/scheduler/mocks/job_repository.go b/internal/scheduler/mocks/job_repository.go index 898cbebe7f1..21dba092880 100644 --- a/internal/scheduler/mocks/job_repository.go +++ b/internal/scheduler/mocks/job_repository.go @@ -52,6 +52,22 @@ func (mr *MockJobRepositoryMockRecorder) CountReceivedPartitions(arg0, arg1 inte return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountReceivedPartitions", reflect.TypeOf((*MockJobRepository)(nil).CountReceivedPartitions), arg0, arg1) } +// FetchInitialJobs mocks base method. +func (m *MockJobRepository) FetchInitialJobs(arg0 *armadacontext.Context) ([]database.Job, []database.Run, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FetchInitialJobs", arg0) + ret0, _ := ret[0].([]database.Job) + ret1, _ := ret[1].([]database.Run) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// FetchInitialJobs indicates an expected call of FetchInitialJobs. +func (mr *MockJobRepositoryMockRecorder) FetchInitialJobs(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchInitialJobs", reflect.TypeOf((*MockJobRepository)(nil).FetchInitialJobs), arg0) +} + // FetchJobRunErrors mocks base method. func (m *MockJobRepository) FetchJobRunErrors(arg0 *armadacontext.Context, arg1 []string) (map[string]*armadaevents.Error, error) { m.ctrl.T.Helper() diff --git a/internal/scheduler/scheduler.go b/internal/scheduler/scheduler.go index f60ee6772e8..e5f86f495f5 100644 --- a/internal/scheduler/scheduler.go +++ b/internal/scheduler/scheduler.go @@ -230,7 +230,7 @@ func (s *Scheduler) cycle(ctx *armadacontext.Context, updateAll bool, leaderToke // Update job state. ctx.Info("Syncing internal state with database") - updatedJobs, jsts, err := s.syncState(ctx) + updatedJobs, jsts, err := s.syncState(ctx, false) if err != nil { return overallSchedulerResult, err } @@ -357,12 +357,21 @@ func (s *Scheduler) cycle(ctx *armadacontext.Context, updateAll bool, leaderToke } // syncState updates jobs in jobDb to match state in postgres and returns all updated jobs. -func (s *Scheduler) syncState(ctx *armadacontext.Context) ([]*jobdb.Job, []jobdb.JobStateTransitions, error) { +func (s *Scheduler) syncState(ctx *armadacontext.Context, initial bool) ([]*jobdb.Job, []jobdb.JobStateTransitions, error) { txn := s.jobDb.WriteTxn() defer txn.Abort() - // Load new and updated jobs from the jobRepo. - updatedJobs, updatedRuns, err := s.jobRepository.FetchJobUpdates(ctx, s.jobsSerial, s.runsSerial) + var updatedJobs []database.Job + var updatedRuns []database.Run + var err error + + if initial { + // Load initial jobs from the jobRepo. + updatedJobs, updatedRuns, err = s.jobRepository.FetchInitialJobs(ctx) + } else { + // Load new and updated jobs from the jobRepo. + updatedJobs, updatedRuns, err = s.jobRepository.FetchJobUpdates(ctx, s.jobsSerial, s.runsSerial) + } if err != nil { return nil, nil, err } @@ -966,7 +975,7 @@ func (s *Scheduler) initialise(ctx *armadacontext.Context) error { case <-ctx.Done(): return nil default: - if _, _, err := s.syncState(ctx); err != nil { + if _, _, err := s.syncState(ctx, true); err != nil { logging.WithStacktrace(ctx, err).Error("failed to initialise; trying again in 1 second") time.Sleep(1 * time.Second) } else { diff --git a/internal/scheduler/scheduler_test.go b/internal/scheduler/scheduler_test.go index 177e85a377a..95dbab6fc8e 100644 --- a/internal/scheduler/scheduler_test.go +++ b/internal/scheduler/scheduler_test.go @@ -1039,6 +1039,143 @@ func TestRun(t *testing.T) { cancel() } +func TestScheduler_TestSyncInitialState(t *testing.T) { + tests := map[string]struct { + initialJobs []database.Job /// jobs in the jobdb at the start of the cycle + initialJobRuns []database.Run // jobs runs in the jobdb at the start of the cycle + expectedInitialJobs []*jobdb.Job + expectedInitialJobDbIds []string + expectedJobsSerial int64 + expectedRunsSerial int64 + }{ + "no initial jobs": { + initialJobs: []database.Job{}, + initialJobRuns: []database.Run{}, + expectedInitialJobs: []*jobdb.Job{}, + expectedInitialJobDbIds: []string{}, + expectedJobsSerial: -1, + expectedRunsSerial: -1, + }, + "initial jobs are present": { + initialJobs: []database.Job{ + { + JobID: queuedJob.Id(), + JobSet: queuedJob.Jobset(), + Queue: queuedJob.Queue(), + Queued: false, + QueuedVersion: 1, + Priority: int64(queuedJob.Priority()), + SchedulingInfo: schedulingInfoBytes, + Serial: 1, + Validated: true, + Submitted: 1, + }, + }, + initialJobRuns: []database.Run{ + { + RunID: leasedJob.LatestRun().Id(), + JobID: queuedJob.Id(), + JobSet: queuedJob.Jobset(), + Executor: "test-executor", + Node: "test-node", + Created: 123, + ScheduledAtPriority: func() *int32 { + scheduledAtPriority := int32(5) + return &scheduledAtPriority + }(), + Serial: 1, + }, + }, + expectedInitialJobs: []*jobdb.Job{ + queuedJob.WithUpdatedRun( + testfixtures.JobDb.CreateRun( + leasedJob.LatestRun().Id(), + queuedJob.Id(), + 123, + "test-executor", + "test-executor-test-node", + "test-node", + "pool", + pointer.Int32(5), + false, + false, + false, + false, + false, + false, + false, + false, + nil, + nil, + nil, + nil, + nil, + false, + false, + )).WithQueued(false).WithQueuedVersion(1), + }, + expectedInitialJobDbIds: []string{queuedJob.Id()}, + expectedJobsSerial: 1, + expectedRunsSerial: 1, + }, + } + for name, tc := range tests { + t.Run(name, func(t *testing.T) { + ctx, cancel := armadacontext.WithTimeout(armadacontext.Background(), 5*time.Second) + defer cancel() + + // Test objects + jobRepo := &testJobRepository{ + initialJobs: tc.initialJobs, + initialRuns: tc.initialJobRuns, + } + schedulingAlgo := &testSchedulingAlgo{} + publisher := &testPublisher{} + clusterRepo := &testExecutorRepository{} + leaderController := leader.NewStandaloneLeaderController() + sched, err := NewScheduler( + testfixtures.NewJobDb(testfixtures.TestResourceListFactory), + jobRepo, + clusterRepo, + schedulingAlgo, + leaderController, + publisher, + nil, + 1*time.Second, + 5*time.Second, + 1*time.Hour, + maxNumberOfAttempts, + nodeIdLabel, + schedulerMetrics, + ) + require.NoError(t, err) + sched.EnableAssertions() + + // The SchedulingKeyGenerator embedded in the jobDb has some randomness, + // which must be consistent within tests. + sched.jobDb = testfixtures.NewJobDb(testfixtures.TestResourceListFactory) + + initialJobs, _, err := sched.syncState(ctx, true) + require.NoError(t, err) + + expectedJobDb := testfixtures.NewJobDbWithJobs(tc.expectedInitialJobs) + actualJobDb := testfixtures.NewJobDbWithJobs(initialJobs) + assert.NoError(t, expectedJobDb.ReadTxn().AssertEqual(actualJobDb.ReadTxn())) + allDbJobs := sched.jobDb.ReadTxn().GetAll() + + expectedIds := stringSet(tc.expectedInitialJobDbIds) + require.Equal(t, len(tc.expectedInitialJobDbIds), len(allDbJobs)) + for _, job := range allDbJobs { + _, ok := expectedIds[job.Id()] + assert.True(t, ok) + } + + require.Equal(t, tc.expectedJobsSerial, sched.jobsSerial) + require.Equal(t, tc.expectedRunsSerial, sched.runsSerial) + }) + } +} + func TestScheduler_TestSyncState(t *testing.T) { tests := map[string]struct { initialJobs []*jobdb.Job // jobs in the jobdb at the start of the cycle @@ -1232,7 +1369,7 @@ func TestScheduler_TestSyncState(t *testing.T) { require.NoError(t, err) txn.Commit() - updatedJobs, _, err := sched.syncState(ctx) + updatedJobs, _, err := sched.syncState(ctx, false) require.NoError(t, err) expectedJobDb := testfixtures.NewJobDbWithJobs(tc.expectedUpdatedJobs) @@ -1268,6 +1405,8 @@ func (t *testSubmitChecker) Check(_ *armadacontext.Context, jobs []*jobdb.Job) ( // Test implementations of the interfaces needed by the Scheduler type testJobRepository struct { + initialJobs []database.Job + initialRuns []database.Run updatedJobs []database.Job updatedRuns []database.Run errors map[string]*armadaevents.Error @@ -1306,6 +1445,13 @@ func (t *testJobRepository) CountReceivedPartitions(ctx *armadacontext.Context, return t.numReceivedPartitions, nil } +func (t *testJobRepository) FetchInitialJobs(ctx *armadacontext.Context) ([]database.Job, []database.Run, error) { + if t.shouldError { + return nil, nil, errors.New("error fetching job updates") + } + return t.initialJobs, t.initialRuns, nil +} + type testExecutorRepository struct { updateTimes map[string]time.Time shouldError bool diff --git a/magefiles/tests.go b/magefiles/tests.go index a30ba7b51ad..e4460c8eee0 100644 --- a/magefiles/tests.go +++ b/magefiles/tests.go @@ -17,6 +17,11 @@ var Gotestsum string var LocalBin = filepath.Join(os.Getenv("PWD"), "/bin") +var ( + redisImage = "redis:6.2.6" + postgresImage = "postgres:14.2" +) + func makeLocalBin() error { if _, err := os.Stat(LocalBin); os.IsNotExist(err) { err = os.MkdirAll(LocalBin, os.ModePerm) @@ -52,12 +57,22 @@ func Tests() error { return err } - err = dockerRun("run", "-d", "--name=redis", docker_Net, "-p=6379:6379", "redis:6.2.6") + redisArgs := []string{"run", "-d", "--name=redis", "-p=6379:6379"} + if len(docker_Net) > 0 { + redisArgs = append(redisArgs, docker_Net) + } + redisArgs = append(redisArgs, redisImage) + err = dockerRun(redisArgs...) if err != nil { return err } - err = dockerRun("run", "-d", "--name=postgres", docker_Net, "-p", "5432:5432", "-e", "POSTGRES_PASSWORD=psw", "postgres:14.2") + postgresArgs := []string{"run", "-d", "--name=postgres", "-p", "5432:5432", "-e", "POSTGRES_PASSWORD=psw"} + if len(docker_Net) > 0 { + postgresArgs = append(postgresArgs, docker_Net) + } + postgresArgs = append(postgresArgs, postgresImage) + err = dockerRun(postgresArgs...) if err != nil { return err } From 133c490d4bbba50f61a58b1fb2fde5a656b3ffd1 Mon Sep 17 00:00:00 2001 From: Dejan Zele Pejchev Date: Thu, 31 Oct 2024 15:18:24 +0100 Subject: [PATCH 8/9] document TrackedNodeLabels and ToleratedTaints executor config fields (#4030) --- internal/executor/configuration/types.go | 25 +++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/internal/executor/configuration/types.go b/internal/executor/configuration/types.go index 96bd3febf81..42bbd05ed9e 100644 --- a/internal/executor/configuration/types.go +++ b/internal/executor/configuration/types.go @@ -12,6 +12,8 @@ import ( ) type ApplicationConfiguration struct { + // ClusterId is the unique identifier for the cluster that the executor is running on. + // It is used to identify the cluster in the scheduler. ClusterId string Pool string SubmitConcurrencyLimit int @@ -50,18 +52,23 @@ type ClientConfiguration struct { } type KubernetesConfiguration struct { - // Wether to impersonate users when creating Kubernetes objects. + // Whether to impersonate users when creating Kubernetes objects. ImpersonateUsers bool // Max number of Kubernetes API queries per second // and max number of concurrent Kubernetes API queries. - QPS float32 - Burst int - Etcd EtcdConfiguration - NodePoolLabel string - NodeTypeLabel string - NodeIdLabel string - TrackedNodeLabels []string - AvoidNodeLabelsOnRetry []string + QPS float32 + Burst int + Etcd EtcdConfiguration + NodePoolLabel string + NodeTypeLabel string + NodeIdLabel string + // TrackedNodeLabels is a list of node labels that the executor should index and track. + // As nodes can have many labels, taking all of them into consideration can be slow down scheduling. + // Only node labels defined in this list can be referenced in Armada job nodeSelector field. + TrackedNodeLabels []string + AvoidNodeLabelsOnRetry []string + // ToleratedTaints specifies taints which are tolerated by the executor. + // If a node has a taint that is not in this list, the executor will consider it for scheduling Armada jobs. ToleratedTaints []string MinimumPodAge time.Duration StuckTerminatingPodExpiry time.Duration From 77f8a0ad5a7afc35b2a8d7710a59e6443ec6b2a7 Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 31 Oct 2024 16:21:12 +0000 Subject: [PATCH 9/9] [Simulator] Allow JobTemplates To Be Repeated (#4031) * [Simulator] Allow JobTemplates To Be Repeated (#267) * wip * wip * wip * F/chrisma/simulator repeated submission (#269) * wip * wip * wip * lint * Update simulator.go --------- Co-authored-by: Christopher Martin --- internal/scheduler/simulator/simulator.go | 33 ++ internal/scheduler/simulator/simulator.pb.go | 476 +++++++++++++++---- internal/scheduler/simulator/simulator.proto | 9 + 3 files changed, 420 insertions(+), 98 deletions(-) diff --git a/internal/scheduler/simulator/simulator.go b/internal/scheduler/simulator/simulator.go index 2dde5298010..d581450aa00 100644 --- a/internal/scheduler/simulator/simulator.go +++ b/internal/scheduler/simulator/simulator.go @@ -10,6 +10,7 @@ import ( "sync/atomic" "time" + "github.com/gogo/protobuf/proto" "github.com/pkg/errors" "golang.org/x/exp/maps" "golang.org/x/exp/slices" @@ -143,6 +144,7 @@ func NewSimulator( if err := validateWorkloadSpec(workloadSpec); err != nil { return nil, err } + workloadSpec = expandRepeatingTemplates(workloadSpec) jobDb := jobdb.NewJobDb( schedulingConfig.PriorityClasses, schedulingConfig.DefaultPriorityClassName, @@ -292,6 +294,14 @@ func validateWorkloadSpec(workloadSpec *WorkloadSpec) error { if template.GangCardinality != 0 && int(template.Number)%int(template.GangCardinality) != 0 { return errors.Errorf("template.Number [%d] is not exactly divisible by template.GangCardinality [%d]", template.Number, template.GangCardinality) } + if template.Repeat != nil { + if template.Repeat.Period == nil { + return errors.Errorf("template.Repeat.Period is unset") + } + if template.Repeat.NumTimes < 1 { + return errors.Errorf("template.Repeat.NumTimes must be greater than 0") + } + } } } return nil @@ -1035,3 +1045,26 @@ func (s *Simulator) removeJobFromDemand(job *jobdb.Job) { r.SubV1ResourceList(job.PodRequirements().ResourceRequirements.Requests) } } + +func expandRepeatingTemplates(w *WorkloadSpec) *WorkloadSpec { + workload := proto.Clone(w).(*WorkloadSpec) + for _, q := range workload.GetQueues() { + var templates []*JobTemplate + for _, template := range q.GetJobTemplates() { + if template.Repeat != nil { + period := *template.Repeat.Period + for i := 0; i < int(template.Repeat.NumTimes); i++ { + t := proto.Clone(template).(*JobTemplate) + t.Repeat = nil + t.Id = fmt.Sprintf("%s-repeat-%d", t.Id, i) + t.EarliestSubmitTime = t.EarliestSubmitTime + time.Duration(i)*period + templates = append(templates, t) + } + } else { + templates = append(templates, template) + } + } + q.JobTemplates = templates + } + return workload +} diff --git a/internal/scheduler/simulator/simulator.pb.go b/internal/scheduler/simulator/simulator.pb.go index 472ca2cf906..22f78aea673 100644 --- a/internal/scheduler/simulator/simulator.pb.go +++ b/internal/scheduler/simulator/simulator.pb.go @@ -384,6 +384,8 @@ type JobTemplate struct { GangCardinality uint32 `protobuf:"varint,13,opt,name=gang_cardinality,json=gangCardinality,proto3" json:"gangCardinality,omitempty"` // Node Uniformity label when scheduling gangs. Only applies if gang_cardinality is non-zero. If unset it defaults to armadaproject.io/clusterName GangNodeUniformityLabel string `protobuf:"bytes,14,opt,name=gang_node_uniformity_label,json=gangNodeUniformityLabel,proto3" json:"gangNodeUniformityLabel,omitempty"` + // If set then the template will be repeated at some frequency. If null then the template will be submitted a single time. + Repeat *RepeatDetails `protobuf:"bytes,15,opt,name=repeat,proto3" json:"repeat,omitempty"` } func (m *JobTemplate) Reset() { *m = JobTemplate{} } @@ -517,6 +519,67 @@ func (m *JobTemplate) GetGangNodeUniformityLabel() string { return "" } +func (m *JobTemplate) GetRepeat() *RepeatDetails { + if m != nil { + return m.Repeat + } + return nil +} + +type RepeatDetails struct { + // The number of times that template should be repeated. Must be > 0 + NumTimes uint32 `protobuf:"varint,1,opt,name=num_times,json=numTimes,proto3" json:"numTimes,omitempty"` + // The period between template submissions. May not be null + Period *time.Duration `protobuf:"bytes,2,opt,name=period,proto3,stdduration" json:"period,omitempty"` +} + +func (m *RepeatDetails) Reset() { *m = RepeatDetails{} } +func (m *RepeatDetails) String() string { return proto.CompactTextString(m) } +func (*RepeatDetails) ProtoMessage() {} +func (*RepeatDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_63baccdfe9127510, []int{6} +} +func (m *RepeatDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RepeatDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_RepeatDetails.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 *RepeatDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_RepeatDetails.Merge(m, src) +} +func (m *RepeatDetails) XXX_Size() int { + return m.Size() +} +func (m *RepeatDetails) XXX_DiscardUnknown() { + xxx_messageInfo_RepeatDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_RepeatDetails proto.InternalMessageInfo + +func (m *RepeatDetails) GetNumTimes() uint32 { + if m != nil { + return m.NumTimes + } + return 0 +} + +func (m *RepeatDetails) GetPeriod() *time.Duration { + if m != nil { + return m.Period + } + return nil +} + type ShiftedExponential struct { Minimum time.Duration `protobuf:"bytes,1,opt,name=minimum,proto3,stdduration" json:"minimum"` TailMean time.Duration `protobuf:"bytes,2,opt,name=tail_mean,json=tailMean,proto3,stdduration" json:"tailMean"` @@ -526,7 +589,7 @@ func (m *ShiftedExponential) Reset() { *m = ShiftedExponential{} } func (m *ShiftedExponential) String() string { return proto.CompactTextString(m) } func (*ShiftedExponential) ProtoMessage() {} func (*ShiftedExponential) Descriptor() ([]byte, []int) { - return fileDescriptor_63baccdfe9127510, []int{6} + return fileDescriptor_63baccdfe9127510, []int{7} } func (m *ShiftedExponential) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -577,6 +640,7 @@ func init() { proto.RegisterMapType((map[string]string)(nil), "simulator.NodeTemplate.LabelsEntry") proto.RegisterType((*Queue)(nil), "simulator.Queue") proto.RegisterType((*JobTemplate)(nil), "simulator.JobTemplate") + proto.RegisterType((*RepeatDetails)(nil), "simulator.RepeatDetails") proto.RegisterType((*ShiftedExponential)(nil), "simulator.ShiftedExponential") } @@ -585,84 +649,89 @@ func init() { } var fileDescriptor_63baccdfe9127510 = []byte{ - // 1228 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x6e, 0x13, 0x47, - 0x18, 0xcf, 0xc6, 0xe0, 0xc4, 0x63, 0x27, 0x84, 0x49, 0x14, 0x8c, 0x11, 0x5e, 0x63, 0x54, 0xe4, - 0x56, 0x61, 0x2d, 0xa8, 0x54, 0x51, 0x54, 0x21, 0x75, 0x13, 0x10, 0x42, 0x40, 0xc1, 0xa1, 0x45, - 0x2a, 0x87, 0xd5, 0xec, 0xee, 0x17, 0x67, 0xe2, 0xdd, 0x1d, 0x33, 0x3b, 0x0b, 0xf5, 0xa1, 0xef, - 0xd0, 0x4b, 0x51, 0x1f, 0xa2, 0x97, 0x5e, 0x78, 0x06, 0x8e, 0x1c, 0x7b, 0xda, 0x56, 0x70, 0xdb, - 0x27, 0xe8, 0xb1, 0xda, 0xd9, 0x59, 0x67, 0x8c, 0x03, 0x24, 0x27, 0xfb, 0xfb, 0x7d, 0x7f, 0xf6, - 0x37, 0xdf, 0xbf, 0x19, 0xb4, 0x45, 0x23, 0x01, 0x3c, 0x22, 0x41, 0x3f, 0xf6, 0xf6, 0xc1, 0x4f, - 0x02, 0xe0, 0xfd, 0x98, 0x86, 0x49, 0x40, 0x04, 0xd3, 0xfe, 0x59, 0x63, 0xce, 0x04, 0xc3, 0xb5, - 0x29, 0xd0, 0x6a, 0x0f, 0x19, 0x1b, 0x06, 0xd0, 0x97, 0x0a, 0x37, 0xd9, 0xeb, 0xfb, 0x09, 0x27, - 0x82, 0xb2, 0xa8, 0x30, 0x6d, 0x75, 0x47, 0x37, 0x62, 0x8b, 0xb2, 0x3e, 0x19, 0xd3, 0xbe, 0xc7, - 0x38, 0xf4, 0x5f, 0x5c, 0xeb, 0x0f, 0x21, 0x02, 0x4e, 0x04, 0xf8, 0xca, 0xe6, 0xea, 0x90, 0x8a, - 0xfd, 0xc4, 0xb5, 0x3c, 0x16, 0xf6, 0x87, 0x6c, 0xc8, 0x0e, 0x83, 0xe5, 0x92, 0x14, 0xe4, 0x3f, - 0x65, 0x7e, 0xf3, 0x28, 0xae, 0xe5, 0x3f, 0xe6, 0x1e, 0x80, 0x27, 0xe2, 0x39, 0xa0, 0xf0, 0xed, - 0xfe, 0x5e, 0x41, 0xf5, 0xed, 0x20, 0x89, 0x05, 0xf0, 0xdd, 0x31, 0x78, 0xf8, 0x0a, 0x3a, 0x15, - 0x91, 0x10, 0x9a, 0x46, 0xc7, 0xe8, 0xd5, 0x6c, 0x9c, 0xa5, 0xe6, 0x6a, 0x2e, 0x6f, 0xb1, 0x90, - 0x0a, 0x08, 0xc7, 0x62, 0x32, 0x90, 0x7a, 0x7c, 0x07, 0x2d, 0x7b, 0x85, 0x5b, 0xdc, 0x5c, 0xec, - 0x54, 0x7a, 0xf5, 0xeb, 0xd8, 0x3a, 0xcc, 0x8a, 0x8a, 0x68, 0x6f, 0x66, 0xa9, 0x89, 0x4b, 0x3b, - 0x2d, 0xc6, 0xd4, 0x17, 0xbf, 0x32, 0xd0, 0xe5, 0x97, 0x8c, 0x8f, 0xf6, 0x02, 0xf6, 0xd2, 0x09, - 0x49, 0x44, 0x86, 0xc0, 0x1d, 0x1f, 0x02, 0x32, 0x71, 0x7c, 0x1a, 0x0b, 0x4e, 0xdd, 0x24, 0x4f, - 0x5e, 0xb3, 0xd2, 0x31, 0x7a, 0xf5, 0xeb, 0x17, 0xb5, 0x6f, 0xec, 0xee, 0xd3, 0x3d, 0x01, 0xfe, - 0xed, 0x5f, 0xc6, 0x2c, 0x82, 0x48, 0x50, 0x12, 0xd8, 0xbd, 0x37, 0xa9, 0xb9, 0x90, 0xa5, 0x66, - 0xa7, 0x8c, 0xf8, 0xa0, 0x08, 0xb8, 0x93, 0xc7, 0xdb, 0xd1, 0xc2, 0x0d, 0x3e, 0x6b, 0x81, 0x7f, - 0x45, 0xad, 0x31, 0x44, 0x3e, 0x8d, 0x86, 0x47, 0xd1, 0x39, 0x75, 0x1c, 0x3a, 0x1d, 0x45, 0xa7, - 0xa9, 0x02, 0xcd, 0xd3, 0xf8, 0xa8, 0xa6, 0xfb, 0xda, 0x40, 0x4b, 0x2a, 0x8b, 0xc7, 0xae, 0xc9, - 0x15, 0x74, 0x6a, 0xcc, 0x58, 0x20, 0x73, 0xa5, 0xec, 0x72, 0x59, 0xb7, 0xcb, 0x65, 0xfc, 0x0c, - 0xad, 0x46, 0xcc, 0x07, 0x27, 0x07, 0x03, 0x22, 0xa0, 0xac, 0xe0, 0x39, 0xed, 0x38, 0x0f, 0x99, - 0x0f, 0x4f, 0x94, 0xde, 0xbe, 0x90, 0xa5, 0xe6, 0xb9, 0x48, 0x43, 0xf4, 0x5a, 0xae, 0xcc, 0x28, - 0xba, 0x7f, 0x19, 0xa8, 0xf1, 0x94, 0xf1, 0x51, 0xc0, 0x88, 0x7f, 0xa2, 0x8e, 0xfa, 0x16, 0xd5, - 0x39, 0x89, 0x7c, 0x16, 0x3a, 0x31, 0x80, 0xdf, 0x5c, 0xec, 0x18, 0xbd, 0x8a, 0xdd, 0xcc, 0x52, - 0x73, 0xa3, 0x80, 0x77, 0x01, 0x7c, 0xcd, 0x09, 0x1d, 0xa2, 0xf8, 0x16, 0xaa, 0x3e, 0x4f, 0x20, - 0x81, 0xb8, 0x59, 0x91, 0x07, 0x59, 0xd3, 0x0e, 0xf2, 0x38, 0x57, 0xd8, 0x1b, 0x59, 0x6a, 0xae, - 0x15, 0x36, 0x5a, 0x0c, 0xe5, 0xd5, 0x7d, 0x55, 0x41, 0x0d, 0xfd, 0xc0, 0x78, 0x0b, 0x55, 0xa3, - 0x24, 0x74, 0x81, 0x4b, 0xd6, 0x95, 0xc2, 0xbd, 0x40, 0x74, 0xf7, 0x02, 0xc1, 0xdf, 0xa3, 0xaa, - 0x20, 0x34, 0x12, 0x65, 0x1e, 0xcf, 0x5b, 0xc5, 0x8c, 0x5b, 0x64, 0x4c, 0xad, 0x7c, 0xc6, 0xad, - 0x17, 0xd7, 0xac, 0x27, 0xb9, 0x85, 0xbd, 0xaa, 0x5a, 0x42, 0x39, 0x0c, 0xd4, 0x2f, 0x7e, 0x8c, - 0xaa, 0x01, 0x71, 0x21, 0x28, 0x4f, 0x70, 0xf9, 0x23, 0xa5, 0xb0, 0xee, 0x4b, 0xab, 0xdb, 0x91, - 0xe0, 0x93, 0x82, 0x55, 0xe1, 0xa6, 0xb3, 0x2a, 0x10, 0xec, 0xa0, 0x33, 0x82, 0x09, 0x12, 0x38, - 0x1c, 0x62, 0x96, 0x70, 0x0f, 0x62, 0xd5, 0xb5, 0x6d, 0x6b, 0x6e, 0x17, 0x0c, 0x94, 0xc9, 0x7d, - 0x1a, 0x0b, 0x7b, 0x53, 0x71, 0x5c, 0x95, 0xee, 0xa5, 0x2a, 0x1e, 0x7c, 0x20, 0xb7, 0x08, 0xaa, - 0x6b, 0x6c, 0xf0, 0x65, 0x54, 0x19, 0xc1, 0x44, 0x95, 0xf9, 0x6c, 0x96, 0x9a, 0x2b, 0x23, 0x98, - 0x68, 0xbc, 0x72, 0x2d, 0xfe, 0x12, 0x9d, 0x7e, 0x41, 0x82, 0x04, 0x64, 0x79, 0x6b, 0xf6, 0x7a, - 0x96, 0x9a, 0x67, 0x24, 0xa0, 0x19, 0x16, 0x16, 0x37, 0x17, 0x6f, 0x18, 0xf9, 0x14, 0x9c, 0x96, - 0x05, 0x3c, 0x76, 0x17, 0x6d, 0xa1, 0xea, 0x4b, 0xa0, 0xc3, 0x7d, 0x21, 0xbf, 0x60, 0x14, 0x39, - 0x2a, 0x10, 0x3d, 0x47, 0x05, 0x82, 0x9f, 0xa2, 0x95, 0x03, 0xe6, 0x6a, 0x83, 0x50, 0x64, 0x7f, - 0x53, 0xcb, 0xfe, 0x3d, 0xe6, 0x4e, 0xe7, 0xa0, 0x95, 0xa5, 0xe6, 0xe6, 0xc1, 0x21, 0xa0, 0xa7, - 0xbd, 0xa1, 0xe3, 0xdd, 0xff, 0x96, 0x51, 0x5d, 0xf3, 0x3c, 0x61, 0x43, 0xdd, 0x43, 0x4a, 0xb7, - 0x9b, 0x78, 0x1e, 0xc4, 0xf1, 0x5e, 0x12, 0xa8, 0x79, 0x68, 0x67, 0xa9, 0xd9, 0xfa, 0x50, 0xa7, - 0x45, 0x98, 0xf3, 0xcb, 0x33, 0x2e, 0xbb, 0x5c, 0x6d, 0x05, 0x99, 0x71, 0x09, 0xe8, 0x19, 0x97, - 0x00, 0xee, 0xa0, 0x45, 0xea, 0xcb, 0x26, 0xa9, 0xd9, 0x6b, 0x59, 0x6a, 0x36, 0xa8, 0x3e, 0x70, - 0x8b, 0xd4, 0xc7, 0x57, 0xd1, 0x52, 0x9e, 0xaf, 0x18, 0x44, 0xf3, 0xb4, 0x34, 0x93, 0xe7, 0x38, - 0x60, 0xee, 0x2e, 0xcc, 0xa4, 0xb7, 0x40, 0xb0, 0x8d, 0x56, 0x65, 0x64, 0x67, 0xcc, 0x29, 0xe3, - 0x54, 0x4c, 0x9a, 0xd5, 0x8e, 0xd1, 0x5b, 0x29, 0xf6, 0x89, 0xd4, 0x3c, 0x52, 0x0a, 0x7d, 0x9f, - 0xcc, 0x28, 0xf0, 0x0f, 0x68, 0xbd, 0xf4, 0x76, 0xbc, 0x80, 0xc4, 0xb1, 0x23, 0xfb, 0x60, 0x49, - 0x7e, 0xde, 0xcc, 0x52, 0xf3, 0x42, 0xa9, 0xde, 0xce, 0xb5, 0x0f, 0x67, 0x9b, 0xe2, 0xec, 0x9c, - 0x12, 0x3f, 0x43, 0x0d, 0x0e, 0xcf, 0x13, 0xca, 0x21, 0x84, 0x7c, 0x66, 0x97, 0xe5, 0x50, 0x5c, - 0x9a, 0x1f, 0x8a, 0x47, 0xcc, 0x1f, 0x68, 0x86, 0xf6, 0x86, 0x9a, 0x8b, 0x19, 0xf7, 0xc1, 0x8c, - 0x84, 0x6f, 0xa1, 0x86, 0x0f, 0xf9, 0x52, 0x87, 0xc8, 0xa3, 0x10, 0x37, 0x6b, 0x9d, 0x4a, 0xaf, - 0x56, 0xf4, 0x8d, 0x8e, 0xeb, 0x7d, 0xa3, 0xe3, 0x78, 0x84, 0x36, 0x80, 0xf0, 0x80, 0x42, 0x2c, - 0x9c, 0x38, 0x71, 0x43, 0x2a, 0x1c, 0x41, 0x43, 0x68, 0x22, 0x49, 0xf2, 0xbc, 0x55, 0x3c, 0x2e, - 0xac, 0xf2, 0x3d, 0x60, 0xed, 0xa8, 0xc7, 0x85, 0xdd, 0x56, 0xe4, 0x70, 0xe9, 0xbe, 0x2b, 0xbd, - 0x9f, 0xd0, 0x10, 0xfe, 0xf8, 0xc7, 0x34, 0x06, 0x47, 0xe0, 0xf8, 0xb5, 0x81, 0xfa, 0x47, 0x7d, - 0xcd, 0xd9, 0xe3, 0x2c, 0x74, 0xa6, 0xbc, 0x26, 0x8e, 0xc7, 0xc2, 0x71, 0x00, 0xf2, 0xe2, 0xab, - 0x7f, 0x8e, 0xc8, 0x37, 0x8a, 0xc8, 0x57, 0xf3, 0x1f, 0xbc, 0xc3, 0x59, 0xb8, 0x33, 0x8d, 0xba, - 0x3d, 0x0d, 0x2a, 0x09, 0x9e, 0xc0, 0x1e, 0x87, 0x68, 0x83, 0x27, 0x91, 0xa4, 0x3a, 0x73, 0x2b, - 0x37, 0x8e, 0x73, 0x2b, 0x5f, 0x50, 0x04, 0xd7, 0x55, 0x88, 0x99, 0x0b, 0xf9, 0x28, 0x10, 0xdf, - 0x45, 0x6b, 0x43, 0x12, 0x0d, 0x1d, 0x8f, 0x70, 0x9f, 0x46, 0x24, 0xc8, 0x1b, 0x79, 0x45, 0x36, - 0xf2, 0xc5, 0x2c, 0x35, 0xcf, 0xe7, 0xba, 0xed, 0x43, 0x95, 0x56, 0xdb, 0x33, 0x1f, 0xa8, 0xb0, - 0x8b, 0x5a, 0x32, 0x92, 0xbc, 0x7e, 0x93, 0x88, 0xee, 0x31, 0x1e, 0xe6, 0x8d, 0x2d, 0x57, 0x76, - 0x73, 0x55, 0xf6, 0xf4, 0x17, 0x59, 0x6a, 0x5e, 0xca, 0xad, 0xf2, 0xbd, 0xff, 0xe3, 0xd4, 0x46, - 0xae, 0x5a, 0x2d, 0xf6, 0xb9, 0x8f, 0x98, 0x74, 0xff, 0x34, 0x10, 0x9e, 0x3f, 0x36, 0xbe, 0x8b, - 0x96, 0x42, 0x1a, 0xd1, 0x30, 0x09, 0xe5, 0x0a, 0xfa, 0x64, 0x0d, 0xd7, 0x55, 0x8a, 0x4a, 0x0f, - 0x59, 0xa0, 0x52, 0xc0, 0xf7, 0x51, 0x4d, 0x10, 0x1a, 0x38, 0x21, 0x90, 0x48, 0xae, 0xa5, 0x4f, - 0xc6, 0x2a, 0xa7, 0x66, 0x39, 0xf7, 0x79, 0x00, 0xa4, 0xa8, 0xf6, 0x54, 0xb2, 0x7f, 0x7a, 0xf3, - 0xae, 0x6d, 0xbc, 0x7d, 0xd7, 0x36, 0xfe, 0x7d, 0xd7, 0x36, 0x7e, 0x7b, 0xdf, 0x5e, 0x78, 0xfb, - 0xbe, 0xbd, 0xf0, 0xf7, 0xfb, 0xf6, 0xc2, 0xcf, 0xdf, 0x69, 0xaf, 0x60, 0xc2, 0x43, 0xe2, 0x93, - 0x31, 0x67, 0xf9, 0x68, 0x2a, 0xa9, 0xff, 0xa9, 0x27, 0xba, 0x5b, 0x95, 0x54, 0xbe, 0xfe, 0x3f, - 0x00, 0x00, 0xff, 0xff, 0xb2, 0x34, 0xbe, 0x7a, 0xc9, 0x0b, 0x00, 0x00, + // 1310 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x57, 0x41, 0x6f, 0x13, 0xc7, + 0x17, 0xcf, 0xc6, 0xe0, 0xe0, 0x49, 0x9c, 0x84, 0x49, 0x14, 0x16, 0x23, 0xbc, 0xc6, 0xe8, 0x8f, + 0xfc, 0xaf, 0xc2, 0x5a, 0x80, 0x54, 0x51, 0x54, 0x21, 0x75, 0x13, 0x10, 0xa2, 0x40, 0xc1, 0xa1, + 0x45, 0x2a, 0x87, 0xd5, 0xd8, 0xfb, 0xe2, 0x4c, 0xb2, 0xbb, 0x63, 0x66, 0x67, 0xa1, 0x3e, 0xf4, + 0x3b, 0xb4, 0x87, 0xa2, 0x7e, 0x88, 0x5e, 0x7a, 0xe1, 0xdc, 0x23, 0x47, 0x8e, 0x3d, 0x6d, 0x2b, + 0xb8, 0xed, 0xa7, 0xa8, 0x76, 0x66, 0xd6, 0x19, 0xe3, 0x00, 0xe1, 0x14, 0xcf, 0xef, 0xfd, 0xde, + 0xdb, 0xdf, 0xbc, 0x79, 0xef, 0xcd, 0x04, 0x6d, 0xd2, 0x58, 0x00, 0x8f, 0x49, 0xd8, 0x4d, 0x06, + 0x7b, 0x10, 0xa4, 0x21, 0xf0, 0x6e, 0x42, 0xa3, 0x34, 0x24, 0x82, 0x19, 0xbf, 0xdc, 0x11, 0x67, + 0x82, 0xe1, 0xda, 0x04, 0x68, 0x34, 0x87, 0x8c, 0x0d, 0x43, 0xe8, 0x4a, 0x43, 0x3f, 0xdd, 0xed, + 0x06, 0x29, 0x27, 0x82, 0xb2, 0x58, 0x51, 0x1b, 0xed, 0x83, 0xeb, 0x89, 0x4b, 0x59, 0x97, 0x8c, + 0x68, 0x77, 0xc0, 0x38, 0x74, 0x9f, 0x5f, 0xe9, 0x0e, 0x21, 0x06, 0x4e, 0x04, 0x04, 0x9a, 0x73, + 0x79, 0x48, 0xc5, 0x5e, 0xda, 0x77, 0x07, 0x2c, 0xea, 0x0e, 0xd9, 0x90, 0x1d, 0x06, 0x2b, 0x56, + 0x72, 0x21, 0x7f, 0x69, 0xfa, 0x8d, 0xa3, 0xb4, 0x96, 0xbf, 0x58, 0x7f, 0x1f, 0x06, 0x22, 0x99, + 0x01, 0x94, 0x6f, 0xfb, 0xb7, 0x0a, 0x5a, 0xdc, 0x0a, 0xd3, 0x44, 0x00, 0xdf, 0x19, 0xc1, 0x00, + 0x5f, 0x42, 0x27, 0x62, 0x12, 0x81, 0x6d, 0xb5, 0xac, 0x4e, 0xcd, 0xc3, 0x79, 0xe6, 0x2c, 0x17, + 0xeb, 0x4d, 0x16, 0x51, 0x01, 0xd1, 0x48, 0x8c, 0x7b, 0xd2, 0x8e, 0x6f, 0xa3, 0x53, 0x03, 0xe5, + 0x96, 0xd8, 0xf3, 0xad, 0x4a, 0x67, 0xf1, 0x2a, 0x76, 0x0f, 0xb3, 0xa2, 0x23, 0x7a, 0x1b, 0x79, + 0xe6, 0xe0, 0x92, 0x67, 0xc4, 0x98, 0xf8, 0xe2, 0x97, 0x16, 0xba, 0xf8, 0x82, 0xf1, 0x83, 0xdd, + 0x90, 0xbd, 0xf0, 0x23, 0x12, 0x93, 0x21, 0x70, 0x3f, 0x80, 0x90, 0x8c, 0xfd, 0x80, 0x26, 0x82, + 0xd3, 0x7e, 0x5a, 0x24, 0xcf, 0xae, 0xb4, 0xac, 0xce, 0xe2, 0xd5, 0xf3, 0xc6, 0x37, 0x76, 0xf6, + 0xe8, 0xae, 0x80, 0xe0, 0xd6, 0x4f, 0x23, 0x16, 0x43, 0x2c, 0x28, 0x09, 0xbd, 0xce, 0xeb, 0xcc, + 0x99, 0xcb, 0x33, 0xa7, 0x55, 0x46, 0xbc, 0xaf, 0x02, 0x6e, 0x17, 0xf1, 0xb6, 0x8d, 0x70, 0xbd, + 0x4f, 0x32, 0xf0, 0xcf, 0xa8, 0x31, 0x82, 0x38, 0xa0, 0xf1, 0xf0, 0x28, 0x39, 0x27, 0x8e, 0x23, + 0xa7, 0xa5, 0xe5, 0xd8, 0x3a, 0xd0, 0xac, 0x8c, 0x0f, 0x5a, 0xda, 0xaf, 0x2c, 0xb4, 0xa0, 0xb3, + 0x78, 0xec, 0x33, 0xb9, 0x84, 0x4e, 0x8c, 0x18, 0x0b, 0x65, 0xae, 0x34, 0xaf, 0x58, 0x9b, 0xbc, + 0x62, 0x8d, 0x9f, 0xa2, 0xe5, 0x98, 0x05, 0xe0, 0x17, 0x60, 0x48, 0x04, 0x94, 0x27, 0x78, 0xc6, + 0xd8, 0xce, 0x03, 0x16, 0xc0, 0x63, 0x6d, 0xf7, 0xce, 0xe5, 0x99, 0x73, 0x26, 0x36, 0x10, 0xf3, + 0x2c, 0xeb, 0x53, 0x86, 0xf6, 0x9f, 0x16, 0x5a, 0x7a, 0xc2, 0xf8, 0x41, 0xc8, 0x48, 0xf0, 0x59, + 0x15, 0xf5, 0x15, 0x5a, 0xe4, 0x24, 0x0e, 0x58, 0xe4, 0x27, 0x00, 0x81, 0x3d, 0xdf, 0xb2, 0x3a, + 0x15, 0xcf, 0xce, 0x33, 0x67, 0x5d, 0xc1, 0x3b, 0x00, 0x81, 0xe1, 0x84, 0x0e, 0x51, 0x7c, 0x13, + 0x55, 0x9f, 0xa5, 0x90, 0x42, 0x62, 0x57, 0xe4, 0x46, 0x56, 0x8d, 0x8d, 0x3c, 0x2a, 0x0c, 0xde, + 0x7a, 0x9e, 0x39, 0xab, 0x8a, 0x63, 0xc4, 0xd0, 0x5e, 0xed, 0x97, 0x15, 0xb4, 0x64, 0x6e, 0x18, + 0x6f, 0xa2, 0x6a, 0x9c, 0x46, 0x7d, 0xe0, 0x52, 0x75, 0x45, 0xb9, 0x2b, 0xc4, 0x74, 0x57, 0x08, + 0xfe, 0x06, 0x55, 0x05, 0xa1, 0xb1, 0x28, 0xf3, 0x78, 0xd6, 0x55, 0x3d, 0xee, 0x92, 0x11, 0x75, + 0x8b, 0x1e, 0x77, 0x9f, 0x5f, 0x71, 0x1f, 0x17, 0x0c, 0x6f, 0x59, 0x97, 0x84, 0x76, 0xe8, 0xe9, + 0xbf, 0xf8, 0x11, 0xaa, 0x86, 0xa4, 0x0f, 0x61, 0xb9, 0x83, 0x8b, 0x1f, 0x38, 0x0a, 0xf7, 0x9e, + 0x64, 0xdd, 0x8a, 0x05, 0x1f, 0x2b, 0x55, 0xca, 0xcd, 0x54, 0xa5, 0x10, 0xec, 0xa3, 0x15, 0xc1, + 0x04, 0x09, 0x7d, 0x0e, 0x09, 0x4b, 0xf9, 0x00, 0x12, 0x5d, 0xb5, 0x4d, 0x77, 0x66, 0x16, 0xf4, + 0x34, 0xe5, 0x1e, 0x4d, 0x84, 0xb7, 0xa1, 0x35, 0x2e, 0x4b, 0xf7, 0xd2, 0x94, 0xf4, 0xde, 0x5b, + 0x37, 0x08, 0x5a, 0x34, 0xd4, 0xe0, 0x8b, 0xa8, 0x72, 0x00, 0x63, 0x7d, 0xcc, 0xa7, 0xf3, 0xcc, + 0xa9, 0x1f, 0xc0, 0xd8, 0xd0, 0x55, 0x58, 0xf1, 0xff, 0xd1, 0xc9, 0xe7, 0x24, 0x4c, 0x41, 0x1e, + 0x6f, 0xcd, 0x5b, 0xcb, 0x33, 0x67, 0x45, 0x02, 0x06, 0x51, 0x31, 0x6e, 0xcc, 0x5f, 0xb7, 0x8a, + 0x2e, 0x38, 0x29, 0x0f, 0xf0, 0xd8, 0x55, 0xb4, 0x89, 0xaa, 0x2f, 0x80, 0x0e, 0xf7, 0x84, 0xfc, + 0x82, 0xa5, 0x72, 0xa4, 0x10, 0x33, 0x47, 0x0a, 0xc1, 0x4f, 0x50, 0x7d, 0x9f, 0xf5, 0x8d, 0x46, + 0x50, 0xd9, 0xdf, 0x30, 0xb2, 0x7f, 0x97, 0xf5, 0x27, 0x7d, 0xd0, 0xc8, 0x33, 0x67, 0x63, 0xff, + 0x10, 0x30, 0xd3, 0xbe, 0x64, 0xe2, 0xed, 0xbf, 0x6a, 0x68, 0xd1, 0xf0, 0xfc, 0xcc, 0x82, 0xba, + 0x8b, 0xb4, 0x6d, 0x27, 0x1d, 0x0c, 0x20, 0x49, 0x76, 0xd3, 0x50, 0xf7, 0x43, 0x33, 0xcf, 0x9c, + 0xc6, 0xfb, 0x36, 0x23, 0xc2, 0x8c, 0x5f, 0x91, 0x71, 0x59, 0xe5, 0x7a, 0x2a, 0xc8, 0x8c, 0x4b, + 0xc0, 0xcc, 0xb8, 0x04, 0x70, 0x0b, 0xcd, 0xd3, 0x40, 0x16, 0x49, 0xcd, 0x5b, 0xcd, 0x33, 0x67, + 0x89, 0x9a, 0x0d, 0x37, 0x4f, 0x03, 0x7c, 0x19, 0x2d, 0x14, 0xf9, 0x4a, 0x40, 0xd8, 0x27, 0x25, + 0x4d, 0xee, 0x63, 0x9f, 0xf5, 0x77, 0x60, 0x2a, 0xbd, 0x0a, 0xc1, 0x1e, 0x5a, 0x96, 0x91, 0xfd, + 0x11, 0xa7, 0x8c, 0x53, 0x31, 0xb6, 0xab, 0x2d, 0xab, 0x53, 0x57, 0xf3, 0x44, 0x5a, 0x1e, 0x6a, + 0x83, 0x39, 0x4f, 0xa6, 0x0c, 0xf8, 0x3b, 0xb4, 0x56, 0x7a, 0xfb, 0x83, 0x90, 0x24, 0x89, 0x2f, + 0xeb, 0x60, 0x41, 0x7e, 0xde, 0xc9, 0x33, 0xe7, 0x5c, 0x69, 0xde, 0x2a, 0xac, 0x0f, 0xa6, 0x8b, + 0xe2, 0xf4, 0x8c, 0x11, 0x3f, 0x45, 0x4b, 0x1c, 0x9e, 0xa5, 0x94, 0x43, 0x04, 0x45, 0xcf, 0x9e, + 0x92, 0x4d, 0x71, 0x61, 0xb6, 0x29, 0x1e, 0xb2, 0xa0, 0x67, 0x10, 0xbd, 0x75, 0xdd, 0x17, 0x53, + 0xee, 0xbd, 0xa9, 0x15, 0xbe, 0x89, 0x96, 0x02, 0x28, 0x86, 0x3a, 0xc4, 0x03, 0x0a, 0x89, 0x5d, + 0x6b, 0x55, 0x3a, 0x35, 0x55, 0x37, 0x26, 0x6e, 0xd6, 0x8d, 0x89, 0xe3, 0x03, 0xb4, 0x0e, 0x84, + 0x87, 0x14, 0x12, 0xe1, 0x27, 0x69, 0x3f, 0xa2, 0xc2, 0x17, 0x34, 0x02, 0x1b, 0x49, 0x91, 0x67, + 0x5d, 0xf5, 0xb8, 0x70, 0xcb, 0xf7, 0x80, 0xbb, 0xad, 0x1f, 0x17, 0x5e, 0x53, 0x8b, 0xc3, 0xa5, + 0xfb, 0x8e, 0xf4, 0x7e, 0x4c, 0x23, 0xf8, 0xfd, 0x1f, 0xc7, 0xea, 0x1d, 0x81, 0xe3, 0x57, 0x16, + 0xea, 0x1e, 0xf5, 0x35, 0x7f, 0x97, 0xb3, 0xc8, 0x9f, 0xe8, 0x1a, 0xfb, 0x03, 0x16, 0x8d, 0x42, + 0x90, 0x17, 0xdf, 0xe2, 0xa7, 0x84, 0x7c, 0xa9, 0x85, 0x7c, 0x31, 0xfb, 0xc1, 0xdb, 0x9c, 0x45, + 0xdb, 0x93, 0xa8, 0x5b, 0x93, 0xa0, 0x52, 0xe0, 0x67, 0xf0, 0x71, 0x84, 0xd6, 0x79, 0x1a, 0x4b, + 0xa9, 0x53, 0xb7, 0xf2, 0xd2, 0x71, 0x6e, 0xe5, 0x73, 0x5a, 0xe0, 0x9a, 0x0e, 0x31, 0x75, 0x21, + 0x1f, 0x05, 0xe2, 0x3b, 0x68, 0x75, 0x48, 0xe2, 0xa1, 0x3f, 0x20, 0x3c, 0xa0, 0x31, 0x09, 0x8b, + 0x42, 0xae, 0xcb, 0x42, 0x3e, 0x9f, 0x67, 0xce, 0xd9, 0xc2, 0xb6, 0x75, 0x68, 0x32, 0xce, 0x76, + 0xe5, 0x3d, 0x13, 0xee, 0xa3, 0x86, 0x8c, 0x24, 0xaf, 0xdf, 0x34, 0xa6, 0xbb, 0x8c, 0x47, 0x45, + 0x61, 0xcb, 0x91, 0x6d, 0x2f, 0xcb, 0x9a, 0xfe, 0x5f, 0x9e, 0x39, 0x17, 0x0a, 0x56, 0x31, 0xf7, + 0xbf, 0x9f, 0x70, 0xe4, 0xa8, 0x35, 0x62, 0x9f, 0xf9, 0x00, 0x05, 0xdf, 0x46, 0x55, 0x0e, 0x23, + 0x20, 0xc2, 0x5e, 0x91, 0xe9, 0xb0, 0x8d, 0x74, 0xf4, 0xa4, 0x61, 0x1b, 0x04, 0xa1, 0x61, 0xa2, + 0x9a, 0x57, 0x71, 0xcd, 0xe6, 0x55, 0x48, 0xfb, 0x57, 0x0b, 0xd5, 0xa7, 0xf8, 0xf8, 0x1a, 0xaa, + 0xc5, 0x69, 0x24, 0x4b, 0x24, 0x91, 0x73, 0xac, 0xae, 0x1e, 0x78, 0x71, 0x1a, 0x15, 0xc7, 0x35, + 0xf5, 0xc0, 0x2b, 0x31, 0xfc, 0x2d, 0xaa, 0x8e, 0x80, 0x53, 0xa6, 0x6e, 0xf4, 0x8f, 0x96, 0x4e, + 0x71, 0xd9, 0xaf, 0x2a, 0xf2, 0x61, 0x28, 0x59, 0x1c, 0x3a, 0x44, 0xfb, 0x0f, 0x0b, 0xe1, 0xd9, + 0x23, 0xc5, 0x77, 0xd0, 0x42, 0x44, 0x63, 0x1a, 0xa5, 0x91, 0x94, 0xf5, 0xd1, 0x8f, 0xac, 0xe9, + 0xe3, 0x2f, 0x3d, 0x64, 0xfc, 0x72, 0x81, 0xef, 0xa1, 0x5a, 0xb1, 0x57, 0x3f, 0x02, 0x12, 0x7f, + 0x5a, 0x70, 0x39, 0x11, 0x4e, 0x15, 0x3e, 0xf7, 0x81, 0xa8, 0x4a, 0x9e, 0xac, 0xbc, 0x1f, 0x5e, + 0xbf, 0x6d, 0x5a, 0x6f, 0xde, 0x36, 0xad, 0x7f, 0xdf, 0x36, 0xad, 0x5f, 0xde, 0x35, 0xe7, 0xde, + 0xbc, 0x6b, 0xce, 0xfd, 0xfd, 0xae, 0x39, 0xf7, 0xe3, 0xd7, 0xc6, 0x0b, 0x9f, 0xf0, 0x88, 0x04, + 0x64, 0xc4, 0x59, 0x31, 0x76, 0xf4, 0xaa, 0xfb, 0xb1, 0x7f, 0x3f, 0xfa, 0x55, 0x29, 0xe5, 0xda, + 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xb1, 0x00, 0x01, 0x6c, 0xa5, 0x0c, 0x00, 0x00, } func (m *ClusterSpec) Marshal() (dAtA []byte, err error) { @@ -970,6 +1039,18 @@ func (m *JobTemplate) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Repeat != nil { + { + size, err := m.Repeat.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSimulator(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x7a + } if len(m.GangNodeUniformityLabel) > 0 { i -= len(m.GangNodeUniformityLabel) copy(dAtA[i:], m.GangNodeUniformityLabel) @@ -992,21 +1073,21 @@ func (m *JobTemplate) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i-- dAtA[i] = 0x62 - n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.EarliestSubmitTimeFromDependencyCompletion, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTimeFromDependencyCompletion):]) - if err5 != nil { - return 0, err5 - } - i -= n5 - i = encodeVarintSimulator(dAtA, i, uint64(n5)) - i-- - dAtA[i] = 0x5a - n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.EarliestSubmitTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTime):]) + n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.EarliestSubmitTimeFromDependencyCompletion, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.EarliestSubmitTimeFromDependencyCompletion):]) 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 -= n7 + i = encodeVarintSimulator(dAtA, i, uint64(n7)) + i-- dAtA[i] = 0x52 if len(m.Dependencies) > 0 { for iNdEx := len(m.Dependencies) - 1; iNdEx >= 0; iNdEx-- { @@ -1073,6 +1154,44 @@ func (m *JobTemplate) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *RepeatDetails) 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 *RepeatDetails) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RepeatDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Period != nil { + n9, err9 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.Period, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.Period):]) + if err9 != nil { + return 0, err9 + } + i -= n9 + i = encodeVarintSimulator(dAtA, i, uint64(n9)) + i-- + dAtA[i] = 0x12 + } + if m.NumTimes != 0 { + i = encodeVarintSimulator(dAtA, i, uint64(m.NumTimes)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *ShiftedExponential) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1093,20 +1212,20 @@ func (m *ShiftedExponential) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n8, err8 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TailMean, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TailMean):]) - if err8 != nil { - return 0, err8 + n10, err10 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.TailMean, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.TailMean):]) + if err10 != nil { + return 0, err10 } - i -= n8 - i = encodeVarintSimulator(dAtA, i, uint64(n8)) + i -= n10 + i = encodeVarintSimulator(dAtA, i, uint64(n10)) i-- dAtA[i] = 0x12 - n9, err9 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Minimum, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Minimum):]) - if err9 != nil { - return 0, err9 + n11, err11 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Minimum, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Minimum):]) + if err11 != nil { + return 0, err11 } - i -= n9 - i = encodeVarintSimulator(dAtA, i, uint64(n9)) + i -= n11 + i = encodeVarintSimulator(dAtA, i, uint64(n11)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -1293,6 +1412,26 @@ func (m *JobTemplate) Size() (n int) { if l > 0 { n += 1 + l + sovSimulator(uint64(l)) } + if m.Repeat != nil { + l = m.Repeat.Size() + n += 1 + l + sovSimulator(uint64(l)) + } + return n +} + +func (m *RepeatDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NumTimes != 0 { + n += 1 + sovSimulator(uint64(m.NumTimes)) + } + if m.Period != nil { + l = github_com_gogo_protobuf_types.SizeOfStdDuration(*m.Period) + n += 1 + l + sovSimulator(uint64(l)) + } return n } @@ -2599,6 +2738,147 @@ func (m *JobTemplate) Unmarshal(dAtA []byte) error { } m.GangNodeUniformityLabel = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Repeat", 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.Repeat == nil { + m.Repeat = &RepeatDetails{} + } + if err := m.Repeat.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 *RepeatDetails) 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: RepeatDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RepeatDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NumTimes", wireType) + } + m.NumTimes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSimulator + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NumTimes |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Period", 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.Period == nil { + m.Period = new(time.Duration) + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(m.Period, 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 7aea2c46321..cc71e807855 100644 --- a/internal/scheduler/simulator/simulator.proto +++ b/internal/scheduler/simulator/simulator.proto @@ -79,6 +79,15 @@ message JobTemplate { uint32 gang_cardinality = 13; // Node Uniformity label when scheduling gangs. Only applies if gang_cardinality is non-zero. If unset it defaults to armadaproject.io/clusterName string gang_node_uniformity_label = 14; + // If set then the template will be repeated at some frequency. If null then the template will be submitted a single time. + RepeatDetails repeat = 15; +} + +message RepeatDetails { + // The number of times that template should be repeated. Must be > 0 + uint32 num_times = 1; + // The period between template submissions. May not be null + google.protobuf.Duration period = 2 [(gogoproto.stdduration) = true]; } message ShiftedExponential {