From 807569f8f00584ec626e4ab15257cdafc1e52132 Mon Sep 17 00:00:00 2001 From: Albin Severinson Date: Fri, 23 Jun 2023 10:20:30 +0100 Subject: [PATCH] Separate resource limits for each priority class (#2601) * Fix per-pc resource limits * Lint * Cleanup * Fix tests --- internal/armada/configuration/types.go | 6 +- internal/armada/server/lease.go | 209 +++++--- .../utilisation/cluster_utilisation.go | 4 +- internal/scheduler/common.go | 40 -- internal/scheduler/constraints/constraints.go | 30 +- internal/scheduler/context/context.go | 107 ++-- internal/scheduler/context/context_test.go | 6 +- internal/scheduler/gang_scheduler_test.go | 26 +- internal/scheduler/nodedb/nodedb_test.go | 4 +- .../scheduler/preempting_queue_scheduler.go | 4 +- .../preempting_queue_scheduler_test.go | 131 +++-- internal/scheduler/queue_scheduler_test.go | 87 +--- internal/scheduler/reports.go | 8 +- internal/scheduler/reports_test.go | 14 +- .../schedulerobjects/resourcelist.go | 94 ++-- .../schedulerobjects/resourcelist_test.go | 187 ++++--- .../schedulerobjects/schedulerobjects.pb.go | 483 ++++++++++++------ .../schedulerobjects/schedulerobjects.proto | 6 +- internal/scheduler/scheduling_algo.go | 62 +-- internal/scheduler/scheduling_algo_test.go | 38 +- .../scheduler/testfixtures/testfixtures.go | 18 +- 21 files changed, 867 insertions(+), 697 deletions(-) diff --git a/internal/armada/configuration/types.go b/internal/armada/configuration/types.go index 7c8da6afefc..dc93ae7ec4d 100644 --- a/internal/armada/configuration/types.go +++ b/internal/armada/configuration/types.go @@ -233,12 +233,8 @@ type PriorityClass struct { Priority int32 // If true, Armada may preempt jobs of this class to improve fairness. Preemptible bool - // Limits resources assigned to jobs of priority equal to or lower than that of this priority class. + // Limits resources assigned to jobs of this priority class. // Specifically, jobs of this priority class are only scheduled if doing so does not exceed this limit. - // - // For example, if priority is 10 and MaximumResourceFractionPerQueue is map[string]float64{"cpu": 0.3}, - // jobs of this priority class are not scheduled if doing so would cause the total resources assigned - // to jobs of priority 10 or lower from the same queue to exceed 30% of the total. MaximumResourceFractionPerQueue map[string]float64 // Per-pool override of MaximumResourceFractionPerQueue. // If missing for a particular pool, MaximumResourceFractionPerQueue is used instead for that pool. diff --git a/internal/armada/server/lease.go b/internal/armada/server/lease.go index a8559ea6932..572cc890bc7 100644 --- a/internal/armada/server/lease.go +++ b/internal/armada/server/lease.go @@ -40,6 +40,7 @@ import ( schedulerconstraints "github.com/armadaproject/armada/internal/scheduler/constraints" schedulercontext "github.com/armadaproject/armada/internal/scheduler/context" "github.com/armadaproject/armada/internal/scheduler/database" + "github.com/armadaproject/armada/internal/scheduler/interfaces" schedulerinterfaces "github.com/armadaproject/armada/internal/scheduler/interfaces" "github.com/armadaproject/armada/internal/scheduler/nodedb" "github.com/armadaproject/armada/internal/scheduler/schedulerobjects" @@ -282,10 +283,31 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL }) } + // Map queue names to priority factor for all active queues, i.e., + // all queues for which the jobs queue has not been deleted automatically by Redis. + queues, err := q.queueRepository.GetAllQueues() + if err != nil { + return nil, err + } + priorityFactorByQueue := make(map[string]float64, len(queues)) + apiQueues := make([]*api.Queue, len(queues)) + for i, queue := range queues { + priorityFactorByQueue[queue.Name] = float64(queue.PriorityFactor) + apiQueues[i] = &api.Queue{Name: queue.Name} + } + activeQueues, err := q.jobRepository.FilterActiveQueues(apiQueues) + if err != nil { + return nil, err + } + priorityFactorByActiveQueue := make(map[string]float64, len(activeQueues)) + for _, queue := range activeQueues { + priorityFactorByActiveQueue[queue.Name] = priorityFactorByQueue[queue.Name] + } + // Nodes to be considered by the scheduler. lastSeen := q.clock.Now() nodes := make([]*schedulerobjects.Node, 0, len(req.Nodes)) - allocatedByQueueForCluster := make(map[string]schedulerobjects.QuantityByPriorityAndResourceType) + allocatedByQueueAndPriorityClassForCluster := make(map[string]schedulerobjects.QuantityByTAndResourceType[string], len(queues)) jobIdsByGangId := make(map[string]map[string]bool) gangIdByJobId := make(map[string]string) nodeIdByJobId := make(map[string]string) @@ -332,11 +354,9 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL } // Aggregate total resources allocated by queue for this cluster. - allocatedByQueueForCluster = scheduler.UpdateUsage( - allocatedByQueueForCluster, - jobs, - q.schedulingConfig.Preemption.PriorityClasses, - scheduler.Add, + allocatedByQueueAndPriorityClassForCluster = updateAllocatedByQueueAndPriorityClass( + allocatedByQueueAndPriorityClassForCluster, + add, jobs, ) // Group gangs. @@ -398,30 +418,36 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL return nil, err } - // Load executor reports for all clusters, and insert an updated report for this cluster. + // Load allocation reports for all executors from Redis. reportsByExecutor, err := q.usageRepository.GetClusterQueueResourceUsage() if err != nil { return nil, err } - executorReport := &schedulerobjects.ClusterResourceUsageReport{ + + // Insert an updated report for the current executor, which includes information received in this lease call. + currentExecutorReport := &schedulerobjects.ClusterResourceUsageReport{ Pool: req.Pool, Created: q.clock.Now(), - ResourcesByQueue: make(map[string]*schedulerobjects.QueueClusterResourceUsage), + ResourcesByQueue: make(map[string]*schedulerobjects.QueueClusterResourceUsage, len(queues)), } - for queue, allocated := range allocatedByQueueForCluster { - executorReport.ResourcesByQueue[queue] = &schedulerobjects.QueueClusterResourceUsage{ - Created: executorReport.Created, - Queue: queue, - ExecutorId: req.ClusterId, - ResourcesByPriority: allocated.DeepCopy(), + for queue, allocatedByPriorityClass := range allocatedByQueueAndPriorityClassForCluster { + currentExecutorReport.ResourcesByQueue[queue] = &schedulerobjects.QueueClusterResourceUsage{ + Created: currentExecutorReport.Created, + Queue: queue, + ExecutorId: req.ClusterId, + ResourcesByPriorityClassName: armadamaps.DeepCopy(allocatedByPriorityClass), } } - reportsByExecutor[req.ClusterId] = executorReport - if err := q.usageRepository.UpdateClusterQueueResourceUsage(req.ClusterId, executorReport); err != nil { + reportsByExecutor[req.ClusterId] = currentExecutorReport + + // Write the updated report into Redis to make the information available to other replicas of the server. + if err := q.usageRepository.UpdateClusterQueueResourceUsage(req.ClusterId, currentExecutorReport); err != nil { return nil, errors.WithMessagef(err, "failed to update cluster usage for cluster %s", req.ClusterId) } - allocatedByQueueForPool := q.aggregateUsage(reportsByExecutor, req.Pool) - log.Infof("allocated resources per queue for pool %s before scheduling: %v", req.Pool, allocatedByQueueForPool) + + // Aggregate allocation across all clusters. + allocatedByQueueAndPriorityClassForPool := q.aggregateAllocationAcrossExecutor(reportsByExecutor, req.Pool) + log.Infof("allocated resources per queue for pool %s before scheduling: %v", req.Pool, allocatedByQueueAndPriorityClassForPool) // Store executor details in Redis so they can be used by submit checks and the new scheduler. if err := q.executorRepository.StoreExecutor(ctx, &schedulerobjects.Executor{ @@ -435,29 +461,7 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL log.WithError(err).Warnf("could not store executor details for cluster %s", req.ClusterId) } - // Map queue names to priority factor for all active queues, i.e., - // all queues for which the jobs queue has not been deleted automatically by Redis. - queues, err := q.queueRepository.GetAllQueues() - if err != nil { - return nil, err - } - priorityFactorByQueue := make(map[string]float64, len(queues)) - apiQueues := make([]*api.Queue, len(queues)) - for i, queue := range queues { - priorityFactorByQueue[queue.Name] = float64(queue.PriorityFactor) - apiQueues[i] = &api.Queue{Name: queue.Name} - } - activeQueues, err := q.jobRepository.FilterActiveQueues(apiQueues) - if err != nil { - return nil, err - } - priorityFactorByActiveQueue := make(map[string]float64, len(activeQueues)) - for _, queue := range activeQueues { - priorityFactorByActiveQueue[queue.Name] = priorityFactorByQueue[queue.Name] - } - - // Give Schedule() a 3 second shorter deadline than ctx, - // to give it a chance to finish up before ctx is cancelled. + // Give Schedule() a 3 second shorter deadline than ctx to give it a chance to finish up before ctx deadline. if deadline, ok := ctx.Deadline(); ok { var cancel context.CancelFunc ctx, cancel = context.WithDeadline(ctx, deadline.Add(-3*time.Second)) @@ -473,7 +477,7 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL schedulerobjects.ResourceList{Resources: totalCapacity}, ) for queue, priorityFactor := range priorityFactorByQueue { - if err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueForPool[queue]); err != nil { + if err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriorityClassForPool[queue]); err != nil { return nil, err } } @@ -629,38 +633,37 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL } // Update resource cluster report to account for preempted/leased jobs and write it to Redis. - allocatedByQueueForCluster = scheduler.UpdateUsage( - allocatedByQueueForCluster, - result.PreemptedJobs, - q.schedulingConfig.Preemption.PriorityClasses, - scheduler.Subtract, + allocatedByQueueAndPriorityClassForCluster = updateAllocatedByQueueAndPriorityClass( + allocatedByQueueAndPriorityClassForCluster, + subtract, result.PreemptedJobs, ) - for queue, m := range allocatedByQueueForCluster { + for queue, m := range allocatedByQueueAndPriorityClassForCluster { // Any quantity in the negative indicates a resource accounting problem. - if !m.IsStrictlyNonNegative() { - log.Errorf("unexpected negative resource quantity for queue %s: %v", queue, m) + for _, rl := range m { + if !rl.IsStrictlyNonNegative() { + return nil, errors.Errorf("unexpected negative resource quantity for queue %s: %v", queue, m) + } } } - allocatedByQueueForCluster = scheduler.UpdateUsage( - allocatedByQueueForCluster, - successfullyLeasedApiJobs, - q.schedulingConfig.Preemption.PriorityClasses, - scheduler.Add, + allocatedByQueueAndPriorityClassForCluster = updateAllocatedByQueueAndPriorityClass( + allocatedByQueueAndPriorityClassForCluster, + add, successfullyLeasedApiJobs, ) - executorReport.Created = q.clock.Now() - for queue, usage := range allocatedByQueueForCluster { - executorReport.ResourcesByQueue[queue] = &schedulerobjects.QueueClusterResourceUsage{ - Created: executorReport.Created, - Queue: queue, - ExecutorId: req.ClusterId, - ResourcesByPriority: usage.DeepCopy(), + currentExecutorReport.Created = q.clock.Now() + for queue, usage := range allocatedByQueueAndPriorityClassForCluster { + currentExecutorReport.ResourcesByQueue[queue] = &schedulerobjects.QueueClusterResourceUsage{ + Created: currentExecutorReport.Created, + Queue: queue, + ExecutorId: req.ClusterId, + ResourcesByPriorityClassName: armadamaps.DeepCopy(usage), } } - if err := q.usageRepository.UpdateClusterQueueResourceUsage(req.ClusterId, executorReport); err != nil { + if err := q.usageRepository.UpdateClusterQueueResourceUsage(req.ClusterId, currentExecutorReport); err != nil { logging.WithStacktrace(log, err).Errorf("failed to update cluster usage") } - allocatedByQueueForPool = q.aggregateUsage(reportsByExecutor, req.Pool) - log.Infof("allocated resources per queue for pool %s after scheduling: %v", req.Pool, allocatedByQueueForPool) + + allocatedByQueueAndPriorityClassForPool = q.aggregateAllocationAcrossExecutor(reportsByExecutor, req.Pool) + log.Infof("allocated resources per queue for pool %s after scheduling: %v", req.Pool, allocatedByQueueAndPriorityClassForPool) // Optionally set node id selectors on scheduled jobs. if q.schedulingConfig.Preemption.SetNodeIdSelector { @@ -742,31 +745,69 @@ func (q *AggregatedQueueServer) getJobs(ctx context.Context, req *api.StreamingL return successfullyLeasedApiJobs, nil } -// aggregateUsage Creates a map of resource usage first by cluster and then by queue. -// Clusters in pools other than pool are excluded. -func (q *AggregatedQueueServer) aggregateUsage(reportsByCluster map[string]*schedulerobjects.ClusterResourceUsageReport, pool string) map[string]schedulerobjects.QuantityByPriorityAndResourceType { - const activeClusterExpiry = 10 * time.Minute +type addOrSubtract int + +const ( + add addOrSubtract = iota + subtract +) + +func updateAllocatedByQueueAndPriorityClass[T interfaces.LegacySchedulerJob]( + allocatedByQueueAndPriorityClass map[string]schedulerobjects.QuantityByTAndResourceType[string], + op addOrSubtract, + jobs []T, +) map[string]schedulerobjects.QuantityByTAndResourceType[string] { + if allocatedByQueueAndPriorityClass == nil { + allocatedByQueueAndPriorityClass = make(map[string]schedulerobjects.QuantityByTAndResourceType[string], 256) + } + for _, job := range jobs { + allocatedByPriorityClassName := allocatedByQueueAndPriorityClass[job.GetQueue()] + if allocatedByPriorityClassName == nil { + allocatedByPriorityClassName = make(map[string]schedulerobjects.ResourceList) + allocatedByQueueAndPriorityClass[job.GetQueue()] = allocatedByPriorityClassName + } + allocated := allocatedByPriorityClassName[job.GetPriorityClassName()] + if op == add { + allocated.AddV1ResourceList(job.GetResourceRequirements().Requests) + } else if op == subtract { + allocated.SubV1ResourceList(job.GetResourceRequirements().Requests) + } else { + panic(fmt.Sprintf("unknown op %d", op)) + } + allocatedByPriorityClassName[job.GetPriorityClassName()] = allocated + } + return allocatedByQueueAndPriorityClass +} + +func (q *AggregatedQueueServer) aggregateAllocationAcrossExecutor(reportsByExecutor map[string]*schedulerobjects.ClusterResourceUsageReport, pool string) map[string]schedulerobjects.QuantityByTAndResourceType[string] { now := q.clock.Now() - aggregatedUsageByQueue := make(map[string]schedulerobjects.QuantityByPriorityAndResourceType) - for _, clusterReport := range reportsByCluster { - if clusterReport.Pool != pool { - // Separate resource accounting per pool. + allocatedByQueueAndPriorityClass := make(map[string]schedulerobjects.QuantityByTAndResourceType[string]) + for _, executorReport := range reportsByExecutor { + if executorReport.Pool != pool { + // Only consider executors in the specified pool. continue } - if !clusterReport.Created.Add(activeClusterExpiry).After(now) { - // Stale report; omit. - continue + if q.schedulingConfig.ExecutorTimeout != 0 { + reportAge := now.Sub(executorReport.Created) + if reportAge > q.schedulingConfig.ExecutorTimeout { + // Stale report; omit. + continue + } } - for queue, report := range clusterReport.ResourcesByQueue { - quantityByPriorityAndResourceType, ok := aggregatedUsageByQueue[queue] - if !ok { - quantityByPriorityAndResourceType = make(schedulerobjects.QuantityByPriorityAndResourceType) - aggregatedUsageByQueue[queue] = quantityByPriorityAndResourceType + for queue, queueReport := range executorReport.ResourcesByQueue { + allocatedByPriorityClass := allocatedByQueueAndPriorityClass[queue] + if allocatedByPriorityClass == nil { + allocatedByPriorityClass = make(map[string]schedulerobjects.ResourceList) + allocatedByQueueAndPriorityClass[queue] = allocatedByPriorityClass + } + for priorityClassName, allocated := range queueReport.ResourcesByPriorityClassName { + rl := allocatedByPriorityClass[priorityClassName] + rl.Add(allocated) + allocatedByPriorityClass[priorityClassName] = rl } - quantityByPriorityAndResourceType.Add(report.ResourcesByPriority) } } - return aggregatedUsageByQueue + return allocatedByQueueAndPriorityClass } func (q *AggregatedQueueServer) decompressJobOwnershipGroups(jobs []*api.Job) error { diff --git a/internal/executor/utilisation/cluster_utilisation.go b/internal/executor/utilisation/cluster_utilisation.go index d6893cf7487..9c7f544b311 100644 --- a/internal/executor/utilisation/cluster_utilisation.go +++ b/internal/executor/utilisation/cluster_utilisation.go @@ -281,8 +281,8 @@ func groupPodsByNodes(pods []*v1.Pod) map[string][]*v1.Pod { return podsByNodes } -func allocatedByPriorityAndResourceTypeFromPods(pods []*v1.Pod) schedulerobjects.QuantityByPriorityAndResourceType { - rv := make(schedulerobjects.QuantityByPriorityAndResourceType) +func allocatedByPriorityAndResourceTypeFromPods(pods []*v1.Pod) schedulerobjects.QuantityByTAndResourceType[int32] { + rv := make(schedulerobjects.QuantityByTAndResourceType[int32]) for _, pod := range pods { var priority int32 = 0 if pod.Spec.Priority != nil { diff --git a/internal/scheduler/common.go b/internal/scheduler/common.go index e569412f866..f8a29114936 100644 --- a/internal/scheduler/common.go +++ b/internal/scheduler/common.go @@ -116,46 +116,6 @@ func JobsSummary(jobs []interfaces.LegacySchedulerJob) string { ) } -type AddOrSubtract int - -const ( - Add AddOrSubtract = iota - Subtract -) - -func UpdateUsage[S ~[]E, E interfaces.LegacySchedulerJob]( - usage map[string]schedulerobjects.QuantityByPriorityAndResourceType, - jobs S, - priorityClasses map[string]configuration.PriorityClass, - addOrSubtract AddOrSubtract, -) map[string]schedulerobjects.QuantityByPriorityAndResourceType { - if usage == nil { - usage = make(map[string]schedulerobjects.QuantityByPriorityAndResourceType) - } - for _, job := range jobs { - req := PodRequirementFromLegacySchedulerJob(job, priorityClasses) - if req == nil { - continue - } - requests := schedulerobjects.ResourceListFromV1ResourceList(req.ResourceRequirements.Requests) - queue := job.GetQueue() - m := usage[queue] - if m == nil { - m = make(schedulerobjects.QuantityByPriorityAndResourceType) - } - switch addOrSubtract { - case Add: - m.Add(schedulerobjects.QuantityByPriorityAndResourceType{req.Priority: requests}) - case Subtract: - m.Sub(schedulerobjects.QuantityByPriorityAndResourceType{req.Priority: requests}) - default: - panic(fmt.Sprintf("invalid operation %d", addOrSubtract)) - } - usage[queue] = m - } - return usage -} - func jobSchedulingContextsFromJobs[T interfaces.LegacySchedulerJob](jobs []T, executorId string, priorityClasses map[string]configuration.PriorityClass) []*schedulercontext.JobSchedulingContext { if jobs == nil { return nil diff --git a/internal/scheduler/constraints/constraints.go b/internal/scheduler/constraints/constraints.go index 2325737962d..477b121b2e4 100644 --- a/internal/scheduler/constraints/constraints.go +++ b/internal/scheduler/constraints/constraints.go @@ -53,11 +53,8 @@ type SchedulingConstraints struct { type PriorityClassSchedulingConstraints struct { PriorityClassName string PriorityClassPriority int32 - // Prevents jobs of this priority class from being scheduled if doing so would exceed - // cumulative resource usage at priority priorityClassPriority for the queue the job originates from. - // - // Cumulative resource usage at priority x includes resources allocated to jobs of priorityClassPriority x or lower. - MaximumCumulativeResourcesPerQueue schedulerobjects.ResourceList + // Limits total resources allocated to jobs of this priority class per queue. + MaximumResourcesPerQueue schedulerobjects.ResourceList } func SchedulingConstraintsFromSchedulingConfig( @@ -74,9 +71,9 @@ func SchedulingConstraintsFromSchedulingConfig( maximumResourceFractionPerQueue = m } priorityClassSchedulingConstraintsByPriorityClassName[name] = PriorityClassSchedulingConstraints{ - PriorityClassName: name, - PriorityClassPriority: priorityClass.Priority, - MaximumCumulativeResourcesPerQueue: absoluteFromRelativeLimits(totalResources, maximumResourceFractionPerQueue), + PriorityClassName: name, + PriorityClassPriority: priorityClass.Priority, + MaximumResourcesPerQueue: absoluteFromRelativeLimits(totalResources, maximumResourceFractionPerQueue), } } maximumResourceFractionToSchedule := config.MaximumResourceFractionToSchedule @@ -114,7 +111,7 @@ func (constraints *SchedulingConstraints) CheckRoundConstraints(sctx *schedulerc } // MaximumResourcesToSchedule check. - if exceedsResourceLimits(sctx.ScheduledResources, constraints.MaximumResourcesToSchedule) { + if !sctx.ScheduledResources.IsStrictlyLessOrEqual(constraints.MaximumResourcesToSchedule) { return false, UnschedulableReasonMaximumResourcesScheduled, nil } return true, "", nil @@ -132,26 +129,13 @@ func (constraints *SchedulingConstraints) CheckPerQueueAndPriorityClassConstrain // PriorityClassSchedulingConstraintsByPriorityClassName check. if priorityClassConstraint, ok := constraints.PriorityClassSchedulingConstraintsByPriorityClassName[priorityClassName]; ok { - allocatedByPriorityAndResourceType := schedulerobjects.NewAllocatedByPriorityAndResourceType([]int32{priorityClassConstraint.PriorityClassPriority}) - for p, rl := range qctx.AllocatedByPriority { - allocatedByPriorityAndResourceType.MarkAllocated(p, rl) - } - if exceedsResourceLimits( - // TODO: Avoid allocation. - schedulerobjects.QuantityByPriorityAndResourceType(allocatedByPriorityAndResourceType).AggregateByResource(), - priorityClassConstraint.MaximumCumulativeResourcesPerQueue, - ) { + if !qctx.AllocatedByPriorityClass[priorityClassName].IsStrictlyLessOrEqual(priorityClassConstraint.MaximumResourcesPerQueue) { return false, UnschedulableReasonMaximumResourcesPerQueueExceeded, nil } } return true, "", nil } -// exceedsResourceLimits returns true if used/total > limits for some resource. -func exceedsResourceLimits(used, limits schedulerobjects.ResourceList) bool { - return !used.IsStrictlyLessOrEqual(limits) -} - // ScaleQuantity scales q in-place by a factor f. // This functions overflows for quantities the milli value of which can't be expressed as an int64. // E.g., 1Pi is ok, but not 10Pi. diff --git a/internal/scheduler/context/context.go b/internal/scheduler/context/context.go index 40902b4eb00..a34caac890c 100644 --- a/internal/scheduler/context/context.go +++ b/internal/scheduler/context/context.go @@ -42,11 +42,11 @@ type SchedulingContext struct { // Total resources across all clusters available at the start of the scheduling cycle. TotalResources schedulerobjects.ResourceList // Resources assigned across all queues during this scheduling cycle. - ScheduledResources schedulerobjects.ResourceList - ScheduledResourcesByPriority schedulerobjects.QuantityByPriorityAndResourceType + ScheduledResources schedulerobjects.ResourceList + ScheduledResourcesByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] // Resources evicted across all queues during this scheduling cycle. - EvictedResources schedulerobjects.ResourceList - EvictedResourcesByPriority schedulerobjects.QuantityByPriorityAndResourceType + EvictedResources schedulerobjects.ResourceList + EvictedResourcesByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] // Total number of successfully scheduled jobs. NumScheduledJobs int // Total number of successfully scheduled gangs. @@ -73,19 +73,19 @@ func NewSchedulingContext( totalResources schedulerobjects.ResourceList, ) *SchedulingContext { return &SchedulingContext{ - Started: time.Now(), - ExecutorId: executorId, - Pool: pool, - PriorityClasses: priorityClasses, - DefaultPriorityClass: defaultPriorityClass, - ResourceScarcity: resourceScarcity, - QueueSchedulingContexts: make(map[string]*QueueSchedulingContext), - TotalResources: totalResources.DeepCopy(), - ScheduledResources: schedulerobjects.NewResourceListWithDefaultSize(), - ScheduledResourcesByPriority: make(schedulerobjects.QuantityByPriorityAndResourceType), - EvictedResourcesByPriority: make(schedulerobjects.QuantityByPriorityAndResourceType), - SchedulingKeyGenerator: schedulerobjects.NewSchedulingKeyGenerator(), - UnfeasibleSchedulingKeys: make(map[schedulerobjects.SchedulingKey]*JobSchedulingContext), + Started: time.Now(), + ExecutorId: executorId, + Pool: pool, + PriorityClasses: priorityClasses, + DefaultPriorityClass: defaultPriorityClass, + ResourceScarcity: resourceScarcity, + QueueSchedulingContexts: make(map[string]*QueueSchedulingContext), + TotalResources: totalResources.DeepCopy(), + ScheduledResources: schedulerobjects.NewResourceListWithDefaultSize(), + ScheduledResourcesByPriorityClass: make(schedulerobjects.QuantityByTAndResourceType[string]), + EvictedResourcesByPriorityClass: make(schedulerobjects.QuantityByTAndResourceType[string]), + SchedulingKeyGenerator: schedulerobjects.NewSchedulingKeyGenerator(), + UnfeasibleSchedulingKeys: make(map[schedulerobjects.SchedulingKey]*JobSchedulingContext), } } @@ -107,7 +107,7 @@ func (sctx *SchedulingContext) ClearUnfeasibleSchedulingKeys() { sctx.UnfeasibleSchedulingKeys = make(map[schedulerobjects.SchedulingKey]*JobSchedulingContext) } -func (sctx *SchedulingContext) AddQueueSchedulingContext(queue string, priorityFactor float64, initialAllocatedByPriority schedulerobjects.QuantityByPriorityAndResourceType) error { +func (sctx *SchedulingContext) AddQueueSchedulingContext(queue string, priorityFactor float64, initialAllocatedByPriorityClass schedulerobjects.QuantityByTAndResourceType[string]) error { if _, ok := sctx.QueueSchedulingContexts[queue]; ok { return errors.WithStack(&armadaerrors.ErrInvalidArgument{ Name: "queue", @@ -115,10 +115,14 @@ func (sctx *SchedulingContext) AddQueueSchedulingContext(queue string, priorityF Message: fmt.Sprintf("there already exists a context for queue %s", queue), }) } - if initialAllocatedByPriority == nil { - initialAllocatedByPriority = make(schedulerobjects.QuantityByPriorityAndResourceType) + if initialAllocatedByPriorityClass == nil { + initialAllocatedByPriorityClass = make(schedulerobjects.QuantityByTAndResourceType[string]) } else { - initialAllocatedByPriority = initialAllocatedByPriority.DeepCopy() + initialAllocatedByPriorityClass = initialAllocatedByPriorityClass.DeepCopy() + } + allocated := schedulerobjects.NewResourceListWithDefaultSize() + for _, rl := range initialAllocatedByPriorityClass { + allocated.Add(rl) } qctx := &QueueSchedulingContext{ SchedulingContext: sctx, @@ -126,10 +130,10 @@ func (sctx *SchedulingContext) AddQueueSchedulingContext(queue string, priorityF ExecutorId: sctx.ExecutorId, Queue: queue, PriorityFactor: priorityFactor, - Allocated: initialAllocatedByPriority.AggregateByResource(), - AllocatedByPriority: initialAllocatedByPriority, - ScheduledResourcesByPriority: make(schedulerobjects.QuantityByPriorityAndResourceType), - EvictedResourcesByPriority: make(schedulerobjects.QuantityByPriorityAndResourceType), + Allocated: allocated, + AllocatedByPriorityClass: initialAllocatedByPriorityClass, + ScheduledResourcesByPriorityClass: make(schedulerobjects.QuantityByTAndResourceType[string]), + EvictedResourcesByPriorityClass: make(schedulerobjects.QuantityByTAndResourceType[string]), SuccessfulJobSchedulingContexts: make(map[string]*JobSchedulingContext), UnsuccessfulJobSchedulingContexts: make(map[string]*JobSchedulingContext), EvictedJobsById: make(map[string]bool), @@ -220,11 +224,11 @@ func (sctx *SchedulingContext) AddJobSchedulingContext(jctx *JobSchedulingContex if jctx.IsSuccessful() { if evictedInThisRound { sctx.EvictedResources.SubV1ResourceList(jctx.Req.ResourceRequirements.Requests) - sctx.EvictedResourcesByPriority.SubV1ResourceList(jctx.Req.Priority, jctx.Req.ResourceRequirements.Requests) + sctx.EvictedResourcesByPriorityClass.SubV1ResourceList(jctx.Job.GetPriorityClassName(), jctx.Req.ResourceRequirements.Requests) sctx.NumEvictedJobs-- } else { sctx.ScheduledResources.AddV1ResourceList(jctx.Req.ResourceRequirements.Requests) - sctx.ScheduledResourcesByPriority.AddV1ResourceList(jctx.Req.Priority, jctx.Req.ResourceRequirements.Requests) + sctx.ScheduledResourcesByPriorityClass.AddV1ResourceList(jctx.Job.GetPriorityClassName(), jctx.Req.ResourceRequirements.Requests) sctx.NumScheduledJobs++ } } @@ -255,14 +259,14 @@ func (sctx *SchedulingContext) EvictJob(job interfaces.LegacySchedulerJob) (bool if err != nil { return false, err } - priority, rl := priorityAndRequestsFromLegacySchedulerJob(job, sctx.PriorityClasses) + rl := job.GetResourceRequirements().Requests if scheduledInThisRound { sctx.ScheduledResources.SubV1ResourceList(rl) - sctx.ScheduledResourcesByPriority.SubV1ResourceList(priority, rl) + sctx.ScheduledResourcesByPriorityClass.SubV1ResourceList(job.GetPriorityClassName(), rl) sctx.NumScheduledJobs-- } else { sctx.EvictedResources.AddV1ResourceList(rl) - sctx.EvictedResourcesByPriority.AddV1ResourceList(priority, rl) + sctx.EvictedResourcesByPriorityClass.AddV1ResourceList(job.GetPriorityClassName(), rl) sctx.NumEvictedJobs++ } return scheduledInThisRound, nil @@ -286,14 +290,14 @@ func (sctx *SchedulingContext) SuccessfulJobSchedulingContexts() []*JobSchedulin } // AllocatedByQueueAndPriority returns map from queue name and priority to resources allocated. -func (sctx *SchedulingContext) AllocatedByQueueAndPriority() map[string]schedulerobjects.QuantityByPriorityAndResourceType { +func (sctx *SchedulingContext) AllocatedByQueueAndPriority() map[string]schedulerobjects.QuantityByTAndResourceType[string] { rv := make( - map[string]schedulerobjects.QuantityByPriorityAndResourceType, + map[string]schedulerobjects.QuantityByTAndResourceType[string], len(sctx.QueueSchedulingContexts), ) for queue, qctx := range sctx.QueueSchedulingContexts { - if len(qctx.AllocatedByPriority) > 0 { - rv[queue] = qctx.AllocatedByPriority.DeepCopy() + if !qctx.AllocatedByPriorityClass.IsZero() { + rv[queue] = qctx.AllocatedByPriorityClass.DeepCopy() } } return rv @@ -315,13 +319,13 @@ type QueueSchedulingContext struct { // Total resources assigned to the queue across all clusters by priority class priority. // Includes jobs scheduled during this invocation of the scheduler. Allocated schedulerobjects.ResourceList - // Total resources assigned to the queue across all clusters by priority class priority. + // Total resources assigned to the queue across all clusters by priority class. // Includes jobs scheduled during this invocation of the scheduler. - AllocatedByPriority schedulerobjects.QuantityByPriorityAndResourceType + AllocatedByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] // Resources assigned to this queue during this scheduling cycle. - ScheduledResourcesByPriority schedulerobjects.QuantityByPriorityAndResourceType + ScheduledResourcesByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] // Resources evicted from this queue during this scheduling cycle. - EvictedResourcesByPriority schedulerobjects.QuantityByPriorityAndResourceType + EvictedResourcesByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] // Job scheduling contexts associated with successful scheduling attempts. SuccessfulJobSchedulingContexts map[string]*JobSchedulingContext // Job scheduling contexts associated with unsuccessful scheduling attempts. @@ -350,13 +354,13 @@ func (qctx *QueueSchedulingContext) ReportString(verbosity int32) string { fmt.Fprintf(w, "Time:\t%s\n", qctx.Created) fmt.Fprintf(w, "Queue:\t%s\n", qctx.Queue) } - fmt.Fprintf(w, "Scheduled resources:\t%s\n", qctx.ScheduledResourcesByPriority.AggregateByResource().CompactString()) - fmt.Fprintf(w, "Scheduled resources (by priority):\t%s\n", qctx.ScheduledResourcesByPriority.String()) - fmt.Fprintf(w, "Preempted resources:\t%s\n", qctx.EvictedResourcesByPriority.AggregateByResource().CompactString()) - fmt.Fprintf(w, "Preempted resources (by priority):\t%s\n", qctx.EvictedResourcesByPriority.String()) + fmt.Fprintf(w, "Scheduled resources:\t%s\n", qctx.ScheduledResourcesByPriorityClass.AggregateByResource().CompactString()) + fmt.Fprintf(w, "Scheduled resources (by priority):\t%s\n", qctx.ScheduledResourcesByPriorityClass.String()) + fmt.Fprintf(w, "Preempted resources:\t%s\n", qctx.EvictedResourcesByPriorityClass.AggregateByResource().CompactString()) + fmt.Fprintf(w, "Preempted resources (by priority):\t%s\n", qctx.EvictedResourcesByPriorityClass.String()) if verbosity >= 0 { - fmt.Fprintf(w, "Total allocated resources after scheduling:\t%s\n", qctx.AllocatedByPriority.AggregateByResource().CompactString()) - fmt.Fprintf(w, "Total allocated resources after scheduling (by priority):\t%s\n", qctx.AllocatedByPriority.String()) + fmt.Fprintf(w, "Total allocated resources after scheduling:\t%s\n", qctx.Allocated.CompactString()) + fmt.Fprintf(w, "Total allocated resources after scheduling by priority class:\t%s\n", qctx.AllocatedByPriorityClass) fmt.Fprintf(w, "Number of jobs scheduled:\t%d\n", len(qctx.SuccessfulJobSchedulingContexts)) fmt.Fprintf(w, "Number of jobs preempted:\t%d\n", len(qctx.EvictedJobsById)) fmt.Fprintf(w, "Number of jobs that could not be scheduled:\t%d\n", len(qctx.UnsuccessfulJobSchedulingContexts)) @@ -438,16 +442,16 @@ func (qctx *QueueSchedulingContext) AddJobSchedulingContext(jctx *JobSchedulingC // Always update ResourcesByPriority. // Since ResourcesByPriority is used to order queues by fraction of fair share. qctx.Allocated.AddV1ResourceList(jctx.Req.ResourceRequirements.Requests) - qctx.AllocatedByPriority.AddV1ResourceList(jctx.Req.Priority, jctx.Req.ResourceRequirements.Requests) + qctx.AllocatedByPriorityClass.AddV1ResourceList(jctx.Job.GetPriorityClassName(), jctx.Req.ResourceRequirements.Requests) // Only if the job is not evicted, update ScheduledResourcesByPriority. // Since ScheduledResourcesByPriority is used to control per-round scheduling constraints. if evictedInThisRound { delete(qctx.EvictedJobsById, jctx.JobId) - qctx.EvictedResourcesByPriority.SubV1ResourceList(jctx.Req.Priority, jctx.Req.ResourceRequirements.Requests) + qctx.EvictedResourcesByPriorityClass.SubV1ResourceList(jctx.Job.GetPriorityClassName(), jctx.Req.ResourceRequirements.Requests) } else { qctx.SuccessfulJobSchedulingContexts[jctx.JobId] = jctx - qctx.ScheduledResourcesByPriority.AddV1ResourceList(jctx.Req.Priority, jctx.Req.ResourceRequirements.Requests) + qctx.ScheduledResourcesByPriorityClass.AddV1ResourceList(jctx.Job.GetPriorityClassName(), jctx.Req.ResourceRequirements.Requests) } } else { qctx.UnsuccessfulJobSchedulingContexts[jctx.JobId] = jctx @@ -457,7 +461,7 @@ func (qctx *QueueSchedulingContext) AddJobSchedulingContext(jctx *JobSchedulingC func (qctx *QueueSchedulingContext) EvictJob(job interfaces.LegacySchedulerJob) (bool, error) { jobId := job.GetId() - priority, rl := priorityAndRequestsFromLegacySchedulerJob(job, qctx.SchedulingContext.PriorityClasses) + _, rl := priorityAndRequestsFromLegacySchedulerJob(job, qctx.SchedulingContext.PriorityClasses) if _, ok := qctx.UnsuccessfulJobSchedulingContexts[jobId]; ok { return false, errors.Errorf("failed evicting job %s from queue: job already marked unsuccessful", jobId) } @@ -466,17 +470,18 @@ func (qctx *QueueSchedulingContext) EvictJob(job interfaces.LegacySchedulerJob) } _, scheduledInThisRound := qctx.SuccessfulJobSchedulingContexts[jobId] if scheduledInThisRound { - qctx.ScheduledResourcesByPriority.SubV1ResourceList(priority, rl) + qctx.ScheduledResourcesByPriorityClass.SubV1ResourceList(job.GetPriorityClassName(), rl) delete(qctx.SuccessfulJobSchedulingContexts, jobId) } else { - qctx.EvictedResourcesByPriority.AddV1ResourceList(priority, rl) + qctx.EvictedResourcesByPriorityClass.AddV1ResourceList(job.GetPriorityClassName(), rl) qctx.EvictedJobsById[jobId] = true } qctx.Allocated.SubV1ResourceList(rl) - qctx.AllocatedByPriority.SubV1ResourceList(priority, rl) + qctx.AllocatedByPriorityClass.SubV1ResourceList(job.GetPriorityClassName(), rl) return scheduledInThisRound, nil } +// TODO: Remove? func priorityAndRequestsFromLegacySchedulerJob(job interfaces.LegacySchedulerJob, priorityClasses map[string]configuration.PriorityClass) (int32, v1.ResourceList) { req := job.GetRequirements(priorityClasses) for _, r := range req.ObjectRequirements { diff --git a/internal/scheduler/context/context_test.go b/internal/scheduler/context/context_test.go index 150eb593b05..0c8704f7de9 100644 --- a/internal/scheduler/context/context_test.go +++ b/internal/scheduler/context/context_test.go @@ -42,13 +42,13 @@ func TestSchedulingContextAccounting(t *testing.T) { schedulerobjects.ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, ) priorityFactorByQueue := map[string]float64{"A": 1, "B": 1} - allocatedByQueueAndPriority := map[string]schedulerobjects.QuantityByPriorityAndResourceType{ + allocatedByQueueAndPriorityClass := map[string]schedulerobjects.QuantityByTAndResourceType[string]{ "A": { - 0: schedulerobjects.ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, + "foo": schedulerobjects.ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, } for _, queue := range []string{"A", "B"} { - err := sctx.AddQueueSchedulingContext(queue, priorityFactorByQueue[queue], allocatedByQueueAndPriority[queue]) + err := sctx.AddQueueSchedulingContext(queue, priorityFactorByQueue[queue], allocatedByQueueAndPriorityClass[queue]) require.NoError(t, err) } diff --git a/internal/scheduler/gang_scheduler_test.go b/internal/scheduler/gang_scheduler_test.go index d94d0436dd2..8a9d7f2f3f6 100644 --- a/internal/scheduler/gang_scheduler_test.go +++ b/internal/scheduler/gang_scheduler_test.go @@ -117,26 +117,26 @@ func TestGangScheduler(t *testing.T) { }, "MaximumResourceFractionPerQueue": { SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 1.0}, - 1: {"cpu": 15.0 / 32.0}, - 2: {"cpu": 10.0 / 32.0}, - 3: {"cpu": 3.0 / 32.0}, + map[string]map[string]float64{ + testfixtures.PriorityClass0: {"cpu": 1.0 / 32.0}, + testfixtures.PriorityClass1: {"cpu": 2.0 / 32.0}, + testfixtures.PriorityClass2: {"cpu": 3.0 / 32.0}, + testfixtures.PriorityClass3: {"cpu": 4.0 / 32.0}, }, testfixtures.TestSchedulingConfig(), ), Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), Gangs: [][]*jobdb.Job{ + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 1), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 2), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 2), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 3), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 3), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 4), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 4), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 3), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 8), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 7), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 6), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 5), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 18), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 17), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 5), }, - ExpectedScheduledIndices: []int{1, 3, 5, 7}, + ExpectedScheduledIndices: []int{0, 2, 4, 6}, }, "resolution has no impact on jobs of size a multiple of the resolution": { SchedulingConfig: testfixtures.WithIndexedResourcesConfig( diff --git a/internal/scheduler/nodedb/nodedb_test.go b/internal/scheduler/nodedb/nodedb_test.go index c62e0abfe6a..9f932aecc01 100644 --- a/internal/scheduler/nodedb/nodedb_test.go +++ b/internal/scheduler/nodedb/nodedb_test.go @@ -188,10 +188,10 @@ func assertNodeAccountingEqual(t *testing.T, node1, node2 *schedulerobjects.Node rv := true rv = rv && assert.True( t, - schedulerobjects.QuantityByPriorityAndResourceType( + schedulerobjects.QuantityByTAndResourceType[int32]( node1.AllocatableByPriorityAndResource, ).Equal( - schedulerobjects.QuantityByPriorityAndResourceType( + schedulerobjects.QuantityByTAndResourceType[int32]( node2.AllocatableByPriorityAndResource, ), ), diff --git a/internal/scheduler/preempting_queue_scheduler.go b/internal/scheduler/preempting_queue_scheduler.go index 78931edd37c..b41a7a6b361 100644 --- a/internal/scheduler/preempting_queue_scheduler.go +++ b/internal/scheduler/preempting_queue_scheduler.go @@ -426,8 +426,8 @@ func (sch *PreemptingQueueScheduler) setEvictedGangCardinality(evictedJobsById m func (sch *PreemptingQueueScheduler) evictionAssertions(evictedJobsById map[string]interfaces.LegacySchedulerJob, affectedNodesById map[string]*schedulerobjects.Node) error { for _, qctx := range sch.schedulingContext.QueueSchedulingContexts { - if !qctx.AllocatedByPriority.IsStrictlyNonNegative() { - return errors.Errorf("negative allocation for queue %s after eviction: %s", qctx.Queue, qctx.AllocatedByPriority) + if !qctx.AllocatedByPriorityClass.IsStrictlyNonNegative() { + return errors.Errorf("negative allocation for queue %s after eviction: %s", qctx.Queue, qctx.AllocatedByPriorityClass) } } evictedJobIdsByGangId := make(map[string]map[string]bool) diff --git a/internal/scheduler/preempting_queue_scheduler_test.go b/internal/scheduler/preempting_queue_scheduler_test.go index ff594403e58..b2151e7e9b6 100644 --- a/internal/scheduler/preempting_queue_scheduler_test.go +++ b/internal/scheduler/preempting_queue_scheduler_test.go @@ -111,10 +111,9 @@ func TestPreemptingQueueScheduler(t *testing.T) { Rounds []SchedulingRound // Map from queue to the priority factor associated with that queue. PriorityFactorByQueue map[string]float64 - // Initial resource usage for all queues. - // This value is used across all rounds, + // Initial resource usage for all queues. This value is used across all rounds, // i.e., we don't update it based on preempted/scheduled jobs. - InitialAllocationByQueue map[string]schedulerobjects.QuantityByPriorityAndResourceType + InitialAllocationByQueueAndPriorityClass map[string]schedulerobjects.QuantityByTAndResourceType[string] // Total resources across all clusters. // If empty, it is computed as the total resources across the provided nodes. TotalResources schedulerobjects.ResourceList @@ -818,25 +817,30 @@ func TestPreemptingQueueScheduler(t *testing.T) { "A": 1, }, }, - "per-priority class limits": { + "MaximumResourceFractionPerQueue": { SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 60.0 / 64.0}, - 1: {"cpu": 20.0 / 64.0}, + map[string]map[string]float64{ + testfixtures.PriorityClass0: {"cpu": 1.0 / 32.0}, + testfixtures.PriorityClass1: {"cpu": 2.0 / 32.0}, + testfixtures.PriorityClass2: {"cpu": 3.0 / 32.0}, + testfixtures.PriorityClass3: {"cpu": 4.0 / 32.0}, }, testfixtures.TestSchedulingConfig(), ), - Nodes: testfixtures.N32CpuNodes(2, testfixtures.TestPriorities), + Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), Rounds: []SchedulingRound{ { JobsByQueue: map[string][]*jobdb.Job{ - "A": append( - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 64), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 64)..., + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), ), }, ExpectedScheduledIndices: map[string][]int{ - "A": append(testfixtures.IntRange(0, 19), testfixtures.IntRange(64, 103)...), + "A": {0, 32, 33, 64, 65, 66, 96, 97, 98, 99}, }, }, { @@ -849,11 +853,13 @@ func TestPreemptingQueueScheduler(t *testing.T) { "A": 1, }, }, - "per-priority class limits multiple rounds": { + "MaximumResourceFractionPerQueue multiple rounds": { SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 30.0 / 32.0}, - 1: {"cpu": 10.0 / 32.0}, + map[string]map[string]float64{ + testfixtures.PriorityClass0: {"cpu": 1.0 / 32.0}, + testfixtures.PriorityClass1: {"cpu": 2.0 / 32.0}, + testfixtures.PriorityClass2: {"cpu": 3.0 / 32.0}, + testfixtures.PriorityClass3: {"cpu": 4.0 / 32.0}, }, testfixtures.TestSchedulingConfig(), ), @@ -861,31 +867,57 @@ func TestPreemptingQueueScheduler(t *testing.T) { Rounds: []SchedulingRound{ { JobsByQueue: map[string][]*jobdb.Job{ - "A": append( - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 5), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 10)..., + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), + ), + }, + ExpectedScheduledIndices: map[string][]int{ + "A": testfixtures.IntRange(0, 0), + }, + }, + { + JobsByQueue: map[string][]*jobdb.Job{ + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), + ), + }, + ExpectedScheduledIndices: map[string][]int{ + "A": testfixtures.IntRange(0, 1), + }, + }, + { + JobsByQueue: map[string][]*jobdb.Job{ + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), ), }, ExpectedScheduledIndices: map[string][]int{ - "A": testfixtures.IntRange(0, 14), + "A": testfixtures.IntRange(0, 2), }, }, { JobsByQueue: map[string][]*jobdb.Job{ - "A": append( + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 32), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 32), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32)..., + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), ), }, ExpectedScheduledIndices: map[string][]int{ - "A": append(testfixtures.IntRange(0, 4), testfixtures.IntRange(32, 41)...), + "A": testfixtures.IntRange(0, 3), }, }, { JobsByQueue: map[string][]*jobdb.Job{ - "A": append( + "A": armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 32), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 32), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 32), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32)..., + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 32), ), }, }, @@ -1118,7 +1150,7 @@ func TestPreemptingQueueScheduler(t *testing.T) { // Accounting across scheduling rounds. roundByJobId := make(map[string]int) indexByJobId := make(map[string]int) - allocatedByQueueAndPriority := armadamaps.DeepCopy(tc.InitialAllocationByQueue) + allocatedByQueueAndPriorityClass := armadamaps.DeepCopy(tc.InitialAllocationByQueueAndPriorityClass) nodeIdByJobId := make(map[string]string) var jobIdsByGangId map[string]map[string]bool var gangIdByJobId map[string]string @@ -1189,7 +1221,7 @@ func TestPreemptingQueueScheduler(t *testing.T) { tc.TotalResources, ) for queue, priorityFactor := range tc.PriorityFactorByQueue { - err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriority[queue]) + err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriorityClass[queue]) require.NoError(t, err) } constraints := schedulerconstraints.SchedulingConstraintsFromSchedulingConfig( @@ -1217,35 +1249,30 @@ func TestPreemptingQueueScheduler(t *testing.T) { // Test resource accounting. for _, job := range result.PreemptedJobs { - req := PodRequirementFromLegacySchedulerJob(job, tc.SchedulingConfig.Preemption.PriorityClasses) - requests := schedulerobjects.ResourceListFromV1ResourceList(req.ResourceRequirements.Requests) - quantityByPriorityAndResourceType := schedulerobjects.QuantityByPriorityAndResourceType{ - req.Priority: requests, + m := allocatedByQueueAndPriorityClass[job.GetQueue()] + if m == nil { + m = make(schedulerobjects.QuantityByTAndResourceType[string]) + allocatedByQueueAndPriorityClass[job.GetQueue()] = m } - allocatedByQueueAndPriority[job.GetQueue()].Sub(quantityByPriorityAndResourceType) + m.SubV1ResourceList( + job.GetPriorityClassName(), + job.GetResourceRequirements().Requests, + ) } for _, job := range result.ScheduledJobs { - req := PodRequirementFromLegacySchedulerJob(job, tc.SchedulingConfig.Preemption.PriorityClasses) - requests := schedulerobjects.ResourceListFromV1ResourceList(req.ResourceRequirements.Requests) - quantityByPriorityAndResourceType := schedulerobjects.QuantityByPriorityAndResourceType{ - req.Priority: requests, - } - m := allocatedByQueueAndPriority[job.GetQueue()] + m := allocatedByQueueAndPriorityClass[job.GetQueue()] if m == nil { - m = make(schedulerobjects.QuantityByPriorityAndResourceType) + m = make(schedulerobjects.QuantityByTAndResourceType[string]) + allocatedByQueueAndPriorityClass[job.GetQueue()] = m } - m.Add(quantityByPriorityAndResourceType) - allocatedByQueueAndPriority[job.GetQueue()] = m - } - for queue, allocated := range allocatedByQueueAndPriority { - // Filter out explicit zeros to enable comparing with expected allocation. - allocatedByQueueAndPriority[queue] = armadamaps.Filter( - allocated, - func(_ int32, rl schedulerobjects.ResourceList) bool { - return !rl.IsZero() - }, + m.AddV1ResourceList( + job.GetPriorityClassName(), + job.GetResourceRequirements().Requests, ) } + for queue, qctx := range sctx.QueueSchedulingContexts { + assert.True(t, qctx.AllocatedByPriorityClass.Equal(allocatedByQueueAndPriorityClass[queue])) + } // Test that jobs are mapped to nodes correctly. for _, job := range result.PreemptedJobs { @@ -1428,7 +1455,7 @@ func BenchmarkPreemptingQueueScheduler(b *testing.B) { nodeDb, err := CreateNodeDb(tc.Nodes) require.NoError(b, err) repo := NewInMemoryJobRepository(testfixtures.TestPriorityClasses) - allocatedByQueueAndPriority := make(map[string]schedulerobjects.QuantityByPriorityAndResourceType) + allocatedByQueueAndPriorityClass := make(map[string]schedulerobjects.QuantityByTAndResourceType[string]) jobs := make([]interfaces.LegacySchedulerJob, 0) for _, queueJobs := range jobsByQueue { @@ -1447,7 +1474,7 @@ func BenchmarkPreemptingQueueScheduler(b *testing.B) { nodeDb.TotalResources(), ) for queue, priorityFactor := range priorityFactorByQueue { - err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriority[queue]) + err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriorityClass[queue]) require.NoError(b, err) } constraints := schedulerconstraints.SchedulingConstraintsFromSchedulingConfig( @@ -1496,7 +1523,7 @@ func BenchmarkPreemptingQueueScheduler(b *testing.B) { nodeDb.TotalResources(), ) for queue, priorityFactor := range priorityFactorByQueue { - err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriority[queue]) + err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByQueueAndPriorityClass[queue]) require.NoError(b, err) } sch := NewPreemptingQueueScheduler( diff --git a/internal/scheduler/queue_scheduler_test.go b/internal/scheduler/queue_scheduler_test.go index 6489e743b6a..9980c402423 100644 --- a/internal/scheduler/queue_scheduler_test.go +++ b/internal/scheduler/queue_scheduler_test.go @@ -34,7 +34,7 @@ func TestQueueScheduler(t *testing.T) { // Map from queue to the priority factor associated with that queue. PriorityFactorByQueue map[string]float64 // Initial resource usage for all queues. - InitialAllocatedByQueueAndPriority map[string]schedulerobjects.QuantityByPriorityAndResourceType + InitialAllocatedByQueueAndPriorityClass map[string]schedulerobjects.QuantityByTAndResourceType[string] // Nodes to be considered by the scheduler. Nodes []*schedulerobjects.Node // Jobs to try scheduling. @@ -138,79 +138,34 @@ func TestQueueScheduler(t *testing.T) { }, "PerPriorityLimits": { SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 1.0}, - 1: {"cpu": 15.0 / 32.0}, - 2: {"cpu": 10.0 / 32.0}, - 3: {"cpu": 3.0 / 32.0}, + map[string]map[string]float64{ + testfixtures.PriorityClass0: {"cpu": 1.0 / 32.0}, + testfixtures.PriorityClass1: {"cpu": 2.0 / 32.0}, + testfixtures.PriorityClass2: {"cpu": 3.0 / 32.0}, + testfixtures.PriorityClass3: {"cpu": 4.0 / 32.0}, }, testfixtures.TestSchedulingConfig(), ), PriorityFactorByQueue: map[string]float64{"A": 1.0}, Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), Jobs: armadaslices.Concatenate( + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 1), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 2), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 2), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 3), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 3), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 3), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 4), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass3, 4), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass2, 8), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 6), - testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 18), + testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 1), ), ExpectedScheduledIndices: armadaslices.Concatenate( - testfixtures.IntRange(0, 2), - testfixtures.IntRange(4, 10), - testfixtures.IntRange(12, 16), - testfixtures.IntRange(18, 34), + testfixtures.IntRange(0, 0), + testfixtures.IntRange(3, 4), + testfixtures.IntRange(8, 10), + testfixtures.IntRange(14, 17), ), }, - "PerPriorityLimits equal MaximumResourceFractionToSchedule": { - SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 0.9}, // 28 cpu - 1: {"cpu": 0.9}, - }, testfixtures.TestSchedulingConfig()), - Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), - Jobs: armadaslices.Concatenate(testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 5), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 5)), - PriorityFactorByQueue: map[string]float64{"A": 1}, - InitialAllocatedByQueueAndPriority: map[string]schedulerobjects.QuantityByPriorityAndResourceType{ - "A": { - 0: schedulerobjects.ResourceList{ - Resources: map[string]resource.Quantity{ - "cpu": resource.MustParse("13"), - }, - }, - 1: schedulerobjects.ResourceList{ - Resources: map[string]resource.Quantity{ - "cpu": resource.MustParse("14"), - }, - }, - }, - }, - ExpectedScheduledIndices: []int{0}, - }, - "limit hit at higher priority doesn't block jobs at lower priority": { - SchedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - 0: {"cpu": 0.9}, // 28 cpu - 1: {"cpu": 0.5}, // 14 cpu - }, testfixtures.TestSchedulingConfig()), - Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), - Jobs: armadaslices.Concatenate(testfixtures.N1CpuJobs("A", testfixtures.PriorityClass1, 1), testfixtures.N1CpuJobs("A", testfixtures.PriorityClass0, 5)), - PriorityFactorByQueue: map[string]float64{"A": 1}, - InitialAllocatedByQueueAndPriority: map[string]schedulerobjects.QuantityByPriorityAndResourceType{ - "A": { - 0: schedulerobjects.ResourceList{ - Resources: map[string]resource.Quantity{ - "cpu": resource.MustParse("7"), // out of 28 - }, - }, - 1: schedulerobjects.ResourceList{ - Resources: map[string]resource.Quantity{ - "cpu": resource.MustParse("20"), // out of 14, i.e., over the limit - }, - }, - }, - }, - ExpectedScheduledIndices: []int{1}, - }, "fairness two queues": { SchedulingConfig: testfixtures.TestSchedulingConfig(), Nodes: testfixtures.N32CpuNodes(1, testfixtures.TestPriorities), @@ -283,9 +238,9 @@ func TestQueueScheduler(t *testing.T) { "A": 1, "B": 1, }, - InitialAllocatedByQueueAndPriority: map[string]schedulerobjects.QuantityByPriorityAndResourceType{ + InitialAllocatedByQueueAndPriorityClass: map[string]schedulerobjects.QuantityByTAndResourceType[string]{ "A": { - 0: schedulerobjects.ResourceList{ + testfixtures.PriorityClass0: schedulerobjects.ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("100"), }, @@ -498,7 +453,7 @@ func TestQueueScheduler(t *testing.T) { tc.TotalResources, ) for queue, priorityFactor := range tc.PriorityFactorByQueue { - err := sctx.AddQueueSchedulingContext(queue, priorityFactor, tc.InitialAllocatedByQueueAndPriority[queue]) + err := sctx.AddQueueSchedulingContext(queue, priorityFactor, tc.InitialAllocatedByQueueAndPriorityClass[queue]) require.NoError(t, err) } constraints := schedulerconstraints.SchedulingConstraintsFromSchedulingConfig( diff --git a/internal/scheduler/reports.go b/internal/scheduler/reports.go index 0fd5415de4d..aefef6eb884 100644 --- a/internal/scheduler/reports.go +++ b/internal/scheduler/reports.go @@ -136,13 +136,13 @@ func (repo *SchedulingContextRepository) addSchedulingContext(sctx *schedulercon mostRecentSuccessfulByExecutor := *repo.mostRecentSuccessfulByExecutor.Load() mostRecentSuccessfulByExecutor = maps.Clone(mostRecentSuccessfulByExecutor) - if !sctx.ScheduledResourcesByPriority.IsZero() { + if !sctx.ScheduledResourcesByPriorityClass.IsZero() { mostRecentSuccessfulByExecutor[sctx.ExecutorId] = sctx } mostRecentPreemptingByExecutor := *repo.mostRecentPreemptingByExecutor.Load() mostRecentPreemptingByExecutor = maps.Clone(mostRecentPreemptingByExecutor) - if !sctx.EvictedResourcesByPriority.IsZero() { + if !sctx.EvictedResourcesByPriorityClass.IsZero() { mostRecentPreemptingByExecutor[sctx.ExecutorId] = sctx } @@ -190,7 +190,7 @@ func (repo *SchedulingContextRepository) addSchedulingContextForQueues(sctx *sch mostRecentByExecutorByQueue[queue] = SchedulingContextByExecutor{executorId: sctx} } - if !qctx.ScheduledResourcesByPriority.IsZero() { + if !qctx.ScheduledResourcesByPriorityClass.IsZero() { if previous := mostRecentSuccessfulByExecutorByQueue[queue]; previous != nil { previous = maps.Clone(previous) previous[executorId] = sctx @@ -200,7 +200,7 @@ func (repo *SchedulingContextRepository) addSchedulingContextForQueues(sctx *sch } } - if !qctx.EvictedResourcesByPriority.IsZero() { + if !qctx.EvictedResourcesByPriorityClass.IsZero() { if previous := mostRecentPreemptingByExecutorByQueue[queue]; previous != nil { previous = maps.Clone(previous) previous[executorId] = sctx diff --git a/internal/scheduler/reports_test.go b/internal/scheduler/reports_test.go index 110ed1fe2d5..a3178449ed9 100644 --- a/internal/scheduler/reports_test.go +++ b/internal/scheduler/reports_test.go @@ -246,7 +246,7 @@ func withSuccessfulJobSchedulingContext(sctx *schedulercontext.SchedulingContext } qctx := sctx.QueueSchedulingContexts[queue] if qctx == nil { - if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByPriorityAndResourceType)); err != nil { + if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByTAndResourceType[string])); err != nil { panic(err) } qctx = sctx.QueueSchedulingContexts[queue] @@ -258,8 +258,8 @@ func withSuccessfulJobSchedulingContext(sctx *schedulercontext.SchedulingContext JobId: jobId, } rl := schedulerobjects.ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}} - qctx.ScheduledResourcesByPriority.AddResourceList(0, rl) - sctx.ScheduledResourcesByPriority.AddResourceList(0, rl) + qctx.ScheduledResourcesByPriorityClass.AddResourceList("foo", rl) + sctx.ScheduledResourcesByPriorityClass.AddResourceList("foo", rl) return sctx } @@ -269,7 +269,7 @@ func withPreemptingJobSchedulingContext(sctx *schedulercontext.SchedulingContext } qctx := sctx.QueueSchedulingContexts[queue] if qctx == nil { - if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByPriorityAndResourceType)); err != nil { + if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByTAndResourceType[string])); err != nil { panic(err) } qctx = sctx.QueueSchedulingContexts[queue] @@ -278,8 +278,8 @@ func withPreemptingJobSchedulingContext(sctx *schedulercontext.SchedulingContext } qctx.EvictedJobsById[jobId] = true rl := schedulerobjects.ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}} - qctx.EvictedResourcesByPriority.AddResourceList(0, rl) - sctx.EvictedResourcesByPriority.AddResourceList(0, rl) + qctx.EvictedResourcesByPriorityClass.AddResourceList("foo", rl) + sctx.EvictedResourcesByPriorityClass.AddResourceList("foo", rl) return sctx } @@ -289,7 +289,7 @@ func withUnsuccessfulJobSchedulingContext(sctx *schedulercontext.SchedulingConte } qctx := sctx.QueueSchedulingContexts[queue] if qctx == nil { - if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByPriorityAndResourceType)); err != nil { + if err := sctx.AddQueueSchedulingContext(queue, 1.0, make(schedulerobjects.QuantityByTAndResourceType[string])); err != nil { panic(err) } qctx = sctx.QueueSchedulingContexts[queue] diff --git a/internal/scheduler/schedulerobjects/resourcelist.go b/internal/scheduler/schedulerobjects/resourcelist.go index af90c4ffc8a..7c98a95eff3 100644 --- a/internal/scheduler/schedulerobjects/resourcelist.go +++ b/internal/scheduler/schedulerobjects/resourcelist.go @@ -39,75 +39,77 @@ func V1ResourceListFromResourceList(rl ResourceList) v1.ResourceList { return rv } -type QuantityByPriorityAndResourceType map[int32]ResourceList +type QuantityByTAndResourceType[T comparable] map[T]ResourceList -func (a QuantityByPriorityAndResourceType) DeepCopy() QuantityByPriorityAndResourceType { - rv := make(QuantityByPriorityAndResourceType) - for p, rl := range a { - rv[p] = rl.DeepCopy() +// type QuantityByPriorityAndResourceType QuantityByTAndResourceType[int32] + +func (a QuantityByTAndResourceType[T]) Add(b QuantityByTAndResourceType[T]) { + for p, rlb := range b { + a.AddResourceList(p, rlb) + } +} + +func (a QuantityByTAndResourceType[T]) AddResourceList(t T, rlb ResourceList) { + rla := a[t] + rla.Add(rlb) + a[t] = rla +} + +func (a QuantityByTAndResourceType[T]) DeepCopy() QuantityByTAndResourceType[T] { + rv := make(QuantityByTAndResourceType[T]) + for t, rl := range a { + rv[t] = rl.DeepCopy() } return rv } -func (a QuantityByPriorityAndResourceType) String() string { +func (a QuantityByTAndResourceType[T]) String() string { var sb strings.Builder i := 0 sb.WriteString("{") - for p, rl := range a { + for t, rl := range a { if i < len(a)-1 { - sb.WriteString(fmt.Sprintf("%d: %s, ", p, rl.CompactString())) + sb.WriteString(fmt.Sprintf("%v: %s, ", t, rl.CompactString())) } else { - sb.WriteString(fmt.Sprintf("%d: %s", p, rl.CompactString())) + sb.WriteString(fmt.Sprintf("%v: %s", t, rl.CompactString())) } } sb.WriteString("}") return sb.String() } -func (a QuantityByPriorityAndResourceType) Add(b QuantityByPriorityAndResourceType) { - for p, rlb := range b { - a.AddResourceList(p, rlb) - } -} - -func (a QuantityByPriorityAndResourceType) Sub(b QuantityByPriorityAndResourceType) { - for p, rlb := range b { - a.SubResourceList(p, rlb) +func (a QuantityByTAndResourceType[T]) Sub(b QuantityByTAndResourceType[T]) { + for t, rlb := range b { + a.SubResourceList(t, rlb) } } -func (a QuantityByPriorityAndResourceType) AddResourceList(priority int32, rlb ResourceList) { - rla := a[priority] - rla.Add(rlb) - a[priority] = rla -} - -func (a QuantityByPriorityAndResourceType) AddV1ResourceList(priority int32, rlb v1.ResourceList) { - rla := a[priority] +func (a QuantityByTAndResourceType[T]) AddV1ResourceList(t T, rlb v1.ResourceList) { + rla := a[t] rla.AddV1ResourceList(rlb) - a[priority] = rla + a[t] = rla } -func (a QuantityByPriorityAndResourceType) SubResourceList(priority int32, rlb ResourceList) { - rla := a[priority] +func (a QuantityByTAndResourceType[T]) SubResourceList(t T, rlb ResourceList) { + rla := a[t] rla.Sub(rlb) - a[priority] = rla + a[t] = rla } -func (a QuantityByPriorityAndResourceType) SubV1ResourceList(priority int32, rlb v1.ResourceList) { - rla := a[priority] +func (a QuantityByTAndResourceType[T]) SubV1ResourceList(t T, rlb v1.ResourceList) { + rla := a[t] rla.SubV1ResourceList(rlb) - a[priority] = rla + a[t] = rla } -func (a QuantityByPriorityAndResourceType) Equal(b QuantityByPriorityAndResourceType) bool { - for p, rla := range a { - if !rla.Equal(b[p]) { +func (a QuantityByTAndResourceType[T]) Equal(b QuantityByTAndResourceType[T]) bool { + for t, rla := range a { + if !rla.Equal(b[t]) { return false } } - for p, rlb := range b { - if !rlb.Equal(a[p]) { + for t, rlb := range b { + if !rlb.Equal(a[t]) { return false } } @@ -115,7 +117,7 @@ func (a QuantityByPriorityAndResourceType) Equal(b QuantityByPriorityAndResource } // IsZero returns true if all quantities in a are zero. -func (a QuantityByPriorityAndResourceType) IsZero() bool { +func (a QuantityByTAndResourceType[T]) IsZero() bool { for _, rl := range a { if !rl.IsZero() { return false @@ -125,7 +127,7 @@ func (a QuantityByPriorityAndResourceType) IsZero() bool { } // IsStrictlyNonNegative returns true if there are no quantities in a with value less than zero. -func (a QuantityByPriorityAndResourceType) IsStrictlyNonNegative() bool { +func (a QuantityByTAndResourceType[T]) IsStrictlyNonNegative() bool { for _, rl := range a { if !rl.IsStrictlyNonNegative() { return false @@ -134,7 +136,7 @@ func (a QuantityByPriorityAndResourceType) IsStrictlyNonNegative() bool { return true } -func (a QuantityByPriorityAndResourceType) AggregateByResource() ResourceList { +func (a QuantityByTAndResourceType[T]) AggregateByResource() ResourceList { rv := NewResourceListWithDefaultSize() for _, rl := range a { rv.Add(rl) @@ -147,7 +149,7 @@ func (a QuantityByPriorityAndResourceType) AggregateByResource() ResourceList { // where p1, ..., pn are the priorities in a, for each resource set explicitly in rl. // // If necessary to add resources to make up the difference, those resources are added at priority p. -func (a QuantityByPriorityAndResourceType) MaxAggregatedByResource(p int32, rl ResourceList) { +func (a QuantityByTAndResourceType[T]) MaxAggregatedByResource(t T, rl ResourceList) { aggregate := a.AggregateByResource() var difference ResourceList for t, q := range rl.Resources { @@ -158,7 +160,7 @@ func (a QuantityByPriorityAndResourceType) MaxAggregatedByResource(p int32, rl R } } if len(difference.Resources) > 0 { - a.AddResourceList(p, difference) + a.AddResourceList(t, difference) } } @@ -275,7 +277,7 @@ func (a ResourceList) IsStrictlyNonNegative() bool { return true } -// IsStrictlyLessOrEqual returns true if all quantities in a are strictly less or equal than those in b. +// IsStrictlyLessOrEqual returns false if there is a quantity in b greater than that in a and true otherwise. func (a ResourceList) IsStrictlyLessOrEqual(b ResourceList) bool { for t, q := range b.Resources { if q.Cmp(a.Get(t)) == -1 { @@ -310,7 +312,7 @@ func (rl *ResourceList) initialise() { // AllocatableByPriorityAndResourceType accounts for resources that can be allocated to pods of a given priority. // E.g., AllocatableByPriorityAndResourceType[5]["cpu"] is the amount of CPU available to pods with priority 5, // where alloctable resources = unused resources + resources allocated to lower-priority pods. -type AllocatableByPriorityAndResourceType QuantityByPriorityAndResourceType +type AllocatableByPriorityAndResourceType QuantityByTAndResourceType[int32] func NewAllocatableByPriorityAndResourceType(priorities []int32, rl ResourceList) AllocatableByPriorityAndResourceType { rv := make(AllocatableByPriorityAndResourceType) @@ -366,7 +368,7 @@ func (m AllocatableByPriorityAndResourceType) MarkAllocatableV1ResourceList(p in // AllocatedByPriorityAndResourceType accounts for resources allocated to pods of a given priority or lower. // E.g., AllocatedByPriorityAndResourceType[5]["cpu"] is the amount of CPU allocated to pods with priority 5 or lower. -type AllocatedByPriorityAndResourceType QuantityByPriorityAndResourceType +type AllocatedByPriorityAndResourceType QuantityByTAndResourceType[int32] func NewAllocatedByPriorityAndResourceType(priorities []int32) AllocatedByPriorityAndResourceType { rv := make(AllocatedByPriorityAndResourceType) diff --git a/internal/scheduler/schedulerobjects/resourcelist_test.go b/internal/scheduler/schedulerobjects/resourcelist_test.go index 7308124ba83..f744686ea66 100644 --- a/internal/scheduler/schedulerobjects/resourcelist_test.go +++ b/internal/scheduler/schedulerobjects/resourcelist_test.go @@ -8,11 +8,11 @@ import ( "k8s.io/apimachinery/pkg/api/resource" ) -func TestQuantityByPriorityAndResourceTypeAdd(t *testing.T) { +func TestQuantityByTAndResourceTypeAdd(t *testing.T) { tests := map[string]struct { - a QuantityByPriorityAndResourceType - b QuantityByPriorityAndResourceType - expected QuantityByPriorityAndResourceType + a QuantityByTAndResourceType[int32] + b QuantityByTAndResourceType[int32] + expected QuantityByTAndResourceType[int32] }{ "nil and nil": { a: nil, @@ -20,34 +20,34 @@ func TestQuantityByPriorityAndResourceTypeAdd(t *testing.T) { expected: nil, }, "empty and nil": { - a: QuantityByPriorityAndResourceType{}, + a: QuantityByTAndResourceType[int32]{}, b: nil, - expected: QuantityByPriorityAndResourceType{}, + expected: QuantityByTAndResourceType[int32]{}, }, "nil and empty": { a: nil, - b: QuantityByPriorityAndResourceType{}, + b: QuantityByTAndResourceType[int32]{}, expected: nil, }, "matching": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("4")}}, }, }, "mismatched resources": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"bar": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "foo": resource.MustParse("3"), @@ -57,13 +57,13 @@ func TestQuantityByPriorityAndResourceTypeAdd(t *testing.T) { }, }, "mismatched priorities": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 1: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, @@ -77,11 +77,11 @@ func TestQuantityByPriorityAndResourceTypeAdd(t *testing.T) { } } -func TestQuantityByPriorityAndResourceTypeSub(t *testing.T) { +func TestQuantityByTAndResourceTypeSub(t *testing.T) { tests := map[string]struct { - a QuantityByPriorityAndResourceType - b QuantityByPriorityAndResourceType - expected QuantityByPriorityAndResourceType + a QuantityByTAndResourceType[int32] + b QuantityByTAndResourceType[int32] + expected QuantityByTAndResourceType[int32] }{ "nil and nil": { a: nil, @@ -89,34 +89,34 @@ func TestQuantityByPriorityAndResourceTypeSub(t *testing.T) { expected: nil, }, "empty and nil": { - a: QuantityByPriorityAndResourceType{}, + a: QuantityByTAndResourceType[int32]{}, b: nil, - expected: QuantityByPriorityAndResourceType{}, + expected: QuantityByTAndResourceType[int32]{}, }, "nil and empty": { a: nil, - b: QuantityByPriorityAndResourceType{}, + b: QuantityByTAndResourceType[int32]{}, expected: nil, }, "matching": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("2")}}, }, }, "mismatched resources": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"bar": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "foo": resource.MustParse("3"), @@ -126,13 +126,13 @@ func TestQuantityByPriorityAndResourceTypeSub(t *testing.T) { }, }, "mismatched priorities": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 1: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("3")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("-1")}}, }, @@ -146,66 +146,66 @@ func TestQuantityByPriorityAndResourceTypeSub(t *testing.T) { } } -func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { +func TestQuantityByTAndResourceTypeEqual(t *testing.T) { tests := map[string]struct { - a QuantityByPriorityAndResourceType - b QuantityByPriorityAndResourceType + a QuantityByTAndResourceType[int32] + b QuantityByTAndResourceType[int32] expected bool }{ "both empty": { - a: QuantityByPriorityAndResourceType{}, - b: QuantityByPriorityAndResourceType{}, + a: QuantityByTAndResourceType[int32]{}, + b: QuantityByTAndResourceType[int32]{}, expected: true, }, "both with an empty map": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{}, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{}, }, expected: true, }, "one empty map": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{}, }, - b: QuantityByPriorityAndResourceType{}, + b: QuantityByTAndResourceType[int32]{}, expected: true, }, "zero equals empty": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "foo": resource.MustParse("0"), }, }, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{}, }, expected: true, }, "zero equals missing": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{}, }, - b: QuantityByPriorityAndResourceType{}, + b: QuantityByTAndResourceType[int32]{}, expected: true, }, "zero equals missing with empty ResourceList": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "foo": resource.MustParse("0"), }, }, }, - b: QuantityByPriorityAndResourceType{}, + b: QuantityByTAndResourceType[int32]{}, expected: true, }, "simple equal": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -214,7 +214,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { }, }, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -226,7 +226,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { expected: true, }, "equal with two priorities": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -242,7 +242,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { }, }, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -261,7 +261,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { expected: true, }, "simple unequal": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -270,7 +270,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { }, }, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -282,7 +282,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { expected: false, }, "unequal differing priority": { - a: QuantityByPriorityAndResourceType{ + a: QuantityByTAndResourceType[int32]{ 0: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -291,7 +291,7 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { }, }, }, - b: QuantityByPriorityAndResourceType{ + b: QuantityByTAndResourceType[int32]{ 1: ResourceList{ Resources: map[string]resource.Quantity{ "cpu": resource.MustParse("1"), @@ -311,9 +311,9 @@ func TestQuantityByPriorityAndResourceTypeEqual(t *testing.T) { } } -func TestQuantityByPriorityAndResourceTypeIsStrictlyNonNegative(t *testing.T) { +func TestQuantityByTAndResourceTypeIsStrictlyNonNegative(t *testing.T) { tests := map[string]struct { - m QuantityByPriorityAndResourceType + m QuantityByTAndResourceType[int32] expected bool }{ "nil": { @@ -321,23 +321,23 @@ func TestQuantityByPriorityAndResourceTypeIsStrictlyNonNegative(t *testing.T) { expected: true, }, "empty": { - m: QuantityByPriorityAndResourceType{}, + m: QuantityByTAndResourceType[int32]{}, expected: true, }, "simple zero": { - m: QuantityByPriorityAndResourceType{ + m: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("0")}}, }, expected: true, }, "simple positive": { - m: QuantityByPriorityAndResourceType{ + m: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, }, expected: true, }, "simple positive and negative": { - m: QuantityByPriorityAndResourceType{ + m: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"foo": resource.MustParse("1")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"bar": resource.MustParse("-1")}}, }, @@ -351,80 +351,80 @@ func TestQuantityByPriorityAndResourceTypeIsStrictlyNonNegative(t *testing.T) { } } -func TestQuantityByPriorityAndResourceTypeMaxAggregatedByResource(t *testing.T) { +func TestQuantityByTAndResourceTypeMaxAggregatedByResource(t *testing.T) { tests := map[string]struct { - q QuantityByPriorityAndResourceType + q QuantityByTAndResourceType[int32] p int32 rl ResourceList - expected QuantityByPriorityAndResourceType + expected QuantityByTAndResourceType[int32] }{ "no change": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, p: 1, rl: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, }, "empty": { - q: QuantityByPriorityAndResourceType{}, + q: QuantityByTAndResourceType[int32]{}, p: 0, rl: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, }, "add same resource at same priority": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, p: 0, rl: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("2")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("2")}}, }, }, "add different resource at same priority": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, p: 0, rl: ResourceList{Resources: map[string]resource.Quantity{"memory": resource.MustParse("1Gi")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1"), "memory": resource.MustParse("1Gi")}}, }, }, "add same resource at different priority": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, p: 1, rl: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("2")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, }, "add different resource at different priority": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, }, p: 1, rl: ResourceList{Resources: map[string]resource.Quantity{"memory": resource.MustParse("1Gi")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("1")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"memory": resource.MustParse("1Gi")}}, }, }, "multiple resources": { - q: QuantityByPriorityAndResourceType{ + q: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("100m"), "memory": resource.MustParse("50Mi")}}, }, p: 1, rl: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("10"), "memory": resource.MustParse("4000Mi")}}, - expected: QuantityByPriorityAndResourceType{ + expected: QuantityByTAndResourceType[int32]{ 0: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("100m"), "memory": resource.MustParse("50Mi")}}, 1: ResourceList{Resources: map[string]resource.Quantity{"cpu": resource.MustParse("9900m"), "memory": resource.MustParse("3950Mi")}}, }, @@ -940,3 +940,34 @@ func BenchmarkResourceListZeroAdd(b *testing.B) { rla.Add(rlb) } } + +func BenchmarkQuantityByTAndResourceTypeAdd(b *testing.B) { + dst := make(QuantityByTAndResourceType[string], 3) + src := QuantityByTAndResourceType[string]{ + "1": ResourceList{ + Resources: map[string]resource.Quantity{ + "foo": resource.MustParse("1"), + "bar": resource.MustParse("2"), + "baz": resource.MustParse("3"), + }, + }, + "2": ResourceList{ + Resources: map[string]resource.Quantity{ + "foo": resource.MustParse("1"), + "bar": resource.MustParse("2"), + "baz": resource.MustParse("3"), + }, + }, + "3": ResourceList{ + Resources: map[string]resource.Quantity{ + "foo": resource.MustParse("1"), + "bar": resource.MustParse("2"), + "baz": resource.MustParse("3"), + }, + }, + } + b.ResetTimer() + for n := 0; n < b.N; n++ { + dst.Add(src) + } +} diff --git a/internal/scheduler/schedulerobjects/schedulerobjects.pb.go b/internal/scheduler/schedulerobjects/schedulerobjects.pb.go index ad9614deea0..18cc6ced935 100644 --- a/internal/scheduler/schedulerobjects/schedulerobjects.pb.go +++ b/internal/scheduler/schedulerobjects/schedulerobjects.pb.go @@ -443,13 +443,13 @@ func (m *NodeType) GetUnsetIndexedLabels() map[string]string { return nil } -// Captures the resource usage of a particular queue -// in a given cluster. +// Captures the resource usage of a particular queue in a given cluster. type QueueClusterResourceUsage struct { - Created time.Time `protobuf:"bytes,1,opt,name=created,proto3,stdtime" json:"created"` - Queue string `protobuf:"bytes,2,opt,name=queue,proto3" json:"queue,omitempty"` - ExecutorId string `protobuf:"bytes,3,opt,name=executorId,proto3" json:"executorId,omitempty"` - ResourcesByPriority map[int32]ResourceList `protobuf:"bytes,4,rep,name=resourcesByPriority,proto3" json:"resourcesByPriority" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Created time.Time `protobuf:"bytes,1,opt,name=created,proto3,stdtime" json:"created"` + Queue string `protobuf:"bytes,2,opt,name=queue,proto3" json:"queue,omitempty"` + ExecutorId string `protobuf:"bytes,3,opt,name=executorId,proto3" json:"executorId,omitempty"` + ResourcesByPriority map[int32]ResourceList `protobuf:"bytes,4,rep,name=resourcesByPriority,proto3" json:"resourcesByPriority" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Deprecated: Do not use. + ResourcesByPriorityClassName map[string]ResourceList `protobuf:"bytes,5,rep,name=resourcesByPriorityClassName,proto3" json:"resourcesByPriorityClassName" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (m *QueueClusterResourceUsage) Reset() { *m = QueueClusterResourceUsage{} } @@ -506,6 +506,7 @@ func (m *QueueClusterResourceUsage) GetExecutorId() string { return "" } +// Deprecated: Do not use. func (m *QueueClusterResourceUsage) GetResourcesByPriority() map[int32]ResourceList { if m != nil { return m.ResourcesByPriority @@ -513,6 +514,13 @@ func (m *QueueClusterResourceUsage) GetResourcesByPriority() map[int32]ResourceL return nil } +func (m *QueueClusterResourceUsage) GetResourcesByPriorityClassName() map[string]ResourceList { + if m != nil { + return m.ResourcesByPriorityClassName + } + return nil +} + // A collection of QueueClusterResourceUsage // This is only needed to brige the gap between the redis based scheduler and the new scheduler. type ClusterResourceUsageReport struct { @@ -989,6 +997,7 @@ func init() { proto.RegisterMapType((map[string]string)(nil), "schedulerobjects.NodeType.LabelsEntry") proto.RegisterMapType((map[string]string)(nil), "schedulerobjects.NodeType.UnsetIndexedLabelsEntry") proto.RegisterType((*QueueClusterResourceUsage)(nil), "schedulerobjects.QueueClusterResourceUsage") + proto.RegisterMapType((map[string]ResourceList)(nil), "schedulerobjects.QueueClusterResourceUsage.ResourcesByPriorityClassNameEntry") proto.RegisterMapType((map[int32]ResourceList)(nil), "schedulerobjects.QueueClusterResourceUsage.ResourcesByPriorityEntry") proto.RegisterType((*ClusterResourceUsageReport)(nil), "schedulerobjects.ClusterResourceUsageReport") proto.RegisterMapType((map[string]*QueueClusterResourceUsage)(nil), "schedulerobjects.ClusterResourceUsageReport.ResourcesByQueueEntry") @@ -1007,141 +1016,143 @@ func init() { } var fileDescriptor_97dadc5fbd620721 = []byte{ - // 2132 bytes of a gzipped FileDescriptorProto + // 2173 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0x4d, 0x6f, 0x1b, 0xc7, - 0x55, 0x2b, 0x52, 0x12, 0x39, 0x94, 0x25, 0x6a, 0x64, 0xd9, 0x2b, 0xda, 0xe6, 0x32, 0x4c, 0x1a, - 0x28, 0x8d, 0x43, 0x36, 0x4e, 0x81, 0x1a, 0x6e, 0x2f, 0xa2, 0xa5, 0xd6, 0x94, 0x1d, 0x4a, 0x5e, - 0x4a, 0x2d, 0x5a, 0xa0, 0x59, 0x2c, 0xb9, 0x23, 0x7a, 0xa3, 0xe5, 0x0c, 0xbd, 0x3b, 0xab, 0x86, - 0x39, 0xb7, 0x87, 0x22, 0x40, 0x1a, 0xb4, 0x41, 0x11, 0xa0, 0x40, 0x8b, 0xdc, 0xfa, 0x0b, 0x7a, - 0xe9, 0xad, 0xa7, 0x1c, 0x73, 0xec, 0x89, 0x2d, 0xec, 0x1b, 0x8f, 0x45, 0x7f, 0x40, 0x31, 0x33, - 0xbb, 0xdc, 0xe1, 0xee, 0x52, 0x94, 0xd3, 0xba, 0x3e, 0x71, 0xe7, 0x7d, 0x7f, 0xcd, 0x9b, 0x37, - 0x43, 0x70, 0xcf, 0xc6, 0x14, 0xb9, 0xd8, 0x74, 0xea, 0x5e, 0xf7, 0x09, 0xb2, 0x7c, 0x07, 0xb9, - 0xd1, 0x17, 0xe9, 0x7c, 0x88, 0xba, 0xd4, 0x4b, 0x00, 0x6a, 0x03, 0x97, 0x50, 0x02, 0x8b, 0x71, - 0x78, 0x49, 0xeb, 0x11, 0xd2, 0x73, 0x50, 0x9d, 0xe3, 0x3b, 0xfe, 0x69, 0x9d, 0xda, 0x7d, 0xe4, - 0x51, 0xb3, 0x3f, 0x10, 0x2c, 0xa5, 0xea, 0xd9, 0x5d, 0xaf, 0x66, 0x93, 0xba, 0x39, 0xb0, 0xeb, - 0x5d, 0xe2, 0xa2, 0xfa, 0xf9, 0xbb, 0xf5, 0x1e, 0xc2, 0xc8, 0x35, 0x29, 0xb2, 0x02, 0x9a, 0xef, - 0x46, 0x34, 0x7d, 0xb3, 0xfb, 0xc4, 0xc6, 0xc8, 0x1d, 0xd6, 0x07, 0x67, 0x3d, 0xce, 0xe4, 0x22, - 0x8f, 0xf8, 0x6e, 0x17, 0x25, 0xb8, 0xde, 0xe9, 0xd9, 0xf4, 0x89, 0xdf, 0xa9, 0x75, 0x49, 0xbf, - 0xde, 0x23, 0x3d, 0x12, 0xd9, 0xc0, 0x56, 0x7c, 0xc1, 0xbf, 0x04, 0x79, 0xf5, 0xcf, 0x19, 0x90, - 0xdb, 0xff, 0x08, 0x75, 0x7d, 0x4a, 0x5c, 0x58, 0x01, 0x8b, 0xb6, 0xa5, 0x2a, 0x15, 0x65, 0x27, - 0xdf, 0x28, 0x8e, 0x47, 0xda, 0xaa, 0x6d, 0xdd, 0x26, 0x7d, 0x9b, 0xa2, 0xfe, 0x80, 0x0e, 0xf5, - 0x45, 0xdb, 0x82, 0x6f, 0x82, 0xec, 0x80, 0x10, 0x47, 0x5d, 0xe4, 0x34, 0x70, 0x3c, 0xd2, 0xd6, - 0xd8, 0x5a, 0xa2, 0xe2, 0x78, 0xb8, 0x0b, 0x96, 0x30, 0xb1, 0x90, 0xa7, 0x66, 0x2a, 0x99, 0x9d, - 0xc2, 0x9d, 0x6b, 0xb5, 0x44, 0xe8, 0x5a, 0xc4, 0x42, 0x8d, 0xcd, 0xf1, 0x48, 0x5b, 0xe7, 0x84, - 0x92, 0x04, 0xc1, 0x09, 0x3f, 0x00, 0x6b, 0x7d, 0x1b, 0xdb, 0x7d, 0xbf, 0x7f, 0x40, 0x3a, 0x6d, - 0xfb, 0x63, 0xa4, 0x66, 0x2b, 0xca, 0x4e, 0xe1, 0x4e, 0x39, 0x29, 0x4b, 0x0f, 0x82, 0xf1, 0xc8, - 0xf6, 0x68, 0xe3, 0xda, 0x57, 0x23, 0x6d, 0x81, 0x19, 0x36, 0xcd, 0xad, 0xc7, 0xd6, 0x4c, 0xbe, - 0x63, 0x7a, 0xf4, 0x64, 0x60, 0x99, 0x14, 0x1d, 0xdb, 0x7d, 0xa4, 0x2e, 0x71, 0xf9, 0xa5, 0x9a, - 0x48, 0x5e, 0x2d, 0x0c, 0x5c, 0xed, 0x38, 0x4c, 0x5e, 0xa3, 0x14, 0xca, 0x9e, 0xe6, 0xfc, 0xec, - 0x1f, 0x9a, 0xa2, 0xc7, 0x60, 0xf0, 0x10, 0x6c, 0xfa, 0xd8, 0xf4, 0x3c, 0xbb, 0x87, 0x91, 0x65, - 0x7c, 0x48, 0x3a, 0x86, 0xeb, 0x63, 0x4f, 0xcd, 0x57, 0x32, 0x3b, 0xf9, 0x86, 0x36, 0x1e, 0x69, - 0x37, 0x22, 0xf4, 0x01, 0xe9, 0xe8, 0x3e, 0x96, 0x83, 0xb0, 0x91, 0x40, 0x56, 0xff, 0xbd, 0x05, - 0xb2, 0x2c, 0x6a, 0x97, 0x4b, 0x13, 0x36, 0xfb, 0x48, 0x5d, 0x8d, 0xd2, 0xc4, 0xd6, 0x72, 0x9a, - 0xd8, 0x1a, 0xde, 0x05, 0x80, 0x05, 0x7b, 0xaf, 0xf3, 0x10, 0x0d, 0x3d, 0x15, 0x56, 0x32, 0x3b, - 0xab, 0x0d, 0x75, 0x3c, 0xd2, 0xae, 0x46, 0x50, 0x89, 0x47, 0xa2, 0x85, 0xef, 0x83, 0x3c, 0xf3, - 0xd7, 0xf0, 0x10, 0xc2, 0xbc, 0x1a, 0x2e, 0x0e, 0xdc, 0xd5, 0x20, 0x70, 0x39, 0xc6, 0xd4, 0x46, - 0x08, 0xf3, 0x90, 0x4d, 0x56, 0xf0, 0x10, 0xe4, 0x99, 0x70, 0x83, 0x0e, 0x07, 0x48, 0xcd, 0x04, - 0xe2, 0x52, 0x6b, 0xe6, 0x78, 0x38, 0x40, 0x8d, 0x6b, 0xe3, 0x91, 0x06, 0x71, 0xb0, 0x92, 0x2c, - 0xcc, 0x85, 0x30, 0x78, 0x0f, 0xac, 0x4e, 0x04, 0x1a, 0xb6, 0xc5, 0x6b, 0x27, 0x1b, 0xf9, 0xc6, - 0x68, 0x9a, 0x56, 0xdc, 0x37, 0x01, 0x85, 0xbb, 0x60, 0x99, 0x9a, 0x36, 0xa6, 0x9e, 0xba, 0xc4, - 0xab, 0x77, 0xbb, 0x26, 0x76, 0x62, 0xcd, 0x1c, 0xd8, 0x35, 0xb6, 0x5b, 0x6b, 0xe7, 0xef, 0xd6, - 0x8e, 0x19, 0x45, 0x63, 0x2d, 0xf0, 0x2b, 0x60, 0xd0, 0x83, 0x5f, 0x78, 0x04, 0x96, 0x1d, 0xb3, - 0x83, 0x1c, 0x4f, 0x5d, 0xe6, 0x22, 0xaa, 0xe9, 0xce, 0xd4, 0x1e, 0x71, 0xa2, 0x7d, 0x4c, 0xdd, - 0x61, 0xe3, 0xea, 0x78, 0xa4, 0x15, 0x05, 0x97, 0x64, 0x58, 0x20, 0x07, 0x1a, 0x60, 0x9d, 0x12, - 0x6a, 0x3a, 0x46, 0xb8, 0xf3, 0x3d, 0x75, 0xe5, 0xc5, 0xf6, 0x03, 0x67, 0x0f, 0x51, 0x9e, 0x1e, - 0x5b, 0xc3, 0xbf, 0x28, 0xe0, 0x0d, 0xd3, 0x71, 0x48, 0xd7, 0xa4, 0x66, 0xc7, 0x41, 0x46, 0x67, - 0x68, 0x0c, 0x5c, 0x9b, 0xb8, 0x36, 0x1d, 0x1a, 0x26, 0xb6, 0x26, 0x7a, 0xd5, 0x1c, 0xf7, 0xe8, - 0x07, 0x33, 0x3c, 0xda, 0x8d, 0x44, 0x34, 0x86, 0x47, 0x81, 0x80, 0x5d, 0x6c, 0x85, 0x8a, 0x84, - 0xaf, 0x3b, 0x81, 0x51, 0x15, 0x73, 0x0e, 0xb9, 0x3e, 0x97, 0x02, 0xba, 0x60, 0xd3, 0xa3, 0x26, - 0xe5, 0x16, 0x07, 0xdb, 0x8c, 0x65, 0x3c, 0xcf, 0xcd, 0x7c, 0x7b, 0x86, 0x99, 0x6d, 0xc6, 0xd1, - 0x18, 0x8a, 0xbd, 0xd5, 0xb4, 0x84, 0x55, 0xd7, 0x03, 0xab, 0xd6, 0xbd, 0x69, 0xac, 0x1e, 0x07, - 0x40, 0x1f, 0x6c, 0x06, 0x76, 0x21, 0x2b, 0xd4, 0x6b, 0x5b, 0x2a, 0xe0, 0x3a, 0x6f, 0x5f, 0x1c, - 0x1a, 0x64, 0x71, 0x41, 0xa1, 0x52, 0x35, 0x50, 0x5a, 0x34, 0x63, 0x68, 0x3d, 0x01, 0x81, 0x14, - 0xc0, 0x29, 0xb5, 0x4f, 0x7d, 0xe4, 0x23, 0xb5, 0x70, 0x59, 0xad, 0x8f, 0x19, 0xf9, 0x6c, 0xad, - 0x1c, 0xad, 0x27, 0x20, 0xcc, 0x59, 0x74, 0x6e, 0x77, 0x69, 0xd4, 0xc6, 0x0c, 0xdb, 0xf2, 0xd4, - 0xb5, 0x0b, 0xd5, 0xee, 0x0b, 0x8e, 0x30, 0x62, 0x5e, 0x4c, 0x2d, 0x8a, 0xa1, 0xf5, 0x04, 0x04, - 0x7e, 0xa9, 0x80, 0x32, 0x26, 0xd8, 0x30, 0xdd, 0xbe, 0x69, 0x99, 0x46, 0xe4, 0x78, 0xb4, 0x03, - 0xae, 0x70, 0x13, 0xbe, 0x37, 0xc3, 0x84, 0x16, 0xc1, 0xbb, 0x9c, 0x77, 0x12, 0x82, 0x49, 0xb5, - 0x0b, 0x6b, 0x5e, 0x0f, 0xac, 0xb9, 0x81, 0x67, 0x53, 0xea, 0x17, 0x21, 0xe1, 0x2e, 0xb8, 0xe2, - 0xe3, 0x40, 0x3b, 0xab, 0x50, 0x75, 0xbd, 0xa2, 0xec, 0xe4, 0x1a, 0x37, 0xc6, 0x23, 0xed, 0xfa, - 0x14, 0x42, 0xda, 0xd1, 0xd3, 0x1c, 0xf0, 0x13, 0x05, 0x5c, 0x0f, 0x3d, 0x32, 0x7c, 0xcf, 0xec, - 0xa1, 0x28, 0xb3, 0x45, 0xee, 0xdf, 0x77, 0x66, 0xf8, 0x17, 0x9a, 0x71, 0xc2, 0x98, 0xa6, 0xb2, - 0x5b, 0x1d, 0x8f, 0xb4, 0xb2, 0x9b, 0x82, 0x96, 0xcc, 0xb8, 0x9a, 0x86, 0x67, 0xa7, 0x96, 0x8b, - 0x06, 0xc4, 0xa5, 0x36, 0xee, 0x19, 0x51, 0x4b, 0xde, 0xe0, 0x07, 0x09, 0x3f, 0xb5, 0x26, 0xe8, - 0x56, 0xb2, 0xff, 0x6e, 0x24, 0x90, 0x25, 0x13, 0x14, 0xa4, 0x26, 0x07, 0x5f, 0x07, 0x99, 0x33, - 0x34, 0x0c, 0x0e, 0xaf, 0x8d, 0xf1, 0x48, 0xbb, 0x72, 0x86, 0x86, 0x92, 0x04, 0x86, 0x85, 0x6f, - 0x81, 0xa5, 0x73, 0xd3, 0xf1, 0x51, 0x30, 0x66, 0xf0, 0x29, 0x81, 0x03, 0xe4, 0x29, 0x81, 0x03, - 0xee, 0x2d, 0xde, 0x55, 0x4a, 0x7f, 0x50, 0xc0, 0xb7, 0x2e, 0xd5, 0x76, 0x64, 0xed, 0x4b, 0x33, - 0xb5, 0x37, 0x65, 0xed, 0xf3, 0xfb, 0xeb, 0x3c, 0xeb, 0x7e, 0xad, 0x80, 0xab, 0x69, 0xdd, 0xe6, - 0x72, 0xa1, 0x78, 0x20, 0x1b, 0xb3, 0x76, 0xe7, 0x56, 0xd2, 0x18, 0x21, 0x54, 0x68, 0x98, 0x67, - 0xcb, 0x27, 0x0a, 0xd8, 0x4a, 0xed, 0x42, 0x97, 0x33, 0xe6, 0x7f, 0x1c, 0x99, 0x98, 0x35, 0x51, - 0xfd, 0xbe, 0x12, 0x6b, 0xce, 0xc0, 0x56, 0x6a, 0xcf, 0xfa, 0x06, 0x25, 0x9b, 0x9b, 0xab, 0xec, - 0xf7, 0x0a, 0xa8, 0xcc, 0x6b, 0x4f, 0xaf, 0xa4, 0x5a, 0x7f, 0xa3, 0x80, 0xed, 0x99, 0x7d, 0xe5, - 0x55, 0xe4, 0xa5, 0xfa, 0xc7, 0x2c, 0xc8, 0x85, 0xdd, 0x84, 0x8d, 0xbe, 0x4d, 0x31, 0xfa, 0x66, - 0xc5, 0xe8, 0x3b, 0x35, 0xc4, 0x2d, 0x4e, 0x0d, 0x6f, 0x8b, 0xdf, 0x74, 0x78, 0x3b, 0x9e, 0x0c, - 0x6f, 0xe2, 0xf6, 0xf2, 0xe6, 0xec, 0x49, 0xf4, 0x05, 0x06, 0xb8, 0x5f, 0x2a, 0x00, 0xfa, 0xd8, - 0x43, 0xb4, 0x89, 0x2d, 0xf4, 0x11, 0xb2, 0x04, 0xa7, 0x9a, 0xe5, 0x2a, 0xee, 0x5c, 0xa0, 0xe2, - 0x24, 0xc1, 0x24, 0xd4, 0x55, 0xc6, 0x23, 0xed, 0x66, 0x52, 0xa2, 0xa4, 0x3a, 0x45, 0xdf, 0xff, - 0xa3, 0x1f, 0xf7, 0xc1, 0xf5, 0x19, 0x36, 0xbf, 0x0c, 0x75, 0xd5, 0x7f, 0x65, 0xc0, 0x36, 0xaf, - 0xd1, 0xfb, 0x8e, 0xef, 0x51, 0xe4, 0x4e, 0x95, 0x2f, 0x6c, 0x82, 0x95, 0xae, 0x8b, 0xd8, 0xee, - 0xe2, 0x5a, 0x2f, 0xbe, 0xa6, 0x6c, 0x06, 0x15, 0x11, 0xb2, 0xf0, 0x5b, 0x4a, 0xb8, 0x60, 0x76, - 0x89, 0x63, 0x59, 0xb2, 0xeb, 0x69, 0xec, 0x54, 0x15, 0x14, 0xec, 0x62, 0x85, 0x82, 0x5b, 0x75, - 0xd3, 0xe2, 0x17, 0x9a, 0xbc, 0xb8, 0x7c, 0x44, 0x50, 0xf9, 0xf2, 0x11, 0x41, 0xe1, 0xef, 0x14, - 0x76, 0x02, 0x07, 0x7d, 0x20, 0x3a, 0xca, 0x82, 0x3a, 0xd9, 0x4b, 0xd6, 0xc9, 0x4c, 0xd7, 0x27, - 0xdb, 0x4c, 0x12, 0x23, 0x2a, 0xe7, 0x46, 0xe0, 0x66, 0x9a, 0x22, 0x3d, 0x0d, 0x58, 0xfa, 0x54, - 0x01, 0xea, 0x2c, 0x71, 0xaf, 0xa2, 0x4f, 0x55, 0xff, 0x9a, 0x01, 0xa5, 0x34, 0xa7, 0x75, 0x3e, - 0x80, 0x4c, 0xde, 0x29, 0x94, 0x39, 0xef, 0x14, 0x52, 0x75, 0x2c, 0xfe, 0x97, 0xd5, 0xf1, 0xa9, - 0x02, 0x8a, 0x52, 0xe8, 0x78, 0x5a, 0x82, 0x06, 0xd2, 0x48, 0x3a, 0x3b, 0xdb, 0x76, 0x39, 0x6d, - 0xd2, 0x48, 0x57, 0x1e, 0x8f, 0xb4, 0x52, 0x5c, 0xbe, 0xe4, 0x4f, 0x42, 0x77, 0xe9, 0x0b, 0x05, - 0x6c, 0xa5, 0xca, 0xba, 0xdc, 0x2e, 0xfc, 0xf1, 0x74, 0xc2, 0xde, 0x7e, 0x81, 0xca, 0x9b, 0x9b, - 0xbd, 0x5f, 0x2d, 0x82, 0x55, 0x39, 0xdd, 0xf0, 0x03, 0x90, 0x8f, 0xa6, 0x7a, 0x85, 0x07, 0xed, - 0x9d, 0x8b, 0x2b, 0xa4, 0x16, 0x9b, 0xe5, 0x37, 0x82, 0xe4, 0x44, 0x72, 0xf4, 0xe8, 0xb3, 0xf4, - 0xb9, 0x02, 0xd6, 0x66, 0x9f, 0xae, 0xb3, 0x83, 0xf0, 0xd3, 0xe9, 0x20, 0xd4, 0xa4, 0xc3, 0x64, - 0xf2, 0x26, 0x57, 0x1b, 0x9c, 0xf5, 0xf8, 0xe9, 0x12, 0xaa, 0xab, 0x3d, 0xf6, 0x4d, 0x4c, 0x6d, - 0x3a, 0x9c, 0x1b, 0x87, 0xcf, 0x97, 0xc0, 0xc6, 0x01, 0xe9, 0xb4, 0x85, 0xa3, 0x36, 0xee, 0x35, - 0xf1, 0x29, 0x81, 0x77, 0x40, 0xce, 0xb1, 0x4f, 0x11, 0xb5, 0xfb, 0x88, 0x9b, 0x77, 0x45, 0xbc, - 0x77, 0x84, 0x30, 0xf9, 0xbd, 0x23, 0x84, 0xc1, 0x7b, 0x60, 0xd5, 0xa4, 0x46, 0x9f, 0x78, 0xd4, - 0x20, 0xb8, 0x1b, 0x8e, 0x21, 0xbc, 0xe5, 0x98, 0xf4, 0x7d, 0xe2, 0xd1, 0x43, 0xdc, 0x95, 0x39, - 0x41, 0x04, 0x85, 0xdf, 0x07, 0x85, 0x81, 0x8b, 0x18, 0xdc, 0x66, 0x57, 0x98, 0x0c, 0x67, 0xdd, - 0x1e, 0x8f, 0xb4, 0x2d, 0x09, 0x2c, 0xf1, 0xca, 0xd4, 0xf0, 0x01, 0x28, 0x76, 0x09, 0xee, 0xfa, - 0xae, 0x8b, 0x70, 0x77, 0x68, 0x78, 0xe6, 0xa9, 0x78, 0xa8, 0xcb, 0x35, 0x6e, 0x8d, 0x47, 0xda, - 0xb6, 0x84, 0x6b, 0x9b, 0xa7, 0xb2, 0x94, 0xf5, 0x18, 0x8a, 0x5d, 0x3d, 0x26, 0x0f, 0x0e, 0x5d, - 0xc7, 0xf4, 0x3c, 0x83, 0xbf, 0x61, 0x2d, 0x47, 0x57, 0x8f, 0x10, 0x7d, 0x9f, 0x61, 0x5b, 0xd3, - 0x0f, 0x5a, 0x1b, 0x09, 0x24, 0x6c, 0x83, 0x82, 0xe7, 0x77, 0xfa, 0x36, 0x35, 0x78, 0x28, 0x57, - 0xe6, 0x6e, 0xf0, 0xf0, 0xa9, 0x04, 0x08, 0xb6, 0xc9, 0xd3, 0x9e, 0xb4, 0x66, 0xc9, 0x09, 0x35, - 0xa9, 0xb9, 0x28, 0x39, 0x21, 0x4c, 0x4e, 0x4e, 0x08, 0x83, 0xbf, 0x00, 0x9b, 0xa2, 0x84, 0x0d, - 0x17, 0x3d, 0xf5, 0x6d, 0x17, 0xf5, 0x51, 0xf4, 0xba, 0xf4, 0x46, 0xb2, 0xce, 0x0f, 0xf9, 0xaf, - 0x2e, 0xd1, 0x8a, 0xc3, 0x9e, 0x24, 0xe0, 0xf2, 0x61, 0x9f, 0xc4, 0xc2, 0x3a, 0x58, 0x39, 0x47, - 0xae, 0x67, 0x13, 0xac, 0xe6, 0xb9, 0xad, 0x5b, 0xe3, 0x91, 0xb6, 0x11, 0x80, 0x24, 0xde, 0x90, - 0xea, 0x5e, 0xf6, 0x8b, 0x2f, 0x35, 0xa5, 0xfa, 0x5b, 0x05, 0xc0, 0xa4, 0x0d, 0xd0, 0x01, 0xeb, - 0x03, 0x62, 0xc9, 0xa0, 0xe0, 0x48, 0x7d, 0x2d, 0xe9, 0xc2, 0xd1, 0x34, 0xa1, 0x28, 0x86, 0x18, - 0x77, 0x64, 0xc0, 0x83, 0x05, 0x3d, 0x2e, 0xba, 0xb1, 0x06, 0x56, 0xe5, 0x68, 0x55, 0xff, 0xb6, - 0x02, 0xd6, 0x63, 0x52, 0xa1, 0x27, 0x5e, 0xf9, 0xda, 0xc8, 0x41, 0x5d, 0x4a, 0xdc, 0xa0, 0x73, - 0xbc, 0x37, 0xd7, 0x1c, 0x3e, 0x5c, 0x85, 0x5c, 0xa2, 0x7f, 0x94, 0xc6, 0x23, 0xed, 0x9a, 0x2c, - 0x4c, 0x0a, 0xcf, 0x94, 0x12, 0x78, 0x04, 0x72, 0xe6, 0xe9, 0xa9, 0x8d, 0x59, 0x05, 0x88, 0xb6, - 0x70, 0x33, 0x6d, 0xc6, 0xdc, 0x0d, 0x68, 0x44, 0x7d, 0x84, 0x1c, 0x72, 0x7d, 0x84, 0x30, 0x78, - 0x02, 0x0a, 0x94, 0x38, 0xc8, 0x35, 0xa9, 0x4d, 0x70, 0x38, 0x75, 0x96, 0x53, 0x07, 0xd7, 0x09, - 0xd9, 0xe4, 0x34, 0x92, 0x59, 0x75, 0x79, 0x01, 0x09, 0x28, 0x98, 0x18, 0x13, 0x1a, 0x88, 0x5d, - 0x99, 0x35, 0x69, 0xc6, 0x83, 0xb3, 0x1b, 0x31, 0x89, 0xd8, 0xf0, 0x5e, 0x20, 0x89, 0x92, 0x7b, - 0x81, 0x04, 0x9e, 0xda, 0x1b, 0x59, 0x3e, 0x0d, 0xcc, 0xdf, 0x1b, 0x07, 0xa0, 0x18, 0xb6, 0x13, - 0x82, 0x8f, 0x88, 0x63, 0x77, 0x87, 0xfc, 0x21, 0x3e, 0x2f, 0x4e, 0xbc, 0x38, 0x4e, 0x3e, 0xf1, - 0xe2, 0x38, 0xf8, 0x31, 0x98, 0x3c, 0x6a, 0x4c, 0x55, 0xe9, 0x32, 0xcf, 0xd2, 0x4e, 0x5a, 0x40, - 0xf5, 0x14, 0xfa, 0xc6, 0xcd, 0x20, 0xb4, 0xa9, 0xd2, 0xf4, 0x54, 0x28, 0x6c, 0x83, 0xcd, 0xae, - 0xc9, 0x22, 0x1b, 0x35, 0xf3, 0x87, 0x48, 0xb4, 0x88, 0xd5, 0xc6, 0x6b, 0xe3, 0x91, 0x76, 0x2b, - 0x05, 0x2d, 0x79, 0x93, 0xc6, 0x5d, 0xea, 0x81, 0x8d, 0x44, 0xa5, 0xbe, 0x94, 0x91, 0xfd, 0x14, - 0x14, 0xe3, 0x59, 0x7f, 0x29, 0xb3, 0xfa, 0x9f, 0x14, 0xb0, 0x7d, 0xe4, 0x3b, 0x9e, 0xe9, 0xb6, - 0xc3, 0x2a, 0x3c, 0x20, 0x9d, 0x3d, 0x44, 0x4d, 0xdb, 0xf1, 0x98, 0x30, 0xfe, 0x24, 0x11, 0xe8, - 0xe4, 0xc2, 0x38, 0x40, 0x16, 0x26, 0x5e, 0x42, 0xdf, 0x02, 0x4b, 0x8f, 0xe3, 0xb3, 0x78, 0x7c, - 0x24, 0x12, 0x14, 0xf0, 0x36, 0x58, 0x66, 0x67, 0x2c, 0xa2, 0xc1, 0x1c, 0xce, 0xaf, 0x69, 0x02, - 0x22, 0x5f, 0xd3, 0x04, 0xe4, 0xdb, 0x87, 0xa0, 0x20, 0xbd, 0xa8, 0xc0, 0x02, 0x58, 0x39, 0x69, - 0x3d, 0x6c, 0x1d, 0xfe, 0xa4, 0x55, 0x5c, 0x60, 0x8b, 0xa3, 0xfd, 0xd6, 0x5e, 0xb3, 0xf5, 0xa3, - 0xa2, 0xc2, 0x16, 0xfa, 0x49, 0xab, 0xc5, 0x16, 0x8b, 0xf0, 0x0a, 0xc8, 0xb7, 0x4f, 0xee, 0xdf, - 0xdf, 0xdf, 0xdf, 0xdb, 0xdf, 0x2b, 0x66, 0x20, 0x00, 0xcb, 0x3f, 0xdc, 0x6d, 0x3e, 0xda, 0xdf, - 0x2b, 0x66, 0x1b, 0x3f, 0xff, 0xea, 0x59, 0x59, 0xf9, 0xfa, 0x59, 0x59, 0xf9, 0xe7, 0xb3, 0xb2, - 0xf2, 0xd9, 0xf3, 0xf2, 0xc2, 0xd7, 0xcf, 0xcb, 0x0b, 0x7f, 0x7f, 0x5e, 0x5e, 0xf8, 0xd9, 0x7d, - 0xe9, 0xaf, 0x3a, 0xf1, 0xc8, 0x39, 0x70, 0x09, 0xdb, 0x92, 0xc1, 0xaa, 0x7e, 0x89, 0xff, 0x24, - 0x3b, 0xcb, 0xfc, 0x1c, 0x7b, 0xef, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xfd, 0xfc, 0x96, 0xed, - 0xc1, 0x1c, 0x00, 0x00, + 0x55, 0x2b, 0x52, 0x12, 0x39, 0x94, 0x25, 0x6a, 0x64, 0xd9, 0x2b, 0xc6, 0xe6, 0xd2, 0x8a, 0x1b, + 0x28, 0x8d, 0x43, 0x36, 0x4e, 0x81, 0x1a, 0x6e, 0x2f, 0xa2, 0xa5, 0xd6, 0xf4, 0x07, 0x25, 0x2f, + 0xa5, 0x16, 0x2d, 0xd0, 0x2c, 0x96, 0xdc, 0x11, 0xbd, 0xd1, 0x72, 0x86, 0xde, 0x9d, 0x55, 0xc3, + 0x9c, 0xdb, 0x43, 0x11, 0x20, 0x0d, 0x8a, 0xb4, 0x0d, 0x50, 0xa0, 0x45, 0x6e, 0xfd, 0x05, 0xed, + 0xa1, 0xb7, 0x9e, 0x7c, 0xcc, 0xb1, 0x27, 0xb6, 0xb0, 0x6f, 0x3c, 0xf7, 0x07, 0x14, 0x33, 0xb3, + 0xcb, 0x1d, 0xee, 0x2e, 0x45, 0x39, 0xa9, 0xab, 0x13, 0x39, 0xef, 0x7b, 0xde, 0x7b, 0xf3, 0xe6, + 0xbd, 0x59, 0x70, 0xd7, 0xc6, 0x14, 0xb9, 0xd8, 0x74, 0x6a, 0x5e, 0xe7, 0x29, 0xb2, 0x7c, 0x07, + 0xb9, 0xd1, 0x3f, 0xd2, 0xfe, 0x10, 0x75, 0xa8, 0x97, 0x00, 0x54, 0xfb, 0x2e, 0xa1, 0x04, 0x16, + 0xe3, 0xf0, 0x92, 0xd6, 0x25, 0xa4, 0xeb, 0xa0, 0x1a, 0xc7, 0xb7, 0xfd, 0xe3, 0x1a, 0xb5, 0x7b, + 0xc8, 0xa3, 0x66, 0xaf, 0x2f, 0x58, 0x4a, 0x5b, 0x27, 0x77, 0xbc, 0xaa, 0x4d, 0x6a, 0x66, 0xdf, + 0xae, 0x75, 0x88, 0x8b, 0x6a, 0xa7, 0xef, 0xd5, 0xba, 0x08, 0x23, 0xd7, 0xa4, 0xc8, 0x0a, 0x68, + 0xbe, 0x1b, 0xd1, 0xf4, 0xcc, 0xce, 0x53, 0x1b, 0x23, 0x77, 0x50, 0xeb, 0x9f, 0x74, 0x39, 0x93, + 0x8b, 0x3c, 0xe2, 0xbb, 0x1d, 0x94, 0xe0, 0x7a, 0xb7, 0x6b, 0xd3, 0xa7, 0x7e, 0xbb, 0xda, 0x21, + 0xbd, 0x5a, 0x97, 0x74, 0x49, 0x64, 0x03, 0x5b, 0xf1, 0x05, 0xff, 0x27, 0xc8, 0xb7, 0xfe, 0x92, + 0x01, 0xb9, 0xbd, 0x8f, 0x50, 0xc7, 0xa7, 0xc4, 0x85, 0x15, 0x30, 0x6f, 0x5b, 0xaa, 0x52, 0x51, + 0xb6, 0xf3, 0xf5, 0xe2, 0x68, 0xa8, 0x2d, 0xdb, 0xd6, 0x2d, 0xd2, 0xb3, 0x29, 0xea, 0xf5, 0xe9, + 0x40, 0x9f, 0xb7, 0x2d, 0xf8, 0x16, 0xc8, 0xf6, 0x09, 0x71, 0xd4, 0x79, 0x4e, 0x03, 0x47, 0x43, + 0x6d, 0x85, 0xad, 0x25, 0x2a, 0x8e, 0x87, 0x3b, 0x60, 0x01, 0x13, 0x0b, 0x79, 0x6a, 0xa6, 0x92, + 0xd9, 0x2e, 0xdc, 0xbe, 0x52, 0x4d, 0xb8, 0xae, 0x49, 0x2c, 0x54, 0x5f, 0x1f, 0x0d, 0xb5, 0x55, + 0x4e, 0x28, 0x49, 0x10, 0x9c, 0xf0, 0x03, 0xb0, 0xd2, 0xb3, 0xb1, 0xdd, 0xf3, 0x7b, 0x0f, 0x48, + 0xbb, 0x65, 0x7f, 0x8c, 0xd4, 0x6c, 0x45, 0xd9, 0x2e, 0xdc, 0x2e, 0x27, 0x65, 0xe9, 0x81, 0x33, + 0x1e, 0xd9, 0x1e, 0xad, 0x5f, 0x79, 0x3e, 0xd4, 0xe6, 0x98, 0x61, 0x93, 0xdc, 0x7a, 0x6c, 0xcd, + 0xe4, 0x3b, 0xa6, 0x47, 0x8f, 0xfa, 0x96, 0x49, 0xd1, 0xa1, 0xdd, 0x43, 0xea, 0x02, 0x97, 0x5f, + 0xaa, 0x8a, 0xe0, 0x55, 0x43, 0xc7, 0x55, 0x0f, 0xc3, 0xe0, 0xd5, 0x4b, 0xa1, 0xec, 0x49, 0xce, + 0xcf, 0xfe, 0xa5, 0x29, 0x7a, 0x0c, 0x06, 0xf7, 0xc1, 0xba, 0x8f, 0x4d, 0xcf, 0xb3, 0xbb, 0x18, + 0x59, 0xc6, 0x87, 0xa4, 0x6d, 0xb8, 0x3e, 0xf6, 0xd4, 0x7c, 0x25, 0xb3, 0x9d, 0xaf, 0x6b, 0xa3, + 0xa1, 0xf6, 0x46, 0x84, 0x7e, 0x40, 0xda, 0xba, 0x8f, 0x65, 0x27, 0xac, 0x25, 0x90, 0x5b, 0xff, + 0xd9, 0x00, 0x59, 0xe6, 0xb5, 0xf3, 0x85, 0x09, 0x9b, 0x3d, 0xa4, 0x2e, 0x47, 0x61, 0x62, 0x6b, + 0x39, 0x4c, 0x6c, 0x0d, 0xef, 0x00, 0xc0, 0x9c, 0xbd, 0xdb, 0x7e, 0x88, 0x06, 0x9e, 0x0a, 0x2b, + 0x99, 0xed, 0xe5, 0xba, 0x3a, 0x1a, 0x6a, 0x97, 0x23, 0xa8, 0xc4, 0x23, 0xd1, 0xc2, 0xc7, 0x20, + 0xcf, 0xf6, 0x6b, 0x78, 0x08, 0x61, 0x9e, 0x0d, 0x67, 0x3b, 0xee, 0x72, 0xe0, 0xb8, 0x1c, 0x63, + 0x6a, 0x21, 0x84, 0xb9, 0xcb, 0xc6, 0x2b, 0xb8, 0x0f, 0xf2, 0x4c, 0xb8, 0x41, 0x07, 0x7d, 0xa4, + 0x66, 0x02, 0x71, 0xa9, 0x39, 0x73, 0x38, 0xe8, 0xa3, 0xfa, 0x95, 0xd1, 0x50, 0x83, 0x38, 0x58, + 0x49, 0x16, 0xe6, 0x42, 0x18, 0xbc, 0x0b, 0x96, 0xc7, 0x02, 0x0d, 0xdb, 0xe2, 0xb9, 0x93, 0x8d, + 0xf6, 0xc6, 0x68, 0x1a, 0x56, 0x7c, 0x6f, 0x02, 0x0a, 0x77, 0xc0, 0x22, 0x35, 0x6d, 0x4c, 0x3d, + 0x75, 0x81, 0x67, 0xef, 0x66, 0x55, 0x9c, 0xc4, 0xaa, 0xd9, 0xb7, 0xab, 0xec, 0xb4, 0x56, 0x4f, + 0xdf, 0xab, 0x1e, 0x32, 0x8a, 0xfa, 0x4a, 0xb0, 0xaf, 0x80, 0x41, 0x0f, 0x7e, 0xe1, 0x01, 0x58, + 0x74, 0xcc, 0x36, 0x72, 0x3c, 0x75, 0x91, 0x8b, 0xd8, 0x4a, 0xdf, 0x4c, 0xf5, 0x11, 0x27, 0xda, + 0xc3, 0xd4, 0x1d, 0xd4, 0x2f, 0x8f, 0x86, 0x5a, 0x51, 0x70, 0x49, 0x86, 0x05, 0x72, 0xa0, 0x01, + 0x56, 0x29, 0xa1, 0xa6, 0x63, 0x84, 0x27, 0xdf, 0x53, 0x97, 0x5e, 0xed, 0x3c, 0x70, 0xf6, 0x10, + 0xe5, 0xe9, 0xb1, 0x35, 0xfc, 0xab, 0x02, 0x6e, 0x9a, 0x8e, 0x43, 0x3a, 0x26, 0x35, 0xdb, 0x0e, + 0x32, 0xda, 0x03, 0xa3, 0xef, 0xda, 0xc4, 0xb5, 0xe9, 0xc0, 0x30, 0xb1, 0x35, 0xd6, 0xab, 0xe6, + 0xf8, 0x8e, 0x7e, 0x30, 0x65, 0x47, 0x3b, 0x91, 0x88, 0xfa, 0xe0, 0x20, 0x10, 0xb0, 0x83, 0xad, + 0x50, 0x91, 0xd8, 0xeb, 0x76, 0x60, 0x54, 0xc5, 0x9c, 0x41, 0xae, 0xcf, 0xa4, 0x80, 0x2e, 0x58, + 0xf7, 0xa8, 0x49, 0xb9, 0xc5, 0xc1, 0x31, 0x63, 0x11, 0xcf, 0x73, 0x33, 0xdf, 0x99, 0x62, 0x66, + 0x8b, 0x71, 0xd4, 0x07, 0xe2, 0x6c, 0x35, 0x2c, 0x61, 0xd5, 0xd5, 0xc0, 0xaa, 0x55, 0x6f, 0x12, + 0xab, 0xc7, 0x01, 0xd0, 0x07, 0xeb, 0x81, 0x5d, 0xc8, 0x0a, 0xf5, 0xda, 0x96, 0x0a, 0xb8, 0xce, + 0x5b, 0x67, 0xbb, 0x06, 0x59, 0x5c, 0x50, 0xa8, 0x54, 0x0d, 0x94, 0x16, 0xcd, 0x18, 0x5a, 0x4f, + 0x40, 0x20, 0x05, 0x70, 0x42, 0xed, 0x33, 0x1f, 0xf9, 0x48, 0x2d, 0x9c, 0x57, 0xeb, 0x13, 0x46, + 0x3e, 0x5d, 0x2b, 0x47, 0xeb, 0x09, 0x08, 0xdb, 0x2c, 0x3a, 0xb5, 0x3b, 0x34, 0x2a, 0x63, 0x86, + 0x6d, 0x79, 0xea, 0xca, 0x99, 0x6a, 0xf7, 0x04, 0x47, 0xe8, 0x31, 0x2f, 0xa6, 0x16, 0xc5, 0xd0, + 0x7a, 0x02, 0x02, 0xbf, 0x54, 0x40, 0x19, 0x13, 0x6c, 0x98, 0x6e, 0xcf, 0xb4, 0x4c, 0x23, 0xda, + 0x78, 0x74, 0x02, 0x2e, 0x71, 0x13, 0xbe, 0x37, 0xc5, 0x84, 0x26, 0xc1, 0x3b, 0x9c, 0x77, 0xec, + 0x82, 0x71, 0xb6, 0x0b, 0x6b, 0xde, 0x0c, 0xac, 0x79, 0x03, 0x4f, 0xa7, 0xd4, 0xcf, 0x42, 0xc2, + 0x1d, 0x70, 0xc9, 0xc7, 0x81, 0x76, 0x96, 0xa1, 0xea, 0x6a, 0x45, 0xd9, 0xce, 0xd5, 0xdf, 0x18, + 0x0d, 0xb5, 0xab, 0x13, 0x08, 0xe9, 0x44, 0x4f, 0x72, 0xc0, 0x4f, 0x14, 0x70, 0x35, 0xdc, 0x91, + 0xe1, 0x7b, 0x66, 0x17, 0x45, 0x91, 0x2d, 0xf2, 0xfd, 0x7d, 0x67, 0xca, 0xfe, 0x42, 0x33, 0x8e, + 0x18, 0xd3, 0x44, 0x74, 0xb7, 0x46, 0x43, 0xad, 0xec, 0xa6, 0xa0, 0x25, 0x33, 0x2e, 0xa7, 0xe1, + 0xd9, 0xad, 0xe5, 0xa2, 0x3e, 0x71, 0xa9, 0x8d, 0xbb, 0x46, 0x54, 0x92, 0xd7, 0xf8, 0x45, 0xc2, + 0x6f, 0xad, 0x31, 0xba, 0x99, 0xac, 0xbf, 0x6b, 0x09, 0x64, 0xc9, 0x04, 0x05, 0xa9, 0xc8, 0xc1, + 0x37, 0x41, 0xe6, 0x04, 0x0d, 0x82, 0xcb, 0x6b, 0x6d, 0x34, 0xd4, 0x2e, 0x9d, 0xa0, 0x81, 0x24, + 0x81, 0x61, 0xe1, 0xdb, 0x60, 0xe1, 0xd4, 0x74, 0x7c, 0x14, 0xb4, 0x19, 0xbc, 0x4b, 0xe0, 0x00, + 0xb9, 0x4b, 0xe0, 0x80, 0xbb, 0xf3, 0x77, 0x94, 0xd2, 0x1f, 0x15, 0xf0, 0xad, 0x73, 0x95, 0x1d, + 0x59, 0xfb, 0xc2, 0x54, 0xed, 0x0d, 0x59, 0xfb, 0xec, 0xfa, 0x3a, 0xcb, 0xba, 0x5f, 0x2b, 0xe0, + 0x72, 0x5a, 0xb5, 0x39, 0x9f, 0x2b, 0xee, 0xcb, 0xc6, 0xac, 0xdc, 0xbe, 0x9e, 0x34, 0x46, 0x08, + 0x15, 0x1a, 0x66, 0xd9, 0xf2, 0x89, 0x02, 0x36, 0x52, 0xab, 0xd0, 0xf9, 0x8c, 0xf9, 0x1f, 0x7b, + 0x26, 0x66, 0x4d, 0x94, 0xbf, 0x17, 0x62, 0xcd, 0x09, 0xd8, 0x48, 0xad, 0x59, 0x5f, 0x23, 0x65, + 0x73, 0x33, 0x95, 0xfd, 0x5e, 0x01, 0x95, 0x59, 0xe5, 0xe9, 0x42, 0xb2, 0xf5, 0x37, 0x0a, 0xd8, + 0x9c, 0x5a, 0x57, 0x2e, 0x22, 0x2e, 0x5b, 0x7f, 0xca, 0x82, 0x5c, 0x58, 0x4d, 0x58, 0xeb, 0xdb, + 0x10, 0xad, 0x6f, 0x56, 0xb4, 0xbe, 0x13, 0x4d, 0xdc, 0xfc, 0x44, 0xf3, 0x36, 0xff, 0x75, 0x9b, + 0xb7, 0xc3, 0x71, 0xf3, 0x26, 0xa6, 0x97, 0xb7, 0xa6, 0x77, 0xa2, 0xaf, 0xd0, 0xc0, 0xfd, 0x52, + 0x01, 0xd0, 0xc7, 0x1e, 0xa2, 0x0d, 0x6c, 0xa1, 0x8f, 0x90, 0x25, 0x38, 0xd5, 0x2c, 0x57, 0x71, + 0xfb, 0x0c, 0x15, 0x47, 0x09, 0x26, 0xa1, 0xae, 0x32, 0x1a, 0x6a, 0xd7, 0x92, 0x12, 0x25, 0xd5, + 0x29, 0xfa, 0xfe, 0x1f, 0xf5, 0xb8, 0x07, 0xae, 0x4e, 0xb1, 0xf9, 0x75, 0xa8, 0xdb, 0x7a, 0xbe, + 0x08, 0x36, 0x79, 0x8e, 0xde, 0x73, 0x7c, 0x8f, 0x22, 0x77, 0x22, 0x7d, 0x61, 0x03, 0x2c, 0x75, + 0x5c, 0xc4, 0x4e, 0x17, 0xd7, 0x7a, 0xf6, 0x98, 0xb2, 0x1e, 0x64, 0x44, 0xc8, 0xc2, 0xa7, 0x94, + 0x70, 0xc1, 0xec, 0x12, 0xd7, 0xb2, 0x64, 0xd7, 0xb3, 0xd8, 0xad, 0x2a, 0x28, 0xd8, 0x60, 0x85, + 0x82, 0xa9, 0xba, 0x61, 0xf1, 0x81, 0x26, 0x2f, 0x86, 0x8f, 0x08, 0x2a, 0x0f, 0x1f, 0x11, 0x14, + 0xfe, 0x4e, 0x61, 0x37, 0x70, 0x50, 0x07, 0xa2, 0xab, 0x2c, 0xc8, 0x93, 0xdd, 0x64, 0x9e, 0x4c, + 0xdd, 0xfa, 0xf8, 0x98, 0x49, 0x62, 0x44, 0xe6, 0x5c, 0x0f, 0xb6, 0x99, 0xaa, 0x48, 0xd1, 0xd3, + 0xc0, 0xf0, 0x6f, 0x0a, 0xb8, 0x96, 0x02, 0xbf, 0xe7, 0x98, 0x9e, 0xd7, 0x34, 0xf9, 0xf4, 0xcc, + 0x0c, 0x7c, 0xfc, 0x0d, 0x0d, 0x1c, 0xcb, 0x13, 0x96, 0xde, 0x0c, 0x2c, 0x3d, 0x53, 0xb5, 0x7e, + 0x26, 0xb6, 0xf4, 0xa9, 0x02, 0xd4, 0x69, 0xae, 0xb8, 0x90, 0x1a, 0xfb, 0x07, 0x05, 0xdc, 0x98, + 0xb9, 0xf5, 0x0b, 0xa9, 0xb5, 0x7f, 0xcf, 0x80, 0x52, 0x5a, 0xa4, 0x74, 0xde, 0xd6, 0x8d, 0x5f, + 0x7f, 0x94, 0x19, 0xaf, 0x3f, 0xd2, 0x99, 0x9b, 0xff, 0x86, 0x67, 0xee, 0x53, 0x05, 0x14, 0xa5, + 0xe8, 0xf2, 0x5c, 0x0a, 0xca, 0x72, 0x3d, 0xb9, 0xd9, 0xe9, 0xb6, 0xcb, 0xb9, 0x26, 0x35, 0xca, + 0xe5, 0xd1, 0x50, 0x2b, 0xc5, 0xe5, 0x4b, 0xfb, 0x49, 0xe8, 0x2e, 0x7d, 0xa1, 0x80, 0x8d, 0x54, + 0x59, 0xe7, 0x0b, 0xd8, 0x8f, 0x27, 0x03, 0xf6, 0xce, 0x2b, 0x1c, 0x97, 0x99, 0xd1, 0xfb, 0xd5, + 0x3c, 0x58, 0x96, 0xc3, 0x0d, 0x3f, 0x00, 0xf9, 0x68, 0x56, 0x52, 0xb8, 0xd3, 0xde, 0x3d, 0x3b, + 0x43, 0xaa, 0xb1, 0x09, 0x69, 0x2d, 0x08, 0x4e, 0x24, 0x47, 0x8f, 0xfe, 0x96, 0x3e, 0x57, 0xc0, + 0xca, 0xf4, 0x9e, 0x65, 0xba, 0x13, 0x7e, 0x3a, 0xe9, 0x84, 0xaa, 0x74, 0x45, 0x8f, 0x5f, 0x3a, + 0xab, 0xfd, 0x93, 0x2e, 0xbf, 0xb3, 0x43, 0x75, 0xd5, 0x27, 0xbe, 0x89, 0xa9, 0x4d, 0x07, 0x33, + 0xfd, 0xf0, 0xf9, 0x02, 0x58, 0x7b, 0x40, 0xda, 0x2d, 0xb1, 0x51, 0x1b, 0x77, 0x1b, 0xf8, 0x98, + 0xc0, 0xdb, 0x20, 0xe7, 0xd8, 0xc7, 0x88, 0xda, 0x3d, 0xc4, 0xcd, 0xbb, 0x24, 0x5e, 0x91, 0x42, + 0x98, 0xfc, 0x8a, 0x14, 0xc2, 0xe0, 0x5d, 0xb0, 0x6c, 0x52, 0xa3, 0x47, 0x3c, 0x6a, 0x10, 0xdc, + 0x09, 0x9b, 0x3b, 0x5e, 0xc8, 0x4d, 0xfa, 0x98, 0x78, 0x74, 0x1f, 0x77, 0x64, 0x4e, 0x10, 0x41, + 0xe1, 0xf7, 0x41, 0xa1, 0xef, 0x22, 0x06, 0xb7, 0xd9, 0x60, 0x98, 0xe1, 0xac, 0x9b, 0xa3, 0xa1, + 0xb6, 0x21, 0x81, 0x25, 0x5e, 0x99, 0x1a, 0xde, 0x07, 0xc5, 0x0e, 0xc1, 0x1d, 0xdf, 0x75, 0x11, + 0xee, 0x0c, 0x0c, 0xcf, 0x3c, 0x16, 0xcf, 0x9f, 0xb9, 0xfa, 0xf5, 0xd1, 0x50, 0xdb, 0x94, 0x70, + 0x2d, 0xf3, 0x58, 0x96, 0xb2, 0x1a, 0x43, 0xb1, 0x81, 0x6e, 0xfc, 0x8c, 0xd3, 0x61, 0x15, 0xc6, + 0xe0, 0x2f, 0x83, 0x8b, 0xd1, 0x40, 0xd7, 0x8f, 0xd7, 0x1f, 0x79, 0xa0, 0x4b, 0x20, 0x61, 0x0b, + 0x14, 0x3c, 0xbf, 0xdd, 0xb3, 0xa9, 0xc1, 0x5d, 0xb9, 0x34, 0xf3, 0x80, 0x87, 0x0f, 0x50, 0x40, + 0xb0, 0x8d, 0x1f, 0x4c, 0xa5, 0x35, 0x0b, 0x4e, 0xa8, 0x49, 0xcd, 0x45, 0xc1, 0x09, 0x61, 0x72, + 0x70, 0x42, 0x18, 0xfc, 0x05, 0x58, 0x17, 0x29, 0x6c, 0xb8, 0xe8, 0x99, 0x6f, 0xbb, 0xa8, 0x87, + 0xa2, 0x37, 0xbb, 0x9b, 0xc9, 0x3c, 0xdf, 0xe7, 0xbf, 0xba, 0x44, 0x2b, 0x5a, 0x28, 0x92, 0x80, + 0xcb, 0x2d, 0x54, 0x12, 0x0b, 0x6b, 0x60, 0xe9, 0x14, 0xb9, 0x9e, 0x4d, 0xb0, 0x9a, 0xe7, 0xb6, + 0x6e, 0x8c, 0x86, 0xda, 0x5a, 0x00, 0x92, 0x78, 0x43, 0xaa, 0xbb, 0xd9, 0x2f, 0xbe, 0xd4, 0x94, + 0xad, 0xdf, 0x2a, 0x00, 0x26, 0x6d, 0x80, 0x0e, 0x58, 0xed, 0x13, 0x4b, 0x06, 0x05, 0x8d, 0xca, + 0x8d, 0xe4, 0x16, 0x0e, 0x26, 0x09, 0x45, 0x32, 0xc4, 0xb8, 0x23, 0x03, 0xee, 0xcf, 0xe9, 0x71, + 0xd1, 0xf5, 0x15, 0xb0, 0x2c, 0x7b, 0x6b, 0xeb, 0x1f, 0x4b, 0x60, 0x35, 0x26, 0x15, 0x7a, 0xe2, + 0xed, 0xb4, 0x85, 0x1c, 0xd4, 0xa1, 0xc4, 0x0d, 0x2a, 0xc7, 0xfb, 0x33, 0xcd, 0xe1, 0x2d, 0x6b, + 0xc8, 0x25, 0xea, 0x47, 0x69, 0x34, 0xd4, 0xae, 0xc8, 0xc2, 0x24, 0xf7, 0x4c, 0x28, 0x81, 0x07, + 0x20, 0x67, 0x1e, 0x1f, 0xdb, 0x98, 0x65, 0x80, 0x28, 0x0b, 0xd7, 0xd2, 0x3a, 0xf7, 0x9d, 0x80, + 0x46, 0xe4, 0x47, 0xc8, 0x21, 0xe7, 0x47, 0x08, 0x83, 0x47, 0xa0, 0x40, 0x89, 0x83, 0x5c, 0x93, + 0xda, 0x04, 0x87, 0xbd, 0x7c, 0x39, 0x75, 0x1c, 0x18, 0x93, 0x8d, 0x6f, 0x23, 0x99, 0x55, 0x97, + 0x17, 0x90, 0x80, 0x82, 0x89, 0x31, 0xa1, 0x81, 0xd8, 0xa5, 0x69, 0xfd, 0x7b, 0xdc, 0x39, 0x3b, + 0x11, 0x93, 0xf0, 0x0d, 0xaf, 0x05, 0x92, 0x28, 0xb9, 0x16, 0x48, 0xe0, 0x89, 0xb3, 0x91, 0xe5, + 0x7d, 0xca, 0xec, 0xb3, 0xf1, 0x00, 0x14, 0xc3, 0x72, 0x42, 0xf0, 0x01, 0x71, 0xec, 0xce, 0x80, + 0x7f, 0xde, 0xc8, 0x8b, 0x1b, 0x2f, 0x8e, 0x93, 0x6f, 0xbc, 0x38, 0x0e, 0x7e, 0x0c, 0xc6, 0x4f, + 0x45, 0x13, 0x59, 0xba, 0xc8, 0xa3, 0xb4, 0x9d, 0xe6, 0x50, 0x3d, 0x85, 0xbe, 0x7e, 0x2d, 0x70, + 0x6d, 0xaa, 0x34, 0x3d, 0x15, 0x0a, 0x5b, 0x60, 0xbd, 0x63, 0x32, 0xcf, 0x46, 0xc5, 0xfc, 0x21, + 0x12, 0x25, 0x62, 0xb9, 0x7e, 0x63, 0x34, 0xd4, 0xae, 0xa7, 0xa0, 0xa5, 0xdd, 0xa4, 0x71, 0x97, + 0xba, 0x60, 0x2d, 0x91, 0xa9, 0xaf, 0x65, 0x10, 0x3a, 0x06, 0xc5, 0x78, 0xd4, 0x5f, 0xcb, 0x04, + 0xf4, 0x67, 0x05, 0x6c, 0x1e, 0xf8, 0x8e, 0x67, 0xba, 0xad, 0x30, 0x0b, 0x1f, 0x90, 0xf6, 0x2e, + 0xa2, 0xa6, 0xed, 0x78, 0x4c, 0x18, 0x7f, 0xe8, 0x09, 0x74, 0x72, 0x61, 0x1c, 0x20, 0x0b, 0x13, + 0xef, 0xcb, 0x6f, 0x83, 0x85, 0x27, 0xf1, 0x09, 0x27, 0xde, 0x12, 0x09, 0x0a, 0x78, 0x0b, 0x2c, + 0xb2, 0x3b, 0x16, 0xd1, 0x60, 0xba, 0xe1, 0xc3, 0xaf, 0x80, 0xc8, 0xc3, 0xaf, 0x80, 0x7c, 0x7b, + 0x1f, 0x14, 0xa4, 0x77, 0x2a, 0x58, 0x00, 0x4b, 0x47, 0xcd, 0x87, 0xcd, 0xfd, 0x9f, 0x34, 0x8b, + 0x73, 0x6c, 0x71, 0xb0, 0xd7, 0xdc, 0x6d, 0x34, 0x7f, 0x54, 0x54, 0xd8, 0x42, 0x3f, 0x6a, 0x36, + 0xd9, 0x62, 0x1e, 0x5e, 0x02, 0xf9, 0xd6, 0xd1, 0xbd, 0x7b, 0x7b, 0x7b, 0xbb, 0x7b, 0xbb, 0xc5, + 0x0c, 0x04, 0x60, 0xf1, 0x87, 0x3b, 0x8d, 0x47, 0x7b, 0xbb, 0xc5, 0x6c, 0xfd, 0xe7, 0xcf, 0x5f, + 0x94, 0x95, 0xaf, 0x5e, 0x94, 0x95, 0x7f, 0xbf, 0x28, 0x2b, 0x9f, 0xbd, 0x2c, 0xcf, 0x7d, 0xf5, + 0xb2, 0x3c, 0xf7, 0xcf, 0x97, 0xe5, 0xb9, 0x9f, 0xdd, 0x93, 0x3e, 0x80, 0x8a, 0xa7, 0xe3, 0xbe, + 0x4b, 0xd8, 0x91, 0x0c, 0x56, 0xb5, 0x73, 0x7c, 0xe9, 0x6d, 0x2f, 0xf2, 0x7b, 0xec, 0xfd, 0xff, + 0x06, 0x00, 0x00, 0xff, 0xff, 0xff, 0x66, 0x0a, 0x62, 0x17, 0x1e, 0x00, 0x00, } func (m *Executor) Marshal() (dAtA []byte, err error) { @@ -1616,6 +1627,30 @@ func (m *QueueClusterResourceUsage) MarshalToSizedBuffer(dAtA []byte) (int, erro _ = i var l int _ = l + if len(m.ResourcesByPriorityClassName) > 0 { + for k := range m.ResourcesByPriorityClassName { + v := m.ResourcesByPriorityClassName[k] + baseI := i + { + size, err := (&v).MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSchedulerobjects(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + i -= len(k) + copy(dAtA[i:], k) + i = encodeVarintSchedulerobjects(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = encodeVarintSchedulerobjects(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x2a + } + } if len(m.ResourcesByPriority) > 0 { for k := range m.ResourcesByPriority { v := m.ResourcesByPriority[k] @@ -1652,12 +1687,12 @@ func (m *QueueClusterResourceUsage) MarshalToSizedBuffer(dAtA []byte) (int, erro i-- dAtA[i] = 0x12 } - n12, err12 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Created, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Created):]) - if err12 != nil { - return 0, err12 + n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Created, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Created):]) + if err13 != nil { + return 0, err13 } - i -= n12 - i = encodeVarintSchedulerobjects(dAtA, i, uint64(n12)) + i -= n13 + i = encodeVarintSchedulerobjects(dAtA, i, uint64(n13)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -1709,12 +1744,12 @@ func (m *ClusterResourceUsageReport) MarshalToSizedBuffer(dAtA []byte) (int, err dAtA[i] = 0x1a } } - n14, err14 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Created, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Created):]) - if err14 != nil { - return 0, err14 + n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Created, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Created):]) + if err15 != nil { + return 0, err15 } - i -= n14 - i = encodeVarintSchedulerobjects(dAtA, i, uint64(n14)) + i -= n15 + i = encodeVarintSchedulerobjects(dAtA, i, uint64(n15)) i-- dAtA[i] = 0x12 if len(m.Pool) > 0 { @@ -1804,12 +1839,12 @@ func (m *JobSchedulingInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x40 } - n16, err16 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.SubmitTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.SubmitTime):]) - if err16 != nil { - return 0, err16 + n17, err17 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.SubmitTime, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.SubmitTime):]) + if err17 != nil { + return 0, err17 } - i -= n16 - i = encodeVarintSchedulerobjects(dAtA, i, uint64(n16)) + i -= n17 + i = encodeVarintSchedulerobjects(dAtA, i, uint64(n17)) i-- dAtA[i] = 0x3a if len(m.PriorityClassName) > 0 { @@ -2307,6 +2342,15 @@ func (m *QueueClusterResourceUsage) Size() (n int) { n += mapEntrySize + 1 + sovSchedulerobjects(uint64(mapEntrySize)) } } + if len(m.ResourcesByPriorityClassName) > 0 { + for k, v := range m.ResourcesByPriorityClassName { + _ = k + _ = v + l = v.Size() + mapEntrySize := 1 + len(k) + sovSchedulerobjects(uint64(len(k))) + 1 + l + sovSchedulerobjects(uint64(l)) + n += mapEntrySize + 1 + sovSchedulerobjects(uint64(mapEntrySize)) + } + } return n } @@ -4663,6 +4707,135 @@ func (m *QueueClusterResourceUsage) Unmarshal(dAtA []byte) error { } m.ResourcesByPriority[mapkey] = *mapvalue iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResourcesByPriorityClassName", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSchedulerobjects + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSchedulerobjects + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSchedulerobjects + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ResourcesByPriorityClassName == nil { + m.ResourcesByPriorityClassName = make(map[string]ResourceList) + } + var mapkey string + mapvalue := &ResourceList{} + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSchedulerobjects + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSchedulerobjects + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLengthSchedulerobjects + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLengthSchedulerobjects + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSchedulerobjects + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthSchedulerobjects + } + postmsgIndex := iNdEx + mapmsglen + if postmsgIndex < 0 { + return ErrInvalidLengthSchedulerobjects + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &ResourceList{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipSchedulerobjects(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSchedulerobjects + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.ResourcesByPriorityClassName[mapkey] = *mapvalue + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSchedulerobjects(dAtA[iNdEx:]) diff --git a/internal/scheduler/schedulerobjects/schedulerobjects.proto b/internal/scheduler/schedulerobjects/schedulerobjects.proto index c9865a28d51..f87f88275e4 100644 --- a/internal/scheduler/schedulerobjects/schedulerobjects.proto +++ b/internal/scheduler/schedulerobjects/schedulerobjects.proto @@ -101,13 +101,13 @@ message NodeType { map unsetIndexedLabels = 4; } -// Captures the resource usage of a particular queue -// in a given cluster. +// Captures the resource usage of a particular queue in a given cluster. message QueueClusterResourceUsage { google.protobuf.Timestamp created = 1 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; string queue = 2; string executorId = 3; - map resourcesByPriority = 4 [(gogoproto.nullable) = false]; + map resourcesByPriority = 4 [(gogoproto.nullable) = false, deprecated = true]; + map resourcesByPriorityClassName = 5 [(gogoproto.nullable) = false]; } // A collection of QueueClusterResourceUsage diff --git a/internal/scheduler/scheduling_algo.go b/internal/scheduler/scheduling_algo.go index 1025d6b21c6..fc46ecd16bc 100644 --- a/internal/scheduler/scheduling_algo.go +++ b/internal/scheduler/scheduling_algo.go @@ -146,7 +146,7 @@ func (l *FairSchedulingAlgo) Schedule( maps.Copy(overallSchedulerResult.NodeIdByJobId, schedulerResult.NodeIdByJobId) // Update accounting. - accounting.totalAllocationByPoolAndQueue[executor.Pool] = sctx.AllocatedByQueueAndPriority() + accounting.allocationByPoolAndQueueAndPriorityClass[executor.Pool] = sctx.AllocatedByQueueAndPriority() // Update result to mark this executor as scheduled l.previousScheduleClusterId = executor.Id @@ -175,14 +175,14 @@ func (it *JobQueueIteratorAdapter) Next() (interfaces.LegacySchedulerJob, error) } type fairSchedulingAlgoContext struct { - priorityFactorByQueue map[string]float64 - totalCapacity schedulerobjects.ResourceList - jobsByExecutorId map[string][]*jobdb.Job - nodeIdByJobId map[string]string - jobIdsByGangId map[string]map[string]bool - gangIdByJobId map[string]string - totalAllocationByPoolAndQueue map[string]map[string]schedulerobjects.QuantityByPriorityAndResourceType - executors []*schedulerobjects.Executor + priorityFactorByQueue map[string]float64 + totalCapacity schedulerobjects.ResourceList + jobsByExecutorId map[string][]*jobdb.Job + nodeIdByJobId map[string]string + jobIdsByGangId map[string]map[string]bool + gangIdByJobId map[string]string + allocationByPoolAndQueueAndPriorityClass map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string] + executors []*schedulerobjects.Executor } // This function will return executors in the order they should be scheduled in @@ -280,21 +280,21 @@ func (l *FairSchedulingAlgo) newFairSchedulingAlgoContext(ctx context.Context, t } // Used to calculate fair share. - totalAllocationByPoolAndQueue := l.totalAllocationByPoolAndQueue(executors, jobsByExecutorId) + totalAllocationByPoolAndQueue := l.aggregateAllocationByPoolAndQueueAndPriorityClass(executors, jobsByExecutorId) // Filter out any executor that isn't acknowledging jobs in a timely fashion // Note that we do this after aggregating allocation across clusters for fair share. executors = l.filterLaggingExecutors(executors, jobsByExecutorId) return &fairSchedulingAlgoContext{ - priorityFactorByQueue: priorityFactorByQueue, - totalCapacity: totalCapacity, - jobsByExecutorId: jobsByExecutorId, - nodeIdByJobId: nodeIdByJobId, - jobIdsByGangId: jobIdsByGangId, - gangIdByJobId: gangIdByJobId, - totalAllocationByPoolAndQueue: totalAllocationByPoolAndQueue, - executors: executors, + priorityFactorByQueue: priorityFactorByQueue, + totalCapacity: totalCapacity, + jobsByExecutorId: jobsByExecutorId, + nodeIdByJobId: nodeIdByJobId, + jobIdsByGangId: jobIdsByGangId, + gangIdByJobId: gangIdByJobId, + allocationByPoolAndQueueAndPriorityClass: totalAllocationByPoolAndQueue, + executors: executors, }, nil } @@ -323,11 +323,11 @@ func (l *FairSchedulingAlgo) scheduleOnExecutor( accounting.totalCapacity, ) for queue, priorityFactor := range accounting.priorityFactorByQueue { - var allocatedByPriority schedulerobjects.QuantityByPriorityAndResourceType - if allocatedByQueueAndPriority := accounting.totalAllocationByPoolAndQueue[executor.Pool]; allocatedByQueueAndPriority != nil { - allocatedByPriority = allocatedByQueueAndPriority[queue] + var allocatedByPriorityClass schedulerobjects.QuantityByTAndResourceType[string] + if allocatedByQueueAndPriorityClass := accounting.allocationByPoolAndQueueAndPriorityClass[executor.Pool]; allocatedByQueueAndPriorityClass != nil { + allocatedByPriorityClass = allocatedByQueueAndPriorityClass[queue] } - if err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByPriority); err != nil { + if err := sctx.AddQueueSchedulingContext(queue, priorityFactor, allocatedByPriorityClass); err != nil { return nil, nil, err } } @@ -520,30 +520,22 @@ func (l *FairSchedulingAlgo) filterLaggingExecutors( return activeExecutors } -func (l *FairSchedulingAlgo) totalAllocationByPoolAndQueue(executors []*schedulerobjects.Executor, jobsByExecutorId map[string][]*jobdb.Job) map[string]map[string]schedulerobjects.QuantityByPriorityAndResourceType { - rv := make(map[string]map[string]schedulerobjects.QuantityByPriorityAndResourceType) +func (l *FairSchedulingAlgo) aggregateAllocationByPoolAndQueueAndPriorityClass(executors []*schedulerobjects.Executor, jobsByExecutorId map[string][]*jobdb.Job) map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string] { + rv := make(map[string]map[string]schedulerobjects.QuantityByTAndResourceType[string]) for _, executor := range executors { allocationByQueue := rv[executor.Pool] if allocationByQueue == nil { - allocationByQueue = make(map[string]schedulerobjects.QuantityByPriorityAndResourceType) + allocationByQueue = make(map[string]schedulerobjects.QuantityByTAndResourceType[string]) rv[executor.Pool] = allocationByQueue } for _, job := range jobsByExecutorId[executor.Id] { queue := job.Queue() allocation := allocationByQueue[queue] if allocation == nil { - allocation = make(schedulerobjects.QuantityByPriorityAndResourceType) + allocation = make(schedulerobjects.QuantityByTAndResourceType[string]) allocationByQueue[queue] = allocation } - jobSchedulingInfo := job.JobSchedulingInfo() - if jobSchedulingInfo != nil { - priorityClass, ok := l.priorityClasses[jobSchedulingInfo.PriorityClassName] - if ok { - allocation.AddResourceList(priorityClass.Priority, jobSchedulingInfo.GetTotalResourceRequest()) - } else { - log.Errorf("job %s has unknown priority class name %s; ignoring the resources allocated to this job", job.Id(), jobSchedulingInfo.PriorityClassName) - } - } + allocation.AddV1ResourceList(job.GetPriorityClassName(), job.GetResourceRequirements().Requests) } } return rv diff --git a/internal/scheduler/scheduling_algo_test.go b/internal/scheduler/scheduling_algo_test.go index 99560a23d56..7f07f9ded04 100644 --- a/internal/scheduler/scheduling_algo_test.go +++ b/internal/scheduler/scheduling_algo_test.go @@ -111,8 +111,8 @@ func TestLegacySchedulingAlgo_TestSchedule(t *testing.T) { }, "user is at usage cap before scheduling": { schedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - testfixtures.TestPriorityClasses[testfixtures.PriorityClass3].Priority: {"cpu": 0.5}, + map[string]map[string]float64{ + testfixtures.PriorityClass3: {"cpu": 0.5}, }, testfixtures.TestSchedulingConfig(), ), @@ -134,8 +134,8 @@ func TestLegacySchedulingAlgo_TestSchedule(t *testing.T) { }, "user hits usage cap during scheduling": { schedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - testfixtures.TestPriorityClasses[testfixtures.PriorityClass3].Priority: {"cpu": 0.5}, + map[string]map[string]float64{ + testfixtures.PriorityClass3: {"cpu": 0.5}, }, testfixtures.TestSchedulingConfig(), ), @@ -179,8 +179,8 @@ func TestLegacySchedulingAlgo_TestSchedule(t *testing.T) { }, "computation of allocated resources does not confuse priority class with per-queue priority": { schedulingConfig: testfixtures.WithPerPriorityLimitsConfig( - map[int32]map[string]float64{ - testfixtures.TestPriorityClasses[testfixtures.PriorityClass3].Priority: {"cpu": 0.5}, + map[string]map[string]float64{ + testfixtures.PriorityClass3: {"cpu": 0.5}, }, testfixtures.TestSchedulingConfig(), ), @@ -422,15 +422,14 @@ func TestLegacySchedulingAlgo_TestSchedule(t *testing.T) { assert.Equal(t, len(jobIndices), sctx.NumScheduledJobs) expectedScheduledResources := schedulerobjects.ResourceList{} - expectedScheduledResourcesByPriority := schedulerobjects.QuantityByPriorityAndResourceType{} + expectedScheduledResourcesByPriorityClass := make(schedulerobjects.QuantityByTAndResourceType[string]) for _, i := range jobIndices { job := tc.queuedJobs[i] - req := job.JobSchedulingInfo().ObjectRequirements[0].GetPodRequirements() - expectedScheduledResources.AddV1ResourceList(req.ResourceRequirements.Requests) - expectedScheduledResourcesByPriority.AddV1ResourceList(req.Priority, req.ResourceRequirements.Requests) + expectedScheduledResources.AddV1ResourceList(job.GetResourceRequirements().Requests) + expectedScheduledResourcesByPriorityClass.AddV1ResourceList(job.GetPriorityClassName(), job.GetResourceRequirements().Requests) } - assert.Equal(t, expectedScheduledResources, sctx.ScheduledResources) - assert.Equal(t, expectedScheduledResourcesByPriority, sctx.ScheduledResourcesByPriority) + assert.True(t, expectedScheduledResources.Equal(sctx.ScheduledResources)) + assert.True(t, expectedScheduledResourcesByPriorityClass.Equal(sctx.ScheduledResourcesByPriorityClass)) } scheduledJobs := ScheduledJobsFromSchedulerResult[*jobdb.Job](schedulerResult) @@ -469,25 +468,24 @@ func TestLegacySchedulingAlgo_TestSchedule(t *testing.T) { assert.Equal(t, len(tc.expectedPreemptedIndices), numPreemptedJobs) expectedPreemptedResources := schedulerobjects.ResourceList{} - expectedPreemptedResourcesByPriority := schedulerobjects.QuantityByPriorityAndResourceType{} + expectedPreemptedResourcesByPriorityClass := make(schedulerobjects.QuantityByTAndResourceType[string]) for _, i := range tc.expectedPreemptedIndices { job := tc.existingJobs[i] - req := job.JobSchedulingInfo().ObjectRequirements[0].GetPodRequirements() - expectedPreemptedResources.AddV1ResourceList(req.ResourceRequirements.Requests) - expectedPreemptedResourcesByPriority.AddV1ResourceList(req.Priority, req.ResourceRequirements.Requests) + expectedPreemptedResources.AddV1ResourceList(job.GetResourceRequirements().Requests) + expectedPreemptedResourcesByPriorityClass.AddV1ResourceList(job.GetPriorityClassName(), job.GetResourceRequirements().Requests) } preemptedResources := schedulerobjects.ResourceList{} - preemptedResourcesByPriority := schedulerobjects.QuantityByPriorityAndResourceType{} + preemptedResourcesByPriority := make(schedulerobjects.QuantityByTAndResourceType[string]) for _, sctx := range schedulingContextByExecutor { for resourceType, quantity := range sctx.EvictedResources.Resources { preemptedResources.AddQuantity(resourceType, quantity) } - for p, rl := range sctx.EvictedResourcesByPriority { + for p, rl := range sctx.EvictedResourcesByPriorityClass { preemptedResourcesByPriority.AddResourceList(p, rl) } } - assert.Equal(t, expectedPreemptedResources, preemptedResources) - assert.Equal(t, expectedPreemptedResourcesByPriority, preemptedResourcesByPriority) + assert.True(t, expectedPreemptedResources.Equal(preemptedResources)) + assert.True(t, expectedPreemptedResourcesByPriorityClass.Equal(preemptedResourcesByPriority)) }) } } diff --git a/internal/scheduler/testfixtures/testfixtures.go b/internal/scheduler/testfixtures/testfixtures.go index 898e30c0b88..559c14b3d33 100644 --- a/internal/scheduler/testfixtures/testfixtures.go +++ b/internal/scheduler/testfixtures/testfixtures.go @@ -126,12 +126,18 @@ func WithRoundLimitsPoolConfig(limits map[string]map[string]float64, config conf return config } -func WithPerPriorityLimitsConfig(limits map[int32]map[string]float64, config configuration.SchedulingConfig) configuration.SchedulingConfig { - for k, v := range config.Preemption.PriorityClasses { - config.Preemption.PriorityClasses[k] = configuration.PriorityClass{ - Priority: v.Priority, - Preemptible: v.Preemptible, - MaximumResourceFractionPerQueue: limits[v.Priority], +func WithPerPriorityLimitsConfig(limits map[string]map[string]float64, config configuration.SchedulingConfig) configuration.SchedulingConfig { + for priorityClassName, limit := range limits { + priorityClass, ok := config.Preemption.PriorityClasses[priorityClassName] + if !ok { + panic(fmt.Sprintf("no priority class with name %s", priorityClassName)) + } + // We need to make a copy to avoid mutating the priorityClasses, which are used by other tests too. + config.Preemption.PriorityClasses[priorityClassName] = configuration.PriorityClass{ + Priority: priorityClass.Priority, + Preemptible: priorityClass.Preemptible, + MaximumResourceFractionPerQueue: limit, + MaximumResourceFractionPerQueueByPool: priorityClass.MaximumResourceFractionPerQueueByPool, } } return config