Merge pull request #461 from MaciekPytel/gpu_unready_fix
Consider GPU nodes unready until allocatable GPU is > 0
This commit is contained in:
		
						commit
						ded016dfd8
					
				| 
						 | 
				
			
			@ -21,6 +21,7 @@ import (
 | 
			
		|||
	"time"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
	kubeletapis "k8s.io/kubernetes/pkg/kubelet/apis"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -156,7 +157,7 @@ func getAdditionalPrice(resources apiv1.ResourceList, startTime time.Time, endTi
 | 
			
		|||
	}
 | 
			
		||||
	hours := getHours(startTime, endTime)
 | 
			
		||||
	price := 0.0
 | 
			
		||||
	gpu := resources[apiv1.ResourceNvidiaGPU]
 | 
			
		||||
	gpu := resources[gpu.ResourceNvidiaGPU]
 | 
			
		||||
	price += float64(gpu.MilliValue()) / 1000.0 * gpuPricePerHour * hours
 | 
			
		||||
	return price
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,8 +21,8 @@ import (
 | 
			
		|||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/resource"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
| 
						 | 
				
			
			@ -69,14 +69,14 @@ func TestGetNodePrice(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	// regular with gpu
 | 
			
		||||
	node4 := BuildTestNode("sillyname4", 8000, 30*1024*1024*1024)
 | 
			
		||||
	node4.Status.Capacity[apiv1.ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	node4.Status.Capacity[gpu.ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	node4.Labels = labels1
 | 
			
		||||
	price4, err := model.NodePrice(node4, now, now.Add(time.Hour))
 | 
			
		||||
 | 
			
		||||
	// preemptable with gpu
 | 
			
		||||
	node5 := BuildTestNode("sillyname5", 8000, 30*1024*1024*1024)
 | 
			
		||||
	node5.Labels = labels2
 | 
			
		||||
	node5.Status.Capacity[apiv1.ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	node5.Status.Capacity[gpu.ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	price5, err := model.NodePrice(node5, now, now.Add(time.Hour))
 | 
			
		||||
 | 
			
		||||
	// Nodes with GPU are way more expensive than regular.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -30,6 +30,7 @@ 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/utils/gpu"
 | 
			
		||||
	kubeletapis "k8s.io/kubernetes/pkg/kubelet/apis"
 | 
			
		||||
 | 
			
		||||
	"github.com/golang/glog"
 | 
			
		||||
| 
						 | 
				
			
			@ -38,7 +39,6 @@ import (
 | 
			
		|||
const (
 | 
			
		||||
	mbPerGB           = 1000
 | 
			
		||||
	millicoresPerCore = 1000
 | 
			
		||||
	resourceNvidiaGPU = "nvidia.com/gpu"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// builds templates for gce cloud provider
 | 
			
		||||
| 
						 | 
				
			
			@ -98,7 +98,7 @@ func (t *templateBuilder) buildCapacity(machineType string, accelerators []*gce.
 | 
			
		|||
	capacity[apiv1.ResourceMemory] = *resource.NewQuantity(mem, resource.DecimalSI)
 | 
			
		||||
 | 
			
		||||
	if accelerators != nil && len(accelerators) > 0 {
 | 
			
		||||
		capacity[resourceNvidiaGPU] = *resource.NewQuantity(t.getAcceleratorCount(accelerators), resource.DecimalSI)
 | 
			
		||||
		capacity[gpu.ResourceNvidiaGPU] = *resource.NewQuantity(t.getAcceleratorCount(accelerators), resource.DecimalSI)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return capacity, nil
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -20,13 +20,16 @@ import (
 | 
			
		|||
	"fmt"
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	gpuUtils "k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
 | 
			
		||||
	gce "google.golang.org/api/compute/v1"
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/resource"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	kubeletapis "k8s.io/kubernetes/pkg/kubelet/apis"
 | 
			
		||||
	"k8s.io/kubernetes/pkg/quota"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func TestBuildNodeFromTemplateSetsResources(t *testing.T) {
 | 
			
		||||
| 
						 | 
				
			
			@ -486,7 +489,7 @@ func makeResourceList(cpu string, memory string, gpu int64) (apiv1.ResourceList,
 | 
			
		|||
		if err != nil {
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
		result[resourceNvidiaGPU] = resultGpu
 | 
			
		||||
		result[gpuUtils.ResourceNvidiaGPU] = resultGpu
 | 
			
		||||
	}
 | 
			
		||||
	return result, nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -38,7 +38,7 @@ import (
 | 
			
		|||
 | 
			
		||||
const (
 | 
			
		||||
	// MaxNodeStartupTime is the maximum time from the moment the node is registered to the time the node is ready.
 | 
			
		||||
	MaxNodeStartupTime = 5 * time.Minute
 | 
			
		||||
	MaxNodeStartupTime = 15 * time.Minute
 | 
			
		||||
 | 
			
		||||
	// MaxStatusSettingDelayAfterCreation is the maximum time for node to set its initial status after the
 | 
			
		||||
	// node is registered.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -19,16 +19,17 @@ package core
 | 
			
		|||
import (
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/clusterstate/utils"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/metrics"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	kube_client "k8s.io/client-go/kubernetes"
 | 
			
		||||
	kube_record "k8s.io/client-go/tools/record"
 | 
			
		||||
 | 
			
		||||
	"github.com/golang/glog"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// StaticAutoscaler is an autoscaler which has all the core functionality of a CA but without the reconfiguration feature
 | 
			
		||||
| 
						 | 
				
			
			@ -101,17 +102,6 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) errors.AutoscalerError
 | 
			
		|||
		return errors.ToAutoscalerError(errors.CloudProviderError, err)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	readyNodes, err := readyNodeLister.List()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		glog.Errorf("Failed to list ready nodes: %v", err)
 | 
			
		||||
		return errors.ToAutoscalerError(errors.ApiCallError, err)
 | 
			
		||||
	}
 | 
			
		||||
	if len(readyNodes) == 0 {
 | 
			
		||||
		glog.Warningf("No ready nodes in the cluster")
 | 
			
		||||
		scaleDown.CleanUpUnneededNodes()
 | 
			
		||||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	allNodes, err := allNodeLister.List()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		glog.Errorf("Failed to list all nodes: %v", err)
 | 
			
		||||
| 
						 | 
				
			
			@ -123,6 +113,23 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) errors.AutoscalerError
 | 
			
		|||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	readyNodes, err := readyNodeLister.List()
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		glog.Errorf("Failed to list ready nodes: %v", err)
 | 
			
		||||
		return errors.ToAutoscalerError(errors.ApiCallError, err)
 | 
			
		||||
	}
 | 
			
		||||
	// Handle GPU case - allocatable GPU may be equal to 0 up to 15 minutes after
 | 
			
		||||
	// node registers as ready. See https://github.com/kubernetes/kubernetes/issues/54959
 | 
			
		||||
	// Treat those nodes as unready until GPU actually becomes available and let
 | 
			
		||||
	// our normal handling for booting up nodes deal with this.
 | 
			
		||||
	// TODO: Remove this call when we handle dynamically provisioned resources.
 | 
			
		||||
	allNodes, readyNodes = gpu.FilterOutNodesWithUnreadyGpus(allNodes, readyNodes)
 | 
			
		||||
	if len(readyNodes) == 0 {
 | 
			
		||||
		glog.Warningf("No ready nodes in the cluster")
 | 
			
		||||
		scaleDown.CleanUpUnneededNodes()
 | 
			
		||||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	err = a.ClusterStateRegistry.UpdateNodes(allNodes, currentTime)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		glog.Errorf("Failed to update node registry: %v", err)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -0,0 +1,94 @@
 | 
			
		|||
/*
 | 
			
		||||
Copyright 2017 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 gpu
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/kubernetes/pkg/api"
 | 
			
		||||
 | 
			
		||||
	"github.com/golang/glog"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
const (
 | 
			
		||||
	// ResourceNvidiaGPU is the name of the Nvidia GPU resource.
 | 
			
		||||
	ResourceNvidiaGPU = "nvidia.com/gpu"
 | 
			
		||||
	// GPULabel is the label added to nodes with GPU resource on GKE.
 | 
			
		||||
	GPULabel = "cloud.google.com/gke-accelerator"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// FilterOutNodesWithUnreadyGpus removes nodes that should have GPU, but don't have it in allocatable
 | 
			
		||||
// from ready nodes list and updates their status to unready on all nodes list.
 | 
			
		||||
// This is a hack/workaround for nodes with GPU coming up without installed drivers, resulting
 | 
			
		||||
// in GPU missing from their allocatable and capacity.
 | 
			
		||||
func FilterOutNodesWithUnreadyGpus(allNodes, readyNodes []*apiv1.Node) ([]*apiv1.Node, []*apiv1.Node) {
 | 
			
		||||
	newAllNodes := make([]*apiv1.Node, 0)
 | 
			
		||||
	newReadyNodes := make([]*apiv1.Node, 0)
 | 
			
		||||
	nodesWithUnreadyGpu := make(map[string]*apiv1.Node)
 | 
			
		||||
	for _, node := range readyNodes {
 | 
			
		||||
		isUnready := false
 | 
			
		||||
		_, hasGpuLabel := node.Labels[GPULabel]
 | 
			
		||||
		gpuAllocatable, hasGpuAllocatable := node.Status.Allocatable[ResourceNvidiaGPU]
 | 
			
		||||
		// We expect node to have GPU based on label, but it doesn't show up
 | 
			
		||||
		// on node object. Assume the node is still not fully started (installing
 | 
			
		||||
		// GPU drivers).
 | 
			
		||||
		if hasGpuLabel && (!hasGpuAllocatable || gpuAllocatable.IsZero()) {
 | 
			
		||||
			newNode, err := getUnreadyNodeCopy(node)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				glog.Errorf("Failed to override status of node %v with unready GPU: %v",
 | 
			
		||||
					node.Name, err)
 | 
			
		||||
			} else {
 | 
			
		||||
				glog.V(3).Infof("Overriding status of node %v, which seems to have unready GPU",
 | 
			
		||||
					node.Name)
 | 
			
		||||
				nodesWithUnreadyGpu[newNode.Name] = newNode
 | 
			
		||||
				isUnready = true
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
		if !isUnready {
 | 
			
		||||
			newReadyNodes = append(newReadyNodes, node)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	// Override any node with unready GPU with its "unready" copy
 | 
			
		||||
	for _, node := range allNodes {
 | 
			
		||||
		if newNode, found := nodesWithUnreadyGpu[node.Name]; found {
 | 
			
		||||
			newAllNodes = append(newAllNodes, newNode)
 | 
			
		||||
		} else {
 | 
			
		||||
			newAllNodes = append(newAllNodes, node)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return newAllNodes, newReadyNodes
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func getUnreadyNodeCopy(node *apiv1.Node) (*apiv1.Node, error) {
 | 
			
		||||
	nodeCopy, err := api.Scheme.DeepCopy(node)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return nil, err
 | 
			
		||||
	}
 | 
			
		||||
	newNode := nodeCopy.(*apiv1.Node)
 | 
			
		||||
	newReadyCondition := apiv1.NodeCondition{
 | 
			
		||||
		Type:               apiv1.NodeReady,
 | 
			
		||||
		Status:             apiv1.ConditionFalse,
 | 
			
		||||
		LastTransitionTime: node.CreationTimestamp,
 | 
			
		||||
	}
 | 
			
		||||
	newNodeConditions := []apiv1.NodeCondition{newReadyCondition}
 | 
			
		||||
	for _, condition := range newNode.Status.Conditions {
 | 
			
		||||
		if condition.Type != apiv1.NodeReady {
 | 
			
		||||
			newNodeConditions = append(newNodeConditions, condition)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	newNode.Status.Conditions = newNodeConditions
 | 
			
		||||
	return newNode, nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			@ -0,0 +1,151 @@
 | 
			
		|||
/*
 | 
			
		||||
Copyright 2017 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 gpu
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"testing"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/resource"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
 | 
			
		||||
	"github.com/stretchr/testify/assert"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func TestFilterOutNodesWithUnreadyGpus(t *testing.T) {
 | 
			
		||||
	start := time.Now()
 | 
			
		||||
	later := start.Add(10 * time.Minute)
 | 
			
		||||
	expectedReadiness := make(map[string]bool)
 | 
			
		||||
	gpuLabels := map[string]string{
 | 
			
		||||
		GPULabel: "nvidia-tesla-k80",
 | 
			
		||||
	}
 | 
			
		||||
	readyCondition := apiv1.NodeCondition{
 | 
			
		||||
		Type:               apiv1.NodeReady,
 | 
			
		||||
		Status:             apiv1.ConditionTrue,
 | 
			
		||||
		LastTransitionTime: metav1.NewTime(later),
 | 
			
		||||
	}
 | 
			
		||||
	unreadyCondition := apiv1.NodeCondition{
 | 
			
		||||
		Type:               apiv1.NodeReady,
 | 
			
		||||
		Status:             apiv1.ConditionFalse,
 | 
			
		||||
		LastTransitionTime: metav1.NewTime(later),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	nodeGpuReady := &apiv1.Node{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:              "nodeGpuReady",
 | 
			
		||||
			Labels:            gpuLabels,
 | 
			
		||||
			CreationTimestamp: metav1.NewTime(start),
 | 
			
		||||
		},
 | 
			
		||||
		Status: apiv1.NodeStatus{
 | 
			
		||||
			Capacity:    apiv1.ResourceList{},
 | 
			
		||||
			Allocatable: apiv1.ResourceList{},
 | 
			
		||||
			Conditions:  []apiv1.NodeCondition{readyCondition},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	nodeGpuReady.Status.Allocatable[ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	nodeGpuReady.Status.Capacity[ResourceNvidiaGPU] = *resource.NewQuantity(1, resource.DecimalSI)
 | 
			
		||||
	expectedReadiness[nodeGpuReady.Name] = true
 | 
			
		||||
 | 
			
		||||
	nodeGpuUnready := &apiv1.Node{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:              "nodeGpuUnready",
 | 
			
		||||
			Labels:            gpuLabels,
 | 
			
		||||
			CreationTimestamp: metav1.NewTime(start),
 | 
			
		||||
		},
 | 
			
		||||
		Status: apiv1.NodeStatus{
 | 
			
		||||
			Capacity:    apiv1.ResourceList{},
 | 
			
		||||
			Allocatable: apiv1.ResourceList{},
 | 
			
		||||
			Conditions:  []apiv1.NodeCondition{readyCondition},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	nodeGpuUnready.Status.Allocatable[ResourceNvidiaGPU] = *resource.NewQuantity(0, resource.DecimalSI)
 | 
			
		||||
	nodeGpuUnready.Status.Capacity[ResourceNvidiaGPU] = *resource.NewQuantity(0, resource.DecimalSI)
 | 
			
		||||
	expectedReadiness[nodeGpuUnready.Name] = false
 | 
			
		||||
 | 
			
		||||
	nodeGpuUnready2 := &apiv1.Node{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:              "nodeGpuUnready2",
 | 
			
		||||
			Labels:            gpuLabels,
 | 
			
		||||
			CreationTimestamp: metav1.NewTime(start),
 | 
			
		||||
		},
 | 
			
		||||
		Status: apiv1.NodeStatus{
 | 
			
		||||
			Conditions: []apiv1.NodeCondition{readyCondition},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	expectedReadiness[nodeGpuUnready2.Name] = false
 | 
			
		||||
 | 
			
		||||
	nodeNoGpuReady := &apiv1.Node{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:              "nodeNoGpuReady",
 | 
			
		||||
			Labels:            make(map[string]string),
 | 
			
		||||
			CreationTimestamp: metav1.NewTime(start),
 | 
			
		||||
		},
 | 
			
		||||
		Status: apiv1.NodeStatus{
 | 
			
		||||
			Conditions: []apiv1.NodeCondition{readyCondition},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	expectedReadiness[nodeNoGpuReady.Name] = true
 | 
			
		||||
 | 
			
		||||
	nodeNoGpuUnready := &apiv1.Node{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:              "nodeNoGpuUnready",
 | 
			
		||||
			Labels:            make(map[string]string),
 | 
			
		||||
			CreationTimestamp: metav1.NewTime(start),
 | 
			
		||||
		},
 | 
			
		||||
		Status: apiv1.NodeStatus{
 | 
			
		||||
			Conditions: []apiv1.NodeCondition{unreadyCondition},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	expectedReadiness[nodeNoGpuUnready.Name] = false
 | 
			
		||||
 | 
			
		||||
	initialReadyNodes := []*apiv1.Node{
 | 
			
		||||
		nodeGpuReady,
 | 
			
		||||
		nodeGpuUnready,
 | 
			
		||||
		nodeGpuUnready2,
 | 
			
		||||
		nodeNoGpuReady,
 | 
			
		||||
	}
 | 
			
		||||
	initialAllNodes := []*apiv1.Node{
 | 
			
		||||
		nodeGpuReady,
 | 
			
		||||
		nodeGpuUnready,
 | 
			
		||||
		nodeGpuUnready2,
 | 
			
		||||
		nodeNoGpuReady,
 | 
			
		||||
		nodeNoGpuUnready,
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	newAllNodes, newReadyNodes := FilterOutNodesWithUnreadyGpus(initialAllNodes, initialReadyNodes)
 | 
			
		||||
 | 
			
		||||
	foundInReady := make(map[string]bool)
 | 
			
		||||
	for _, node := range newReadyNodes {
 | 
			
		||||
		foundInReady[node.Name] = true
 | 
			
		||||
		assert.True(t, expectedReadiness[node.Name], fmt.Sprintf("Node %s found in ready nodes list (it shouldn't be there)", node.Name))
 | 
			
		||||
	}
 | 
			
		||||
	for nodeName, expected := range expectedReadiness {
 | 
			
		||||
		if expected {
 | 
			
		||||
			assert.True(t, foundInReady[nodeName], fmt.Sprintf("Node %s expected ready, but not found in ready nodes list", nodeName))
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	for _, node := range newAllNodes {
 | 
			
		||||
		assert.Equal(t, len(node.Status.Conditions), 1)
 | 
			
		||||
		if expectedReadiness[node.Name] {
 | 
			
		||||
			assert.Equal(t, node.Status.Conditions[0].Status, apiv1.ConditionTrue, fmt.Sprintf("Unexpected ready condition value for node %s", node.Name))
 | 
			
		||||
		} else {
 | 
			
		||||
			assert.Equal(t, node.Status.Conditions[0].Status, apiv1.ConditionFalse, fmt.Sprintf("Unexpected ready condition value for node %s", node.Name))
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
		Loading…
	
		Reference in New Issue