From cdef63c2cb61b8be328587914418b0e425e452f0 Mon Sep 17 00:00:00 2001 From: Jack Francis Date: Fri, 22 Sep 2023 15:44:13 -0700 Subject: [PATCH] Azure: Remove AKS vmType Signed-off-by: Jack Francis --- .../cloudprovider/azure/README.md | 63 +-- .../cloudprovider/azure/azure_cache.go | 4 +- .../cloudprovider/azure/azure_client.go | 7 - .../cloudprovider/azure/azure_config.go | 16 - .../azure/azure_kubernetes_service_pool.go | 450 ------------------ .../azure_kubernetes_service_pool_test.go | 423 ---------------- .../cloudprovider/azure/azure_manager.go | 3 - .../cloudprovider/azure/azure_manager_test.go | 16 +- .../cloudprovider/azure/azure_util_test.go | 2 +- .../examples/cluster-autoscaler-aks.yaml | 12 - 10 files changed, 11 insertions(+), 985 deletions(-) delete mode 100644 cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool.go delete mode 100644 cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool_test.go diff --git a/cluster-autoscaler/cloudprovider/azure/README.md b/cluster-autoscaler/cloudprovider/azure/README.md index 427fc980b634..bfe964fe176f 100644 --- a/cluster-autoscaler/cloudprovider/azure/README.md +++ b/cluster-autoscaler/cloudprovider/azure/README.md @@ -81,8 +81,7 @@ k8s.io_cluster-autoscaler_node-template_autoscaling-options_scaledownunreadytime Cluster autoscaler supports four Kubernetes cluster options on Azure: - [**vmss**](#vmss-deployment): Autoscale VMSS instances by setting the Azure cloud provider's `vmType` parameter to `vmss` or to an empty string. This supports clusters deployed with [aks-engine][]. -- [**standard**](#standard-deployment): Autoscale VMAS instances by setting the Azure cloud provider's `vmType` parameter to `standard`. This supports clusters deployed with [aks-engine][]. -- [**aks**](#aks-deployment): Supports an Azure Kubernetes Service ([AKS][]) cluster. +- [**standard**](#standard-deployment): Autoscale VMAS (Virtual Machine Availability Set) VMs by setting the Azure cloud provider's `vmType` parameter to `standard`. This supports clusters deployed with [aks-engine][]. > **_NOTE_**: only the `vmss` option supports scaling down to zero nodes. @@ -250,74 +249,21 @@ To run a cluster autoscaler pod with Azure managed service identity (MSI), use [ > **_WARNING_**: Cluster autoscaler depends on user-provided deployment parameters to provision new nodes. After upgrading your Kubernetes cluster, cluster autoscaler must also be redeployed with new parameters to prevent provisioning nodes with an old version. -### AKS deployment +## AKS Autoscaler -#### AKS + VMSS - -Autoscaling VM scale sets with AKS is supported for Kubernetes v1.12.4 and later. The option to enable cluster autoscaler is available in the [Azure Portal][] or with the [Azure CLI][]: +Node Pool Autoscaling is a first class feature of your AKS cluster. The option to enable cluster autoscaler is available in the [Azure Portal][] or with the [Azure CLI][]: ```sh az aks create \ --resource-group myResourceGroup \ --name myAKSCluster \ - --kubernetes-version 1.13.5 \ + --kubernetes-version 1.25.11 \ --node-count 1 \ - --enable-vmss \ --enable-cluster-autoscaler \ --min-count 1 \ --max-count 3 ``` -#### AKS + Availability Set - -The CLI based deployment only support VMSS and manual deployment is needed if availability set is used. - -Prerequisites: - -- Get Azure credentials from the [**Permissions**](#permissions) step above. -- Get the cluster name with the `az aks list` command. -- Get the name of a node pool from the value of the label **agentpool** - -```sh -kubectl get nodes --show-labels -``` - -Make a copy of [cluster-autoscaler-aks.yaml](examples/cluster-autoscaler-aks.yaml). Fill in the placeholder values for -the `cluster-autoscaler-azure` secret data by base64-encoding each of your Azure credential fields. - -- ClientID: `` -- ClientSecret: `` -- ResourceGroup: `` (Note: ResourceGroup is case-sensitive) -- SubscriptionID: `` -- TenantID: `` -- ClusterName: `` -- NodeResourceGroup: `` (Note: node resource group is not resource group and can be obtained in the corresponding label of the nodepool) - -> **_NOTE_**: Use a command such as `echo $CLIENT_ID | base64` to encode each of the fields above. - -In the `cluster-autoscaler` spec, find the `image:` field and replace `{{ ca_version }}` with a specific cluster autoscaler release. - -Below that, in the `command:` section, update the `--nodes=` arguments to reference your node limits and node pool name. For example, if node pool "k8s-nodepool-1" should scale from 1 to 10 nodes: - -```yaml - - --nodes=1:10:k8s-nodepool-1 -``` - -or to autoscale multiple VM scale sets: - -```yaml - - --nodes=1:10:k8s-nodepool-1 - - --nodes=1:10:k8s-nodepool-2 -``` - -Then deploy cluster-autoscaler by running - -```sh -kubectl create -f cluster-autoscaler-aks.yaml -``` - -To deploy in AKS with `Helm 3`, please refer to [helm installation tutorial][]. - Please see the [AKS autoscaler documentation][] for details. ## Rate limit and back-off retries @@ -339,7 +285,6 @@ The new version of [Azure client][] supports rate limit and back-off retries whe > **_NOTE_**: * These rate limit configs can be set per-client. Customizing `QPS` and `Bucket` through environment variables per client is not supported. -[AKS]: https://docs.microsoft.com/azure/aks/ [AKS autoscaler documentation]: https://docs.microsoft.com/azure/aks/autoscaler [aks-engine]: https://github.com/Azure/aks-engine [Azure CLI]: https://docs.microsoft.com/cli/azure/install-azure-cli diff --git a/cluster-autoscaler/cloudprovider/azure/azure_cache.go b/cluster-autoscaler/cloudprovider/azure/azure_cache.go index 5c70f52d2ba6..711e98cb8236 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_cache.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_cache.go @@ -39,7 +39,7 @@ var ( // azureCache is used for caching cluster resources state. // // It is needed to: -// - keep track of node groups (AKS, VM and VMSS types) in the cluster, +// - keep track of node groups (VM and VMSS types) in the cluster, // - keep track of instances and which node group they belong to, // - limit repetitive Azure API calls. type azureCache struct { @@ -174,7 +174,7 @@ func (m *azureCache) fetchAzureResources() error { } else { return err } - case vmTypeStandard, vmTypeAKS: + case vmTypeStandard: // List all VMs in the RG. vmResult, err := m.fetchVirtualMachines() if err == nil { diff --git a/cluster-autoscaler/cloudprovider/azure/azure_client.go b/cluster-autoscaler/cloudprovider/azure/azure_client.go index 6cb02343128d..f8123f3f90b6 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_client.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_client.go @@ -34,7 +34,6 @@ import ( klog "k8s.io/klog/v2" - "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/containerserviceclient" "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/diskclient" "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/interfaceclient" "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/storageaccountclient" @@ -151,7 +150,6 @@ type azClient struct { interfacesClient interfaceclient.Interface disksClient diskclient.Interface storageAccountsClient storageaccountclient.Interface - managedKubernetesServicesClient containerserviceclient.Interface skuClient compute.ResourceSkusClient } @@ -274,10 +272,6 @@ func newAzClient(cfg *Config, env *azure.Environment) (*azClient, error) { disksClient := diskclient.New(diskClientConfig) klog.V(5).Infof("Created disks client with authorizer: %v", disksClient) - aksClientConfig := azClientConfig.WithRateLimiter(cfg.KubernetesServiceRateLimit) - kubernetesServicesClient := containerserviceclient.New(aksClientConfig) - klog.V(5).Infof("Created kubernetes services client with authorizer: %v", kubernetesServicesClient) - // Reference on why selecting ResourceManagerEndpoint as baseURI - // https://github.com/Azure/go-autorest/blob/main/autorest/azure/environments.go skuClient := compute.NewResourceSkusClientWithBaseURI(azClientConfig.ResourceManagerEndpoint, cfg.SubscriptionID) @@ -292,7 +286,6 @@ func newAzClient(cfg *Config, env *azure.Environment) (*azClient, error) { deploymentsClient: deploymentsClient, virtualMachinesClient: virtualMachinesClient, storageAccountsClient: storageAccountsClient, - managedKubernetesServicesClient: kubernetesServicesClient, skuClient: skuClient, }, nil } diff --git a/cluster-autoscaler/cloudprovider/azure/azure_config.go b/cluster-autoscaler/cloudprovider/azure/azure_config.go index 810884dfc691..9fe559d07e14 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_config.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_config.go @@ -111,11 +111,6 @@ type Config struct { Deployment string `json:"deployment" yaml:"deployment"` DeploymentParameters map[string]interface{} `json:"deploymentParameters" yaml:"deploymentParameters"` - //Configs only for AKS - ClusterName string `json:"clusterName" yaml:"clusterName"` - //Config only for AKS - NodeResourceGroup string `json:"nodeResourceGroup" yaml:"nodeResourceGroup"` - // VMSS metadata cache TTL in seconds, only applies for vmss type VmssCacheTTL int64 `json:"vmssCacheTTL" yaml:"vmssCacheTTL"` @@ -174,8 +169,6 @@ func BuildAzureConfig(configReader io.Reader) (*Config, error) { cfg.AADClientCertPath = os.Getenv("ARM_CLIENT_CERT_PATH") cfg.AADClientCertPassword = os.Getenv("ARM_CLIENT_CERT_PASSWORD") cfg.Deployment = os.Getenv("ARM_DEPLOYMENT") - cfg.ClusterName = os.Getenv("AZURE_CLUSTER_NAME") - cfg.NodeResourceGroup = os.Getenv("AZURE_NODE_RESOURCE_GROUP") subscriptionID, err := getSubscriptionIdFromInstanceMetadata() if err != nil { @@ -474,8 +467,6 @@ func (cfg *Config) TrimSpace() { cfg.AADClientCertPath = strings.TrimSpace(cfg.AADClientCertPath) cfg.AADClientCertPassword = strings.TrimSpace(cfg.AADClientCertPassword) cfg.Deployment = strings.TrimSpace(cfg.Deployment) - cfg.ClusterName = strings.TrimSpace(cfg.ClusterName) - cfg.NodeResourceGroup = strings.TrimSpace(cfg.NodeResourceGroup) } func (cfg *Config) validate() error { @@ -493,13 +484,6 @@ func (cfg *Config) validate() error { } } - if cfg.VMType == vmTypeAKS { - // Cluster name is a mandatory param to proceed. - if cfg.ClusterName == "" { - return fmt.Errorf("cluster name not set for type %+v", cfg.VMType) - } - } - if cfg.SubscriptionID == "" { return fmt.Errorf("subscription ID not set") } diff --git a/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool.go b/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool.go deleted file mode 100644 index b128f79e4bd3..000000000000 --- a/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool.go +++ /dev/null @@ -1,450 +0,0 @@ -/* -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 azure - -import ( - "fmt" - "strings" - "sync" - "time" - - "github.com/Azure/azure-sdk-for-go/services/containerservice/mgmt/2021-10-01/containerservice" - klog "k8s.io/klog/v2" - - apiv1 "k8s.io/api/core/v1" - "k8s.io/autoscaler/cluster-autoscaler/cloudprovider" - "k8s.io/autoscaler/cluster-autoscaler/config" - "k8s.io/autoscaler/cluster-autoscaler/config/dynamic" - schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework" -) - -const ( - aksManagedPoolNameTag = "aks-managed-poolName" - legacyAKSPoolNameTag = "poolName" -) - -// AKSAgentPool implements NodeGroup interface for agent pool deployed in AKS -type AKSAgentPool struct { - azureRef - manager *AzureManager - util *AzUtil - - minSize int - maxSize int - clusterName string - resourceGroup string - nodeResourceGroup string - - curSize int - lastRefresh time.Time - mutex sync.Mutex -} - -// NewAKSAgentPool constructs AKSAgentPool from the --node param -// and azure manager -func NewAKSAgentPool(spec *dynamic.NodeGroupSpec, am *AzureManager) (*AKSAgentPool, error) { - asg := &AKSAgentPool{ - azureRef: azureRef{ - Name: spec.Name, - }, - minSize: spec.MinSize, - maxSize: spec.MaxSize, - manager: am, - curSize: -1, - } - - asg.util = &AzUtil{ - manager: am, - } - asg.clusterName = am.config.ClusterName - asg.resourceGroup = am.config.ResourceGroup - asg.nodeResourceGroup = am.config.NodeResourceGroup - - return asg, nil -} - -// GetAKSAgentPool is an internal function which figures out ManagedClusterAgentPoolProfile from the list based on the pool name provided in the --node parameter passed -// to the autoscaler main -func (agentPool *AKSAgentPool) GetAKSAgentPool(agentProfiles *[]containerservice.ManagedClusterAgentPoolProfile) (ret *containerservice.ManagedClusterAgentPoolProfile) { - for _, value := range *agentProfiles { - profileName := *value.Name - klog.V(5).Infof("AKS AgentPool profile name: %s", profileName) - if strings.EqualFold(profileName, agentPool.azureRef.Name) { - return &value - } - } - - return nil -} - -// getAKSNodeCount gets node count for AKS agent pool. -func (agentPool *AKSAgentPool) getAKSNodeCount() (count int, err error) { - ctx, cancel := getContextWithCancel() - defer cancel() - - managedCluster, rerr := agentPool.manager.azClient.managedKubernetesServicesClient.Get(ctx, - agentPool.resourceGroup, - agentPool.clusterName) - if rerr != nil { - klog.Errorf("Failed to get AKS cluster (name:%q): %v", agentPool.clusterName, rerr.Error()) - return -1, rerr.Error() - } - - pool := agentPool.GetAKSAgentPool(managedCluster.AgentPoolProfiles) - if pool == nil { - return -1, fmt.Errorf("could not find pool with name: %s", agentPool.azureRef) - } - - if pool.Count != nil { - return int(*pool.Count), nil - } - - return 0, nil -} - -// setAKSNodeCount sets node count for AKS agent pool. -func (agentPool *AKSAgentPool) setAKSNodeCount(count int) error { - ctx, cancel := getContextWithCancel() - defer cancel() - - managedCluster, rerr := agentPool.manager.azClient.managedKubernetesServicesClient.Get(ctx, - agentPool.resourceGroup, - agentPool.clusterName) - if rerr != nil { - klog.Errorf("Failed to get AKS cluster (name:%q): %v", agentPool.clusterName, rerr.Error()) - return rerr.Error() - } - - pool := agentPool.GetAKSAgentPool(managedCluster.AgentPoolProfiles) - if pool == nil { - return fmt.Errorf("could not find pool with name: %s", agentPool.azureRef) - } - - klog.Infof("Current size: %d, Target size requested: %d", *pool.Count, count) - - updateCtx, updateCancel := getContextWithCancel() - defer updateCancel() - *pool.Count = int32(count) - aksClient := agentPool.manager.azClient.managedKubernetesServicesClient - rerr = aksClient.CreateOrUpdate(updateCtx, agentPool.resourceGroup, - agentPool.clusterName, managedCluster, "") - if rerr != nil { - klog.Errorf("Failed to update AKS cluster (%q): %v", agentPool.clusterName, rerr.Error()) - return rerr.Error() - } - - klog.Infof("aksClient.CreateOrUpdate for aks cluster %q succeeded", agentPool.clusterName) - return nil -} - -// GetNodeCount returns the count of nodes from the managed agent pool profile -func (agentPool *AKSAgentPool) GetNodeCount() (count int, err error) { - return agentPool.getAKSNodeCount() -} - -// SetNodeCount sets the count of nodes in the in memory pool profile -func (agentPool *AKSAgentPool) SetNodeCount(count int) (err error) { - return agentPool.setAKSNodeCount(count) -} - -// GetProviderID converts the name of a node into the form that kubernetes cloud -// provider id is presented in. -func (agentPool *AKSAgentPool) GetProviderID(name string) string { - //TODO: come with a generic way to make it work with provider id formats - // in different version of k8s. - return "azure://" + name -} - -// GetName extracts the name of the node (a format which underlying cloud service understands) -// from the cloud providerID (format which kubernetes understands) -func (agentPool *AKSAgentPool) GetName(providerID string) (string, error) { - // Remove the "azure://" string from it - providerID = strings.TrimPrefix(providerID, "azure://") - ctx, cancel := getContextWithCancel() - defer cancel() - vms, rerr := agentPool.manager.azClient.virtualMachinesClient.List(ctx, agentPool.nodeResourceGroup) - if rerr != nil { - return "", rerr.Error() - } - for _, vm := range vms { - if strings.EqualFold(*vm.ID, providerID) { - return *vm.Name, nil - } - } - return "", fmt.Errorf("VM list empty") -} - -// MaxSize returns the maximum size scale limit provided by --node -// parameter to the autoscaler main -func (agentPool *AKSAgentPool) MaxSize() int { - return agentPool.maxSize -} - -// MinSize returns the minimum size the cluster is allowed to scaled down -// to as provided by the node spec in --node parameter. -func (agentPool *AKSAgentPool) MinSize() int { - return agentPool.minSize -} - -// TargetSize gathers the target node count set for the cluster by -// querying the underlying service. -func (agentPool *AKSAgentPool) TargetSize() (int, error) { - agentPool.mutex.Lock() - defer agentPool.mutex.Unlock() - - if agentPool.lastRefresh.Add(15 * time.Second).After(time.Now()) { - return agentPool.curSize, nil - } - - count, err := agentPool.GetNodeCount() - if err != nil { - return -1, err - } - klog.V(5).Infof("Got new size %d for agent pool (%q)", count, agentPool.Name) - - agentPool.curSize = count - agentPool.lastRefresh = time.Now() - return agentPool.curSize, nil -} - -// SetSize contacts the underlying service and sets the size of the pool. -// This will be called when a scale up occurs and will be called just after -// a delete is performed from a scale down. -func (agentPool *AKSAgentPool) SetSize(targetSize int, isScalingDown bool) (err error) { - agentPool.mutex.Lock() - defer agentPool.mutex.Unlock() - - return agentPool.setSizeInternal(targetSize, isScalingDown) -} - -// setSizeInternal contacts the underlying service and sets the size of the pool. -// It should be called under lock protected. -func (agentPool *AKSAgentPool) setSizeInternal(targetSize int, isScalingDown bool) (err error) { - if isScalingDown && targetSize < agentPool.MinSize() { - klog.Errorf("size-decreasing request of %d is smaller than min size %d", targetSize, agentPool.MinSize()) - return fmt.Errorf("size-decreasing request of %d is smaller than min size %d", targetSize, agentPool.MinSize()) - } - - klog.V(2).Infof("Setting size for cluster (%q) with new count (%d)", agentPool.clusterName, targetSize) - - err = agentPool.setAKSNodeCount(targetSize) - if err != nil { - return err - } - - agentPool.curSize = targetSize - agentPool.lastRefresh = time.Now() - return nil -} - -// IncreaseSize calls in the underlying SetSize to increase the size in response -// to a scale up. It calculates the expected size based on a delta provided as -// parameter -func (agentPool *AKSAgentPool) IncreaseSize(delta int) error { - if delta <= 0 { - return fmt.Errorf("size increase must be +ve") - } - currentSize, err := agentPool.TargetSize() - if err != nil { - return err - } - targetSize := currentSize + delta - if targetSize > agentPool.MaxSize() { - return fmt.Errorf("size-increasing request of %d is bigger than max size %d", targetSize, agentPool.MaxSize()) - } - return agentPool.SetSize(targetSize, false) -} - -// deleteNodesInternal calls the underlying vm service to delete the node. -// It should be called within lock protected. -func (agentPool *AKSAgentPool) deleteNodesInternal(providerIDs []string) (deleted int, err error) { - for _, providerID := range providerIDs { - klog.Infof("ProviderID got to delete: %s", providerID) - nodeName, err := agentPool.GetName(providerID) - if err != nil { - return deleted, err - } - klog.Infof("VM name got to delete: %s", nodeName) - - err = agentPool.util.DeleteVirtualMachine(agentPool.nodeResourceGroup, nodeName) - if err != nil { - klog.Errorf("Failed to delete virtual machine %q with error: %v", nodeName, err) - return deleted, err - } - - // increase the deleted count after delete VM succeed. - deleted++ - } - - return deleted, nil -} - -// DeleteNodes extracts the providerIDs from the node spec and calls into the internal -// delete method. -func (agentPool *AKSAgentPool) DeleteNodes(nodes []*apiv1.Node) error { - agentPool.mutex.Lock() - defer agentPool.mutex.Unlock() - - var providerIDs []string - for _, node := range nodes { - providerIDs = append(providerIDs, node.Spec.ProviderID) - } - - deleted, deleteError := agentPool.deleteNodesInternal(providerIDs) - // Update node count if there're some virtual machines got deleted. - if deleted != 0 { - targetSize := agentPool.curSize - deleted - err := agentPool.setSizeInternal(targetSize, true) - if err != nil { - klog.Errorf("Failed to set size for agent pool %q with error: %v", agentPool.Name, err) - } else { - klog.V(3).Infof("Size for agent pool %q has been updated to %d", agentPool.Name, targetSize) - } - } - return deleteError -} - -// IsAKSNode checks if the tag from the vm matches the agentPool name -func (agentPool *AKSAgentPool) IsAKSNode(tags map[string]*string) bool { - poolName := tags[aksManagedPoolNameTag] - if poolName == nil { - poolName = tags[legacyAKSPoolNameTag] - } - if poolName != nil { - klog.V(5).Infof("Matching agentPool name: %s with tag name: %s", agentPool.azureRef.Name, *poolName) - if strings.EqualFold(*poolName, agentPool.azureRef.Name) { - return true - } - } - return false -} - -// GetNodes extracts the node list from the underlying vm service and returns back -// equivalent providerIDs as list. -func (agentPool *AKSAgentPool) GetNodes() ([]string, error) { - ctx, cancel := getContextWithCancel() - defer cancel() - klog.V(6).Infof("GetNodes: starting list aks node pools in %s", agentPool.nodeResourceGroup) - vmList, rerr := agentPool.manager.azClient.virtualMachinesClient.List(ctx, agentPool.nodeResourceGroup) - klog.V(6).Infof("GetNodes: list finished, len(vmlist) = %d, err = %s", len(vmList), rerr.Error()) - if rerr != nil { - klog.Errorf("Azure client list vm error : %v", rerr.Error()) - return nil, rerr.Error() - } - var nodeArray []string - for _, node := range vmList { - klog.V(5).Infof("Node Name: %s, ID: %s", *node.Name, *node.ID) - if agentPool.IsAKSNode(node.Tags) { - providerID, err := convertResourceGroupNameToLower(agentPool.GetProviderID(*node.ID)) - if err != nil { - // This shouldn't happen. Log a waring message for tracking. - klog.Warningf("GetNodes.convertResourceGroupNameToLower failed with error: %v", err) - continue - } - - klog.V(5).Infof("Returning back the providerID: %s", providerID) - nodeArray = append(nodeArray, providerID) - } - } - return nodeArray, nil -} - -// DecreaseTargetSize requests the underlying service to decrease the node count. -func (agentPool *AKSAgentPool) DecreaseTargetSize(delta int) error { - if delta >= 0 { - klog.Errorf("Size decrease error: %d", delta) - return fmt.Errorf("size decrease must be negative") - } - currentSize, err := agentPool.TargetSize() - if err != nil { - klog.Error(err) - return err - } - klog.V(5).Infof("DecreaseTargetSize get current size %d for agent pool %q", currentSize, agentPool.Name) - - // Get the current nodes in the list - nodes, err := agentPool.GetNodes() - if err != nil { - klog.Error(err) - return err - } - - targetSize := currentSize + delta - klog.V(5).Infof("DecreaseTargetSize get target size %d for agent pool %q", targetSize, agentPool.Name) - if targetSize < len(nodes) { - return fmt.Errorf("attempt to delete existing nodes targetSize:%d delta:%d existingNodes: %d", - currentSize, delta, len(nodes)) - } - return agentPool.SetSize(targetSize, true) -} - -// Id returns the name of the agentPool -func (agentPool *AKSAgentPool) Id() string { - return agentPool.azureRef.Name -} - -// Debug returns a string with basic details of the agentPool -func (agentPool *AKSAgentPool) Debug() string { - return fmt.Sprintf("%s (%d:%d)", agentPool.Id(), agentPool.MinSize(), agentPool.MaxSize()) -} - -// Nodes returns the list of nodes in the agentPool. -func (agentPool *AKSAgentPool) Nodes() ([]cloudprovider.Instance, error) { - instanceNames, err := agentPool.GetNodes() - if err != nil { - return nil, err - } - instances := make([]cloudprovider.Instance, 0, len(instanceNames)) - for _, instanceName := range instanceNames { - instances = append(instances, cloudprovider.Instance{Id: instanceName}) - } - return instances, nil -} - -// TemplateNodeInfo is not implemented. -func (agentPool *AKSAgentPool) TemplateNodeInfo() (*schedulerframework.NodeInfo, error) { - return nil, cloudprovider.ErrNotImplemented -} - -// Exist is always true since we are initialized with an existing agentpool -func (agentPool *AKSAgentPool) Exist() bool { - return true -} - -// Create is returns already exists since we don't support the -// agent pool creation. -func (agentPool *AKSAgentPool) Create() (cloudprovider.NodeGroup, error) { - return nil, cloudprovider.ErrAlreadyExist -} - -// Delete is not implemented since we don't support agent pool -// deletion. -func (agentPool *AKSAgentPool) Delete() error { - return cloudprovider.ErrNotImplemented -} - -// Autoprovisioned is set to false to indicate that this code -// does not create agentPools by itself. -func (agentPool *AKSAgentPool) Autoprovisioned() bool { - return false -} - -// GetOptions returns NodeGroupAutoscalingOptions that should be used for this particular -// NodeGroup. Returning a nil will result in using default options. -func (agentPool *AKSAgentPool) GetOptions(defaults config.NodeGroupAutoscalingOptions) (*config.NodeGroupAutoscalingOptions, error) { - return nil, cloudprovider.ErrNotImplemented -} diff --git a/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool_test.go b/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool_test.go deleted file mode 100644 index 419cae024b87..000000000000 --- a/cluster-autoscaler/cloudprovider/azure/azure_kubernetes_service_pool_test.go +++ /dev/null @@ -1,423 +0,0 @@ -/* -Copyright 2020 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 azure - -import ( - "fmt" - "net/http" - "testing" - "time" - - "github.com/Azure/azure-sdk-for-go/services/compute/mgmt/2022-08-01/compute" - "github.com/Azure/azure-sdk-for-go/services/containerservice/mgmt/2021-10-01/containerservice" - "github.com/Azure/go-autorest/autorest/to" - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/assert" - - apiv1 "k8s.io/api/core/v1" - "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/containerserviceclient/mockcontainerserviceclient" - "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/interfaceclient/mockinterfaceclient" - "sigs.k8s.io/cloud-provider-azure/pkg/azureclients/vmclient/mockvmclient" - "sigs.k8s.io/cloud-provider-azure/pkg/retry" -) - -const ( - testAKSPoolName = "aks" -) - -var ( - errInternal = &retry.Error{HTTPStatusCode: http.StatusInternalServerError} - errInternalRaw = fmt.Errorf("Retriable: false, RetryAfter: 0s, HTTPStatusCode: 500, RawError: %w", nil) -) - -func getTestAKSPool(manager *AzureManager, name string) *AKSAgentPool { - return &AKSAgentPool{ - azureRef: azureRef{ - Name: name, - }, - manager: manager, - minSize: 1, - maxSize: 10, - resourceGroup: "rg", - nodeResourceGroup: "rg", - clusterName: "cluster", - curSize: 5, - util: &AzUtil{ - manager: manager, - }, - } -} - -func getExpectedManagedCluster() containerservice.ManagedCluster { - return containerservice.ManagedCluster{ - Name: to.StringPtr("cluster"), - ManagedClusterProperties: &containerservice.ManagedClusterProperties{ - AgentPoolProfiles: &[]containerservice.ManagedClusterAgentPoolProfile{ - { - Name: to.StringPtr(testAKSPoolName), - Count: to.Int32Ptr(1), - }, - }, - }, - } -} - -func TestSetNodeCount(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - mockAKSClient.EXPECT().CreateOrUpdate( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName, - gomock.Any(), - gomock.Any()).Return(nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - err := aksPool.SetNodeCount(3) - assert.NoError(t, err) - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), errInternal) - err = aksPool.SetNodeCount(3) - expectedErr := errInternalRaw - assert.Equal(t, expectedErr, err) - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(containerservice.ManagedCluster{ - ManagedClusterProperties: &containerservice.ManagedClusterProperties{ - AgentPoolProfiles: &[]containerservice.ManagedClusterAgentPoolProfile{}, - }, - }, nil) - err = aksPool.SetNodeCount(3) - expectedErr = fmt.Errorf("could not find pool with name: {aks}") - assert.Equal(t, expectedErr, err) - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - mockAKSClient.EXPECT().CreateOrUpdate( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName, - gomock.Any(), - gomock.Any()).Return(errInternal) - err = aksPool.SetNodeCount(3) - expectedErr = errInternalRaw - assert.Equal(t, expectedErr, err) -} - -func TestGetNodeCount(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - count, err := aksPool.GetNodeCount() - assert.Equal(t, 1, count) - assert.NoError(t, err) - - mockAKSClient.EXPECT().Get(gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), errInternal) - count, err = aksPool.GetNodeCount() - expectedErr := errInternalRaw - assert.Equal(t, -1, count) - assert.Equal(t, expectedErr, err) - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(containerservice.ManagedCluster{ - ManagedClusterProperties: &containerservice.ManagedClusterProperties{ - AgentPoolProfiles: &[]containerservice.ManagedClusterAgentPoolProfile{}, - }, - }, nil) - count, err = aksPool.GetNodeCount() - expectedErr = fmt.Errorf("could not find pool with name: {aks}") - assert.Equal(t, -1, count) - assert.Equal(t, expectedErr, err) - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(containerservice.ManagedCluster{}, errInternal) - count, err = aksPool.GetNodeCount() - assert.Equal(t, -1, count) - assert.Equal(t, err, errInternalRaw) -} - -func TestAKSTargetSize(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get(gomock.Any(), aksPool.resourceGroup, aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - aksPool.lastRefresh = time.Now() - size, err := aksPool.TargetSize() - assert.Equal(t, 5, size) - assert.NoError(t, err) - - aksPool.lastRefresh = time.Now().Add(-1 * 20 * time.Second) - size, err = aksPool.TargetSize() - assert.Equal(t, 1, size) - assert.NoError(t, err) -} - -func TestAKSSetSize(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - tests := []struct { - targetSize int - isScalingDown bool - expectedSize int - expectedErr error - }{ - { - targetSize: 0, - isScalingDown: true, - expectedSize: 5, - expectedErr: fmt.Errorf("size-decreasing request of 0 is smaller than min size 1"), - }, - { - targetSize: 3, - expectedSize: 3, - }, - } - - for _, test := range tests { - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil).MaxTimes(1) - mockAKSClient.EXPECT().CreateOrUpdate( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName, - gomock.Any(), - gomock.Any()).Return(nil).MaxTimes(1) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - err := aksPool.SetSize(test.targetSize, test.isScalingDown) - assert.Equal(t, test.expectedErr, err) - assert.Equal(t, test.expectedSize, aksPool.curSize) - } -} - -func TestAKSIncreaseSize(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - aksPool.lastRefresh = time.Now() - - err := aksPool.IncreaseSize(-1) - assert.Equal(t, fmt.Errorf("size increase must be +ve"), err) - - err = aksPool.IncreaseSize(6) - assert.Equal(t, fmt.Errorf("size-increasing request of 11 is bigger than max size 10"), err) - - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get(gomock.Any(), aksPool.resourceGroup, aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - mockAKSClient.EXPECT().CreateOrUpdate(gomock.Any(), aksPool.resourceGroup, aksPool.clusterName, gomock.Any(), gomock.Any()).Return(nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - err = aksPool.IncreaseSize(3) - assert.NoError(t, err) -} - -func TestIsAKSNode(t *testing.T) { - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - tags := map[string]*string{aksManagedPoolNameTag: to.StringPtr(testAKSPoolName)} - isAKSNode := aksPool.IsAKSNode(tags) - assert.True(t, isAKSNode) - - tags = map[string]*string{aksManagedPoolNameTag: to.StringPtr("fake")} - isAKSNode = aksPool.IsAKSNode(tags) - assert.False(t, isAKSNode) - - tags = map[string]*string{legacyAKSPoolNameTag: to.StringPtr(testAKSPoolName)} - isAKSNode = aksPool.IsAKSNode(tags) - assert.True(t, isAKSNode) -} - -func TestDeleteNodesAKS(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - expectedVMs := []compute.VirtualMachine{ - { - ID: to.StringPtr("id1"), - Name: to.StringPtr("vm1"), - VirtualMachineProperties: &compute.VirtualMachineProperties{ - NetworkProfile: &compute.NetworkProfile{ - NetworkInterfaces: &[]compute.NetworkInterfaceReference{ - {ID: to.StringPtr("id")}, - }, - }, - StorageProfile: &compute.StorageProfile{ - OsDisk: &compute.OSDisk{ - ManagedDisk: &compute.ManagedDiskParameters{}, - }, - }, - }, - }, - } - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockVMClient := mockvmclient.NewMockInterface(ctrl) - mockVMClient.EXPECT().List(gomock.Any(), aksPool.nodeResourceGroup).Return(expectedVMs, nil) - mockVMClient.EXPECT().Get( - gomock.Any(), - aksPool.nodeResourceGroup, - *expectedVMs[0].Name, - gomock.Any()).Return(expectedVMs[0], nil) - mockVMClient.EXPECT().Delete(gomock.Any(), aksPool.nodeResourceGroup, *expectedVMs[0].Name) - aksPool.manager.azClient.virtualMachinesClient = mockVMClient - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - mockAKSClient.EXPECT().CreateOrUpdate( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName, - gomock.Any(), - gomock.Any()) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - mockNICClient := mockinterfaceclient.NewMockInterface(ctrl) - mockNICClient.EXPECT().Delete( - gomock.Any(), - aksPool.resourceGroup, - "id").Return(nil) - aksPool.manager.azClient.interfacesClient = mockNICClient - - nodes := []*apiv1.Node{ - { - Spec: apiv1.NodeSpec{ - ProviderID: "id1", - }, - }, - } - err := aksPool.DeleteNodes(nodes) - assert.Equal(t, 4, aksPool.curSize) - assert.NoError(t, err) -} - -func TestAKSNodes(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - expectedVMs := []compute.VirtualMachine{ - { - Name: to.StringPtr("name"), - ID: to.StringPtr("/subscriptions/sub/resourceGroups/rg/providers/provider/vm1"), - Tags: map[string]*string{aksManagedPoolNameTag: to.StringPtr(testAKSPoolName)}, - }, - } - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - mockVMClient := mockvmclient.NewMockInterface(ctrl) - mockVMClient.EXPECT().List(gomock.Any(), aksPool.nodeResourceGroup).Return(expectedVMs, nil) - aksPool.manager.azClient.virtualMachinesClient = mockVMClient - - instances, err := aksPool.Nodes() - assert.Equal(t, 1, len(instances)) - assert.NoError(t, err) - - mockVMClient.EXPECT().List(gomock.Any(), aksPool.nodeResourceGroup).Return([]compute.VirtualMachine{}, errInternal) - instances, err = aksPool.Nodes() - expectedErr := errInternalRaw - assert.Nil(t, instances) - assert.Equal(t, expectedErr, err) - - expectedVMs[0].ID = to.StringPtr("fakeID") - mockVMClient.EXPECT().List(gomock.Any(), aksPool.nodeResourceGroup).Return(expectedVMs, nil) - instances, err = aksPool.Nodes() - assert.Equal(t, 0, len(instances)) - assert.NoError(t, err) -} - -func TestAKSDecreaseTargetSize(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - aksPool := getTestAKSPool(newTestAzureManager(t), testAKSPoolName) - - err := aksPool.DecreaseTargetSize(1) - expectedErr := fmt.Errorf("size decrease must be negative") - assert.Equal(t, expectedErr, err) - - aksPool.lastRefresh = time.Now().Add(-1 * 20 * time.Second) - - mockAKSClient := mockcontainerserviceclient.NewMockInterface(ctrl) - expectedMC := getExpectedManagedCluster() - (*expectedMC.AgentPoolProfiles)[0].Count = to.Int32Ptr(2) - mockAKSClient.EXPECT().Get(gomock.Any(), aksPool.resourceGroup, aksPool.clusterName).Return(expectedMC, nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - expectedVMs := []compute.VirtualMachine{ - { - Name: to.StringPtr("name"), - ID: to.StringPtr("/subscriptions/sub/resourceGroups/rg/providers/provider/vm1"), - Tags: map[string]*string{aksManagedPoolNameTag: to.StringPtr(testAKSPoolName)}, - }, - } - mockVMClient := mockvmclient.NewMockInterface(ctrl) - mockVMClient.EXPECT().List(gomock.Any(), aksPool.nodeResourceGroup).Return(expectedVMs, nil) - aksPool.manager.azClient.virtualMachinesClient = mockVMClient - - mockAKSClient.EXPECT().Get( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName).Return(getExpectedManagedCluster(), nil) - mockAKSClient.EXPECT().CreateOrUpdate( - gomock.Any(), - aksPool.resourceGroup, - aksPool.clusterName, - gomock.Any(), - gomock.Any()).Return(nil) - aksPool.manager.azClient.managedKubernetesServicesClient = mockAKSClient - - err = aksPool.DecreaseTargetSize(-1) - assert.Equal(t, 1, aksPool.curSize) - assert.NoError(t, err) -} diff --git a/cluster-autoscaler/cloudprovider/azure/azure_manager.go b/cluster-autoscaler/cloudprovider/azure/azure_manager.go index dc04e1558281..f2fc79b1ca9c 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_manager.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_manager.go @@ -35,7 +35,6 @@ import ( const ( vmTypeVMSS = "vmss" vmTypeStandard = "standard" - vmTypeAKS = "aks" scaleToZeroSupportedStandard = false scaleToZeroSupportedVMSS = true @@ -153,8 +152,6 @@ func (m *AzureManager) buildNodeGroupFromSpec(spec string) (cloudprovider.NodeGr return NewAgentPool(s, m) case vmTypeVMSS: return NewScaleSet(s, m, -1) - case vmTypeAKS: - return NewAKSAgentPool(s, m) default: return nil, fmt.Errorf("vmtype %s not supported", m.config.VMType) } diff --git a/cluster-autoscaler/cloudprovider/azure/azure_manager_test.go b/cluster-autoscaler/cloudprovider/azure/azure_manager_test.go index be92d31d2427..209ade185582 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_manager_test.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_manager_test.go @@ -355,8 +355,6 @@ func TestCreateAzureManagerWithNilConfig(t *testing.T) { AADClientCertPath: "aadClientCertPath", AADClientCertPassword: "aadClientCertPassword", Deployment: "deployment", - ClusterName: "clusterName", - NodeResourceGroup: "resourcegroup", UseManagedIdentityExtension: true, UserAssignedIdentityID: "UserAssignedIdentityID", VmssCacheTTL: 100, @@ -425,15 +423,15 @@ func TestCreateAzureManagerWithNilConfig(t *testing.T) { t.Setenv("LOCATION", "southeastasia") t.Setenv("ARM_SUBSCRIPTION_ID", "subscriptionId") t.Setenv("ARM_RESOURCE_GROUP", "resourceGroup") + t.Setenv("AZURE_TENANT_ID", "tenantId") t.Setenv("ARM_TENANT_ID", "tenantId") + t.Setenv("AZURE_CLIENT_ID", "aadClientId") t.Setenv("ARM_CLIENT_ID", "aadClientId") t.Setenv("ARM_CLIENT_SECRET", "aadClientSecret") t.Setenv("ARM_VM_TYPE", "vmss") t.Setenv("ARM_CLIENT_CERT_PATH", "aadClientCertPath") t.Setenv("ARM_CLIENT_CERT_PASSWORD", "aadClientCertPassword") t.Setenv("ARM_DEPLOYMENT", "deployment") - t.Setenv("AZURE_CLUSTER_NAME", "clusterName") - t.Setenv("AZURE_NODE_RESOURCE_GROUP", "resourcegroup") t.Setenv("ARM_USE_MANAGED_IDENTITY_EXTENSION", "true") t.Setenv("ARM_USER_ASSIGNED_IDENTITY_ID", "UserAssignedIdentityID") t.Setenv("AZURE_VMSS_CACHE_TTL", "100") @@ -699,21 +697,15 @@ func TestGetFilteredAutoscalingGroupsWithInvalidVMType(t *testing.T) { mockVMSSClient := mockvmssclient.NewMockInterface(ctrl) mockVMSSClient.EXPECT().List(gomock.Any(), manager.config.ResourceGroup).Return(expectedScaleSets, nil).AnyTimes() manager.azClient.virtualMachineScaleSetsClient = mockVMSSClient - manager.config.VMType = vmTypeAKS + manager.config.VMType = "invalidVMType" specs, err := ParseLabelAutoDiscoverySpecs(ngdo) assert.NoError(t, err) - expectedErr := fmt.Errorf("vmType \"aks\" does not support autodiscovery") + expectedErr := fmt.Errorf("vmType \"invalidVMType\" does not support autodiscovery") asgs, err2 := manager.getFilteredNodeGroups(specs) assert.Nil(t, asgs) assert.Equal(t, expectedErr, err2, "Not match, expected: %v, actual: %v", expectedErr, err2) - - manager.config.VMType = "invalidVMType" - expectedErr = fmt.Errorf("vmType \"invalidVMType\" does not support autodiscovery") - asgs, err2 = manager.getFilteredNodeGroups(specs) - assert.Nil(t, asgs) - assert.Equal(t, expectedErr, err2, "Not match, expected: %v, actual: %v", expectedErr, err2) } func TestFetchAutoAsgsVmss(t *testing.T) { diff --git a/cluster-autoscaler/cloudprovider/azure/azure_util_test.go b/cluster-autoscaler/cloudprovider/azure/azure_util_test.go index d4c030394e80..983693d49b9f 100644 --- a/cluster-autoscaler/cloudprovider/azure/azure_util_test.go +++ b/cluster-autoscaler/cloudprovider/azure/azure_util_test.go @@ -96,7 +96,7 @@ func TestWindowsVMNameParts(t *testing.T) { t.Fatalf("incorrect poolPrefix. expected=%s actual=%s", d.expectedPoolPrefix, poolPrefix) } if orch != d.expectedOrch { - t.Fatalf("incorrect aks string. expected=%s actual=%s", d.expectedOrch, orch) + t.Fatalf("incorrect orchestrator string. expected=%s actual=%s", d.expectedOrch, orch) } if poolIndex != d.expectedPoolIndex { t.Fatalf("incorrect poolIndex. expected=%d actual=%d", d.expectedPoolIndex, poolIndex) diff --git a/cluster-autoscaler/cloudprovider/azure/examples/cluster-autoscaler-aks.yaml b/cluster-autoscaler/cloudprovider/azure/examples/cluster-autoscaler-aks.yaml index 0b8558a43156..c9c40f6d6df7 100644 --- a/cluster-autoscaler/cloudprovider/azure/examples/cluster-autoscaler-aks.yaml +++ b/cluster-autoscaler/cloudprovider/azure/examples/cluster-autoscaler-aks.yaml @@ -128,8 +128,6 @@ data: SubscriptionID: TenantID: VMType: QUtTCg== - ClusterName: - NodeResourceGroup: kind: Secret metadata: name: cluster-autoscaler-azure @@ -202,14 +200,4 @@ spec: secretKeyRef: key: VMType name: cluster-autoscaler-azure - - name: AZURE_CLUSTER_NAME - valueFrom: - secretKeyRef: - key: ClusterName - name: cluster-autoscaler-azure - - name: AZURE_NODE_RESOURCE_GROUP - valueFrom: - secretKeyRef: - key: NodeResourceGroup - name: cluster-autoscaler-azure restartPolicy: Always