Update flags, Improve tests readability & use Bypass instead of ignore in naming
This commit is contained in:
parent
4635a6dc04
commit
a1ae4d3b57
|
|
@ -277,11 +277,6 @@ type AutoscalingOptions struct {
|
|||
// dynamicNodeDeleteDelayAfterTaintEnabled is used to enable/disable dynamic adjustment of NodeDeleteDelayAfterTaint
|
||||
// based on the latency between the CA and the api-server
|
||||
DynamicNodeDeleteDelayAfterTaintEnabled bool
|
||||
//IgnoreSchedulerProcessing is used to signal whether CA will/won't wait
|
||||
//for scheduler to mark pods as unschedulable and will process both marked & non-marked pods
|
||||
//it will also signal whether we enable/disable waiting for pod time buffers before triggering a scale-up.
|
||||
IgnoreSchedulerProcessing bool
|
||||
//IgnoredSchedulers are used to specify which schedulers to ignore their processing
|
||||
//if IgnoreSchedulerProcessing is set to true
|
||||
IgnoredSchedulers map[string]bool
|
||||
// BypassedSchedulers are used to specify which schedulers to bypass their processing
|
||||
BypassedSchedulers map[string]bool
|
||||
}
|
||||
|
|
|
|||
|
|
@ -36,12 +36,14 @@ func NewFilterOutExpendablePodListProcessor() *filterOutExpendable {
|
|||
func (p *filterOutExpendable) Process(context *context.AutoscalingContext, pods []*apiv1.Pod) ([]*apiv1.Pod, error) {
|
||||
nodes, err := context.AllNodeLister().List()
|
||||
if err != nil {
|
||||
klog.Warningf("Failed to list all nodes while filtering expendable: %v", err)
|
||||
return nil, err
|
||||
}
|
||||
expendablePodsPriorityCutoff := context.AutoscalingOptions.ExpendablePodsPriorityCutoff
|
||||
|
||||
unschedulablePods, waitingForLowerPriorityPreemption := core_utils.FilterOutExpendableAndSplit(pods, nodes, expendablePodsPriorityCutoff)
|
||||
if err = p.addPreemptingPodsToSnapshot(waitingForLowerPriorityPreemption, context); err != nil {
|
||||
klog.Warningf("Failed to add preempting pods to snapshot: %v", err)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -310,8 +310,9 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
|
|||
}
|
||||
originalScheduledPods, unschedulablePods := kube_util.ScheduledPods(pods), kube_util.UnschedulablePods(pods)
|
||||
schedulerUnprocessed := make([]*apiv1.Pod, 0, 0)
|
||||
if a.IgnoreSchedulerProcessing {
|
||||
schedulerUnprocessed = kube_util.SchedulerUnprocessedPods(pods, a.IgnoredSchedulers)
|
||||
isSchedulerProcessingIgnored := len(a.BypassedSchedulers) > 0
|
||||
if isSchedulerProcessingIgnored {
|
||||
schedulerUnprocessed = kube_util.SchedulerUnprocessedPods(pods, a.BypassedSchedulers)
|
||||
}
|
||||
|
||||
// Update cluster resource usage metrics
|
||||
|
|
@ -456,7 +457,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
|
|||
|
||||
// SchedulerUnprocessed might be zero here if it was disabled
|
||||
metrics.UpdateUnschedulablePodsCount(len(unschedulablePods), len(schedulerUnprocessed))
|
||||
if a.IgnoreSchedulerProcessing {
|
||||
if isSchedulerProcessingIgnored {
|
||||
// Treat unknown pods as unschedulable, pod list processor will remove schedulable pods
|
||||
unschedulablePods = append(unschedulablePods, schedulerUnprocessed...)
|
||||
}
|
||||
|
|
@ -543,7 +544,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
|
|||
} else if a.MaxNodesTotal > 0 && len(readyNodes) >= a.MaxNodesTotal {
|
||||
scaleUpStatus.Result = status.ScaleUpNoOptionsAvailable
|
||||
klog.V(1).Info("Max total nodes in cluster reached")
|
||||
} else if !a.IgnoreSchedulerProcessing && allPodsAreNew(unschedulablePodsToHelp, currentTime) {
|
||||
} else if !isSchedulerProcessingIgnored && allPodsAreNew(unschedulablePodsToHelp, currentTime) {
|
||||
// The assumption here is that these pods have been created very recently and probably there
|
||||
// is more pods to come. In theory we could check the newest pod time but then if pod were created
|
||||
// slowly but at the pace of 1 every 2 seconds then no scale up would be triggered for long time.
|
||||
|
|
|
|||
|
|
@ -51,6 +51,7 @@ import (
|
|||
"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
|
||||
kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/utils/scheduler"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/utils/taints"
|
||||
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
|
||||
kube_record "k8s.io/client-go/tools/record"
|
||||
|
|
@ -156,6 +157,122 @@ func setUpScaleDownActuator(ctx *context.AutoscalingContext, autoscalingOptions
|
|||
ctx.ScaleDownActuator = actuation.NewActuator(ctx, nil, deletiontracker.NewNodeDeletionTracker(0*time.Second), deleteOptions, rules.Default(deleteOptions), NewTestProcessors(ctx).NodeGroupConfigProcessor)
|
||||
}
|
||||
|
||||
type nodeGroup struct {
|
||||
name string
|
||||
nodes []*apiv1.Node
|
||||
min int
|
||||
max int
|
||||
}
|
||||
type scaleCall struct {
|
||||
ng string
|
||||
delta int
|
||||
}
|
||||
type testCase struct {
|
||||
nodeGroups []*nodeGroup
|
||||
pods []*apiv1.Pod
|
||||
podListerCallTimes int
|
||||
daemonSets []*appsv1.DaemonSet
|
||||
daemonSetListerCallTimes int
|
||||
pdbs []*policyv1.PodDisruptionBudget
|
||||
pdbListerCallTimes int
|
||||
expectedScaleUps []*scaleCall
|
||||
expectedScaleDowns []*scaleCall
|
||||
now time.Time
|
||||
lastScaleupTime time.Time
|
||||
lastScaleDownFailTime time.Time
|
||||
runAutoscalerAt time.Time
|
||||
autoscalingOptions config.AutoscalingOptions
|
||||
OkTotalUnreadyCount int
|
||||
}
|
||||
|
||||
func testAutoscaler(t *testing.T, tc testCase) {
|
||||
readyNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
allNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
allPodListerMock := &podListerMock{}
|
||||
podDisruptionBudgetListerMock := &podDisruptionBudgetListerMock{}
|
||||
daemonSetListerMock := &daemonSetListerMock{}
|
||||
onScaleUpMock := &onScaleUpMock{}
|
||||
onScaleDownMock := &onScaleDownMock{}
|
||||
deleteFinished := make(chan bool, 1)
|
||||
|
||||
provider := testprovider.NewTestCloudProvider(
|
||||
func(id string, delta int) error {
|
||||
return onScaleUpMock.ScaleUp(id, delta)
|
||||
}, func(id string, name string) error {
|
||||
ret := onScaleDownMock.ScaleDown(id, name)
|
||||
deleteFinished <- true
|
||||
return ret
|
||||
})
|
||||
|
||||
allNodes := make([]*apiv1.Node, 0)
|
||||
for _, ng := range tc.nodeGroups {
|
||||
provider.AddNodeGroup(ng.name, ng.min, ng.max, len(ng.nodes))
|
||||
for _, node := range ng.nodes {
|
||||
allNodes = append(allNodes, node)
|
||||
provider.AddNode(ng.name, node)
|
||||
}
|
||||
reflectedNg := reflect.ValueOf(provider.GetNodeGroup(ng.name)).Interface().(*testprovider.TestNodeGroup)
|
||||
assert.NotNil(t, reflectedNg)
|
||||
}
|
||||
// Create context with mocked lister registry.
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
context, err := NewScaleTestAutoscalingContext(tc.autoscalingOptions, &fake.Clientset{}, nil, provider, processorCallbacks, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
setUpScaleDownActuator(&context, tc.autoscalingOptions)
|
||||
|
||||
listerRegistry := kube_util.NewListerRegistry(allNodeLister, readyNodeLister, allPodListerMock,
|
||||
podDisruptionBudgetListerMock, daemonSetListerMock,
|
||||
nil, nil, nil, nil)
|
||||
context.ListerRegistry = listerRegistry
|
||||
|
||||
clusterStateConfig := clusterstate.ClusterStateRegistryConfig{
|
||||
OkTotalUnreadyCount: tc.OkTotalUnreadyCount,
|
||||
}
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, NewBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(tc.autoscalingOptions.NodeGroupDefaults))
|
||||
|
||||
clusterState.UpdateNodes(allNodes, nil, tc.now)
|
||||
processors := NewTestProcessors(&context)
|
||||
|
||||
sdPlanner, sdActuator := newScaleDownPlannerAndActuator(t, &context, processors, clusterState)
|
||||
suOrchestrator := orchestrator.New()
|
||||
suOrchestrator.Initialize(&context, processors, clusterState, taints.TaintConfig{})
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
clusterStateRegistry: clusterState,
|
||||
lastScaleUpTime: tc.lastScaleupTime,
|
||||
lastScaleDownFailTime: tc.lastScaleDownFailTime,
|
||||
scaleDownPlanner: sdPlanner,
|
||||
scaleDownActuator: sdActuator,
|
||||
scaleUpOrchestrator: suOrchestrator,
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
}
|
||||
|
||||
// Assummes all nodes are ready, to be updated when used in tests which needs non-ready nodes
|
||||
readyNodeLister.SetNodes(allNodes)
|
||||
allNodeLister.SetNodes(allNodes)
|
||||
allPodListerMock.On("List").Return(tc.pods, nil).Times(tc.podListerCallTimes)
|
||||
daemonSetListerMock.On("List", labels.Everything()).Return(tc.daemonSets, nil).Times(tc.daemonSetListerCallTimes)
|
||||
podDisruptionBudgetListerMock.On("List").Return([]*policyv1.PodDisruptionBudget{}, nil).Times(tc.pdbListerCallTimes)
|
||||
|
||||
for _, scaleUpCall := range tc.expectedScaleUps {
|
||||
onScaleUpMock.On("ScaleUp", scaleUpCall.ng, scaleUpCall.delta).Return(nil).Once()
|
||||
}
|
||||
for _, scaleDownCall := range tc.expectedScaleDowns {
|
||||
onScaleDownMock.On("ScaleDown", scaleDownCall.ng, scaleDownCall.delta).Return(nil).Once()
|
||||
}
|
||||
|
||||
err = autoscaler.RunOnce(tc.runAutoscalerAt)
|
||||
assert.NoError(t, err)
|
||||
mock.AssertExpectationsForObjects(t, allPodListerMock,
|
||||
podDisruptionBudgetListerMock, daemonSetListerMock, onScaleUpMock, onScaleDownMock)
|
||||
}
|
||||
|
||||
// TODO: Refactor tests to use testAutoscaler
|
||||
|
||||
func TestStaticAutoscalerRunOnce(t *testing.T) {
|
||||
readyNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
allNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
|
|
@ -992,47 +1109,8 @@ func TestStaticAutoscalerRunOnceWithFilteringOnUpcomingNodesEnabledNoScaleUp(t *
|
|||
}
|
||||
|
||||
func TestStaticAutoscalerRunOnceWithSchedulerProcessingIgnored(t *testing.T) {
|
||||
ignoredScheduler := "ignored-scheduler"
|
||||
nonIgnoredScheduler := "non-ignored-scheduler"
|
||||
|
||||
readyNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
allNodeLister := kubernetes.NewTestNodeLister(nil)
|
||||
allPodListerMock := &podListerMock{}
|
||||
podDisruptionBudgetListerMock := &podDisruptionBudgetListerMock{}
|
||||
daemonSetListerMock := &daemonSetListerMock{}
|
||||
onScaleUpMock := &onScaleUpMock{}
|
||||
onScaleDownMock := &onScaleDownMock{}
|
||||
deleteFinished := make(chan bool, 1)
|
||||
|
||||
now := time.Now()
|
||||
later := now.Add(1 * time.Minute)
|
||||
|
||||
n1 := BuildTestNode("n1", 1000, 1000)
|
||||
SetNodeReadyState(n1, true, time.Now())
|
||||
|
||||
p1 := BuildTestPod("p1", 600, 100)
|
||||
p1.Spec.NodeName = "n1"
|
||||
p2 := BuildTestPod("p2", 600, 100, MarkUnschedulable())
|
||||
p3 := BuildTestPod("p3", 600, 100, AddSchedulerName(apiv1.DefaultSchedulerName)) // Not yet processed by scheduler, default scheduler is ignored
|
||||
p4 := BuildTestPod("p4", 600, 100, AddSchedulerName(ignoredScheduler)) // non-default scheduler & ignored, expects a scale-up
|
||||
p5 := BuildTestPod("p5", 600, 100, AddSchedulerName(nonIgnoredScheduler)) // non-default scheduler & not ignored, shouldn't cause a scale-up
|
||||
|
||||
provider := testprovider.NewTestCloudProvider(
|
||||
func(id string, delta int) error {
|
||||
return onScaleUpMock.ScaleUp(id, delta)
|
||||
}, func(id string, name string) error {
|
||||
ret := onScaleDownMock.ScaleDown(id, name)
|
||||
deleteFinished <- true
|
||||
return ret
|
||||
})
|
||||
provider.AddNodeGroup("ng1", 1, 10, 1)
|
||||
provider.AddNode("ng1", n1)
|
||||
|
||||
ng1 := reflect.ValueOf(provider.GetNodeGroup("ng1")).Interface().(*testprovider.TestNodeGroup)
|
||||
assert.NotNil(t, ng1)
|
||||
assert.NotNil(t, provider)
|
||||
|
||||
// Create context with mocked lister registry.
|
||||
bypassedScheduler := "bypassed-scheduler"
|
||||
nonBypassedScheduler := "non-bypassed-scheduler"
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupDefaults: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
|
|
@ -1040,67 +1118,114 @@ func TestStaticAutoscalerRunOnceWithSchedulerProcessingIgnored(t *testing.T) {
|
|||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
IgnoreSchedulerProcessing: true,
|
||||
IgnoredSchedulers: map[string]bool{
|
||||
apiv1.DefaultSchedulerName: true,
|
||||
ignoredScheduler: true,
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
BypassedSchedulers: scheduler.GetBypassedSchedulersMap([]string{
|
||||
apiv1.DefaultSchedulerName,
|
||||
bypassedScheduler,
|
||||
}),
|
||||
}
|
||||
|
||||
n1 := BuildTestNode("n1", 1000, 1000)
|
||||
SetNodeReadyState(n1, true, time.Now())
|
||||
|
||||
p1 := BuildTestPod("p1", 600, 100)
|
||||
p1.Spec.NodeName = "n1"
|
||||
p2 := BuildTestPod("p2", 100, 100, AddSchedulerName(bypassedScheduler))
|
||||
p3 := BuildTestPod("p3", 600, 100, AddSchedulerName(apiv1.DefaultSchedulerName)) // Not yet processed by scheduler, default scheduler is ignored
|
||||
p4 := BuildTestPod("p4", 600, 100, AddSchedulerName(bypassedScheduler)) // non-default scheduler & ignored, expects a scale-up
|
||||
p5 := BuildTestPod("p5", 600, 100, AddSchedulerName(nonBypassedScheduler))
|
||||
|
||||
testCases := map[string]testCase{
|
||||
"Unprocessed pod with bypassed scheduler doesn't cause a scale-up when there's capacity": {
|
||||
pods: []*apiv1.Pod{p1, p2},
|
||||
podListerCallTimes: 2,
|
||||
nodeGroups: []*nodeGroup{{
|
||||
name: "ng1",
|
||||
min: 1,
|
||||
max: 10,
|
||||
nodes: []*apiv1.Node{n1},
|
||||
}},
|
||||
daemonSetListerCallTimes: 1,
|
||||
pdbListerCallTimes: 1,
|
||||
expectedScaleUps: []*scaleCall{},
|
||||
now: time.Now(),
|
||||
lastScaleupTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
runAutoscalerAt: time.Now().Add(time.Hour),
|
||||
autoscalingOptions: options,
|
||||
},
|
||||
"Unprocessed pod with bypassed scheduler causes a scale-up when there's no capacity - Default Scheduler": {
|
||||
pods: []*apiv1.Pod{p1, p3},
|
||||
podListerCallTimes: 2,
|
||||
nodeGroups: []*nodeGroup{{
|
||||
name: "ng1",
|
||||
min: 1,
|
||||
max: 10,
|
||||
nodes: []*apiv1.Node{n1},
|
||||
}},
|
||||
daemonSetListerCallTimes: 1,
|
||||
pdbListerCallTimes: 1,
|
||||
expectedScaleUps: []*scaleCall{{
|
||||
ng: "ng1",
|
||||
delta: 1,
|
||||
}},
|
||||
now: time.Now(),
|
||||
lastScaleupTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
runAutoscalerAt: time.Now().Add(time.Hour),
|
||||
autoscalingOptions: options,
|
||||
},
|
||||
"Unprocessed pod with bypassed scheduler causes a scale-up when there's no capacity - Non-default Scheduler": {
|
||||
pods: []*apiv1.Pod{p1, p4},
|
||||
podListerCallTimes: 2,
|
||||
nodeGroups: []*nodeGroup{{
|
||||
name: "ng1",
|
||||
min: 1,
|
||||
max: 10,
|
||||
nodes: []*apiv1.Node{n1},
|
||||
}},
|
||||
daemonSetListerCallTimes: 1,
|
||||
pdbListerCallTimes: 1,
|
||||
expectedScaleUps: []*scaleCall{{
|
||||
ng: "ng1",
|
||||
delta: 1,
|
||||
}},
|
||||
now: time.Now(),
|
||||
lastScaleupTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
runAutoscalerAt: time.Now().Add(time.Hour),
|
||||
autoscalingOptions: options,
|
||||
},
|
||||
"Unprocessed pod with non-bypassed scheduler doesn't cause a scale-up when there's no capacity": {
|
||||
pods: []*apiv1.Pod{p1, p5},
|
||||
podListerCallTimes: 2,
|
||||
nodeGroups: []*nodeGroup{{
|
||||
name: "ng1",
|
||||
min: 1,
|
||||
max: 10,
|
||||
nodes: []*apiv1.Node{n1},
|
||||
}},
|
||||
daemonSetListerCallTimes: 1,
|
||||
pdbListerCallTimes: 1,
|
||||
expectedScaleUps: []*scaleCall{},
|
||||
now: time.Now(),
|
||||
lastScaleupTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
runAutoscalerAt: time.Now().Add(time.Hour),
|
||||
autoscalingOptions: options,
|
||||
},
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
context, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, nil, provider, processorCallbacks, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
setUpScaleDownActuator(&context, options)
|
||||
|
||||
listerRegistry := kube_util.NewListerRegistry(allNodeLister, readyNodeLister, allPodListerMock,
|
||||
podDisruptionBudgetListerMock, daemonSetListerMock,
|
||||
nil, nil, nil, nil)
|
||||
context.ListerRegistry = listerRegistry
|
||||
|
||||
clusterStateConfig := clusterstate.ClusterStateRegistryConfig{
|
||||
OkTotalUnreadyCount: 1,
|
||||
}
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, NewBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(options.NodeGroupDefaults))
|
||||
|
||||
nodes := []*apiv1.Node{n1}
|
||||
clusterState.UpdateNodes(nodes, nil, now)
|
||||
|
||||
processors := NewTestProcessors(&context)
|
||||
|
||||
sdPlanner, sdActuator := newScaleDownPlannerAndActuator(t, &context, processors, clusterState)
|
||||
suOrchestrator := orchestrator.New()
|
||||
suOrchestrator.Initialize(&context, processors, clusterState, taints.TaintConfig{})
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
clusterStateRegistry: clusterState,
|
||||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDownPlanner: sdPlanner,
|
||||
scaleDownActuator: sdActuator,
|
||||
scaleUpOrchestrator: suOrchestrator,
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
for tcName, tc := range testCases {
|
||||
t.Run(tcName, func(t *testing.T) {
|
||||
testAutoscaler(t, tc)
|
||||
})
|
||||
}
|
||||
|
||||
// Scale up.
|
||||
readyNodeLister.SetNodes([]*apiv1.Node{n1})
|
||||
allNodeLister.SetNodes([]*apiv1.Node{n1})
|
||||
allPodListerMock.On("List").Return([]*apiv1.Pod{p1, p2, p3, p4, p5}, nil).Twice()
|
||||
daemonSetListerMock.On("List", labels.Everything()).Return([]*appsv1.DaemonSet{}, nil).Once()
|
||||
podDisruptionBudgetListerMock.On("List").Return([]*policyv1.PodDisruptionBudget{}, nil).Once()
|
||||
onScaleUpMock.On("ScaleUp", "ng1", 3).Return(nil).Once()
|
||||
|
||||
err = autoscaler.RunOnce(later.Add(time.Hour))
|
||||
assert.NoError(t, err)
|
||||
mock.AssertExpectationsForObjects(t, allPodListerMock,
|
||||
podDisruptionBudgetListerMock, daemonSetListerMock, onScaleUpMock, onScaleDownMock)
|
||||
}
|
||||
|
||||
func TestStaticAutoscalerInstanceCreationErrors(t *testing.T) {
|
||||
|
|
|
|||
|
|
@ -31,7 +31,6 @@ import (
|
|||
|
||||
"github.com/spf13/pflag"
|
||||
|
||||
apiv1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/meta"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apiserver/pkg/server/mux"
|
||||
|
|
@ -244,8 +243,7 @@ var (
|
|||
maxAllocatableDifferenceRatio = flag.Float64("max-allocatable-difference-ratio", config.DefaultMaxAllocatableDifferenceRatio, "Maximum difference in allocatable resources between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's allocatable resource.")
|
||||
forceDaemonSets = flag.Bool("force-ds", false, "Blocks scale-up of node groups too small for all suitable Daemon Sets pods.")
|
||||
dynamicNodeDeleteDelayAfterTaintEnabled = flag.Bool("dynamic-node-delete-delay-after-taint-enabled", false, "Enables dynamic adjustment of NodeDeleteDelayAfterTaint based of the latency between CA and api-server")
|
||||
ignoreSchedulerProcessing = flag.Bool("ignore-scheduler-processing", false, "If true, cluster autoscaler will not wait for scheduler to mark pods as unschedulable and will process both marked & non-marked pods (Schedulable pods will be filtered before scaling-up) it will also disable waiting for pod time buffers before triggering a scale-up.")
|
||||
ignoredSchedulers = pflag.StringSlice("ignore-schedulers", []string{apiv1.DefaultSchedulerName}, fmt.Sprintf("Names of schedulers to be ignored if '--ignore-scheduler-processing' is set to true. default value '%s' is used", apiv1.DefaultSchedulerName))
|
||||
bypassedSchedulers = pflag.StringSlice("bypassed-scheduler-names", []string{}, fmt.Sprintf("Names of schedulers to bypass. If set to non-empty value, CA will not wait for pods to reach a certain age before triggering a scale-up."))
|
||||
)
|
||||
|
||||
func isFlagPassed(name string) bool {
|
||||
|
|
@ -393,8 +391,7 @@ func createAutoscalingOptions() config.AutoscalingOptions {
|
|||
MaxFreeDifferenceRatio: *maxFreeDifferenceRatio,
|
||||
},
|
||||
DynamicNodeDeleteDelayAfterTaintEnabled: *dynamicNodeDeleteDelayAfterTaintEnabled,
|
||||
IgnoreSchedulerProcessing: *ignoreSchedulerProcessing,
|
||||
IgnoredSchedulers: scheduler_util.GetIgnoredSchedulersMap(*ignoredSchedulers),
|
||||
BypassedSchedulers: scheduler_util.GetBypassedSchedulersMap(*bypassedSchedulers),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -145,26 +145,20 @@ type PodLister interface {
|
|||
}
|
||||
|
||||
// isScheduled checks whether a pod is scheduled on a node or not
|
||||
// This method doesn't check for nil ptr, it's the responsibility of the caller
|
||||
func isScheduled(pod *apiv1.Pod) bool {
|
||||
if pod == nil {
|
||||
return false
|
||||
}
|
||||
return pod.Spec.NodeName != ""
|
||||
}
|
||||
|
||||
// isDeleted checks whether a pod is deleted not
|
||||
// This method doesn't check for nil ptr, it's the responsibility of the caller
|
||||
func isDeleted(pod *apiv1.Pod) bool {
|
||||
if pod == nil {
|
||||
return false
|
||||
}
|
||||
return pod.GetDeletionTimestamp() != nil
|
||||
}
|
||||
|
||||
// isUnschedulable checks whether a pod is unschedulable or not
|
||||
// This method doesn't check for nil ptr, it's the responsibility of the caller
|
||||
func isUnschedulable(pod *apiv1.Pod) bool {
|
||||
if pod == nil {
|
||||
return false
|
||||
}
|
||||
if isScheduled(pod) || isDeleted(pod) {
|
||||
return false
|
||||
}
|
||||
|
|
@ -175,12 +169,6 @@ func isUnschedulable(pod *apiv1.Pod) bool {
|
|||
return true
|
||||
}
|
||||
|
||||
// getIsDefaultSchedulerIgnored checks if the default scheduler should be ignored or not
|
||||
func getIsDefaultSchedulerIgnored(ignoredSchedulers map[string]bool) bool {
|
||||
ignored, ok := ignoredSchedulers[apiv1.DefaultSchedulerName]
|
||||
return ignored && ok
|
||||
}
|
||||
|
||||
// ScheduledPods is a helper method that returns all scheduled pods from given pod list.
|
||||
func ScheduledPods(allPods []*apiv1.Pod) []*apiv1.Pod {
|
||||
var scheduledPods []*apiv1.Pod
|
||||
|
|
@ -193,18 +181,12 @@ func ScheduledPods(allPods []*apiv1.Pod) []*apiv1.Pod {
|
|||
return scheduledPods
|
||||
}
|
||||
|
||||
// SchedulerUnprocessedPods is a helper method that returns all pods which are not yet processed by the specified ignored schedulers
|
||||
func SchedulerUnprocessedPods(allPods []*apiv1.Pod, ignoredSchedulers map[string]bool) []*apiv1.Pod {
|
||||
// SchedulerUnprocessedPods is a helper method that returns all pods which are not yet processed by the specified bypassed schedulers
|
||||
func SchedulerUnprocessedPods(allPods []*apiv1.Pod, bypassedSchedulers map[string]bool) []*apiv1.Pod {
|
||||
var unprocessedPods []*apiv1.Pod
|
||||
|
||||
isDefaultSchedulerIgnored := getIsDefaultSchedulerIgnored(ignoredSchedulers)
|
||||
|
||||
for _, pod := range allPods {
|
||||
// Don't add a pod with a scheduler that isn't specified by the user
|
||||
if !isDefaultSchedulerIgnored && pod.Spec.SchedulerName == "" {
|
||||
continue
|
||||
}
|
||||
if isIgnored, found := ignoredSchedulers[pod.Spec.SchedulerName]; !found || !isIgnored {
|
||||
if canBypass := bypassedSchedulers[pod.Spec.SchedulerName]; !canBypass {
|
||||
continue
|
||||
}
|
||||
// Make sure it's not scheduled or deleted
|
||||
|
|
|
|||
|
|
@ -148,11 +148,15 @@ func ConfigFromPath(path string) (*scheduler_config.KubeSchedulerConfiguration,
|
|||
return cfgObj, nil
|
||||
}
|
||||
|
||||
// GetIgnoredSchedulersMap returns a map of scheduler names that should be ignored as keys, and values are set to true
|
||||
func GetIgnoredSchedulersMap(ignoredSchedulers []string) map[string]bool {
|
||||
ignoredSchedulersMap := make(map[string]bool, len(ignoredSchedulers))
|
||||
for _, scheduler := range ignoredSchedulers {
|
||||
ignoredSchedulersMap[scheduler] = true
|
||||
// GetBypassedSchedulersMap returns a map of scheduler names that should be bypassed as keys, and values are set to true
|
||||
// Also sets "" (empty string) to true if default scheduler is bypassed
|
||||
func GetBypassedSchedulersMap(bypassedSchedulers []string) map[string]bool {
|
||||
bypassedSchedulersMap := make(map[string]bool, len(bypassedSchedulers))
|
||||
for _, scheduler := range bypassedSchedulers {
|
||||
bypassedSchedulersMap[scheduler] = true
|
||||
}
|
||||
return ignoredSchedulersMap
|
||||
if canBypass := bypassedSchedulersMap[apiv1.DefaultSchedulerName]; canBypass {
|
||||
bypassedSchedulersMap[""] = true
|
||||
}
|
||||
return bypassedSchedulersMap
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue