Split out code from simulator package
This commit is contained in:
		
							parent
							
								
									95fd1ed645
								
							
						
					
					
						commit
						18f2e67c4f
					
				| 
						 | 
				
			
			@ -24,7 +24,8 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/estimator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/expander"
 | 
			
		||||
	processor_callbacks "k8s.io/autoscaler/cluster-autoscaler/processors/callbacks"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	kube_client "k8s.io/client-go/kubernetes"
 | 
			
		||||
	kube_record "k8s.io/client-go/tools/record"
 | 
			
		||||
| 
						 | 
				
			
			@ -42,9 +43,9 @@ type AutoscalingContext struct {
 | 
			
		|||
	CloudProvider cloudprovider.CloudProvider
 | 
			
		||||
	// TODO(kgolab) - move away too as it's not config
 | 
			
		||||
	// PredicateChecker to check if a pod can fit into a node.
 | 
			
		||||
	PredicateChecker simulator.PredicateChecker
 | 
			
		||||
	PredicateChecker predicatechecker.PredicateChecker
 | 
			
		||||
	// ClusterSnapshot denotes cluster snapshot used for predicate checking.
 | 
			
		||||
	ClusterSnapshot simulator.ClusterSnapshot
 | 
			
		||||
	ClusterSnapshot clustersnapshot.ClusterSnapshot
 | 
			
		||||
	// ExpanderStrategy is the strategy used to choose which node group to expand when scaling up
 | 
			
		||||
	ExpanderStrategy expander.Strategy
 | 
			
		||||
	// EstimatorBuilder is the builder function for node count estimator to be used.
 | 
			
		||||
| 
						 | 
				
			
			@ -90,8 +91,8 @@ func NewResourceLimiterFromAutoscalingOptions(options config.AutoscalingOptions)
 | 
			
		|||
// NewAutoscalingContext returns an autoscaling context from all the necessary parameters passed via arguments
 | 
			
		||||
func NewAutoscalingContext(
 | 
			
		||||
	options config.AutoscalingOptions,
 | 
			
		||||
	predicateChecker simulator.PredicateChecker,
 | 
			
		||||
	clusterSnapshot simulator.ClusterSnapshot,
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker,
 | 
			
		||||
	clusterSnapshot clustersnapshot.ClusterSnapshot,
 | 
			
		||||
	autoscalingKubeClients *AutoscalingKubeClients,
 | 
			
		||||
	cloudProvider cloudprovider.CloudProvider,
 | 
			
		||||
	expanderStrategy expander.Strategy,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -29,7 +29,8 @@ 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/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/backoff"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	kube_client "k8s.io/client-go/kubernetes"
 | 
			
		||||
| 
						 | 
				
			
			@ -42,8 +43,8 @@ type AutoscalerOptions struct {
 | 
			
		|||
	EventsKubeClient       kube_client.Interface
 | 
			
		||||
	AutoscalingKubeClients *context.AutoscalingKubeClients
 | 
			
		||||
	CloudProvider          cloudprovider.CloudProvider
 | 
			
		||||
	PredicateChecker       simulator.PredicateChecker
 | 
			
		||||
	ClusterSnapshot        simulator.ClusterSnapshot
 | 
			
		||||
	PredicateChecker       predicatechecker.PredicateChecker
 | 
			
		||||
	ClusterSnapshot        clustersnapshot.ClusterSnapshot
 | 
			
		||||
	ExpanderStrategy       expander.Strategy
 | 
			
		||||
	EstimatorBuilder       estimator.EstimatorBuilder
 | 
			
		||||
	Processors             *ca_processors.AutoscalingProcessors
 | 
			
		||||
| 
						 | 
				
			
			@ -91,14 +92,14 @@ func initializeDefaultOptions(opts *AutoscalerOptions) error {
 | 
			
		|||
	}
 | 
			
		||||
	if opts.PredicateChecker == nil {
 | 
			
		||||
		predicateCheckerStopChannel := make(chan struct{})
 | 
			
		||||
		predicateChecker, err := simulator.NewSchedulerBasedPredicateChecker(opts.KubeClient, predicateCheckerStopChannel)
 | 
			
		||||
		predicateChecker, err := predicatechecker.NewSchedulerBasedPredicateChecker(opts.KubeClient, predicateCheckerStopChannel)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
		opts.PredicateChecker = predicateChecker
 | 
			
		||||
	}
 | 
			
		||||
	if opts.ClusterSnapshot == nil {
 | 
			
		||||
		opts.ClusterSnapshot = simulator.NewBasicClusterSnapshot()
 | 
			
		||||
		opts.ClusterSnapshot = clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	}
 | 
			
		||||
	if opts.CloudProvider == nil {
 | 
			
		||||
		opts.CloudProvider = cloudBuilder.NewCloudProvider(opts.AutoscalingOptions)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -24,7 +24,8 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/context"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/core/utils"
 | 
			
		||||
	"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/predicatechecker"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	corev1helpers "k8s.io/component-helpers/scheduling/corev1"
 | 
			
		||||
| 
						 | 
				
			
			@ -98,8 +99,8 @@ func (p *filterOutSchedulablePodListProcessor) CleanUp() {
 | 
			
		|||
// and will be scheduled after lower priority pod preemption.
 | 
			
		||||
func (p *filterOutSchedulablePodListProcessor) filterOutSchedulableByPacking(
 | 
			
		||||
	unschedulableCandidates []*apiv1.Pod,
 | 
			
		||||
	clusterSnapshot simulator.ClusterSnapshot,
 | 
			
		||||
	predicateChecker simulator.PredicateChecker) ([]*apiv1.Pod, error) {
 | 
			
		||||
	clusterSnapshot clustersnapshot.ClusterSnapshot,
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker) ([]*apiv1.Pod, error) {
 | 
			
		||||
	unschedulablePodsCache := utils.NewPodSchedulableMap()
 | 
			
		||||
 | 
			
		||||
	// Sort unschedulable pods by importance
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,7 +21,8 @@ import (
 | 
			
		|||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
| 
						 | 
				
			
			@ -100,8 +101,8 @@ func TestFilterOutSchedulableByPacking(t *testing.T) {
 | 
			
		|||
	}
 | 
			
		||||
	for _, tt := range tests {
 | 
			
		||||
		t.Run(tt.name, func(t *testing.T) {
 | 
			
		||||
			predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
			clusterSnapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
			predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
			clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
 | 
			
		||||
			for _, node := range tt.nodes {
 | 
			
		||||
				err := clusterSnapshot.AddNode(node)
 | 
			
		||||
| 
						 | 
				
			
			@ -215,9 +216,9 @@ func BenchmarkFilterOutSchedulableByPacking(b *testing.B) {
 | 
			
		|||
			pendingPods:   12000,
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	snapshots := map[string]func() simulator.ClusterSnapshot{
 | 
			
		||||
		"basic": func() simulator.ClusterSnapshot { return simulator.NewBasicClusterSnapshot() },
 | 
			
		||||
		"delta": func() simulator.ClusterSnapshot { return simulator.NewDeltaClusterSnapshot() },
 | 
			
		||||
	snapshots := map[string]func() clustersnapshot.ClusterSnapshot{
 | 
			
		||||
		"basic": func() clustersnapshot.ClusterSnapshot { return clustersnapshot.NewBasicClusterSnapshot() },
 | 
			
		||||
		"delta": func() clustersnapshot.ClusterSnapshot { return clustersnapshot.NewDeltaClusterSnapshot() },
 | 
			
		||||
	}
 | 
			
		||||
	for snapshotName, snapshotFactory := range snapshots {
 | 
			
		||||
		for _, tc := range tests {
 | 
			
		||||
| 
						 | 
				
			
			@ -242,7 +243,7 @@ func BenchmarkFilterOutSchedulableByPacking(b *testing.B) {
 | 
			
		|||
					}
 | 
			
		||||
				}
 | 
			
		||||
 | 
			
		||||
				predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
				predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
				assert.NoError(b, err)
 | 
			
		||||
 | 
			
		||||
				clusterSnapshot := snapshotFactory()
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -34,6 +34,7 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/metrics"
 | 
			
		||||
	"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/utilization"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/deletetaint"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
| 
						 | 
				
			
			@ -367,9 +368,9 @@ func (a *Actuator) scheduleDeletion(nodeInfo *framework.NodeInfo, nodeGroupId st
 | 
			
		|||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (a *Actuator) createSnapshot(nodes []*apiv1.Node) (simulator.ClusterSnapshot, error) {
 | 
			
		||||
func (a *Actuator) createSnapshot(nodes []*apiv1.Node) (clustersnapshot.ClusterSnapshot, error) {
 | 
			
		||||
	knownNodes := make(map[string]bool)
 | 
			
		||||
	snapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
	snapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
 | 
			
		||||
	scheduledPods, err := a.ctx.ScheduledPodLister().List()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -40,7 +40,7 @@ import (
 | 
			
		|||
	acontext "k8s.io/autoscaler/cluster-autoscaler/context"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/core/test"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/core/utils"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/daemonset"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
| 
						 | 
				
			
			@ -157,7 +157,7 @@ func TestDaemonSetEvictionForEmptyNodes(t *testing.T) {
 | 
			
		|||
			context, err := NewScaleTestAutoscalingContext(options, fakeClient, registry, provider, nil, nil)
 | 
			
		||||
			assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
			simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, []*apiv1.Node{n1}, dsPods)
 | 
			
		||||
			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, []*apiv1.Node{n1}, dsPods)
 | 
			
		||||
 | 
			
		||||
			evictor := Evictor{
 | 
			
		||||
				DsEvictionEmptyNodeTimeout: scenario.dsEvictionTimeout,
 | 
			
		||||
| 
						 | 
				
			
			@ -572,7 +572,7 @@ func TestPodsToEvict(t *testing.T) {
 | 
			
		|||
		},
 | 
			
		||||
	} {
 | 
			
		||||
		t.Run(tn, func(t *testing.T) {
 | 
			
		||||
			snapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
			snapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
			node := BuildTestNode("test-node", 1000, 1000)
 | 
			
		||||
			err := snapshot.AddNodeWithPods(node, tc.pods)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -27,7 +27,7 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/context"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/unremovable"
 | 
			
		||||
	. "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/utils/deletetaint"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -104,7 +104,7 @@ func TestFilterOutUnremovable(t *testing.T) {
 | 
			
		|||
				provider.AddNode("ng1", n)
 | 
			
		||||
			}
 | 
			
		||||
			context, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, nil, provider, nil, nil)
 | 
			
		||||
			simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, tc.nodes, tc.pods)
 | 
			
		||||
			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, tc.nodes, tc.pods)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				t.Fatalf("Could not create autoscaling context: %v", err)
 | 
			
		||||
			}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -23,6 +23,7 @@ import (
 | 
			
		|||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	autoscaler_errors "k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
 | 
			
		||||
	appsv1 "k8s.io/api/apps/v1"
 | 
			
		||||
| 
						 | 
				
			
			@ -150,7 +151,7 @@ func TestFindUnneededNodes(t *testing.T) {
 | 
			
		|||
	sd := wrapper.sd
 | 
			
		||||
	allNodes := []*apiv1.Node{n1, n2, n3, n4, n5, n7, n8, n9}
 | 
			
		||||
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4, p5, p6})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4, p5, p6})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -171,7 +172,7 @@ func TestFindUnneededNodes(t *testing.T) {
 | 
			
		|||
	sd.unremovableNodes = unremovable.NewNodes()
 | 
			
		||||
	sd.unneededNodes.Update([]simulator.NodeToBeRemoved{{Node: n1}, {Node: n2}, {Node: n3}, {Node: n4}}, time.Now())
 | 
			
		||||
	allNodes = []*apiv1.Node{n1, n2, n3, n4}
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -188,7 +189,7 @@ func TestFindUnneededNodes(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	sd.unremovableNodes = unremovable.NewNodes()
 | 
			
		||||
	scaleDownCandidates := []*apiv1.Node{n1, n3, n4}
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, scaleDownCandidates, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -196,7 +197,7 @@ func TestFindUnneededNodes(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	// Node n1 is unneeded, but should be skipped because it has just recently been found to be unremovable
 | 
			
		||||
	allNodes = []*apiv1.Node{n1}
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -205,7 +206,7 @@ func TestFindUnneededNodes(t *testing.T) {
 | 
			
		|||
	assert.Equal(t, 1, len(sd.unremovableNodes.AsList()))
 | 
			
		||||
 | 
			
		||||
	// But it should be checked after timeout
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now().Add(context.UnremovableNodeRecheckTimeout+time.Second), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -282,7 +283,7 @@ func TestFindUnneededGPUNodes(t *testing.T) {
 | 
			
		|||
	sd := wrapper.sd
 | 
			
		||||
	allNodes := []*apiv1.Node{n1, n2, n3}
 | 
			
		||||
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3})
 | 
			
		||||
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
| 
						 | 
				
			
			@ -397,7 +398,7 @@ func TestFindUnneededWithPerNodeGroupThresholds(t *testing.T) {
 | 
			
		|||
			clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
			wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
			sd := wrapper.sd
 | 
			
		||||
			simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, allPods)
 | 
			
		||||
			clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, allPods)
 | 
			
		||||
 | 
			
		||||
			ng1 := provider.GetNodeGroup("n1").(*testprovider.TestNodeGroup)
 | 
			
		||||
			ng1.SetOptions(tc.n1opts)
 | 
			
		||||
| 
						 | 
				
			
			@ -482,7 +483,7 @@ func TestPodsWithPreemptionsFindUnneededNodes(t *testing.T) {
 | 
			
		|||
	sd := wrapper.sd
 | 
			
		||||
 | 
			
		||||
	allNodes := []*apiv1.Node{n1, n2, n3, n4}
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(allNodes, allNodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	assert.Equal(t, 2, len(sd.unneededNodes.AsList()))
 | 
			
		||||
| 
						 | 
				
			
			@ -547,7 +548,7 @@ func TestFindUnneededMaxCandidates(t *testing.T) {
 | 
			
		|||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	sd := wrapper.sd
 | 
			
		||||
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	assert.Equal(t, numCandidates, len(sd.unneededNodes.AsList()))
 | 
			
		||||
| 
						 | 
				
			
			@ -570,7 +571,7 @@ func TestFindUnneededMaxCandidates(t *testing.T) {
 | 
			
		|||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	// Check that the deleted node was replaced
 | 
			
		||||
| 
						 | 
				
			
			@ -631,7 +632,7 @@ func TestFindUnneededEmptyNodes(t *testing.T) {
 | 
			
		|||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	sd := wrapper.sd
 | 
			
		||||
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	assert.Equal(t, numEmpty+numCandidates, len(sd.unneededNodes.AsList()))
 | 
			
		||||
| 
						 | 
				
			
			@ -687,7 +688,7 @@ func TestFindUnneededNodePool(t *testing.T) {
 | 
			
		|||
	clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	sd := wrapper.sd
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
 | 
			
		||||
	autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	assert.NotEmpty(t, sd.unneededNodes)
 | 
			
		||||
| 
						 | 
				
			
			@ -777,7 +778,7 @@ func TestScaleDown(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
 | 
			
		||||
	autoscalererr = wrapper.UpdateClusterState(nodes, nodes, nil, nil, time.Now().Add(-5*time.Minute))
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	empty, drain := wrapper.NodesToDelete(time.Now())
 | 
			
		||||
| 
						 | 
				
			
			@ -1034,7 +1035,7 @@ func simpleScaleDownEmpty(t *testing.T, config *ScaleTestConfig) {
 | 
			
		|||
 | 
			
		||||
	clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, config.NodeDeletionTracker)
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{})
 | 
			
		||||
	autoscalererr = wrapper.UpdateClusterState(nodes, nodes, nil, nil, time.Now().Add(-5*time.Minute))
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	empty, drain := wrapper.NodesToDelete(time.Now())
 | 
			
		||||
| 
						 | 
				
			
			@ -1128,7 +1129,7 @@ func TestNoScaleDownUnready(t *testing.T) {
 | 
			
		|||
	// N1 is unready so it requires a bigger unneeded time.
 | 
			
		||||
	clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
 | 
			
		||||
	autoscalererr = wrapper.UpdateClusterState(nodes, nodes, nil, nil, time.Now().Add(-5*time.Minute))
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	empty, drain := wrapper.NodesToDelete(time.Now())
 | 
			
		||||
| 
						 | 
				
			
			@ -1152,7 +1153,7 @@ func TestNoScaleDownUnready(t *testing.T) {
 | 
			
		|||
	// N1 has been unready for 2 hours, ok to delete.
 | 
			
		||||
	context.CloudProvider = provider
 | 
			
		||||
	wrapper = newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
 | 
			
		||||
	autoscalererr = wrapper.UpdateClusterState(nodes, nodes, nil, nil, time.Now().Add(-2*time.Hour))
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	empty, drain = wrapper.NodesToDelete(time.Now())
 | 
			
		||||
| 
						 | 
				
			
			@ -1242,7 +1243,7 @@ func TestScaleDownNoMove(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, NewBackoff())
 | 
			
		||||
	wrapper := newWrapperForTesting(&context, clusterStateRegistry, nil)
 | 
			
		||||
	simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
 | 
			
		||||
	autoscalererr = wrapper.UpdateClusterState(nodes, nodes, nil, nil, time.Now().Add(-5*time.Minute))
 | 
			
		||||
	assert.NoError(t, autoscalererr)
 | 
			
		||||
	empty, drain := wrapper.NodesToDelete(time.Now())
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -42,6 +42,8 @@ import (
 | 
			
		|||
	ca_processors "k8s.io/autoscaler/cluster-autoscaler/processors"
 | 
			
		||||
	"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/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/backoff"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/deletetaint"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
| 
						 | 
				
			
			@ -121,8 +123,8 @@ func (callbacks *staticAutoscalerProcessorCallbacks) reset() {
 | 
			
		|||
// NewStaticAutoscaler creates an instance of Autoscaler filled with provided parameters
 | 
			
		||||
func NewStaticAutoscaler(
 | 
			
		||||
	opts config.AutoscalingOptions,
 | 
			
		||||
	predicateChecker simulator.PredicateChecker,
 | 
			
		||||
	clusterSnapshot simulator.ClusterSnapshot,
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker,
 | 
			
		||||
	clusterSnapshot clustersnapshot.ClusterSnapshot,
 | 
			
		||||
	autoscalingKubeClients *context.AutoscalingKubeClients,
 | 
			
		||||
	processors *ca_processors.AutoscalingProcessors,
 | 
			
		||||
	cloudProvider cloudprovider.CloudProvider,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -46,7 +46,8 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/processors/nodeinfosprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/processors/nodes"
 | 
			
		||||
	"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/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/labels"
 | 
			
		||||
| 
						 | 
				
			
			@ -166,14 +167,14 @@ func NewScaleTestAutoscalingContext(
 | 
			
		|||
	// Ignoring error here is safe - if a test doesn't specify valid estimatorName,
 | 
			
		||||
	// it either doesn't need one, or should fail when it turns out to be nil.
 | 
			
		||||
	estimatorBuilder, _ := estimator.NewEstimatorBuilder(options.EstimatorName, estimator.NewThresholdBasedEstimationLimiter(0, 0))
 | 
			
		||||
	predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return context.AutoscalingContext{}, err
 | 
			
		||||
	}
 | 
			
		||||
	if debuggingSnapshotter == nil {
 | 
			
		||||
		debuggingSnapshotter = debuggingsnapshot.NewDebuggingSnapshotter(false)
 | 
			
		||||
	}
 | 
			
		||||
	clusterSnapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	return context.AutoscalingContext{
 | 
			
		||||
		AutoscalingOptions: options,
 | 
			
		||||
		AutoscalingKubeClients: context.AutoscalingKubeClients{
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,7 +21,7 @@ import (
 | 
			
		|||
	"reflect"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	pod_utils "k8s.io/autoscaler/cluster-autoscaler/utils/pod"
 | 
			
		||||
)
 | 
			
		||||
| 
						 | 
				
			
			@ -43,7 +43,7 @@ import (
 | 
			
		|||
type PodSchedulableInfo struct {
 | 
			
		||||
	spec            apiv1.PodSpec
 | 
			
		||||
	labels          map[string]string
 | 
			
		||||
	schedulingError *simulator.PredicateError
 | 
			
		||||
	schedulingError *predicatechecker.PredicateError
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
const maxPodsPerOwnerRef = 10
 | 
			
		||||
| 
						 | 
				
			
			@ -68,7 +68,7 @@ func (psi *PodSchedulableInfo) Match(pod *apiv1.Pod) bool {
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
// Get returns scheduling info for given pod if matching one exists in PodSchedulableMap
 | 
			
		||||
func (p PodSchedulableMap) Get(pod *apiv1.Pod) (*simulator.PredicateError, bool) {
 | 
			
		||||
func (p PodSchedulableMap) Get(pod *apiv1.Pod) (*predicatechecker.PredicateError, bool) {
 | 
			
		||||
	ref := drain.ControllerRef(pod)
 | 
			
		||||
	if ref == nil {
 | 
			
		||||
		return nil, false
 | 
			
		||||
| 
						 | 
				
			
			@ -85,7 +85,7 @@ func (p PodSchedulableMap) Get(pod *apiv1.Pod) (*simulator.PredicateError, bool)
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
// Set sets scheduling info for given pod in PodSchedulableMap
 | 
			
		||||
func (p PodSchedulableMap) Set(pod *apiv1.Pod, err *simulator.PredicateError) {
 | 
			
		||||
func (p PodSchedulableMap) Set(pod *apiv1.Pod, err *predicatechecker.PredicateError) {
 | 
			
		||||
	ref := drain.ControllerRef(pod)
 | 
			
		||||
	if ref == nil || pod_utils.IsDaemonSetPod(pod) {
 | 
			
		||||
		return
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -18,7 +18,7 @@ package utils
 | 
			
		|||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
| 
						 | 
				
			
			@ -65,7 +65,7 @@ func TestPodSchedulableMap(t *testing.T) {
 | 
			
		|||
	assert.True(t, found)
 | 
			
		||||
	assert.Nil(t, err)
 | 
			
		||||
 | 
			
		||||
	cpuErr := simulator.GenericPredicateError()
 | 
			
		||||
	cpuErr := predicatechecker.GenericPredicateError()
 | 
			
		||||
 | 
			
		||||
	// Pod in different RC
 | 
			
		||||
	_, found = pMap.Get(podInRc2)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -27,7 +27,7 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/clusterstate"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/metrics"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/daemonset"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
| 
						 | 
				
			
			@ -37,7 +37,7 @@ import (
 | 
			
		|||
)
 | 
			
		||||
 | 
			
		||||
// GetNodeInfoFromTemplate returns NodeInfo object built base on TemplateNodeInfo returned by NodeGroup.TemplateNodeInfo().
 | 
			
		||||
func GetNodeInfoFromTemplate(nodeGroup cloudprovider.NodeGroup, daemonsets []*appsv1.DaemonSet, predicateChecker simulator.PredicateChecker, ignoredTaints taints.TaintKeySet) (*schedulerframework.NodeInfo, errors.AutoscalerError) {
 | 
			
		||||
func GetNodeInfoFromTemplate(nodeGroup cloudprovider.NodeGroup, daemonsets []*appsv1.DaemonSet, predicateChecker predicatechecker.PredicateChecker, ignoredTaints taints.TaintKeySet) (*schedulerframework.NodeInfo, errors.AutoscalerError) {
 | 
			
		||||
	id := nodeGroup.Id()
 | 
			
		||||
	baseNodeInfo, err := nodeGroup.TemplateNodeInfo()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -23,7 +23,8 @@ import (
 | 
			
		|||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/resource"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/scheduler"
 | 
			
		||||
	klog "k8s.io/klog/v2"
 | 
			
		||||
	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
 | 
			
		||||
| 
						 | 
				
			
			@ -37,15 +38,15 @@ type podInfo struct {
 | 
			
		|||
 | 
			
		||||
// BinpackingNodeEstimator estimates the number of needed nodes to handle the given amount of pods.
 | 
			
		||||
type BinpackingNodeEstimator struct {
 | 
			
		||||
	predicateChecker simulator.PredicateChecker
 | 
			
		||||
	clusterSnapshot  simulator.ClusterSnapshot
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker
 | 
			
		||||
	clusterSnapshot  clustersnapshot.ClusterSnapshot
 | 
			
		||||
	limiter          EstimationLimiter
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// NewBinpackingNodeEstimator builds a new BinpackingNodeEstimator.
 | 
			
		||||
func NewBinpackingNodeEstimator(
 | 
			
		||||
	predicateChecker simulator.PredicateChecker,
 | 
			
		||||
	clusterSnapshot simulator.ClusterSnapshot,
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker,
 | 
			
		||||
	clusterSnapshot clustersnapshot.ClusterSnapshot,
 | 
			
		||||
	limiter EstimationLimiter) *BinpackingNodeEstimator {
 | 
			
		||||
	return &BinpackingNodeEstimator{
 | 
			
		||||
		predicateChecker: predicateChecker,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -23,7 +23,8 @@ import (
 | 
			
		|||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/resource"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/units"
 | 
			
		||||
	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
 | 
			
		||||
| 
						 | 
				
			
			@ -166,11 +167,11 @@ func TestBinpackingEstimate(t *testing.T) {
 | 
			
		|||
	}
 | 
			
		||||
	for _, tc := range testCases {
 | 
			
		||||
		t.Run(tc.name, func(t *testing.T) {
 | 
			
		||||
			clusterSnapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
			clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
			// Add one node in different zone to trigger topology spread constraints
 | 
			
		||||
			clusterSnapshot.AddNode(makeNode(100, 100, "oldnode", "zone-jupiter"))
 | 
			
		||||
 | 
			
		||||
			predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
			predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
			assert.NoError(t, err)
 | 
			
		||||
			limiter := NewThresholdBasedEstimationLimiter(tc.maxNodes, time.Duration(0))
 | 
			
		||||
			estimator := NewBinpackingNodeEstimator(predicateChecker, clusterSnapshot, limiter)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,7 +21,8 @@ import (
 | 
			
		|||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -41,15 +42,15 @@ type Estimator interface {
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
// EstimatorBuilder creates a new estimator object.
 | 
			
		||||
type EstimatorBuilder func(simulator.PredicateChecker, simulator.ClusterSnapshot) Estimator
 | 
			
		||||
type EstimatorBuilder func(predicatechecker.PredicateChecker, clustersnapshot.ClusterSnapshot) Estimator
 | 
			
		||||
 | 
			
		||||
// NewEstimatorBuilder creates a new estimator object from flag.
 | 
			
		||||
func NewEstimatorBuilder(name string, limiter EstimationLimiter) (EstimatorBuilder, error) {
 | 
			
		||||
	switch name {
 | 
			
		||||
	case BinpackingEstimatorName:
 | 
			
		||||
		return func(
 | 
			
		||||
			predicateChecker simulator.PredicateChecker,
 | 
			
		||||
			clusterSnapshot simulator.ClusterSnapshot) Estimator {
 | 
			
		||||
			predicateChecker predicatechecker.PredicateChecker,
 | 
			
		||||
			clusterSnapshot clustersnapshot.ClusterSnapshot) Estimator {
 | 
			
		||||
			return NewBinpackingNodeEstimator(predicateChecker, clusterSnapshot, limiter)
 | 
			
		||||
		}, nil
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -48,7 +48,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/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/units"
 | 
			
		||||
| 
						 | 
				
			
			@ -358,7 +358,7 @@ func buildAutoscaler(debuggingSnapshotter debuggingsnapshot.DebuggingSnapshotter
 | 
			
		|||
 | 
			
		||||
	opts := core.AutoscalerOptions{
 | 
			
		||||
		AutoscalingOptions:   autoscalingOptions,
 | 
			
		||||
		ClusterSnapshot:      simulator.NewDeltaClusterSnapshot(),
 | 
			
		||||
		ClusterSnapshot:      clustersnapshot.NewDeltaClusterSnapshot(),
 | 
			
		||||
		KubeClient:           kubeClient,
 | 
			
		||||
		EventsKubeClient:     eventsKubeClient,
 | 
			
		||||
		DebuggingSnapshotter: debuggingSnapshotter,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -22,7 +22,7 @@ import (
 | 
			
		|||
 | 
			
		||||
	testprovider "k8s.io/autoscaler/cluster-autoscaler/cloudprovider/test"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/context"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -75,7 +75,7 @@ func TestGetNodeInfosForGroups(t *testing.T) {
 | 
			
		|||
	podLister := kube_util.NewTestPodLister([]*apiv1.Pod{})
 | 
			
		||||
	registry := kube_util.NewListerRegistry(nil, nil, podLister, nil, nil, nil, nil, nil, nil, nil)
 | 
			
		||||
 | 
			
		||||
	predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
	ctx := context.AutoscalingContext{
 | 
			
		||||
| 
						 | 
				
			
			@ -160,7 +160,7 @@ func TestGetNodeInfosForGroupsCache(t *testing.T) {
 | 
			
		|||
	podLister := kube_util.NewTestPodLister([]*apiv1.Pod{})
 | 
			
		||||
	registry := kube_util.NewListerRegistry(nil, nil, podLister, nil, nil, nil, nil, nil, nil, nil)
 | 
			
		||||
 | 
			
		||||
	predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
	// Fill cache
 | 
			
		||||
| 
						 | 
				
			
			@ -249,7 +249,7 @@ func TestGetNodeInfosCacheExpired(t *testing.T) {
 | 
			
		|||
	provider := testprovider.NewTestAutoprovisioningCloudProvider(nil, nil, nil, nil, nil, nil)
 | 
			
		||||
	podLister := kube_util.NewTestPodLister([]*apiv1.Pod{})
 | 
			
		||||
	registry := kube_util.NewListerRegistry(nil, nil, podLister, nil, nil, nil, nil, nil, nil, nil)
 | 
			
		||||
	predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
	ctx := context.AutoscalingContext{
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -20,6 +20,8 @@ import (
 | 
			
		|||
	"fmt"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
| 
						 | 
				
			
			@ -86,15 +88,15 @@ const (
 | 
			
		|||
// RemovalSimulator is a helper object for simulating node removal scenarios.
 | 
			
		||||
type RemovalSimulator struct {
 | 
			
		||||
	listers          kube_util.ListerRegistry
 | 
			
		||||
	clusterSnapshot  ClusterSnapshot
 | 
			
		||||
	predicateChecker PredicateChecker
 | 
			
		||||
	clusterSnapshot  clustersnapshot.ClusterSnapshot
 | 
			
		||||
	predicateChecker predicatechecker.PredicateChecker
 | 
			
		||||
	usageTracker     *UsageTracker
 | 
			
		||||
	canPersist       bool
 | 
			
		||||
	deleteOptions    NodeDeleteOptions
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// NewRemovalSimulator returns a new RemovalSimulator.
 | 
			
		||||
func NewRemovalSimulator(listers kube_util.ListerRegistry, clusterSnapshot ClusterSnapshot, predicateChecker PredicateChecker,
 | 
			
		||||
func NewRemovalSimulator(listers kube_util.ListerRegistry, clusterSnapshot clustersnapshot.ClusterSnapshot, predicateChecker predicatechecker.PredicateChecker,
 | 
			
		||||
	usageTracker *UsageTracker, deleteOptions NodeDeleteOptions, persistSuccessfulSimulations bool) *RemovalSimulator {
 | 
			
		||||
	return &RemovalSimulator{
 | 
			
		||||
		listers:          listers,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,17 +21,19 @@ import (
 | 
			
		|||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
	appsv1 "k8s.io/api/apps/v1"
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	policyv1 "k8s.io/api/policy/v1"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
	"k8s.io/kubernetes/pkg/kubelet/types"
 | 
			
		||||
	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func TestFindPlaceAllOk(t *testing.T) {
 | 
			
		||||
| 
						 | 
				
			
			@ -51,10 +53,10 @@ func TestFindPlaceAllOk(t *testing.T) {
 | 
			
		|||
		"n1": true,
 | 
			
		||||
		"n2": true,
 | 
			
		||||
	}
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
		[]*apiv1.Node{node1, node2},
 | 
			
		||||
		[]*apiv1.Pod{pod1})
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -89,10 +91,10 @@ func TestFindPlaceAllBas(t *testing.T) {
 | 
			
		|||
		"n1":   true,
 | 
			
		||||
		"n2":   true,
 | 
			
		||||
	}
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
		[]*apiv1.Node{node1, node2},
 | 
			
		||||
		[]*apiv1.Pod{pod1})
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -122,10 +124,10 @@ func TestFindNone(t *testing.T) {
 | 
			
		|||
		"n2": true,
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
	clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot,
 | 
			
		||||
		[]*apiv1.Node{node1, node2},
 | 
			
		||||
		[]*apiv1.Pod{pod1})
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -159,8 +161,8 @@ func TestFindEmptyNodes(t *testing.T) {
 | 
			
		|||
		types.ConfigMirrorAnnotationKey: "",
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	InitializeClusterSnapshotOrDie(t, clusterSnapshot, []*apiv1.Node{nodes[0], nodes[1], nodes[2], nodes[3]}, []*apiv1.Pod{pod1, pod2})
 | 
			
		||||
	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)
 | 
			
		||||
	emptyNodes := r.FindEmptyNodesToRemove(nodeNames, testTime)
 | 
			
		||||
| 
						 | 
				
			
			@ -249,8 +251,8 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
		DaemonSetPods:    []*apiv1.Pod{},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	tracker := NewUsageTracker()
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -308,7 +310,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			for _, node := range test.allNodes {
 | 
			
		||||
				destinations = append(destinations, node.Name)
 | 
			
		||||
			}
 | 
			
		||||
			InitializeClusterSnapshotOrDie(t, clusterSnapshot, test.allNodes, test.pods)
 | 
			
		||||
			clustersnapshot.InitializeClusterSnapshotOrDie(t, clusterSnapshot, test.allNodes, test.pods)
 | 
			
		||||
			r := NewRemovalSimulator(registry, clusterSnapshot, predicateChecker, tracker, testDeleteOptions(), false)
 | 
			
		||||
			toRemove, unremovable, _, err := r.FindNodesToRemove(
 | 
			
		||||
				test.candidates, destinations, map[string]string{},
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"errors"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,14 +14,13 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package clustersnapshot
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// InitializeClusterSnapshotOrDie clears cluster snapshot and then initializes it with given set of nodes and pods.
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
| 
						 | 
				
			
			@ -14,16 +14,18 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// PredicateChecker checks whether all required predicates pass for given Pod and Node.
 | 
			
		||||
type PredicateChecker interface {
 | 
			
		||||
	FitsAnyNode(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod) (string, error)
 | 
			
		||||
	FitsAnyNodeMatching(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod, nodeMatches func(*schedulerframework.NodeInfo) bool) (string, error)
 | 
			
		||||
	CheckPredicates(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod, nodeName string) *PredicateError
 | 
			
		||||
	FitsAnyNode(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod) (string, error)
 | 
			
		||||
	FitsAnyNodeMatching(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod, nodeMatches func(*schedulerframework.NodeInfo) bool) (string, error)
 | 
			
		||||
	CheckPredicates(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod, nodeName string) *PredicateError
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			@ -14,12 +14,14 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"context"
 | 
			
		||||
	"fmt"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/client-go/informers"
 | 
			
		||||
	kube_client "k8s.io/client-go/kubernetes"
 | 
			
		||||
| 
						 | 
				
			
			@ -78,14 +80,14 @@ func NewSchedulerBasedPredicateChecker(kubeClient kube_client.Interface, stop <-
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
// FitsAnyNode checks if the given pod can be placed on any of the given nodes.
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) FitsAnyNode(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod) (string, error) {
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) FitsAnyNode(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod) (string, error) {
 | 
			
		||||
	return p.FitsAnyNodeMatching(clusterSnapshot, pod, func(*schedulerframework.NodeInfo) bool {
 | 
			
		||||
		return true
 | 
			
		||||
	})
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// FitsAnyNodeMatching checks if the given pod can be placed on any of the given nodes matching the provided function.
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) FitsAnyNodeMatching(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod, nodeMatches func(*schedulerframework.NodeInfo) bool) (string, error) {
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) FitsAnyNodeMatching(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod, nodeMatches func(*schedulerframework.NodeInfo) bool) (string, error) {
 | 
			
		||||
	if clusterSnapshot == nil {
 | 
			
		||||
		return "", fmt.Errorf("ClusterSnapshot not provided")
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			@ -141,7 +143,7 @@ func (p *SchedulerBasedPredicateChecker) FitsAnyNodeMatching(clusterSnapshot Clu
 | 
			
		|||
}
 | 
			
		||||
 | 
			
		||||
// CheckPredicates checks if the given pod can be placed on the given node.
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) CheckPredicates(clusterSnapshot ClusterSnapshot, pod *apiv1.Pod, nodeName string) *PredicateError {
 | 
			
		||||
func (p *SchedulerBasedPredicateChecker) CheckPredicates(clusterSnapshot clustersnapshot.ClusterSnapshot, pod *apiv1.Pod, nodeName string) *PredicateError {
 | 
			
		||||
	if clusterSnapshot == nil {
 | 
			
		||||
		return NewPredicateError(InternalPredicateError, "", "ClusterSnapshot not provided", nil, emptyString)
 | 
			
		||||
	}
 | 
			
		||||
| 
						 | 
				
			
			@ -14,12 +14,13 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
| 
						 | 
				
			
			@ -76,7 +77,7 @@ func TestCheckPredicate(t *testing.T) {
 | 
			
		|||
		t.Run(tt.name, func(t *testing.T) {
 | 
			
		||||
			var err error
 | 
			
		||||
			predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
			clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
			clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
			err = clusterSnapshot.AddNodeWithPods(tt.node, tt.scheduledPods)
 | 
			
		||||
			assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -103,7 +104,7 @@ func TestFitsAnyNode(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	var err error
 | 
			
		||||
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
	err = clusterSnapshot.AddNode(n1000)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	err = clusterSnapshot.AddNode(n2000)
 | 
			
		||||
| 
						 | 
				
			
			@ -144,7 +145,7 @@ func TestDebugInfo(t *testing.T) {
 | 
			
		|||
	predicateChecker, err := NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
 | 
			
		||||
	clusterSnapshot := NewBasicClusterSnapshot()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
 | 
			
		||||
	err = clusterSnapshot.AddNode(node1)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
| 
						 | 
				
			
			@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
 | 
			
		|||
limitations under the License.
 | 
			
		||||
*/
 | 
			
		||||
 | 
			
		||||
package simulator
 | 
			
		||||
package predicatechecker
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	clientsetfake "k8s.io/client-go/kubernetes/fake"
 | 
			
		||||
| 
						 | 
				
			
			@ -20,7 +20,8 @@ import (
 | 
			
		|||
	"fmt"
 | 
			
		||||
	"math/rand"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/clustersnapshot"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
 | 
			
		||||
	appsv1 "k8s.io/api/apps/v1"
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
| 
						 | 
				
			
			@ -35,11 +36,11 @@ const (
 | 
			
		|||
)
 | 
			
		||||
 | 
			
		||||
// GetDaemonSetPodsForNode returns daemonset nodes for the given pod.
 | 
			
		||||
func GetDaemonSetPodsForNode(nodeInfo *schedulerframework.NodeInfo, daemonsets []*appsv1.DaemonSet, predicateChecker simulator.PredicateChecker) ([]*apiv1.Pod, error) {
 | 
			
		||||
func GetDaemonSetPodsForNode(nodeInfo *schedulerframework.NodeInfo, daemonsets []*appsv1.DaemonSet, predicateChecker predicatechecker.PredicateChecker) ([]*apiv1.Pod, error) {
 | 
			
		||||
	result := make([]*apiv1.Pod, 0)
 | 
			
		||||
 | 
			
		||||
	// here we can use empty snapshot
 | 
			
		||||
	clusterSnapshot := simulator.NewBasicClusterSnapshot()
 | 
			
		||||
	clusterSnapshot := clustersnapshot.NewBasicClusterSnapshot()
 | 
			
		||||
 | 
			
		||||
	// add a node with pods - node info is created by cloud provider,
 | 
			
		||||
	// we don't know whether it'll have pods or not.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,7 +21,7 @@ import (
 | 
			
		|||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator/predicatechecker"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
	appsv1 "k8s.io/api/apps/v1"
 | 
			
		||||
| 
						 | 
				
			
			@ -39,7 +39,7 @@ func TestGetDaemonSetPodsForNode(t *testing.T) {
 | 
			
		|||
	nodeInfo := schedulerframework.NewNodeInfo()
 | 
			
		||||
	nodeInfo.SetNode(node)
 | 
			
		||||
 | 
			
		||||
	predicateChecker, err := simulator.NewTestPredicateChecker()
 | 
			
		||||
	predicateChecker, err := predicatechecker.NewTestPredicateChecker()
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	ds1 := newDaemonSet("ds1", "0.1", "100M", nil)
 | 
			
		||||
	ds2 := newDaemonSet("ds2", "0.1", "100M", map[string]string{"foo": "bar"})
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue