Respect GPU limits in scale_down
This commit is contained in:
		
							parent
							
								
									0ca0b1bec3
								
							
						
					
					
						commit
						087a5cc9a9
					
				|  | @ -163,6 +163,13 @@ const ( | |||
| 	ResourceNameMemory = "memory" | ||||
| ) | ||||
| 
 | ||||
| // IsGpuResource checks if given resource name point denotes a gpu type
 | ||||
| func IsGpuResource(resourceName string) bool { | ||||
| 	// hack: we assume anything which is not cpu/memory to be a gpu.
 | ||||
| 	// we are not getting anything more that a map string->limits from the user
 | ||||
| 	return resourceName != ResourceNameCores && resourceName != ResourceNameMemory | ||||
| } | ||||
| 
 | ||||
| // ResourceLimiter contains limits (max, min) for resources (cores, memory etc.).
 | ||||
| type ResourceLimiter struct { | ||||
| 	minLimits map[string]int64 | ||||
|  |  | |||
|  | @ -51,6 +51,16 @@ type AutoscalingContext struct { | |||
| 	LogRecorder *utils.LogEventRecorder | ||||
| } | ||||
| 
 | ||||
| // GpuLimits define lower and upper bound on GPU instances of given type in cluster
 | ||||
| type GpuLimits struct { | ||||
| 	// Type of the GPU (e.g. nvidia-tesla-k80)
 | ||||
| 	GpuType string | ||||
| 	// Lower bound on number of GPUs of given type in cluster
 | ||||
| 	Min int64 | ||||
| 	// Upper bound on number of GPUs of given type in cluster
 | ||||
| 	Max int64 | ||||
| } | ||||
| 
 | ||||
| // AutoscalingOptions contain various options to customize how autoscaling works
 | ||||
| type AutoscalingOptions struct { | ||||
| 	// MaxEmptyBulkDelete is a number of empty nodes that can be removed at the same time.
 | ||||
|  | @ -73,6 +83,8 @@ type AutoscalingOptions struct { | |||
| 	MaxMemoryTotal int64 | ||||
| 	// MinMemoryTotal sets the maximum memory (in bytes) in the whole cluster
 | ||||
| 	MinMemoryTotal int64 | ||||
| 	// GpuTotal is a list of strings with configuration of min/max limits for different GPUs.
 | ||||
| 	GpuTotal []GpuLimits | ||||
| 	// NodeGroupAutoDiscovery represents one or more definition(s) of node group auto-discovery
 | ||||
| 	NodeGroupAutoDiscovery []string | ||||
| 	// EstimatorName is the estimator used to estimate the number of needed nodes in scale up.
 | ||||
|  | @ -134,6 +146,25 @@ type AutoscalingOptions struct { | |||
| 	Regional bool | ||||
| } | ||||
| 
 | ||||
| // NewResourceLimiterFromAutoscalingOptions creates new instance of cloudprovider.ResourceLimiter
 | ||||
| // reading limits from AutoscalingOptions struct.
 | ||||
| func NewResourceLimiterFromAutoscalingOptions(options AutoscalingOptions) *cloudprovider.ResourceLimiter { | ||||
| 	// build min/max maps for resources limits
 | ||||
| 	minResources := make(map[string]int64) | ||||
| 	maxResources := make(map[string]int64) | ||||
| 
 | ||||
| 	minResources[cloudprovider.ResourceNameCores] = options.MinCoresTotal | ||||
| 	minResources[cloudprovider.ResourceNameMemory] = options.MinMemoryTotal | ||||
| 	maxResources[cloudprovider.ResourceNameCores] = options.MaxCoresTotal | ||||
| 	maxResources[cloudprovider.ResourceNameMemory] = options.MaxMemoryTotal | ||||
| 
 | ||||
| 	for _, gpuLimits := range options.GpuTotal { | ||||
| 		minResources[gpuLimits.GpuType] = gpuLimits.Min | ||||
| 		maxResources[gpuLimits.GpuType] = gpuLimits.Max | ||||
| 	} | ||||
| 	return cloudprovider.NewResourceLimiter(minResources, maxResources) | ||||
| } | ||||
| 
 | ||||
| // NewAutoscalingContext returns an autoscaling context from all the necessary parameters passed via arguments
 | ||||
| func NewAutoscalingContext(options AutoscalingOptions, predicateChecker *simulator.PredicateChecker, | ||||
| 	kubeClient kube_client.Interface, kubeEventRecorder kube_record.EventRecorder, | ||||
|  | @ -143,9 +174,7 @@ func NewAutoscalingContext(options AutoscalingOptions, predicateChecker *simulat | |||
| 	cloudProvider := cloudProviderBuilder.Build(cloudprovider.NodeGroupDiscoveryOptions{ | ||||
| 		NodeGroupSpecs:              options.NodeGroups, | ||||
| 		NodeGroupAutoDiscoverySpecs: options.NodeGroupAutoDiscovery}, | ||||
| 		cloudprovider.NewResourceLimiter( | ||||
| 			map[string]int64{cloudprovider.ResourceNameCores: int64(options.MinCoresTotal), cloudprovider.ResourceNameMemory: options.MinMemoryTotal}, | ||||
| 			map[string]int64{cloudprovider.ResourceNameCores: options.MaxCoresTotal, cloudprovider.ResourceNameMemory: options.MaxMemoryTotal})) | ||||
| 		NewResourceLimiterFromAutoscalingOptions(options)) | ||||
| 	expanderStrategy, err := factory.ExpanderStrategyFromString(options.ExpanderName, | ||||
| 		cloudProvider, listerRegistry.AllNodeLister()) | ||||
| 	if err != nil { | ||||
|  |  | |||
|  | @ -44,6 +44,7 @@ import ( | |||
| 
 | ||||
| 	"github.com/golang/glog" | ||||
| 	"k8s.io/apimachinery/pkg/util/sets" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu" | ||||
| ) | ||||
| 
 | ||||
| // ScaleDownResult represents the state of scale down.
 | ||||
|  | @ -109,27 +110,57 @@ type scaleDownResourcesDelta map[string]int64 | |||
| // used as a value in scaleDownResourcesLimits if actual limit could not be obtained due to errors talking to cloud provider
 | ||||
| const limitUnknown = math.MinInt64 | ||||
| 
 | ||||
| func computeScaleDownResourcesLeftLimits(nodes []*apiv1.Node, resourceLimiter *cloudprovider.ResourceLimiter, timestamp time.Time) scaleDownResourcesLimits { | ||||
| 	totals := make(map[string]int64) | ||||
| 	totals[cloudprovider.ResourceNameCores], totals[cloudprovider.ResourceNameMemory] = calculateCoresAndMemoryTotal(nodes, timestamp) | ||||
| 	resultScaleDownLimits := make(scaleDownResourcesLimits) | ||||
| func computeScaleDownResourcesLeftLimits(nodes []*apiv1.Node, resourceLimiter *cloudprovider.ResourceLimiter, cp cloudprovider.CloudProvider, timestamp time.Time) scaleDownResourcesLimits { | ||||
| 	totalCores, totalMem := calculateCoresAndMemoryTotal(nodes, timestamp) | ||||
| 
 | ||||
| 	var totalGpus map[string]int64 | ||||
| 	var totalGpusErr error | ||||
| 	if containsGpuResources(resourceLimiter.GetResources()) { | ||||
| 		totalGpus, totalGpusErr = calculateGpusTotal(nodes, cp, timestamp) | ||||
| 	} | ||||
| 
 | ||||
| 	resultScaleDownLimits := make(scaleDownResourcesLimits) | ||||
| 	for _, resource := range resourceLimiter.GetResources() { | ||||
| 		total := totals[resource] | ||||
| 		min := resourceLimiter.GetMin(resource) | ||||
| 
 | ||||
| 		// we put only actual limits into final map. No entry means no limit.
 | ||||
| 		if min > 0 { | ||||
| 			if total > min { | ||||
| 				resultScaleDownLimits[resource] = total - min | ||||
| 			switch { | ||||
| 			case resource == cloudprovider.ResourceNameCores: | ||||
| 				resultScaleDownLimits[resource] = computeLeft(totalCores, min) | ||||
| 			case resource == cloudprovider.ResourceNameMemory: | ||||
| 				resultScaleDownLimits[resource] = computeLeft(totalMem, min) | ||||
| 			case cloudprovider.IsGpuResource(resource): | ||||
| 				if totalGpusErr != nil { | ||||
| 					resultScaleDownLimits[resource] = limitUnknown | ||||
| 				} else { | ||||
| 				resultScaleDownLimits[resource] = 0 | ||||
| 					resultScaleDownLimits[resource] = computeLeft(totalGpus[resource], min) | ||||
| 				} | ||||
| 			default: | ||||
| 				glog.Errorf("Scale down limits defined for unsupported resource '%s'", resource) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| 	return resultScaleDownLimits | ||||
| } | ||||
| 
 | ||||
| func containsGpuResources(resources []string) bool { | ||||
| 	for _, resource := range resources { | ||||
| 		if cloudprovider.IsGpuResource(resource) { | ||||
| 			return true | ||||
| 		} | ||||
| 	} | ||||
| 	return false | ||||
| } | ||||
| 
 | ||||
| func computeLeft(total int64, min int64) int64 { | ||||
| 	if total > min { | ||||
| 		return total - min | ||||
| 	} | ||||
| 	return 0 | ||||
| 
 | ||||
| } | ||||
| 
 | ||||
| func calculateCoresAndMemoryTotal(nodes []*apiv1.Node, timestamp time.Time) (int64, int64) { | ||||
| 	var coresTotal, memoryTotal int64 | ||||
| 	for _, node := range nodes { | ||||
|  | @ -146,6 +177,60 @@ func calculateCoresAndMemoryTotal(nodes []*apiv1.Node, timestamp time.Time) (int | |||
| 	return coresTotal, memoryTotal | ||||
| } | ||||
| 
 | ||||
| func calculateGpusTotal(nodes []*apiv1.Node, cp cloudprovider.CloudProvider, timestamp time.Time) (map[string]int64, error) { | ||||
| 	type gpuInfo struct { | ||||
| 		name  string | ||||
| 		count int64 | ||||
| 	} | ||||
| 
 | ||||
| 	result := make(map[string]int64) | ||||
| 	ngCache := make(map[string]gpuInfo) | ||||
| 	for _, node := range nodes { | ||||
| 		if isNodeBeingDeleted(node, timestamp) { | ||||
| 			// Nodes being deleted do not count towards total cluster resources
 | ||||
| 			continue | ||||
| 		} | ||||
| 		nodeGroup, err := cp.NodeGroupForNode(node) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.ToAutoscalerError(errors.CloudProviderError, err).AddPrefix("can not get node group for node %v when calculating cluster gpu usage", node.Name) | ||||
| 		} | ||||
| 		if reflect.ValueOf(nodeGroup).IsNil() { | ||||
| 			// We do not trust cloud providers to return properly constructed nil for interface type - hence the reflection check.
 | ||||
| 			// See https://golang.org/doc/faq#nil_error
 | ||||
| 			// TODO[lukaszos] consider creating cloud_provider sanitizer which will wrap cloud provider and ensure sane behaviour.
 | ||||
| 			nodeGroup = nil | ||||
| 		} | ||||
| 
 | ||||
| 		var gpuType string | ||||
| 		var gpuCount int64 | ||||
| 
 | ||||
| 		var cached gpuInfo | ||||
| 		var cacheHit bool | ||||
| 		if nodeGroup != nil { | ||||
| 			cached, cacheHit = ngCache[nodeGroup.Id()] | ||||
| 			if cacheHit { | ||||
| 				gpuType = cached.name | ||||
| 				gpuCount = cached.count | ||||
| 			} | ||||
| 		} | ||||
| 		if !cacheHit { | ||||
| 			gpuType, gpuCount, err = gpu.GetNodeTargetGpus(node, nodeGroup) | ||||
| 			if err != nil { | ||||
| 				return nil, errors.ToAutoscalerError(errors.CloudProviderError, err).AddPrefix("can not get gpu count for node %v when calculating cluster gpu usage") | ||||
| 			} | ||||
| 			if nodeGroup != nil { | ||||
| 				ngCache[nodeGroup.Id()] = gpuInfo{name: gpuType, count: gpuCount} | ||||
| 			} | ||||
| 		} | ||||
| 		if gpuType == "" || gpuCount == 0 { | ||||
| 			continue | ||||
| 		} | ||||
| 		result[gpuType] += gpuCount | ||||
| 	} | ||||
| 
 | ||||
| 	return result, nil | ||||
| } | ||||
| 
 | ||||
| func isNodeBeingDeleted(node *apiv1.Node, timestamp time.Time) bool { | ||||
| 	deleteTime, _ := deletetaint.GetToBeDeletedTime(node) | ||||
| 	return deleteTime != nil && (timestamp.Sub(*deleteTime) < MaxCloudProviderNodeDeletionTime || timestamp.Sub(*deleteTime) < MaxKubernetesEmptyNodeDeletionTime) | ||||
|  | @ -163,13 +248,21 @@ func copyScaleDownResourcesLimits(source scaleDownResourcesLimits) scaleDownReso | |||
| 	return copy | ||||
| } | ||||
| 
 | ||||
| func computeScaleDownResourcesDelta(node *apiv1.Node) (scaleDownResourcesDelta, errors.AutoscalerError) { | ||||
| 	nodeCPU, nodeMemory := getNodeCoresAndMemory(node) | ||||
| func computeScaleDownResourcesDelta(node *apiv1.Node, nodeGroup cloudprovider.NodeGroup, resourcesWithLimits []string) (scaleDownResourcesDelta, errors.AutoscalerError) { | ||||
| 	resultScaleDownDelta := make(scaleDownResourcesDelta) | ||||
| 
 | ||||
| 	return scaleDownResourcesDelta{ | ||||
| 		cloudprovider.ResourceNameCores:  nodeCPU, | ||||
| 		cloudprovider.ResourceNameMemory: nodeMemory, | ||||
| 	}, nil | ||||
| 	nodeCPU, nodeMemory := getNodeCoresAndMemory(node) | ||||
| 	resultScaleDownDelta[cloudprovider.ResourceNameCores] = nodeCPU | ||||
| 	resultScaleDownDelta[cloudprovider.ResourceNameMemory] = nodeMemory | ||||
| 
 | ||||
| 	if containsGpuResources(resourcesWithLimits) { | ||||
| 		gpuType, gpuCount, err := gpu.GetNodeTargetGpus(node, nodeGroup) | ||||
| 		if err != nil { | ||||
| 			return scaleDownResourcesDelta{}, errors.ToAutoscalerError(errors.CloudProviderError, err).AddPrefix("Failed to get node %v gpu: %v", node.Name) | ||||
| 		} | ||||
| 		resultScaleDownDelta[gpuType] = gpuCount | ||||
| 	} | ||||
| 	return resultScaleDownDelta, nil | ||||
| } | ||||
| 
 | ||||
| type limitCheckResult struct { | ||||
|  | @ -490,9 +583,10 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p | |||
| 			errCP) | ||||
| 	} | ||||
| 
 | ||||
| 	scaleDownResourcesLeft := computeScaleDownResourcesLeftLimits(nodesWithoutMaster, resourceLimiter, currentTime) | ||||
| 	scaleDownResourcesLeft := computeScaleDownResourcesLeftLimits(nodesWithoutMaster, resourceLimiter, sd.context.CloudProvider, currentTime) | ||||
| 
 | ||||
| 	nodeGroupSize := getNodeGroupSizeMap(sd.context.CloudProvider) | ||||
| 	resourcesWithLimits := resourceLimiter.GetResources() | ||||
| 	for _, node := range nodesWithoutMaster { | ||||
| 		if val, found := sd.unneededNodes[node.Name]; found { | ||||
| 
 | ||||
|  | @ -538,7 +632,7 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p | |||
| 				continue | ||||
| 			} | ||||
| 
 | ||||
| 			scaleDownResourcesDelta, err := computeScaleDownResourcesDelta(node) | ||||
| 			scaleDownResourcesDelta, err := computeScaleDownResourcesDelta(node, nodeGroup, resourcesWithLimits) | ||||
| 			if err != nil { | ||||
| 				glog.Errorf("Error getting node resources: %v", err) | ||||
| 				continue | ||||
|  | @ -651,6 +745,8 @@ func getEmptyNodes(candidates []*apiv1.Node, pods []*apiv1.Pod, maxEmptyBulkDele | |||
| 	availabilityMap := make(map[string]int) | ||||
| 	result := make([]*apiv1.Node, 0) | ||||
| 	resourcesLimitsCopy := copyScaleDownResourcesLimits(resourcesLimits) // we do not want to modify input parameter
 | ||||
| 	resourcesNames := sets.StringKeySet(resourcesLimits).List() | ||||
| 
 | ||||
| 	for _, node := range emptyNodes { | ||||
| 		nodeGroup, err := cloudProvider.NodeGroupForNode(node) | ||||
| 		if err != nil { | ||||
|  | @ -676,7 +772,7 @@ func getEmptyNodes(candidates []*apiv1.Node, pods []*apiv1.Pod, maxEmptyBulkDele | |||
| 			availabilityMap[nodeGroup.Id()] = available | ||||
| 		} | ||||
| 		if available > 0 { | ||||
| 			resourcesDelta, err := computeScaleDownResourcesDelta(node) | ||||
| 			resourcesDelta, err := computeScaleDownResourcesDelta(node, nodeGroup, resourcesNames) | ||||
| 			if err != nil { | ||||
| 				glog.Errorf("Error: %v", err) | ||||
| 				continue | ||||
|  |  | |||
|  | @ -28,7 +28,6 @@ import ( | |||
| 	"k8s.io/apimachinery/pkg/api/errors" | ||||
| 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" | ||||
| 	"k8s.io/apimachinery/pkg/runtime" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider" | ||||
| 	testprovider "k8s.io/autoscaler/cluster-autoscaler/cloudprovider/test" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/clusterstate" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/clusterstate/utils" | ||||
|  | @ -47,6 +46,7 @@ import ( | |||
| 	"github.com/golang/glog" | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/deletetaint" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu" | ||||
| ) | ||||
| 
 | ||||
| func TestFindUnneededNodes(t *testing.T) { | ||||
|  | @ -925,6 +925,35 @@ func TestScaleDownEmptyMinMemoryLimitHit(t *testing.T) { | |||
| 	simpleScaleDownEmpty(t, config) | ||||
| } | ||||
| 
 | ||||
| func TestScaleDownEmptyMinGpuLimitHit(t *testing.T) { | ||||
| 	options := defaultScaleDownOptions | ||||
| 	options.GpuTotal = []context.GpuLimits{ | ||||
| 		{ | ||||
| 			GpuType: gpu.DefaultGPUType, | ||||
| 			Min:     4, | ||||
| 			Max:     50, | ||||
| 		}, | ||||
| 		{ | ||||
| 			GpuType: "nvidia-tesla-p100", // this one should not trigger
 | ||||
| 			Min:     5, | ||||
| 			Max:     50, | ||||
| 		}, | ||||
| 	} | ||||
| 	config := &scaleTestConfig{ | ||||
| 		nodes: []nodeConfig{ | ||||
| 			{"n1", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 			{"n2", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 			{"n3", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 			{"n4", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 			{"n5", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 			{"n6", 1000, 1000 * MB, 1, true, "ng1"}, | ||||
| 		}, | ||||
| 		options:            options, | ||||
| 		expectedScaleDowns: []string{"n1", "n2"}, | ||||
| 	} | ||||
| 	simpleScaleDownEmpty(t, config) | ||||
| } | ||||
| 
 | ||||
| func TestScaleDownEmptyMinGroupSizeLimitHit(t *testing.T) { | ||||
| 	options := defaultScaleDownOptions | ||||
| 	config := &scaleTestConfig{ | ||||
|  | @ -946,6 +975,9 @@ func simpleScaleDownEmpty(t *testing.T, config *scaleTestConfig) { | |||
| 	groups := make(map[string][]*apiv1.Node) | ||||
| 	for i, n := range config.nodes { | ||||
| 		node := BuildTestNode(n.name, n.cpu, n.memory) | ||||
| 		if n.gpu > 0 { | ||||
| 			AddGpusToNode(node, n.gpu) | ||||
| 		} | ||||
| 		SetNodeReadyState(node, n.ready, time.Time{}) | ||||
| 		nodesMap[n.name] = node | ||||
| 		nodes[i] = node | ||||
|  | @ -985,9 +1017,7 @@ func simpleScaleDownEmpty(t *testing.T, config *scaleTestConfig) { | |||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	resourceLimiter := cloudprovider.NewResourceLimiter( | ||||
| 		map[string]int64{cloudprovider.ResourceNameCores: config.options.MinCoresTotal, cloudprovider.ResourceNameMemory: config.options.MinMemoryTotal}, | ||||
| 		map[string]int64{cloudprovider.ResourceNameCores: config.options.MaxCoresTotal, cloudprovider.ResourceNameMemory: config.options.MaxMemoryTotal}) | ||||
| 	resourceLimiter := context.NewResourceLimiterFromAutoscalingOptions(config.options) | ||||
| 	provider.SetResourceLimiter(resourceLimiter) | ||||
| 
 | ||||
| 	assert.NotNil(t, provider) | ||||
|  |  | |||
|  | @ -113,6 +113,7 @@ var ( | |||
| 	maxNodesTotal     = flag.Int("max-nodes-total", 0, "Maximum number of nodes in all node groups. Cluster autoscaler will not grow the cluster beyond this number.") | ||||
| 	coresTotal        = flag.String("cores-total", minMaxFlagString(0, config.DefaultMaxClusterCores), "Minimum and maximum number of cores in cluster, in the format <min>:<max>. Cluster autoscaler will not scale the cluster beyond these numbers.") | ||||
| 	memoryTotal       = flag.String("memory-total", minMaxFlagString(0, config.DefaultMaxClusterMemory), "Minimum and maximum number of gigabytes of memory in cluster, in the format <min>:<max>. Cluster autoscaler will not scale the cluster beyond these numbers.") | ||||
| 	gpuTotal          = multiStringFlag("gpu-total", "Minimum and maximum number of different GPUs in cluster, in the format <gpu_type>:<min>:<max>. Cluster autoscaler will not scale the cluster beyond these numbers. Can be passed multiple times. CURRENTLY THIS FLAG ONLY WORKS ON GKE.") | ||||
| 	cloudProviderFlag = flag.String("cloud-provider", cloudBuilder.DefaultCloudProvider, | ||||
| 		"Cloud provider type. Available values: ["+strings.Join(cloudBuilder.AvailableCloudProviders, ",")+"]") | ||||
| 	maxEmptyBulkDeleteFlag     = flag.Int("max-empty-bulk-delete", 10, "Maximum number of empty nodes that can be deleted at the same time.") | ||||
|  | @ -161,6 +162,11 @@ func createAutoscalingOptions() context.AutoscalingOptions { | |||
| 	minMemoryTotal = minMemoryTotal * units.Gigabyte | ||||
| 	maxMemoryTotal = maxMemoryTotal * units.Gigabyte | ||||
| 
 | ||||
| 	parsedGpuTotal, err := parseMultipleGpuLimits(*gpuTotal) | ||||
| 	if err != nil { | ||||
| 		glog.Fatalf("Failed to parse flags: %v", err) | ||||
| 	} | ||||
| 
 | ||||
| 	return context.AutoscalingOptions{ | ||||
| 		CloudConfig:                      *cloudConfig, | ||||
| 		CloudProviderName:                *cloudProviderFlag, | ||||
|  | @ -177,6 +183,7 @@ func createAutoscalingOptions() context.AutoscalingOptions { | |||
| 		MinCoresTotal:                    minCoresTotal, | ||||
| 		MaxMemoryTotal:                   maxMemoryTotal, | ||||
| 		MinMemoryTotal:                   minMemoryTotal, | ||||
| 		GpuTotal:                         parsedGpuTotal, | ||||
| 		NodeGroups:                       *nodeGroupsFlag, | ||||
| 		ScaleDownDelayAfterAdd:           *scaleDownDelayAfterAdd, | ||||
| 		ScaleDownDelayAfterDelete:        *scaleDownDelayAfterDelete, | ||||
|  | @ -445,3 +452,46 @@ func validateMinMaxFlag(min, max int64) error { | |||
| func minMaxFlagString(min, max int64) string { | ||||
| 	return fmt.Sprintf("%v:%v", min, max) | ||||
| } | ||||
| 
 | ||||
| func parseMultipleGpuLimits(flags MultiStringFlag) ([]context.GpuLimits, error) { | ||||
| 	parsedFlags := make([]context.GpuLimits, 0, len(flags)) | ||||
| 	for _, flag := range flags { | ||||
| 		parsedFlag, err := parseSingleGpuLimit(flag) | ||||
| 		if err != nil { | ||||
| 			return nil, err | ||||
| 		} | ||||
| 		parsedFlags = append(parsedFlags, parsedFlag) | ||||
| 	} | ||||
| 	return parsedFlags, nil | ||||
| } | ||||
| 
 | ||||
| func parseSingleGpuLimit(config string) (context.GpuLimits, error) { | ||||
| 	parts := strings.Split(config, ":") | ||||
| 	if len(parts) != 3 { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit specification: %v", config) | ||||
| 	} | ||||
| 	gpuType := parts[0] | ||||
| 	minVal, err := strconv.ParseInt(parts[1], 10, 64) | ||||
| 	if err != nil { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit - min is not integer: %v", config) | ||||
| 	} | ||||
| 	maxVal, err := strconv.ParseInt(parts[2], 10, 64) | ||||
| 	if err != nil { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit - max is not integer: %v", config) | ||||
| 	} | ||||
| 	if minVal < 0 { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit - min is less than 0; %v", config) | ||||
| 	} | ||||
| 	if maxVal < 0 { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit - max is less than 0; %v", config) | ||||
| 	} | ||||
| 	if minVal > maxVal { | ||||
| 		return context.GpuLimits{}, fmt.Errorf("Incorrect gpu limit - min is greater than max; %v", config) | ||||
| 	} | ||||
| 	parsedGpuLimits := context.GpuLimits{ | ||||
| 		GpuType: gpuType, | ||||
| 		Min:     minVal, | ||||
| 		Max:     maxVal, | ||||
| 	} | ||||
| 	return parsedGpuLimits, nil | ||||
| } | ||||
|  |  | |||
|  | @ -0,0 +1,91 @@ | |||
| /* | ||||
| Copyright 2018 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 main | ||||
| 
 | ||||
| import ( | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/context" | ||||
| 	"testing" | ||||
| ) | ||||
| 
 | ||||
| func TestParseSingleGpuLimit(t *testing.T) { | ||||
| 	type testcase struct { | ||||
| 		input                string | ||||
| 		expectError          bool | ||||
| 		expectedLimits       context.GpuLimits | ||||
| 		expectedErrorMessage string | ||||
| 	} | ||||
| 
 | ||||
| 	testcases := []testcase{ | ||||
| 		{ | ||||
| 			input:       "gpu:1:10", | ||||
| 			expectError: false, | ||||
| 			expectedLimits: context.GpuLimits{ | ||||
| 				GpuType: "gpu", | ||||
| 				Min:     1, | ||||
| 				Max:     10, | ||||
| 			}, | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:1", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit specification: gpu:1", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:1:10:x", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit specification: gpu:1:10:x", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:x:10", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit - min is not integer: gpu:x:10", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:1:y", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit - max is not integer: gpu:1:y", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:-1:10", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit - min is less than 0; gpu:-1:10", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:1:-10", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit - max is less than 0; gpu:1:-10", | ||||
| 		}, | ||||
| 		{ | ||||
| 			input:                "gpu:10:1", | ||||
| 			expectError:          true, | ||||
| 			expectedErrorMessage: "Incorrect gpu limit - min is greater than max; gpu:10:1", | ||||
| 		}, | ||||
| 	} | ||||
| 
 | ||||
| 	for _, testcase := range testcases { | ||||
| 		limits, err := parseSingleGpuLimit(testcase.input) | ||||
| 		if testcase.expectError { | ||||
| 			assert.NotNil(t, err) | ||||
| 			if err != nil { | ||||
| 				assert.Equal(t, testcase.expectedErrorMessage, err.Error()) | ||||
| 			} | ||||
| 		} else { | ||||
| 			assert.Equal(t, testcase.expectedLimits, limits) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -19,8 +19,10 @@ package gpu | |||
| import ( | ||||
| 	apiv1 "k8s.io/api/core/v1" | ||||
| 	"k8s.io/apimachinery/pkg/api/resource" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider" | ||||
| 
 | ||||
| 	"github.com/golang/glog" | ||||
| 	"k8s.io/autoscaler/cluster-autoscaler/utils/errors" | ||||
| ) | ||||
| 
 | ||||
| const ( | ||||
|  | @ -156,3 +158,47 @@ func GetGpuRequests(pods []*apiv1.Pod) map[string]GpuRequestInfo { | |||
| 	} | ||||
| 	return result | ||||
| } | ||||
| 
 | ||||
| // GetNodeTargetGpus returns the number of gpus on a given node. This includes gpus which are not yet
 | ||||
| // ready to use and visible in kubernetes.
 | ||||
| func GetNodeTargetGpus(node *apiv1.Node, nodeGroup cloudprovider.NodeGroup) (gpuType string, gpuCount int64, err error) { | ||||
| 	gpuLabel, found := node.Labels[GPULabel] | ||||
| 	if !found { | ||||
| 		return "", 0, nil | ||||
| 	} | ||||
| 
 | ||||
| 	gpuAllocatable, found := node.Status.Allocatable[ResourceNvidiaGPU] | ||||
| 	if found && gpuAllocatable.Value() > 0 { | ||||
| 		return gpuLabel, gpuAllocatable.Value(), nil | ||||
| 	} | ||||
| 
 | ||||
| 	// A node is supposed to have GPUs (based on label), but they're not available yet
 | ||||
| 	// (driver haven't installed yet?).
 | ||||
| 	// Unfortunately we can't deduce how many GPUs it will actually have from labels (just
 | ||||
| 	// that it will have some).
 | ||||
| 	// Ready for some evil hacks? Well, you won't be disappointed - let's pretend we haven't
 | ||||
| 	// seen the node and just use the template we use for scale from 0. It'll be our little
 | ||||
| 	// secret.
 | ||||
| 
 | ||||
| 	if nodeGroup == nil { | ||||
| 		// We expect this code path to be triggered by situation when we are looking at a node which is expected to have gpus (has gpu label)
 | ||||
| 		// But those are not yet visible in node's resource (e.g. gpu drivers are still being installed).
 | ||||
| 		// In case of node coming from autoscaled node group we would look and node group template here.
 | ||||
| 		// But for nodes coming from non-autoscaled groups we have no such possibility.
 | ||||
| 		// Let's hope it is a transient error. As long as it exists we will not scale nodes groups with gpus.
 | ||||
| 		return "", 0, errors.NewAutoscalerError(errors.InternalError, "node without with gpu label, without capacity not belonging to autoscaled node group") | ||||
| 	} | ||||
| 
 | ||||
| 	template, err := nodeGroup.TemplateNodeInfo() | ||||
| 	if err != nil { | ||||
| 		glog.Errorf("Failed to build template for getting GPU estimation for node %v: %v", node.Name, err) | ||||
| 		return "", 0, errors.ToAutoscalerError(errors.CloudProviderError, err) | ||||
| 	} | ||||
| 	if gpuCapacity, found := template.Node().Status.Capacity[ResourceNvidiaGPU]; found { | ||||
| 		return gpuLabel, gpuCapacity.Value(), nil | ||||
| 	} | ||||
| 
 | ||||
| 	// if template does not define gpus we assume node will not have any even if ith has gpu label
 | ||||
| 	glog.Warningf("Template does not define gpus even though node from its node group does; node=%v", node.Name) | ||||
| 	return "", 0, nil | ||||
| } | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue