Skip to content

Commit

Permalink
Introduce ProvisioningRequestPodsFilter processor
Browse files Browse the repository at this point in the history
  • Loading branch information
yaroslava-serdiuk committed Jan 2, 2024
1 parent 10fafe7 commit a4bb50c
Show file tree
Hide file tree
Showing 10 changed files with 304 additions and 115 deletions.
2 changes: 2 additions & 0 deletions cluster-autoscaler/config/autoscaling_options.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,6 +281,8 @@ type AutoscalingOptions struct {
DynamicNodeDeleteDelayAfterTaintEnabled bool
// BypassedSchedulers are used to specify which schedulers to bypass their processing
BypassedSchedulers map[string]bool
// ProvisioningRequestEnabled tells if CA processes ProvisioningRequest.
ProvisioningRequestEnabled bool
}

// KubeClientOptions specify options for kube client
Expand Down
30 changes: 3 additions & 27 deletions cluster-autoscaler/core/podlistprocessor/pod_list_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,21 +17,15 @@ limitations under the License.
package podlistprocessor

import (
apiv1 "k8s.io/api/core/v1"
"k8s.io/autoscaler/cluster-autoscaler/context"
"k8s.io/autoscaler/cluster-autoscaler/processors/pods"
"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
)

type defaultPodListProcessor struct {
processors []pods.PodListProcessor
}

// NewDefaultPodListProcessor returns a default implementation of the pod list
// processor, which wraps and sequentially runs other sub-processors.
func NewDefaultPodListProcessor(predicateChecker predicatechecker.PredicateChecker) *defaultPodListProcessor {
return &defaultPodListProcessor{
processors: []pods.PodListProcessor{
func NewDefaultPodListProcessor(predicateChecker predicatechecker.PredicateChecker) *pods.ListedPodListProcessor {
return &pods.ListedPodListProcessor{
Processors: []pods.PodListProcessor{
NewClearTPURequestsPodListProcessor(),
NewFilterOutExpendablePodListProcessor(),
NewCurrentlyDrainedNodesPodListProcessor(),
Expand All @@ -40,21 +34,3 @@ func NewDefaultPodListProcessor(predicateChecker predicatechecker.PredicateCheck
},
}
}

// Process runs sub-processors sequentially
func (p *defaultPodListProcessor) Process(ctx *context.AutoscalingContext, unschedulablePods []*apiv1.Pod) ([]*apiv1.Pod, error) {
var err error
for _, processor := range p.processors {
unschedulablePods, err = processor.Process(ctx, unschedulablePods)
if err != nil {
return nil, err
}
}
return unschedulablePods, nil
}

func (p *defaultPodListProcessor) CleanUp() {
for _, processor := range p.processors {
processor.CleanUp()
}
}
9 changes: 8 additions & 1 deletion cluster-autoscaler/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ import (
ca_processors "k8s.io/autoscaler/cluster-autoscaler/processors"
"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroupset"
"k8s.io/autoscaler/cluster-autoscaler/processors/nodeinfosprovider"
"k8s.io/autoscaler/cluster-autoscaler/processors/provreq"
"k8s.io/autoscaler/cluster-autoscaler/processors/scaledowncandidates"
"k8s.io/autoscaler/cluster-autoscaler/processors/scaledowncandidates/emptycandidates"
"k8s.io/autoscaler/cluster-autoscaler/processors/scaledowncandidates/previouscandidates"
Expand Down Expand Up @@ -250,6 +251,7 @@ var (
"--max-graceful-termination-sec flag should not be set when this flag is set. Not setting this flag will use unordered evictor by default."+
"Priority evictor reuses the concepts of drain logic in kubelet(https://github.com/kubernetes/enhancements/tree/master/keps/sig-node/2712-pod-priority-based-graceful-node-shutdown#migration-from-the-node-graceful-shutdown-feature)."+
"Eg. flag usage: '10000:20,1000:100,0:60'")
provisioningRequestsEnabled = flag.Bool("enable-provisioning-requests", false, "Whether the clusterautoscaler will be handling the ProvisioningRequest CRs.")
)

func isFlagPassed(name string) bool {
Expand Down Expand Up @@ -420,6 +422,7 @@ func createAutoscalingOptions() config.AutoscalingOptions {
},
DynamicNodeDeleteDelayAfterTaintEnabled: *dynamicNodeDeleteDelayAfterTaintEnabled,
BypassedSchedulers: scheduler_util.GetBypassedSchedulersMap(*bypassedSchedulers),
ProvisioningRequestEnabled: *provisioningRequestsEnabled,
}
}

Expand Down Expand Up @@ -475,7 +478,11 @@ func buildAutoscaler(debuggingSnapshotter debuggingsnapshot.DebuggingSnapshotter

opts.Processors = ca_processors.DefaultProcessors(autoscalingOptions)
opts.Processors.TemplateNodeInfoProvider = nodeinfosprovider.NewDefaultTemplateNodeInfoProvider(nodeInfoCacheExpireTime, *forceDaemonSets)
opts.Processors.PodListProcessor = podlistprocessor.NewDefaultPodListProcessor(opts.PredicateChecker)
podListProcessor := podlistprocessor.NewDefaultPodListProcessor(opts.PredicateChecker)
if autoscalingOptions.ProvisioningRequestEnabled {
podListProcessor.AppendProcessor(provreq.NewProvisioningRequestPodsFilter(provreq.NewDefautlEventManager()))
}
opts.Processors.PodListProcessor = podListProcessor
scaleDownCandidatesComparers := []scaledowncandidates.CandidatesComparer{}
if autoscalingOptions.ParallelDrain {
sdCandidatesSorting := previouscandidates.NewPreviousCandidates()
Expand Down
29 changes: 29 additions & 0 deletions cluster-autoscaler/processors/pods/pod_list_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,3 +48,32 @@ func (p *NoOpPodListProcessor) Process(
// CleanUp cleans up the processor's internal structures.
func (p *NoOpPodListProcessor) CleanUp() {
}

// ListedPodListProcessor is a list of PodListProcessors
type ListedPodListProcessor struct {
Processors []PodListProcessor
}

// Process runs sub-processors sequentially
func (p *ListedPodListProcessor) Process(ctx *context.AutoscalingContext, unschedulablePods []*apiv1.Pod) ([]*apiv1.Pod, error) {
var err error
for _, processor := range p.Processors {
unschedulablePods, err = processor.Process(ctx, unschedulablePods)
if err != nil {
return nil, err
}
}
return unschedulablePods, nil
}

// CleanUp cleans up the processor's internal structures.
func (p *ListedPodListProcessor) CleanUp() {
for _, processor := range p.Processors {
processor.CleanUp()
}
}

// AppendProcessor append processor to the list.
func (p *ListedPodListProcessor) AppendProcessor(processor PodListProcessor) {
p.Processors = append(p.Processors, processor)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
Copyright 2023 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package provreq

import (
"fmt"
"time"

apiv1 "k8s.io/api/core/v1"
v1 "k8s.io/api/core/v1"
"k8s.io/autoscaler/cluster-autoscaler/context"
"k8s.io/autoscaler/cluster-autoscaler/processors/pods"
"k8s.io/autoscaler/cluster-autoscaler/utils/klogx"
)

const (
provisioningRequestPodAnnotationKey = "cluster-autoscaler.kubernetes.io/consume-provisioning-request"
maxProvReqEvent = 50
)

// EventManager is an interface for handling events for provisioning request.
type EventManager interface {
LogIgnoredInScaleUpEvent(context *context.AutoscalingContext, now time.Time, pod *apiv1.Pod, prName string)
Reset()
}

type defaultEventManager struct {
loggedEvents int
limit int
}

// NewDefautlEventManager return basic event manager.
func NewDefautlEventManager() *defaultEventManager {
return &defaultEventManager{limit: maxProvReqEvent}
}

// LogIgnoredInScaleUpEvent adds event about ignored scale up for unscheduled pod, that consumes Provisioning Request.
func (e *defaultEventManager) LogIgnoredInScaleUpEvent(context *context.AutoscalingContext, now time.Time, pod *apiv1.Pod, prName string) {
message := fmt.Sprintf("Unschedulable pod ignored in scale-up loop, because it's consuming ProvisioningRequest %s/%s", pod.Namespace, prName)
if e.loggedEvents < e.limit {
context.Recorder.Event(pod, apiv1.EventTypeNormal, "", message)
e.loggedEvents++
}
}

// Reset resets event manager internal structure. It will be called once before handling all pods.
func (e *defaultEventManager) Reset() {
e.loggedEvents = 0
}

// ProvisioningRequestPodsFilter filter out pods that consumes Provisioning Request
type ProvisioningRequestPodsFilter struct {
eventManager EventManager
}

// Process filters out all pods that are consuming a Provisioning Request from unschedulable pods list.
func (p *ProvisioningRequestPodsFilter) Process(
context *context.AutoscalingContext,
unschedulablePods []*apiv1.Pod,
) ([]*apiv1.Pod, error) {
now := time.Now()
p.eventManager.Reset()
loggingQuota := klogx.PodsLoggingQuota()
result := make([]*apiv1.Pod, 0, len(unschedulablePods))
for _, pod := range unschedulablePods {
prName, found := provisioningRequestName(pod)
if !found {
result = append(result, pod)
continue
}
klogx.V(1).UpTo(loggingQuota).Infof("Ignoring unschedulable pod %s/%s as it consumes ProvisioningRequest: %s/%s", pod.Namespace, pod.Name, pod.Namespace, prName)
p.eventManager.LogIgnoredInScaleUpEvent(context, now, pod, prName)
}
klogx.V(1).Over(loggingQuota).Infof("There are also %v other pods which were ignored", -loggingQuota.Left())
return result, nil
}

// CleanUp cleans up the processor's internal structures.
func (p *ProvisioningRequestPodsFilter) CleanUp() {}

// NewProvisioningRequestPodsFilter creates a ProvisioningRequest filter processor.
func NewProvisioningRequestPodsFilter(e EventManager) pods.PodListProcessor {
return &ProvisioningRequestPodsFilter{e}
}

func provisioningRequestName(pod *v1.Pod) (string, bool) {
if pod == nil || pod.Annotations == nil {
return "", false
}
provReqName, found := pod.Annotations[provisioningRequestPodAnnotationKey]
return provReqName, found
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
Copyright 2023 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package provreq

import (
"fmt"
"testing"
"time"

"github.com/stretchr/testify/assert"
apiv1 "k8s.io/api/core/v1"
v1 "k8s.io/api/core/v1"
"k8s.io/autoscaler/cluster-autoscaler/context"
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
"k8s.io/client-go/tools/record"
)

func TestProvisioningRequestPodsFilter(t *testing.T) {
prPod1 := BuildTestPod("pr-pod-1", 500, 10)
prPod1.Annotations[provisioningRequestPodAnnotationKey] = "pr-class"

prPod2 := BuildTestPod("pr-pod-2", 500, 10)
prPod2.Annotations[provisioningRequestPodAnnotationKey] = "pr-class-2"

pod1 := BuildTestPod("pod-1", 500, 10)
pod2 := BuildTestPod("pod-2", 500, 10)

testCases := map[string]struct {
unschedulableCandidates []*apiv1.Pod
expectedUnscheduledPods []*apiv1.Pod
}{
"ProvisioningRequest consumer is filtered out": {
unschedulableCandidates: []*v1.Pod{prPod1, pod1},
expectedUnscheduledPods: []*v1.Pod{pod1},
},
"Different ProvisioningRequest consumers are filtered out": {
unschedulableCandidates: []*v1.Pod{prPod1, prPod2, pod1},
expectedUnscheduledPods: []*v1.Pod{pod1},
},
"No pod is filtered": {
unschedulableCandidates: []*v1.Pod{pod1, pod2},
expectedUnscheduledPods: []*v1.Pod{pod1, pod2},
},
"Empty unschedulable pods list": {
unschedulableCandidates: []*v1.Pod{},
expectedUnscheduledPods: []*v1.Pod{},
},
"All ProvisioningRequest consumers are filtered out": {
unschedulableCandidates: []*v1.Pod{prPod1, prPod2},
expectedUnscheduledPods: []*v1.Pod{},
},
}
for _, test := range testCases {
eventRecorder := record.NewFakeRecorder(10)
ctx := &context.AutoscalingContext{AutoscalingKubeClients: context.AutoscalingKubeClients{Recorder: eventRecorder}}
filter := NewProvisioningRequestPodsFilter(NewDefautlEventManager())
got, _ := filter.Process(ctx, test.unschedulableCandidates)
assert.ElementsMatch(t, got, test.expectedUnscheduledPods)
if len(test.expectedUnscheduledPods) < len(test.expectedUnscheduledPods) {
select {
case event := <-eventRecorder.Events:
assert.Contains(t, event, "Unschedulable pod ignored in scale-up loop, because it's consuming ProvisioningRequest default/pr-class")
case <-time.After(1 * time.Second):
t.Errorf("Timeout waiting for event")
}
}
}
}

func TestEventManager(t *testing.T) {
eventLimit := 5
eventManager := &defaultEventManager{limit: eventLimit}
prFilter := NewProvisioningRequestPodsFilter(eventManager)
eventRecorder := record.NewFakeRecorder(10)
ctx := &context.AutoscalingContext{AutoscalingKubeClients: context.AutoscalingKubeClients{Recorder: eventRecorder}}
unscheduledPods := []*v1.Pod{BuildTestPod("pod", 500, 10)}

for i := 0; i < 10; i++ {
prPod := BuildTestPod(fmt.Sprintf("pr-pod-%d", i), 10, 10)
prPod.Annotations[provisioningRequestPodAnnotationKey] = "pr-class"
unscheduledPods = append(unscheduledPods, prPod)
}
got, err := prFilter.Process(ctx, unscheduledPods)
assert.NoError(t, err)
if len(got) != 1 {
t.Errorf("Want 1 unschedulable pod, got: %v", got)
}
assert.Equal(t, eventManager.loggedEvents, eventLimit)
for i := 0; i < eventLimit; i++ {
select {
case event := <-eventRecorder.Events:
assert.Contains(t, event, "Unschedulable pod ignored in scale-up loop, because it's consuming ProvisioningRequest default/pr-class")
case <-time.After(1 * time.Second):
t.Errorf("Timeout waiting for event")
}
}
select {
case <-eventRecorder.Events:
t.Errorf("Receive event after reaching event limit")
case <-time.After(1 * time.Millisecond):
return
}
}
Loading

0 comments on commit a4bb50c

Please sign in to comment.