Refactor NodeDeleteOptions for use in drainability rules
This commit is contained in:
		
							parent
							
								
									af638733e1
								
							
						
					
					
						commit
						a68b748fd7
					
				|  | @ -31,8 +31,9 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/expander" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/expander/factory" | ||||
| 	ca_processors "k8s.io/autoscaler/cluster-autoscaler/processors" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/backoff" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
|  | @ -57,7 +58,8 @@ type AutoscalerOptions struct { | |||
| 	DebuggingSnapshotter   debuggingsnapshot.DebuggingSnapshotter | ||||
| 	RemainingPdbTracker    pdb.RemainingPdbTracker | ||||
| 	ScaleUpOrchestrator    scaleup.Orchestrator | ||||
| 	DeleteOptions          simulator.NodeDeleteOptions | ||||
| 	DeleteOptions          options.NodeDeleteOptions | ||||
| 	DrainabilityRules      rules.Rules | ||||
| } | ||||
| 
 | ||||
| // Autoscaler is the main component of CA which scales up/down node groups according to its configuration
 | ||||
|  | @ -90,7 +92,9 @@ func NewAutoscaler(opts AutoscalerOptions) (Autoscaler, errors.AutoscalerError) | |||
| 		opts.DebuggingSnapshotter, | ||||
| 		opts.RemainingPdbTracker, | ||||
| 		opts.ScaleUpOrchestrator, | ||||
| 		opts.DeleteOptions), nil | ||||
| 		opts.DeleteOptions, | ||||
| 		opts.DrainabilityRules, | ||||
| 	), nil | ||||
| } | ||||
| 
 | ||||
| // Initialize default options if not provided.
 | ||||
|  |  | |||
|  | @ -35,6 +35,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/metrics" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/utilization" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
|  | @ -47,7 +49,8 @@ type Actuator struct { | |||
| 	clusterState          *clusterstate.ClusterStateRegistry | ||||
| 	nodeDeletionTracker   *deletiontracker.NodeDeletionTracker | ||||
| 	nodeDeletionScheduler *GroupDeletionScheduler | ||||
| 	deleteOptions         simulator.NodeDeleteOptions | ||||
| 	deleteOptions         options.NodeDeleteOptions | ||||
| 	drainabilityRules     rules.Rules | ||||
| 	// TODO: Move budget processor to scaledown planner, potentially merge into PostFilteringScaleDownNodeProcessor
 | ||||
| 	// This is a larger change to the code structure which impacts some existing actuator unit tests
 | ||||
| 	// as well as Cluster Autoscaler implementations that may override ScaleDownSetProcessor
 | ||||
|  | @ -64,15 +67,16 @@ type actuatorNodeGroupConfigGetter interface { | |||
| } | ||||
| 
 | ||||
| // NewActuator returns a new instance of Actuator.
 | ||||
| func NewActuator(ctx *context.AutoscalingContext, csr *clusterstate.ClusterStateRegistry, ndt *deletiontracker.NodeDeletionTracker, deleteOptions simulator.NodeDeleteOptions, configGetter actuatorNodeGroupConfigGetter) *Actuator { | ||||
| func NewActuator(ctx *context.AutoscalingContext, csr *clusterstate.ClusterStateRegistry, ndt *deletiontracker.NodeDeletionTracker, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules, configGetter actuatorNodeGroupConfigGetter) *Actuator { | ||||
| 	ndb := NewNodeDeletionBatcher(ctx, csr, ndt, ctx.NodeDeletionBatcherInterval) | ||||
| 	return &Actuator{ | ||||
| 		ctx:                       ctx, | ||||
| 		clusterState:              csr, | ||||
| 		nodeDeletionTracker:       ndt, | ||||
| 		nodeDeletionScheduler:     NewGroupDeletionScheduler(ctx, ndt, ndb, NewDefaultEvictor(deleteOptions, ndt)), | ||||
| 		nodeDeletionScheduler:     NewGroupDeletionScheduler(ctx, ndt, ndb, NewDefaultEvictor(deleteOptions, drainabilityRules, ndt)), | ||||
| 		budgetProcessor:           budgets.NewScaleDownBudgetProcessor(ctx), | ||||
| 		deleteOptions:             deleteOptions, | ||||
| 		drainabilityRules:         drainabilityRules, | ||||
| 		configGetter:              configGetter, | ||||
| 		nodeDeleteDelayAfterTaint: ctx.NodeDeleteDelayAfterTaint, | ||||
| 	} | ||||
|  | @ -273,7 +277,7 @@ func (a *Actuator) deleteNodesAsync(nodes []*apiv1.Node, nodeGroup cloudprovider | |||
| 			continue | ||||
| 		} | ||||
| 
 | ||||
| 		podsToRemove, _, _, err := simulator.GetPodsToMove(nodeInfo, a.deleteOptions, registry, remainingPdbTracker, time.Now()) | ||||
| 		podsToRemove, _, _, err := simulator.GetPodsToMove(nodeInfo, a.deleteOptions, a.drainabilityRules, registry, remainingPdbTracker, time.Now()) | ||||
| 		if err != nil { | ||||
| 			klog.Errorf("Scale-down: couldn't delete node %q, err: %v", node.Name, err) | ||||
| 			nodeDeleteResult := status.NodeDeleteResult{ResultType: status.NodeDeleteErrorInternal, Err: errors.NewAutoscalerError(errors.InternalError, "GetPodsToMove for %q returned error: %v", node.Name, err)} | ||||
|  |  | |||
|  | @ -32,6 +32,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/status" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/metrics" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/daemonset" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 	pod_util "k8s.io/autoscaler/cluster-autoscaler/utils/pod" | ||||
|  | @ -61,11 +63,12 @@ type Evictor struct { | |||
| 	DsEvictionEmptyNodeTimeout time.Duration | ||||
| 	PodEvictionHeadroom        time.Duration | ||||
| 	evictionRegister           evictionRegister | ||||
| 	deleteOptions              simulator.NodeDeleteOptions | ||||
| 	deleteOptions              options.NodeDeleteOptions | ||||
| 	drainabilityRules          rules.Rules | ||||
| } | ||||
| 
 | ||||
| // NewDefaultEvictor returns an instance of Evictor using the default parameters.
 | ||||
| func NewDefaultEvictor(deleteOptions simulator.NodeDeleteOptions, evictionRegister evictionRegister) Evictor { | ||||
| func NewDefaultEvictor(deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules, evictionRegister evictionRegister) Evictor { | ||||
| 	return Evictor{ | ||||
| 		EvictionRetryTime:          DefaultEvictionRetryTime, | ||||
| 		DsEvictionRetryTime:        DefaultDsEvictionRetryTime, | ||||
|  | @ -73,6 +76,7 @@ func NewDefaultEvictor(deleteOptions simulator.NodeDeleteOptions, evictionRegist | |||
| 		PodEvictionHeadroom:        DefaultPodEvictionHeadroom, | ||||
| 		evictionRegister:           evictionRegister, | ||||
| 		deleteOptions:              deleteOptions, | ||||
| 		drainabilityRules:          drainabilityRules, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
|  | @ -176,7 +180,7 @@ func (e Evictor) DrainNodeWithPods(ctx *acontext.AutoscalingContext, node *apiv1 | |||
| // EvictDaemonSetPods creates eviction objects for all DaemonSet pods on the node.
 | ||||
| func (e Evictor) EvictDaemonSetPods(ctx *acontext.AutoscalingContext, nodeInfo *framework.NodeInfo, timeNow time.Time) error { | ||||
| 	nodeToDelete := nodeInfo.Node() | ||||
| 	_, daemonSetPods, _, err := simulator.GetPodsToMove(nodeInfo, e.deleteOptions, nil, nil, timeNow) | ||||
| 	_, daemonSetPods, _, err := simulator.GetPodsToMove(nodeInfo, e.deleteOptions, e.drainabilityRules, nil, nil, timeNow) | ||||
| 	if err != nil { | ||||
| 		return fmt.Errorf("failed to get DaemonSet pods for %s (error: %v)", nodeToDelete.Name, err) | ||||
| 	} | ||||
|  |  | |||
|  | @ -32,6 +32,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/metrics" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/utilization" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 
 | ||||
|  | @ -55,9 +57,9 @@ type ScaleDown struct { | |||
| } | ||||
| 
 | ||||
| // NewScaleDown builds new ScaleDown object.
 | ||||
| func NewScaleDown(context *context.AutoscalingContext, processors *processors.AutoscalingProcessors, ndt *deletiontracker.NodeDeletionTracker, deleteOptions simulator.NodeDeleteOptions) *ScaleDown { | ||||
| func NewScaleDown(context *context.AutoscalingContext, processors *processors.AutoscalingProcessors, ndt *deletiontracker.NodeDeletionTracker, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules) *ScaleDown { | ||||
| 	usageTracker := simulator.NewUsageTracker() | ||||
| 	removalSimulator := simulator.NewRemovalSimulator(context.ListerRegistry, context.ClusterSnapshot, context.PredicateChecker, usageTracker, deleteOptions, false) | ||||
| 	removalSimulator := simulator.NewRemovalSimulator(context.ListerRegistry, context.ClusterSnapshot, context.PredicateChecker, usageTracker, deleteOptions, drainabilityRules, false) | ||||
| 	unremovableNodes := unremovable.NewNodes() | ||||
| 	resourceLimitsFinder := resource.NewLimitsFinder(processors.CustomResourcesProcessor) | ||||
| 	return &ScaleDown{ | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroupconfig" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	autoscaler_errors "k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 
 | ||||
| 	appsv1 "k8s.io/api/apps/v1" | ||||
|  | @ -1287,14 +1288,13 @@ func newWrapperForTesting(ctx *context.AutoscalingContext, clusterStateRegistry | |||
| 	if ndt == nil { | ||||
| 		ndt = deletiontracker.NewNodeDeletionTracker(0 * time.Second) | ||||
| 	} | ||||
| 	deleteOptions := simulator.NodeDeleteOptions{ | ||||
| 	deleteOptions := options.NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           true, | ||||
| 		SkipNodesWithLocalStorage:         true, | ||||
| 		MinReplicaCount:                   0, | ||||
| 		SkipNodesWithCustomControllerPods: true, | ||||
| 	} | ||||
| 	processors := NewTestProcessors(ctx) | ||||
| 	sd := NewScaleDown(ctx, processors, ndt, deleteOptions) | ||||
| 	actuator := actuation.NewActuator(ctx, clusterStateRegistry, ndt, deleteOptions, processors.NodeGroupConfigProcessor) | ||||
| 	sd := NewScaleDown(ctx, processors, ndt, deleteOptions, nil) | ||||
| 	actuator := actuation.NewActuator(ctx, clusterStateRegistry, ndt, deleteOptions, nil, processors.NodeGroupConfigProcessor) | ||||
| 	return NewScaleDownWrapper(sd, actuator) | ||||
| } | ||||
|  |  | |||
|  | @ -34,6 +34,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/nodes" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/scheduling" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/utilization" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
|  | @ -77,7 +79,7 @@ type Planner struct { | |||
| } | ||||
| 
 | ||||
| // New creates a new Planner object.
 | ||||
| func New(context *context.AutoscalingContext, processors *processors.AutoscalingProcessors, deleteOptions simulator.NodeDeleteOptions) *Planner { | ||||
| func New(context *context.AutoscalingContext, processors *processors.AutoscalingProcessors, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules) *Planner { | ||||
| 	resourceLimitsFinder := resource.NewLimitsFinder(processors.CustomResourcesProcessor) | ||||
| 	minUpdateInterval := context.AutoscalingOptions.NodeGroupDefaults.ScaleDownUnneededTime | ||||
| 	if minUpdateInterval == 0*time.Nanosecond { | ||||
|  | @ -87,7 +89,7 @@ func New(context *context.AutoscalingContext, processors *processors.Autoscaling | |||
| 		context:               context, | ||||
| 		unremovableNodes:      unremovable.NewNodes(), | ||||
| 		unneededNodes:         unneeded.NewNodes(processors.NodeGroupConfigProcessor, resourceLimitsFinder), | ||||
| 		rs:                    simulator.NewRemovalSimulator(context.ListerRegistry, context.ClusterSnapshot, context.PredicateChecker, simulator.NewUsageTracker(), deleteOptions, true), | ||||
| 		rs:                    simulator.NewRemovalSimulator(context.ListerRegistry, context.ClusterSnapshot, context.PredicateChecker, simulator.NewUsageTracker(), deleteOptions, drainabilityRules, true), | ||||
| 		actuationInjector:     scheduling.NewHintingSimulator(context.PredicateChecker), | ||||
| 		eligibilityChecker:    eligibility.NewChecker(processors.NodeGroupConfigProcessor), | ||||
| 		nodeUtilizationMap:    make(map[string]utilization.Info), | ||||
|  |  | |||
|  | @ -37,6 +37,7 @@ import ( | |||
| 	. "k8s.io/autoscaler/cluster-autoscaler/core/test" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/utilization" | ||||
| 	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/taints" | ||||
|  | @ -492,8 +493,8 @@ func TestUpdateClusterState(t *testing.T) { | |||
| 			}, &fake.Clientset{}, registry, provider, nil, nil) | ||||
| 			assert.NoError(t, err) | ||||
| 			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, tc.nodes, tc.pods) | ||||
| 			deleteOptions := simulator.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions) | ||||
| 			deleteOptions := options.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions, nil) | ||||
| 			p.eligibilityChecker = &fakeEligibilityChecker{eligible: asMap(tc.eligible)} | ||||
| 			if tc.isSimulationTimeout { | ||||
| 				context.AutoscalingOptions.ScaleDownSimulationTimeout = 1 * time.Second | ||||
|  | @ -611,8 +612,8 @@ func TestUpdateClusterStatUnneededNodesLimit(t *testing.T) { | |||
| 			}, &fake.Clientset{}, nil, provider, nil, nil) | ||||
| 			assert.NoError(t, err) | ||||
| 			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, nil) | ||||
| 			deleteOptions := simulator.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions) | ||||
| 			deleteOptions := options.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions, nil) | ||||
| 			p.eligibilityChecker = &fakeEligibilityChecker{eligible: asMap(nodeNames(nodes))} | ||||
| 			p.minUpdateInterval = tc.updateInterval | ||||
| 			p.unneededNodes.Update(previouslyUnneeded, time.Now()) | ||||
|  | @ -779,8 +780,8 @@ func TestNodesToDelete(t *testing.T) { | |||
| 			}, &fake.Clientset{}, nil, provider, nil, nil) | ||||
| 			assert.NoError(t, err) | ||||
| 			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, nil) | ||||
| 			deleteOptions := simulator.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions) | ||||
| 			deleteOptions := options.NodeDeleteOptions{} | ||||
| 			p := New(&context, NewTestProcessors(&context), deleteOptions, nil) | ||||
| 			p.latestUpdate = time.Now() | ||||
| 			p.actuationStatus = deletiontracker.NewNodeDeletionTracker(0 * time.Second) | ||||
| 			p.unneededNodes.Update(allRemovables, time.Now().Add(-1*time.Hour)) | ||||
|  |  | |||
|  | @ -51,6 +51,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/status" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/backoff" | ||||
| 	caerrors "k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
|  | @ -142,7 +144,8 @@ func NewStaticAutoscaler( | |||
| 	debuggingSnapshotter debuggingsnapshot.DebuggingSnapshotter, | ||||
| 	remainingPdbTracker pdb.RemainingPdbTracker, | ||||
| 	scaleUpOrchestrator scaleup.Orchestrator, | ||||
| 	deleteOptions simulator.NodeDeleteOptions) *StaticAutoscaler { | ||||
| 	deleteOptions options.NodeDeleteOptions, | ||||
| 	drainabilityRules rules.Rules) *StaticAutoscaler { | ||||
| 
 | ||||
| 	clusterStateConfig := clusterstate.ClusterStateRegistryConfig{ | ||||
| 		MaxTotalUnreadyPercentage: opts.MaxTotalUnreadyPercentage, | ||||
|  | @ -169,14 +172,14 @@ func NewStaticAutoscaler( | |||
| 	// TODO: Populate the ScaleDownActuator/Planner fields in AutoscalingContext
 | ||||
| 	// during the struct creation rather than here.
 | ||||
| 	ndt := deletiontracker.NewNodeDeletionTracker(0 * time.Second) | ||||
| 	scaleDown := legacy.NewScaleDown(autoscalingContext, processors, ndt, deleteOptions) | ||||
| 	actuator := actuation.NewActuator(autoscalingContext, clusterStateRegistry, ndt, deleteOptions, processors.NodeGroupConfigProcessor) | ||||
| 	scaleDown := legacy.NewScaleDown(autoscalingContext, processors, ndt, deleteOptions, drainabilityRules) | ||||
| 	actuator := actuation.NewActuator(autoscalingContext, clusterStateRegistry, ndt, deleteOptions, drainabilityRules, processors.NodeGroupConfigProcessor) | ||||
| 	autoscalingContext.ScaleDownActuator = actuator | ||||
| 
 | ||||
| 	var scaleDownPlanner scaledown.Planner | ||||
| 	var scaleDownActuator scaledown.Actuator | ||||
| 	if opts.ParallelDrain { | ||||
| 		scaleDownPlanner = planner.New(autoscalingContext, processors, deleteOptions) | ||||
| 		scaleDownPlanner = planner.New(autoscalingContext, processors, deleteOptions, drainabilityRules) | ||||
| 		scaleDownActuator = actuator | ||||
| 	} else { | ||||
| 		// TODO: Remove the wrapper once the legacy implementation becomes obsolete.
 | ||||
|  |  | |||
|  | @ -45,6 +45,8 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroupconfig" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/utilization" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
|  | @ -149,9 +151,9 @@ func (m *onNodeGroupDeleteMock) Delete(id string) error { | |||
| 	return args.Error(0) | ||||
| } | ||||
| 
 | ||||
| func setUpScaleDownActuator(ctx *context.AutoscalingContext, options config.AutoscalingOptions) { | ||||
| 	deleteOptions := simulator.NewNodeDeleteOptions(options) | ||||
| 	ctx.ScaleDownActuator = actuation.NewActuator(ctx, nil, deletiontracker.NewNodeDeletionTracker(0*time.Second), deleteOptions, NewTestProcessors(ctx).NodeGroupConfigProcessor) | ||||
| func setUpScaleDownActuator(ctx *context.AutoscalingContext, autoscalingOptions config.AutoscalingOptions) { | ||||
| 	deleteOptions := options.NewNodeDeleteOptions(autoscalingOptions) | ||||
| 	ctx.ScaleDownActuator = actuation.NewActuator(ctx, nil, deletiontracker.NewNodeDeletionTracker(0*time.Second), deleteOptions, rules.Default(), NewTestProcessors(ctx).NodeGroupConfigProcessor) | ||||
| } | ||||
| 
 | ||||
| func TestStaticAutoscalerRunOnce(t *testing.T) { | ||||
|  | @ -1447,11 +1449,11 @@ func TestStaticAutoscalerUpcomingScaleDownCandidates(t *testing.T) { | |||
| 		kubernetes.NewTestPodLister(nil), | ||||
| 		kubernetes.NewTestPodDisruptionBudgetLister(nil), daemonSetLister, nil, nil, nil, nil) | ||||
| 
 | ||||
| 	// Create context with minimal options that guarantee we reach the tested logic.
 | ||||
| 	// We're only testing the input to UpdateClusterState which should be called whenever scale-down is enabled, other options shouldn't matter.
 | ||||
| 	options := config.AutoscalingOptions{ScaleDownEnabled: true} | ||||
| 	// Create context with minimal autoscalingOptions that guarantee we reach the tested logic.
 | ||||
| 	// We're only testing the input to UpdateClusterState which should be called whenever scale-down is enabled, other autoscalingOptions shouldn't matter.
 | ||||
| 	autoscalingOptions := config.AutoscalingOptions{ScaleDownEnabled: true} | ||||
| 	processorCallbacks := newStaticAutoscalerProcessorCallbacks() | ||||
| 	ctx, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, listerRegistry, provider, processorCallbacks, nil) | ||||
| 	ctx, err := NewScaleTestAutoscalingContext(autoscalingOptions, &fake.Clientset{}, listerRegistry, provider, processorCallbacks, nil) | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	// Create CSR with unhealthy cluster protection effectively disabled, to guarantee we reach the tested logic.
 | ||||
|  | @ -1459,7 +1461,7 @@ func TestStaticAutoscalerUpcomingScaleDownCandidates(t *testing.T) { | |||
| 	csr := clusterstate.NewClusterStateRegistry(provider, csrConfig, ctx.LogRecorder, NewBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(config.NodeGroupAutoscalingOptions{MaxNodeProvisionTime: 15 * time.Minute})) | ||||
| 
 | ||||
| 	// Setting the Actuator is necessary for testing any scale-down logic, it shouldn't have anything to do in this test.
 | ||||
| 	actuator := actuation.NewActuator(&ctx, csr, deletiontracker.NewNodeDeletionTracker(0*time.Second), simulator.NodeDeleteOptions{}, NewTestProcessors(&ctx).NodeGroupConfigProcessor) | ||||
| 	actuator := actuation.NewActuator(&ctx, csr, deletiontracker.NewNodeDeletionTracker(0*time.Second), options.NodeDeleteOptions{}, nil, NewTestProcessors(&ctx).NodeGroupConfigProcessor) | ||||
| 	ctx.ScaleDownActuator = actuator | ||||
| 
 | ||||
| 	// Fake planner that keeps track of the scale-down candidates passed to UpdateClusterState.
 | ||||
|  | @ -1847,15 +1849,14 @@ func newScaleDownPlannerAndActuator(t *testing.T, ctx *context.AutoscalingContex | |||
| 	ctx.MaxDrainParallelism = 1 | ||||
| 	ctx.NodeDeletionBatcherInterval = 0 * time.Second | ||||
| 	ctx.NodeDeleteDelayAfterTaint = 1 * time.Second | ||||
| 	deleteOptions := simulator.NodeDeleteOptions{ | ||||
| 	deleteOptions := options.NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           true, | ||||
| 		SkipNodesWithLocalStorage:         true, | ||||
| 		MinReplicaCount:                   0, | ||||
| 		SkipNodesWithCustomControllerPods: true, | ||||
| 	} | ||||
| 	ndt := deletiontracker.NewNodeDeletionTracker(0 * time.Second) | ||||
| 	sd := legacy.NewScaleDown(ctx, p, ndt, deleteOptions) | ||||
| 	actuator := actuation.NewActuator(ctx, cs, ndt, deleteOptions, p.NodeGroupConfigProcessor) | ||||
| 	sd := legacy.NewScaleDown(ctx, p, ndt, deleteOptions, nil) | ||||
| 	actuator := actuation.NewActuator(ctx, cs, ndt, deleteOptions, nil, p.NodeGroupConfigProcessor) | ||||
| 	wrapper := legacy.NewScaleDownWrapper(sd, actuator) | ||||
| 	return wrapper, wrapper | ||||
| } | ||||
|  |  | |||
|  | @ -29,10 +29,6 @@ import ( | |||
| 	"syscall" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/debuggingsnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 
 | ||||
| 	"github.com/spf13/pflag" | ||||
| 
 | ||||
| 	"k8s.io/apimachinery/pkg/api/meta" | ||||
|  | @ -45,6 +41,7 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/config" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/core" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/podlistprocessor" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/debuggingsnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/estimator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/expander" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/metrics" | ||||
|  | @ -55,6 +52,9 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/scaledowncandidates/emptycandidates" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/processors/scaledowncandidates/previouscandidates" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| 	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
| 	scheduler_util "k8s.io/autoscaler/cluster-autoscaler/utils/scheduler" | ||||
|  | @ -68,7 +68,7 @@ import ( | |||
| 	"k8s.io/client-go/tools/leaderelection/resourcelock" | ||||
| 	kube_flag "k8s.io/component-base/cli/flag" | ||||
| 	componentbaseconfig "k8s.io/component-base/config" | ||||
| 	"k8s.io/component-base/config/options" | ||||
| 	componentopts "k8s.io/component-base/config/options" | ||||
| 	"k8s.io/component-base/logs" | ||||
| 	logsapi "k8s.io/component-base/logs/api/v1" | ||||
| 	_ "k8s.io/component-base/logs/json/register" | ||||
|  | @ -461,7 +461,7 @@ func buildAutoscaler(debuggingSnapshotter debuggingsnapshot.DebuggingSnapshotter | |||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| 	} | ||||
| 	deleteOptions := simulator.NewNodeDeleteOptions(autoscalingOptions) | ||||
| 	deleteOptions := options.NewNodeDeleteOptions(autoscalingOptions) | ||||
| 
 | ||||
| 	opts := core.AutoscalerOptions{ | ||||
| 		AutoscalingOptions:   autoscalingOptions, | ||||
|  | @ -481,7 +481,7 @@ func buildAutoscaler(debuggingSnapshotter debuggingsnapshot.DebuggingSnapshotter | |||
| 	if autoscalingOptions.ParallelDrain { | ||||
| 		sdCandidatesSorting := previouscandidates.NewPreviousCandidates() | ||||
| 		scaleDownCandidatesComparers = []scaledowncandidates.CandidatesComparer{ | ||||
| 			emptycandidates.NewEmptySortingProcessor(emptycandidates.NewNodeInfoGetter(opts.ClusterSnapshot), deleteOptions), | ||||
| 			emptycandidates.NewEmptySortingProcessor(emptycandidates.NewNodeInfoGetter(opts.ClusterSnapshot), deleteOptions, rules.Default()), | ||||
| 			sdCandidatesSorting, | ||||
| 		} | ||||
| 		opts.Processors.ScaleDownCandidatesNotifier.Register(sdCandidatesSorting) | ||||
|  | @ -575,7 +575,7 @@ func main() { | |||
| 
 | ||||
| 	leaderElection := defaultLeaderElectionConfiguration() | ||||
| 	leaderElection.LeaderElect = true | ||||
| 	options.BindLeaderElectionFlags(&leaderElection, pflag.CommandLine) | ||||
| 	componentopts.BindLeaderElectionFlags(&leaderElection, pflag.CommandLine) | ||||
| 
 | ||||
| 	featureGate := utilfeature.DefaultMutableFeatureGate | ||||
| 	loggingConfig := logsapi.NewLoggingConfiguration() | ||||
|  |  | |||
|  | @ -22,6 +22,8 @@ import ( | |||
| 	apiv1 "k8s.io/api/core/v1" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework" | ||||
| ) | ||||
| 
 | ||||
|  | @ -45,12 +47,17 @@ func NewNodeInfoGetter(c clustersnapshot.ClusterSnapshot) *nodeInfoGetterImpl { | |||
| // EmptySorting is sorting scale down candidates so that empty nodes appear first.
 | ||||
| type EmptySorting struct { | ||||
| 	nodeInfoGetter | ||||
| 	deleteOptions simulator.NodeDeleteOptions | ||||
| 	deleteOptions     options.NodeDeleteOptions | ||||
| 	drainabilityRules rules.Rules | ||||
| } | ||||
| 
 | ||||
| // NewEmptySortingProcessor return EmptySorting struct.
 | ||||
| func NewEmptySortingProcessor(n nodeInfoGetter, deleteOptions simulator.NodeDeleteOptions) *EmptySorting { | ||||
| 	return &EmptySorting{n, deleteOptions} | ||||
| func NewEmptySortingProcessor(n nodeInfoGetter, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules) *EmptySorting { | ||||
| 	return &EmptySorting{ | ||||
| 		nodeInfoGetter:    n, | ||||
| 		deleteOptions:     deleteOptions, | ||||
| 		drainabilityRules: drainabilityRules, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // ScaleDownEarlierThan return true if node1 is empty and node2 isn't.
 | ||||
|  | @ -66,7 +73,7 @@ func (p *EmptySorting) isNodeEmpty(node *apiv1.Node) bool { | |||
| 	if err != nil { | ||||
| 		return false | ||||
| 	} | ||||
| 	podsToRemove, _, _, err := simulator.GetPodsToMove(nodeInfo, p.deleteOptions, nil, nil, time.Now()) | ||||
| 	podsToRemove, _, _, err := simulator.GetPodsToMove(nodeInfo, p.deleteOptions, p.drainabilityRules, nil, nil, time.Now()) | ||||
| 	if err == nil && len(podsToRemove) == 0 { | ||||
| 		return true | ||||
| 	} | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import ( | |||
| 	"testing" | ||||
| 
 | ||||
| 	v1 "k8s.io/api/core/v1" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	. "k8s.io/autoscaler/cluster-autoscaler/utils/test" | ||||
| 	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework" | ||||
| ) | ||||
|  | @ -62,13 +62,15 @@ func TestScaleDownEarlierThan(t *testing.T) { | |||
| 
 | ||||
| 	niGetter := testNodeInfoGetter{map[string]*schedulerframework.NodeInfo{nodeEmptyName: niEmpty, nodeNonEmptyName: niNonEmpty, nodeEmptyName2: niEmpty2}} | ||||
| 
 | ||||
| 	deleteOptions := simulator.NodeDeleteOptions{ | ||||
| 	deleteOptions := options.NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           true, | ||||
| 		SkipNodesWithLocalStorage:         true, | ||||
| 		MinReplicaCount:                   0, | ||||
| 		SkipNodesWithCustomControllerPods: true, | ||||
| 	} | ||||
| 	p := EmptySorting{&niGetter, deleteOptions} | ||||
| 	p := EmptySorting{ | ||||
| 		nodeInfoGetter: &niGetter, | ||||
| 		deleteOptions:  deleteOptions, | ||||
| 	} | ||||
| 
 | ||||
| 	tests := []struct { | ||||
| 		name        string | ||||
|  | @ -98,19 +100,16 @@ func TestScaleDownEarlierThan(t *testing.T) { | |||
| 			name:  "Non-empty node is not earlier that node without nodeInfo", | ||||
| 			node1: nodeNonEmpty, | ||||
| 			node2: noNodeInfoNode, | ||||
| 			wantEarlier: false, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name:  "Node without nodeInfo is not earlier that non-empty node", | ||||
| 			node1: noNodeInfoNode, | ||||
| 			node2: nodeNonEmpty, | ||||
| 			wantEarlier: false, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name:  "Empty node is not earlier that another empty node", | ||||
| 			node1: nodeEmpty, | ||||
| 			node2: nodeEmpty2, | ||||
| 			wantEarlier: false, | ||||
| 		}, | ||||
| 	} | ||||
| 	for _, test := range tests { | ||||
|  |  | |||
|  | @ -22,6 +22,8 @@ import ( | |||
| 
 | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/pdb" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/scheduling" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/drain" | ||||
|  | @ -95,19 +97,21 @@ type RemovalSimulator struct { | |||
| 	clusterSnapshot     clustersnapshot.ClusterSnapshot | ||||
| 	usageTracker        *UsageTracker | ||||
| 	canPersist          bool | ||||
| 	deleteOptions       NodeDeleteOptions | ||||
| 	deleteOptions       options.NodeDeleteOptions | ||||
| 	drainabilityRules   rules.Rules | ||||
| 	schedulingSimulator *scheduling.HintingSimulator | ||||
| } | ||||
| 
 | ||||
| // NewRemovalSimulator returns a new RemovalSimulator.
 | ||||
| func NewRemovalSimulator(listers kube_util.ListerRegistry, clusterSnapshot clustersnapshot.ClusterSnapshot, predicateChecker predicatechecker.PredicateChecker, | ||||
| 	usageTracker *UsageTracker, deleteOptions NodeDeleteOptions, persistSuccessfulSimulations bool) *RemovalSimulator { | ||||
| 	usageTracker *UsageTracker, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules, persistSuccessfulSimulations bool) *RemovalSimulator { | ||||
| 	return &RemovalSimulator{ | ||||
| 		listers:             listers, | ||||
| 		clusterSnapshot:     clusterSnapshot, | ||||
| 		usageTracker:        usageTracker, | ||||
| 		canPersist:          persistSuccessfulSimulations, | ||||
| 		deleteOptions:       deleteOptions, | ||||
| 		drainabilityRules:   drainabilityRules, | ||||
| 		schedulingSimulator: scheduling.NewHintingSimulator(predicateChecker), | ||||
| 	} | ||||
| } | ||||
|  | @ -159,7 +163,7 @@ func (r *RemovalSimulator) SimulateNodeRemoval( | |||
| 		return nil, &UnremovableNode{Node: nodeInfo.Node(), Reason: UnexpectedError} | ||||
| 	} | ||||
| 
 | ||||
| 	podsToRemove, daemonSetPods, blockingPod, err := GetPodsToMove(nodeInfo, r.deleteOptions, r.listers, remainingPdbTracker, timestamp) | ||||
| 	podsToRemove, daemonSetPods, blockingPod, err := GetPodsToMove(nodeInfo, r.deleteOptions, r.drainabilityRules, r.listers, remainingPdbTracker, timestamp) | ||||
| 	if err != nil { | ||||
| 		klog.V(2).Infof("node %s cannot be removed: %v", nodeName, err) | ||||
| 		if blockingPod != nil { | ||||
|  | @ -193,7 +197,7 @@ func (r *RemovalSimulator) FindEmptyNodesToRemove(candidates []string, timestamp | |||
| 			continue | ||||
| 		} | ||||
| 		// Should block on all pods
 | ||||
| 		podsToRemove, _, _, err := GetPodsToMove(nodeInfo, r.deleteOptions, nil, nil, timestamp) | ||||
| 		podsToRemove, _, _, err := GetPodsToMove(nodeInfo, r.deleteOptions, r.drainabilityRules, nil, nil, timestamp) | ||||
| 		if err == nil && len(podsToRemove) == 0 { | ||||
| 			result = append(result, node) | ||||
| 		} | ||||
|  |  | |||
|  | @ -22,6 +22,7 @@ import ( | |||
| 	"time" | ||||
| 
 | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/drain" | ||||
| 	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
|  | @ -58,7 +59,7 @@ func TestFindEmptyNodes(t *testing.T) { | |||
| 	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot() | ||||
| 	clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot, []*apiv1.Node{nodes[0], nodes[1], nodes[2], nodes[3]}, []*apiv1.Pod{pod1, pod2}) | ||||
| 	testTime := time.Date(2020, time.December, 18, 17, 0, 0, 0, time.UTC) | ||||
| 	r := NewRemovalSimulator(nil, clusterSnapshot, nil, nil, testDeleteOptions(), false) | ||||
| 	r := NewRemovalSimulator(nil, clusterSnapshot, nil, nil, testDeleteOptions(), nil, false) | ||||
| 	emptyNodes := r.FindEmptyNodesToRemove(nodeNames, testTime) | ||||
| 	assert.Equal(t, []string{nodeNames[0], nodeNames[2], nodeNames[3]}, emptyNodes) | ||||
| } | ||||
|  | @ -205,7 +206,7 @@ func TestFindNodesToRemove(t *testing.T) { | |||
| 				destinations = append(destinations, node.Name) | ||||
| 			} | ||||
| 			clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot, test.allNodes, test.pods) | ||||
| 			r := NewRemovalSimulator(registry, clusterSnapshot, predicateChecker, tracker, testDeleteOptions(), false) | ||||
| 			r := NewRemovalSimulator(registry, clusterSnapshot, predicateChecker, tracker, testDeleteOptions(), nil, false) | ||||
| 			toRemove, unremovable := r.FindNodesToRemove(test.candidates, destinations, time.Now(), nil) | ||||
| 			fmt.Printf("Test scenario: %s, found len(toRemove)=%v, expected len(test.toRemove)=%v\n", test.name, len(toRemove), len(test.toRemove)) | ||||
| 			assert.Equal(t, toRemove, test.toRemove) | ||||
|  | @ -214,11 +215,10 @@ func TestFindNodesToRemove(t *testing.T) { | |||
| 	} | ||||
| } | ||||
| 
 | ||||
| func testDeleteOptions() NodeDeleteOptions { | ||||
| 	return NodeDeleteOptions{ | ||||
| func testDeleteOptions() options.NodeDeleteOptions { | ||||
| 	return options.NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           true, | ||||
| 		SkipNodesWithLocalStorage:         true, | ||||
| 		MinReplicaCount:                   0, | ||||
| 		SkipNodesWithCustomControllerPods: true, | ||||
| 	} | ||||
| } | ||||
|  |  | |||
|  | @ -24,42 +24,16 @@ import ( | |||
| 	policyv1 "k8s.io/api/policy/v1" | ||||
| 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | ||||
| 	"k8s.io/apimachinery/pkg/labels" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/config" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/pdb" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/drain" | ||||
| 	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes" | ||||
| 	pod_util "k8s.io/autoscaler/cluster-autoscaler/utils/pod" | ||||
| 	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework" | ||||
| ) | ||||
| 
 | ||||
| // NodeDeleteOptions contains various options to customize how draining will behave
 | ||||
| type NodeDeleteOptions struct { | ||||
| 	// SkipNodesWithSystemPods tells if nodes with pods from kube-system should be deleted (except for DaemonSet or mirror pods)
 | ||||
| 	SkipNodesWithSystemPods bool | ||||
| 	// SkipNodesWithLocalStorage tells if nodes with pods with local storage, e.g. EmptyDir or HostPath, should be deleted
 | ||||
| 	SkipNodesWithLocalStorage bool | ||||
| 	// SkipNodesWithCustomControllerPods tells if nodes with custom-controller owned pods should be skipped from deletion (skip if 'true')
 | ||||
| 	SkipNodesWithCustomControllerPods bool | ||||
| 	// MinReplicaCount controls the minimum number of replicas that a replica set or replication controller should have
 | ||||
| 	// to allow their pods deletion in scale down
 | ||||
| 	MinReplicaCount int | ||||
| 	// DrainabilityRules contain a list of checks that are used to verify whether a pod can be drained from node.
 | ||||
| 	DrainabilityRules rules.Rules | ||||
| } | ||||
| 
 | ||||
| // NewNodeDeleteOptions returns new node delete options extracted from autoscaling options
 | ||||
| func NewNodeDeleteOptions(opts config.AutoscalingOptions) NodeDeleteOptions { | ||||
| 	return NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           opts.SkipNodesWithSystemPods, | ||||
| 		SkipNodesWithLocalStorage:         opts.SkipNodesWithLocalStorage, | ||||
| 		MinReplicaCount:                   opts.MinReplicaCount, | ||||
| 		SkipNodesWithCustomControllerPods: opts.SkipNodesWithCustomControllerPods, | ||||
| 		DrainabilityRules:                 rules.Default(), | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // GetPodsToMove returns a list of pods that should be moved elsewhere
 | ||||
| // and a list of DaemonSet pods that should be evicted if the node
 | ||||
| // is drained. Raises error if there is an unreplicated pod.
 | ||||
|  | @ -68,10 +42,8 @@ func NewNodeDeleteOptions(opts config.AutoscalingOptions) NodeDeleteOptions { | |||
| // If listers is not nil it checks whether RC, DS, Jobs and RS that created these pods
 | ||||
| // still exist.
 | ||||
| // TODO(x13n): Rewrite GetPodsForDeletionOnNodeDrain into a set of DrainabilityRules.
 | ||||
| func GetPodsToMove(nodeInfo *schedulerframework.NodeInfo, deleteOptions NodeDeleteOptions, listers kube_util.ListerRegistry, | ||||
| 	remainingPdbTracker pdb.RemainingPdbTracker, timestamp time.Time) (pods []*apiv1.Pod, daemonSetPods []*apiv1.Pod, blockingPod *drain.BlockingPod, err error) { | ||||
| func GetPodsToMove(nodeInfo *schedulerframework.NodeInfo, deleteOptions options.NodeDeleteOptions, drainabilityRules rules.Rules, listers kube_util.ListerRegistry, remainingPdbTracker pdb.RemainingPdbTracker, timestamp time.Time) (pods []*apiv1.Pod, daemonSetPods []*apiv1.Pod, blockingPod *drain.BlockingPod, err error) { | ||||
| 	var drainPods, drainDs []*apiv1.Pod | ||||
| 	drainabilityRules := deleteOptions.DrainabilityRules | ||||
| 	if drainabilityRules == nil { | ||||
| 		drainabilityRules = rules.Default() | ||||
| 	} | ||||
|  | @ -80,6 +52,7 @@ func GetPodsToMove(nodeInfo *schedulerframework.NodeInfo, deleteOptions NodeDele | |||
| 	} | ||||
| 	drainCtx := &drainability.DrainContext{ | ||||
| 		RemainingPdbTracker: remainingPdbTracker, | ||||
| 		DeleteOptions:       deleteOptions, | ||||
| 	} | ||||
| 	for _, podInfo := range nodeInfo.Pods { | ||||
| 		pod := podInfo.Pod | ||||
|  |  | |||
|  | @ -28,6 +28,7 @@ import ( | |||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/pdb" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/drainability/rules" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/drain" | ||||
| 	. "k8s.io/autoscaler/cluster-autoscaler/utils/test" | ||||
| 	"k8s.io/kubernetes/pkg/kubelet/types" | ||||
|  | @ -306,16 +307,14 @@ func TestGetPodsToMove(t *testing.T) { | |||
| 	} | ||||
| 	for _, tc := range testCases { | ||||
| 		t.Run(tc.desc, func(t *testing.T) { | ||||
| 			deleteOptions := NodeDeleteOptions{ | ||||
| 			deleteOptions := options.NodeDeleteOptions{ | ||||
| 				SkipNodesWithSystemPods:           true, | ||||
| 				SkipNodesWithLocalStorage:         true, | ||||
| 				MinReplicaCount:                   0, | ||||
| 				SkipNodesWithCustomControllerPods: true, | ||||
| 				DrainabilityRules:                 tc.rules, | ||||
| 			} | ||||
| 			tracker := pdb.NewBasicRemainingPdbTracker() | ||||
| 			tracker.SetPdbs(tc.pdbs) | ||||
| 			p, d, b, err := GetPodsToMove(schedulerframework.NewNodeInfo(tc.pods...), deleteOptions, nil, tracker, testTime) | ||||
| 			p, d, b, err := GetPodsToMove(schedulerframework.NewNodeInfo(tc.pods...), deleteOptions, tc.rules, nil, tracker, testTime) | ||||
| 			if tc.wantErr { | ||||
| 				assert.Error(t, err) | ||||
| 			} else { | ||||
|  |  | |||
|  | @ -18,9 +18,11 @@ package drainability | |||
| 
 | ||||
| import ( | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/pdb" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/simulator/options" | ||||
| ) | ||||
| 
 | ||||
| // DrainContext contains parameters for drainability rules.
 | ||||
| type DrainContext struct { | ||||
| 	RemainingPdbTracker pdb.RemainingPdbTracker | ||||
| 	DeleteOptions       options.NodeDeleteOptions | ||||
| } | ||||
|  |  | |||
|  | @ -0,0 +1,48 @@ | |||
| /* | ||||
| 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 options | ||||
| 
 | ||||
| import ( | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/config" | ||||
| ) | ||||
| 
 | ||||
| // NodeDeleteOptions contains various options to customize how draining will behave
 | ||||
| type NodeDeleteOptions struct { | ||||
| 	// SkipNodesWithSystemPods is true if nodes with kube-system pods should be
 | ||||
| 	// deleted (except for DaemonSet or mirror pods).
 | ||||
| 	SkipNodesWithSystemPods bool | ||||
| 	// SkipNodesWithLocalStorage is true if nodes with pods using local storage
 | ||||
| 	// (e.g. EmptyDir or HostPath) should be deleted.
 | ||||
| 	SkipNodesWithLocalStorage bool | ||||
| 	// SkipNodesWithCustomControllerPods is true if nodes with
 | ||||
| 	// custom-controller-owned pods should be skipped.
 | ||||
| 	SkipNodesWithCustomControllerPods bool | ||||
| 	// MinReplicaCount determines the minimum number of replicas that a replica
 | ||||
| 	// set or replication controller should have to allow pod deletion during
 | ||||
| 	// scale down.
 | ||||
| 	MinReplicaCount int | ||||
| } | ||||
| 
 | ||||
| // NewNodeDeleteOptions returns new node delete options extracted from autoscaling options.
 | ||||
| func NewNodeDeleteOptions(opts config.AutoscalingOptions) NodeDeleteOptions { | ||||
| 	return NodeDeleteOptions{ | ||||
| 		SkipNodesWithSystemPods:           opts.SkipNodesWithSystemPods, | ||||
| 		SkipNodesWithLocalStorage:         opts.SkipNodesWithLocalStorage, | ||||
| 		MinReplicaCount:                   opts.MinReplicaCount, | ||||
| 		SkipNodesWithCustomControllerPods: opts.SkipNodesWithCustomControllerPods, | ||||
| 	} | ||||
| } | ||||
		Loading…
	
		Reference in New Issue