Skip to content

Commit 6db4044

Browse files
committed
Scheduler changes:
1. Use pod-level resource when feature is enabled and resources are set at pod-level 2. Edge case handling: When a pod defines only CPU or memory limits at pod-level (but not both), and container-level requests/limits are unset, the pod-level requests stay empty for the resource without a pod-limit. The container's request for that resource is then set to the default request value from schedutil.
1 parent 8a8dc27 commit 6db4044

File tree

9 files changed

+451
-32
lines changed

9 files changed

+451
-32
lines changed

pkg/scheduler/eventhandlers.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -623,7 +623,9 @@ func preCheckForNode(nodeInfo *framework.NodeInfo) queue.PreEnqueueCheck {
623623
// returns all failures.
624624
func AdmissionCheck(pod *v1.Pod, nodeInfo *framework.NodeInfo, includeAllFailures bool) []AdmissionResult {
625625
var admissionResults []AdmissionResult
626-
insufficientResources := noderesources.Fits(pod, nodeInfo)
626+
insufficientResources := noderesources.Fits(pod, nodeInfo, noderesources.ResourceRequestsOptions{
627+
EnablePodLevelResources: utilfeature.DefaultFeatureGate.Enabled(features.PodLevelResources),
628+
})
627629
if len(insufficientResources) != 0 {
628630
for i := range insufficientResources {
629631
admissionResults = append(admissionResults, AdmissionResult{InsufficientResource: &insufficientResources[i]})

pkg/scheduler/framework/plugins/feature/feature.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,4 +29,5 @@ type Features struct {
2929
EnableSidecarContainers bool
3030
EnableSchedulingQueueHint bool
3131
EnableAsyncPreemption bool
32+
EnablePodLevelResources bool
3233
}

pkg/scheduler/framework/plugins/noderesources/fit.go

Lines changed: 22 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -90,6 +90,7 @@ type Fit struct {
9090
enableInPlacePodVerticalScaling bool
9191
enableSidecarContainers bool
9292
enableSchedulingQueueHint bool
93+
enablePodLevelResources bool
9394
handle framework.Handle
9495
resourceAllocationScorer
9596
}
@@ -176,10 +177,15 @@ func NewFit(_ context.Context, plArgs runtime.Object, h framework.Handle, fts fe
176177
enableSidecarContainers: fts.EnableSidecarContainers,
177178
enableSchedulingQueueHint: fts.EnableSchedulingQueueHint,
178179
handle: h,
180+
enablePodLevelResources: fts.EnablePodLevelResources,
179181
resourceAllocationScorer: *scorePlugin(args),
180182
}, nil
181183
}
182184

185+
type ResourceRequestsOptions struct {
186+
EnablePodLevelResources bool
187+
}
188+
183189
// computePodResourceRequest returns a framework.Resource that covers the largest
184190
// width in each resource dimension. Because init-containers run sequentially, we collect
185191
// the max in each dimension iteratively. In contrast, we sum the resource vectors for
@@ -207,9 +213,14 @@ func NewFit(_ context.Context, plArgs runtime.Object, h framework.Handle, fts fe
207213
// Memory: 1G
208214
//
209215
// Result: CPU: 3, Memory: 3G
210-
func computePodResourceRequest(pod *v1.Pod) *preFilterState {
216+
// TODO(ndixita): modify computePodResourceRequest to accept opts of type
217+
// ResourceRequestOptions as the second parameter.
218+
func computePodResourceRequest(pod *v1.Pod, opts ResourceRequestsOptions) *preFilterState {
211219
// pod hasn't scheduled yet so we don't need to worry about InPlacePodVerticalScalingEnabled
212-
reqs := resource.PodRequests(pod, resource.PodResourcesOptions{})
220+
reqs := resource.PodRequests(pod, resource.PodResourcesOptions{
221+
// SkipPodLevelResources is set to false when PodLevelResources feature is enabled.
222+
SkipPodLevelResources: !opts.EnablePodLevelResources,
223+
})
213224
result := &preFilterState{}
214225
result.SetMaxResource(reqs)
215226
return result
@@ -225,7 +236,7 @@ func (f *Fit) PreFilter(ctx context.Context, cycleState *framework.CycleState, p
225236
// and the older (before v1.28) kubelet, make the Pod unschedulable.
226237
return nil, framework.NewStatus(framework.UnschedulableAndUnresolvable, "Pod has a restartable init container and the SidecarContainers feature is disabled")
227238
}
228-
cycleState.Write(preFilterStateKey, computePodResourceRequest(pod))
239+
cycleState.Write(preFilterStateKey, computePodResourceRequest(pod, ResourceRequestsOptions{EnablePodLevelResources: f.enablePodLevelResources}))
229240
return nil, nil
230241
}
231242

@@ -370,7 +381,7 @@ func (f *Fit) isSchedulableAfterNodeChange(logger klog.Logger, pod *v1.Pod, oldO
370381
return framework.Queue, err
371382
}
372383
// Leaving in the queue, since the pod won't fit into the modified node anyway.
373-
if !isFit(pod, modifiedNode) {
384+
if !isFit(pod, modifiedNode, ResourceRequestsOptions{EnablePodLevelResources: f.enablePodLevelResources}) {
374385
logger.V(5).Info("node was created or updated, but it doesn't have enough resource(s) to accommodate this pod", "pod", klog.KObj(pod), "node", klog.KObj(modifiedNode))
375386
return framework.QueueSkip, nil
376387
}
@@ -380,7 +391,7 @@ func (f *Fit) isSchedulableAfterNodeChange(logger klog.Logger, pod *v1.Pod, oldO
380391
return framework.Queue, nil
381392
}
382393
// The pod will fit, but since there was no increase in available resources, the change won't make the pod schedulable.
383-
if !haveAnyRequestedResourcesIncreased(pod, originalNode, modifiedNode) {
394+
if !haveAnyRequestedResourcesIncreased(pod, originalNode, modifiedNode, ResourceRequestsOptions{EnablePodLevelResources: f.enablePodLevelResources}) {
384395
logger.V(5).Info("node was updated, but haven't changed the pod's resource requestments fit assessment", "pod", klog.KObj(pod), "node", klog.KObj(modifiedNode))
385396
return framework.QueueSkip, nil
386397
}
@@ -390,8 +401,8 @@ func (f *Fit) isSchedulableAfterNodeChange(logger klog.Logger, pod *v1.Pod, oldO
390401
}
391402

392403
// haveAnyRequestedResourcesIncreased returns true if any of the resources requested by the pod have increased or if allowed pod number increased.
393-
func haveAnyRequestedResourcesIncreased(pod *v1.Pod, originalNode, modifiedNode *v1.Node) bool {
394-
podRequest := computePodResourceRequest(pod)
404+
func haveAnyRequestedResourcesIncreased(pod *v1.Pod, originalNode, modifiedNode *v1.Node, opts ResourceRequestsOptions) bool {
405+
podRequest := computePodResourceRequest(pod, opts)
395406
originalNodeInfo := framework.NewNodeInfo()
396407
originalNodeInfo.SetNode(originalNode)
397408
modifiedNodeInfo := framework.NewNodeInfo()
@@ -429,13 +440,13 @@ func haveAnyRequestedResourcesIncreased(pod *v1.Pod, originalNode, modifiedNode
429440

430441
// isFit checks if the pod fits the node. If the node is nil, it returns false.
431442
// It constructs a fake NodeInfo object for the node and checks if the pod fits the node.
432-
func isFit(pod *v1.Pod, node *v1.Node) bool {
443+
func isFit(pod *v1.Pod, node *v1.Node, opts ResourceRequestsOptions) bool {
433444
if node == nil {
434445
return false
435446
}
436447
nodeInfo := framework.NewNodeInfo()
437448
nodeInfo.SetNode(node)
438-
return len(Fits(pod, nodeInfo)) == 0
449+
return len(Fits(pod, nodeInfo, opts)) == 0
439450
}
440451

441452
// Filter invoked at the filter extension point.
@@ -481,8 +492,8 @@ type InsufficientResource struct {
481492
}
482493

483494
// Fits checks if node have enough resources to host the pod.
484-
func Fits(pod *v1.Pod, nodeInfo *framework.NodeInfo) []InsufficientResource {
485-
return fitsRequest(computePodResourceRequest(pod), nodeInfo, nil, nil)
495+
func Fits(pod *v1.Pod, nodeInfo *framework.NodeInfo, opts ResourceRequestsOptions) []InsufficientResource {
496+
return fitsRequest(computePodResourceRequest(pod, opts), nodeInfo, nil, nil)
486497
}
487498

488499
func fitsRequest(podRequest *preFilterState, nodeInfo *framework.NodeInfo, ignoredExtendedResources, ignoredResourceGroups sets.Set[string]) []InsufficientResource {

pkg/scheduler/framework/plugins/noderesources/fit_test.go

Lines changed: 94 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -119,12 +119,18 @@ var defaultScoringStrategy = &config.ScoringStrategy{
119119
},
120120
}
121121

122+
func newPodLevelResourcesPod(pod *v1.Pod, podResources v1.ResourceRequirements) *v1.Pod {
123+
pod.Spec.Resources = &podResources
124+
return pod
125+
}
126+
122127
func TestEnoughRequests(t *testing.T) {
123128
enoughPodsTests := []struct {
124129
pod *v1.Pod
125130
nodeInfo *framework.NodeInfo
126131
name string
127132
args config.NodeResourcesFitArgs
133+
podLevelResourcesEnabled bool
128134
wantInsufficientResources []InsufficientResource
129135
wantStatus *framework.Status
130136
}{
@@ -478,6 +484,7 @@ func TestEnoughRequests(t *testing.T) {
478484
wantInsufficientResources: []InsufficientResource{},
479485
},
480486
{
487+
podLevelResourcesEnabled: true,
481488
pod: newResourcePod(
482489
framework.Resource{
483490
ScalarResources: map[v1.ResourceName]int64{
@@ -488,10 +495,74 @@ func TestEnoughRequests(t *testing.T) {
488495
name: "skip checking resource request with quantity zero",
489496
wantInsufficientResources: []InsufficientResource{},
490497
},
498+
{
499+
podLevelResourcesEnabled: true,
500+
pod: newPodLevelResourcesPod(
501+
newResourcePod(framework.Resource{MilliCPU: 1, Memory: 1}),
502+
v1.ResourceRequirements{
503+
Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("1m"), v1.ResourceMemory: resource.MustParse("2")},
504+
},
505+
),
506+
nodeInfo: framework.NewNodeInfo(
507+
newResourcePod(framework.Resource{MilliCPU: 5, Memory: 5})),
508+
name: "both pod-level and container-level resources fit",
509+
wantInsufficientResources: []InsufficientResource{},
510+
},
511+
{
512+
podLevelResourcesEnabled: true,
513+
pod: newPodLevelResourcesPod(
514+
newResourcePod(framework.Resource{MilliCPU: 1, Memory: 1}),
515+
v1.ResourceRequirements{
516+
Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("7m"), v1.ResourceMemory: resource.MustParse("2")},
517+
},
518+
),
519+
nodeInfo: framework.NewNodeInfo(
520+
newResourcePod(framework.Resource{MilliCPU: 5, Memory: 5})),
521+
name: "pod-level cpu resource not fit",
522+
wantStatus: framework.NewStatus(framework.Unschedulable, getErrReason(v1.ResourceCPU)),
523+
wantInsufficientResources: []InsufficientResource{{
524+
ResourceName: v1.ResourceCPU, Reason: getErrReason(v1.ResourceCPU), Requested: 7, Used: 5, Capacity: 10},
525+
},
526+
},
527+
{
528+
podLevelResourcesEnabled: true,
529+
pod: newPodLevelResourcesPod(
530+
newResourcePod(framework.Resource{MilliCPU: 1, Memory: 1}),
531+
v1.ResourceRequirements{
532+
Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("3m"), v1.ResourceMemory: resource.MustParse("2")},
533+
},
534+
),
535+
nodeInfo: framework.NewNodeInfo(
536+
newResourcePod(framework.Resource{MilliCPU: 5, Memory: 19})),
537+
name: "pod-level memory resource not fit",
538+
wantStatus: framework.NewStatus(framework.Unschedulable, getErrReason(v1.ResourceMemory)),
539+
wantInsufficientResources: []InsufficientResource{{
540+
ResourceName: v1.ResourceMemory, Reason: getErrReason(v1.ResourceMemory), Requested: 2, Used: 19, Capacity: 20},
541+
},
542+
},
543+
{
544+
podLevelResourcesEnabled: true,
545+
pod: newResourceInitPod(newPodLevelResourcesPod(
546+
newResourcePod(framework.Resource{MilliCPU: 1, Memory: 1}),
547+
v1.ResourceRequirements{
548+
Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("3m"), v1.ResourceMemory: resource.MustParse("2")},
549+
},
550+
),
551+
framework.Resource{MilliCPU: 1, Memory: 1},
552+
),
553+
nodeInfo: framework.NewNodeInfo(
554+
newResourcePod(framework.Resource{MilliCPU: 5, Memory: 19})),
555+
name: "one pod-level cpu resource fits and all init and non-init containers resources fit",
556+
wantStatus: framework.NewStatus(framework.Unschedulable, getErrReason(v1.ResourceMemory)),
557+
wantInsufficientResources: []InsufficientResource{{
558+
ResourceName: v1.ResourceMemory, Reason: getErrReason(v1.ResourceMemory), Requested: 2, Used: 19, Capacity: 20},
559+
},
560+
},
491561
}
492562

493563
for _, test := range enoughPodsTests {
494564
t.Run(test.name, func(t *testing.T) {
565+
495566
node := v1.Node{Status: v1.NodeStatus{Capacity: makeResources(10, 20, 32, 5, 20, 5), Allocatable: makeAllocatableResources(10, 20, 32, 5, 20, 5)}}
496567
test.nodeInfo.SetNode(&node)
497568

@@ -502,7 +573,7 @@ func TestEnoughRequests(t *testing.T) {
502573
_, ctx := ktesting.NewTestContext(t)
503574
ctx, cancel := context.WithCancel(ctx)
504575
defer cancel()
505-
p, err := NewFit(ctx, &test.args, nil, plfeature.Features{})
576+
p, err := NewFit(ctx, &test.args, nil, plfeature.Features{EnablePodLevelResources: test.podLevelResourcesEnabled})
506577
if err != nil {
507578
t.Fatal(err)
508579
}
@@ -517,7 +588,7 @@ func TestEnoughRequests(t *testing.T) {
517588
t.Errorf("status does not match: %v, want: %v", gotStatus, test.wantStatus)
518589
}
519590

520-
gotInsufficientResources := fitsRequest(computePodResourceRequest(test.pod), test.nodeInfo, p.(*Fit).ignoredResources, p.(*Fit).ignoredResourceGroups)
591+
gotInsufficientResources := fitsRequest(computePodResourceRequest(test.pod, ResourceRequestsOptions{EnablePodLevelResources: test.podLevelResourcesEnabled}), test.nodeInfo, p.(*Fit).ignoredResources, p.(*Fit).ignoredResourceGroups)
521592
if !reflect.DeepEqual(gotInsufficientResources, test.wantInsufficientResources) {
522593
t.Errorf("insufficient resources do not match: %+v, want: %v", gotInsufficientResources, test.wantInsufficientResources)
523594
}
@@ -1434,9 +1505,10 @@ func Test_isSchedulableAfterNodeChange(t *testing.T) {
14341505

14351506
func TestIsFit(t *testing.T) {
14361507
testCases := map[string]struct {
1437-
pod *v1.Pod
1438-
node *v1.Node
1439-
expected bool
1508+
pod *v1.Pod
1509+
node *v1.Node
1510+
podLevelResourcesEnabled bool
1511+
expected bool
14401512
}{
14411513
"nil node": {
14421514
pod: &v1.Pod{},
@@ -1452,11 +1524,26 @@ func TestIsFit(t *testing.T) {
14521524
node: st.MakeNode().Capacity(map[v1.ResourceName]string{v1.ResourceCPU: "2"}).Obj(),
14531525
expected: true,
14541526
},
1527+
"insufficient pod-level resource": {
1528+
pod: st.MakePod().Resources(
1529+
v1.ResourceRequirements{Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("2")}},
1530+
).Obj(),
1531+
node: st.MakeNode().Capacity(map[v1.ResourceName]string{v1.ResourceCPU: "1"}).Obj(),
1532+
podLevelResourcesEnabled: true,
1533+
expected: false,
1534+
},
1535+
"sufficient pod-level resource": {
1536+
pod: st.MakePod().Resources(
1537+
v1.ResourceRequirements{Requests: v1.ResourceList{v1.ResourceCPU: resource.MustParse("2")}},
1538+
).Obj(),
1539+
node: st.MakeNode().Capacity(map[v1.ResourceName]string{v1.ResourceCPU: "2"}).Obj(),
1540+
expected: true,
1541+
},
14551542
}
14561543

14571544
for name, tc := range testCases {
14581545
t.Run(name, func(t *testing.T) {
1459-
if got := isFit(tc.pod, tc.node); got != tc.expected {
1546+
if got := isFit(tc.pod, tc.node, ResourceRequestsOptions{tc.podLevelResourcesEnabled}); got != tc.expected {
14601547
t.Errorf("expected: %v, got: %v", tc.expected, got)
14611548
}
14621549
})
@@ -1589,7 +1676,7 @@ func TestHaveAnyRequestedResourcesIncreased(t *testing.T) {
15891676
}
15901677
for name, tc := range testCases {
15911678
t.Run(name, func(t *testing.T) {
1592-
if got := haveAnyRequestedResourcesIncreased(tc.pod, tc.originalNode, tc.modifiedNode); got != tc.expected {
1679+
if got := haveAnyRequestedResourcesIncreased(tc.pod, tc.originalNode, tc.modifiedNode, ResourceRequestsOptions{}); got != tc.expected {
15931680
t.Errorf("expected: %v, got: %v", tc.expected, got)
15941681
}
15951682
})

pkg/scheduler/framework/plugins/noderesources/resource_allocation.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -119,7 +119,10 @@ func (r *resourceAllocationScorer) calculatePodResourceRequest(pod *v1.Pod, reso
119119

120120
opts := resourcehelper.PodResourcesOptions{
121121
UseStatusResources: utilfeature.DefaultFeatureGate.Enabled(features.InPlacePodVerticalScaling),
122+
// SkipPodLevelResources is set to false when PodLevelResources feature is enabled.
123+
SkipPodLevelResources: !utilfeature.DefaultFeatureGate.Enabled(features.PodLevelResources),
122124
}
125+
123126
if !r.useRequested {
124127
opts.NonMissingContainerRequests = v1.ResourceList{
125128
v1.ResourceCPU: *resource.NewMilliQuantity(schedutil.DefaultMilliCPURequest, resource.DecimalSI),

pkg/scheduler/framework/plugins/registry.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@ func NewInTreeRegistry() runtime.Registry {
5555
EnableSidecarContainers: feature.DefaultFeatureGate.Enabled(features.SidecarContainers),
5656
EnableSchedulingQueueHint: feature.DefaultFeatureGate.Enabled(features.SchedulerQueueingHints),
5757
EnableAsyncPreemption: feature.DefaultFeatureGate.Enabled(features.SchedulerAsyncPreemption),
58+
EnablePodLevelResources: feature.DefaultFeatureGate.Enabled(features.PodLevelResources),
5859
}
5960

6061
registry := runtime.Registry{

pkg/scheduler/framework/types.go

Lines changed: 64 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1052,19 +1052,74 @@ func (n *NodeInfo) update(pod *v1.Pod, sign int64) {
10521052
n.Generation = nextGeneration()
10531053
}
10541054

1055+
// getNonMissingContainerRequests returns the default non-zero CPU and memory
1056+
// requests for a container that the scheduler uses when container-level and
1057+
// pod-level requests are not set for a resource. It returns a ResourceList that
1058+
// includes these default non-zero requests, which are essential for the
1059+
// scheduler to function correctly.
1060+
// The method's behavior depends on whether pod-level resources are set or not:
1061+
// 1. When the pod level resources are not set, the method returns a ResourceList
1062+
// with the following defaults:
1063+
// - CPU: schedutil.DefaultMilliCPURequest
1064+
// - Memory: schedutil.DefaultMemoryRequest
1065+
//
1066+
// These defaults ensure that each container has a minimum resource request,
1067+
// allowing the scheduler to aggregate these requests and find a suitable node
1068+
// for the pod.
1069+
//
1070+
// 2. When the pod level resources are set, if a CPU or memory request is
1071+
// missing at the container-level *and* at the pod-level, the corresponding
1072+
// default value (schedutil.DefaultMilliCPURequest or schedutil.DefaultMemoryRequest)
1073+
// is included in the returned ResourceList.
1074+
// Note that these default values are not set in the Pod object itself, they are only used
1075+
// by the scheduler during node selection.
1076+
func getNonMissingContainerRequests(requests v1.ResourceList, podLevelResourcesSet bool) v1.ResourceList {
1077+
if !podLevelResourcesSet {
1078+
return v1.ResourceList{
1079+
v1.ResourceCPU: *resource.NewMilliQuantity(schedutil.DefaultMilliCPURequest, resource.DecimalSI),
1080+
v1.ResourceMemory: *resource.NewQuantity(schedutil.DefaultMemoryRequest, resource.DecimalSI),
1081+
}
1082+
}
1083+
1084+
nonMissingContainerRequests := make(v1.ResourceList, 2)
1085+
// DefaultMilliCPURequest serves as the fallback value when both
1086+
// pod-level and container-level CPU requests are not set.
1087+
// Note that the apiserver defaulting logic will propagate a non-zero
1088+
// container-level CPU request to the pod level if a pod-level request
1089+
// is not explicitly set.
1090+
if _, exists := requests[v1.ResourceCPU]; !exists {
1091+
nonMissingContainerRequests[v1.ResourceCPU] = *resource.NewMilliQuantity(schedutil.DefaultMilliCPURequest, resource.DecimalSI)
1092+
}
1093+
1094+
// DefaultMemoryRequest serves as the fallback value when both
1095+
// pod-level and container-level CPU requests are unspecified.
1096+
// Note that the apiserver defaulting logic will propagate a non-zero
1097+
// container-level memory request to the pod level if a pod-level request
1098+
// is not explicitly set.
1099+
if _, exists := requests[v1.ResourceMemory]; !exists {
1100+
nonMissingContainerRequests[v1.ResourceMemory] = *resource.NewQuantity(schedutil.DefaultMemoryRequest, resource.DecimalSI)
1101+
}
1102+
return nonMissingContainerRequests
1103+
1104+
}
1105+
10551106
func calculateResource(pod *v1.Pod) (Resource, int64, int64) {
10561107
requests := resourcehelper.PodRequests(pod, resourcehelper.PodResourcesOptions{
10571108
UseStatusResources: utilfeature.DefaultFeatureGate.Enabled(features.InPlacePodVerticalScaling),
1109+
// SkipPodLevelResources is set to false when PodLevelResources feature is enabled.
1110+
SkipPodLevelResources: !utilfeature.DefaultFeatureGate.Enabled(features.PodLevelResources),
10581111
})
1059-
1060-
non0Requests := resourcehelper.PodRequests(pod, resourcehelper.PodResourcesOptions{
1061-
UseStatusResources: utilfeature.DefaultFeatureGate.Enabled(features.InPlacePodVerticalScaling),
1062-
NonMissingContainerRequests: map[v1.ResourceName]resource.Quantity{
1063-
v1.ResourceCPU: *resource.NewMilliQuantity(schedutil.DefaultMilliCPURequest, resource.DecimalSI),
1064-
v1.ResourceMemory: *resource.NewQuantity(schedutil.DefaultMemoryRequest, resource.DecimalSI),
1065-
},
1066-
})
1067-
1112+
isPodLevelResourcesSet := utilfeature.DefaultFeatureGate.Enabled(features.PodLevelResources) && resourcehelper.IsPodLevelRequestsSet(pod)
1113+
nonMissingContainerRequests := getNonMissingContainerRequests(requests, isPodLevelResourcesSet)
1114+
non0Requests := requests
1115+
if len(nonMissingContainerRequests) > 0 {
1116+
non0Requests = resourcehelper.PodRequests(pod, resourcehelper.PodResourcesOptions{
1117+
UseStatusResources: utilfeature.DefaultFeatureGate.Enabled(features.InPlacePodVerticalScaling),
1118+
// SkipPodLevelResources is set to false when PodLevelResources feature is enabled.
1119+
SkipPodLevelResources: !utilfeature.DefaultFeatureGate.Enabled(features.PodLevelResources),
1120+
NonMissingContainerRequests: nonMissingContainerRequests,
1121+
})
1122+
}
10681123
non0CPU := non0Requests[v1.ResourceCPU]
10691124
non0Mem := non0Requests[v1.ResourceMemory]
10701125

0 commit comments

Comments
 (0)