Tainting unneeded nodes as PreferNoSchedule

This commit is contained in:
Jacek Kaniuk 2019-01-11 13:58:14 +01:00
parent 4a2ddef96e
commit 0c64e0932a
10 changed files with 565 additions and 69 deletions

View File

@ -121,4 +121,9 @@ type AutoscalingOptions struct {
Regional bool
// Pods newer than this will not be considered as unschedulable for scale-up.
NewPodScaleUpDelay time.Duration
// MaxBulkSoftTaint sets the maximum number of nodes that can be (un)tainted PreferNoSchedule during single scaling down run.
// Value of 0 turns turn off such tainting.
MaxBulkSoftTaintCount int
// MaxBulkSoftTaintTime sets the maximum duration of single run of PreferNoSchedule tainting.
MaxBulkSoftTaintTime time.Duration
}

View File

@ -76,6 +76,9 @@ type NodeDeleteStatus struct {
nodeDeleteResults map[string]error
}
// Get current time. Proxy for unit tests.
var now func() time.Time = time.Now
// IsDeleteInProgress returns true if a node is being deleted.
func (n *NodeDeleteStatus) IsDeleteInProgress() bool {
n.Lock()
@ -572,6 +575,50 @@ func (sd *ScaleDown) mapNodesToStatusScaleDownNodes(nodes []*apiv1.Node, nodeGro
return result
}
// SoftTaintUnneededNodes manage soft taints of unneeded nodes.
func (sd *ScaleDown) SoftTaintUnneededNodes(allNodes []*apiv1.Node) (errors []error) {
defer metrics.UpdateDurationFromStart(metrics.ScaleDownSoftTaintUnneeded, time.Now())
apiCallBudget := sd.context.AutoscalingOptions.MaxBulkSoftTaintCount
timeBudget := sd.context.AutoscalingOptions.MaxBulkSoftTaintTime
skippedNodes := 0
startTime := now()
for _, node := range allNodes {
if deletetaint.HasToBeDeletedTaint(node) {
// Do not consider nodes that are scheduled to be deleted
continue
}
alreadyTainted := deletetaint.HasDeletionCandidateTaint(node)
_, unneeded := sd.unneededNodes[node.Name]
// Check if expected taints match existing taints
if unneeded != alreadyTainted {
if apiCallBudget <= 0 || now().Sub(startTime) >= timeBudget {
skippedNodes++
continue
}
apiCallBudget--
if unneeded && !alreadyTainted {
err := deletetaint.MarkDeletionCandidate(node, sd.context.ClientSet)
if err != nil {
errors = append(errors, err)
klog.Warningf("Soft taint on %s adding error %v", node.Name, err)
}
}
if !unneeded && alreadyTainted {
_, err := deletetaint.CleanDeletionCandidate(node, sd.context.ClientSet)
if err != nil {
errors = append(errors, err)
klog.Warningf("Soft taint on %s removal error %v", node.Name, err)
}
}
}
}
if skippedNodes > 0 {
klog.V(4).Infof("Skipped adding/removing soft taints on %v nodes - API call limit exceeded", skippedNodes)
}
return
}
// TryToScaleDown tries to scale down the cluster. It returns a result inside a ScaleDownStatus indicating if any node was
// removed and error if such occurred.
func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, pdbs []*policyv1.PodDisruptionBudget, currentTime time.Time) (*status.ScaleDownStatus, errors.AutoscalerError) {
@ -1011,21 +1058,6 @@ func drainNode(node *apiv1.Node, pods []*apiv1.Pod, client kube_client.Interface
errors.TransientError, "Failed to drain node %s/%s: pods remaining after timeout", node.Namespace, node.Name)
}
// cleanToBeDeleted cleans ToBeDeleted taints.
func cleanToBeDeleted(nodes []*apiv1.Node, client kube_client.Interface, recorder kube_record.EventRecorder) {
for _, node := range nodes {
cleaned, err := deletetaint.CleanToBeDeleted(node, client)
if err != nil {
klog.Warningf("Error while releasing taints on node %v: %v", node.Name, err)
recorder.Eventf(node, apiv1.EventTypeWarning, "ClusterAutoscalerCleanup",
"failed to clean toBeDeletedTaint: %v", err)
} else if cleaned {
klog.V(1).Infof("Successfully released toBeDeletedTaint on node %v", node.Name)
recorder.Eventf(node, apiv1.EventTypeNormal, "ClusterAutoscalerCleanup", "marking the node as schedulable")
}
}
}
// Removes the given node from cloud provider. No extra pre-deletion actions are executed on
// the Kubernetes side.
func deleteNodeFromCloudProvider(node *apiv1.Node, cloudProvider cloudprovider.CloudProvider,

View File

@ -1182,39 +1182,16 @@ func getStringFromChanImmediately(c chan string) string {
}
}
func TestCleanToBeDeleted(t *testing.T) {
n1 := BuildTestNode("n1", 1000, 10)
n2 := BuildTestNode("n2", 1000, 10)
n2.Spec.Taints = []apiv1.Taint{{Key: deletetaint.ToBeDeletedTaint, Value: strconv.FormatInt(time.Now().Unix()-301, 10)}}
fakeClient := &fake.Clientset{}
fakeClient.Fake.AddReactor("get", "nodes", func(action core.Action) (bool, runtime.Object, error) {
getAction := action.(core.GetAction)
switch getAction.GetName() {
case n1.Name:
return true, n1, nil
case n2.Name:
return true, n2, nil
func getCountOfChan(c chan string) int {
count := 0
for {
select {
case <-c:
count++
default:
return count
}
return true, nil, fmt.Errorf("Wrong node: %v", getAction.GetName())
})
fakeClient.Fake.AddReactor("update", "nodes", func(action core.Action) (bool, runtime.Object, error) {
update := action.(core.UpdateAction)
obj := update.GetObject().(*apiv1.Node)
switch obj.Name {
case n1.Name:
n1 = obj
case n2.Name:
n2 = obj
}
return true, obj, nil
})
fakeRecorder := kube_util.CreateEventRecorder(fakeClient)
cleanToBeDeleted([]*apiv1.Node{n1, n2}, fakeClient, fakeRecorder)
assert.Equal(t, 0, len(n1.Spec.Taints))
assert.Equal(t, 0, len(n2.Spec.Taints))
}
}
func TestCalculateCoresAndMemoryTotal(t *testing.T) {
@ -1339,3 +1316,242 @@ func TestCheckScaleDownDeltaWithinLimits(t *testing.T) {
}
}
}
func TestSoftTaint(t *testing.T) {
updatedNodes := make(chan string, 10)
deletedNodes := make(chan string, 10)
taintedNodes := make(chan string, 10)
fakeClient := &fake.Clientset{}
job := batchv1.Job{
ObjectMeta: metav1.ObjectMeta{
Name: "job",
Namespace: "default",
SelfLink: "/apivs/batch/v1/namespaces/default/jobs/job",
},
}
n1000 := BuildTestNode("n1000", 1000, 1000)
SetNodeReadyState(n1000, true, time.Time{})
n2000 := BuildTestNode("n2000", 2000, 1000)
SetNodeReadyState(n2000, true, time.Time{})
p500 := BuildTestPod("p500", 500, 0)
p700 := BuildTestPod("p700", 700, 0)
p1200 := BuildTestPod("p1200", 1200, 0)
p500.Spec.NodeName = "n2000"
p700.Spec.NodeName = "n1000"
p1200.Spec.NodeName = "n2000"
fakeClient.Fake.AddReactor("get", "nodes", func(action core.Action) (bool, runtime.Object, error) {
getAction := action.(core.GetAction)
switch getAction.GetName() {
case n1000.Name:
return true, n1000, nil
case n2000.Name:
return true, n2000, nil
}
return true, nil, fmt.Errorf("Wrong node: %v", getAction.GetName())
})
fakeClient.Fake.AddReactor("update", "nodes", func(action core.Action) (bool, runtime.Object, error) {
update := action.(core.UpdateAction)
obj := update.GetObject().(*apiv1.Node)
if deletetaint.HasDeletionCandidateTaint(obj) {
taintedNodes <- obj.Name
}
updatedNodes <- obj.Name
return true, obj, nil
})
provider := testprovider.NewTestCloudProvider(nil, func(nodeGroup string, node string) error {
deletedNodes <- node
return nil
})
provider.AddNodeGroup("ng1", 1, 10, 2)
provider.AddNode("ng1", n1000)
provider.AddNode("ng1", n2000)
assert.NotNil(t, provider)
options := config.AutoscalingOptions{
ScaleDownUtilizationThreshold: 0.5,
ScaleDownUnneededTime: 10 * time.Minute,
MaxGracefulTerminationSec: 60,
MaxBulkSoftTaintCount: 1,
MaxBulkSoftTaintTime: 3 * time.Second,
}
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
assert.NoError(t, err)
registry := kube_util.NewListerRegistry(nil, nil, nil, nil, nil, nil, nil, jobLister, nil, nil)
context := NewScaleTestAutoscalingContext(options, fakeClient, registry, provider)
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
scaleDown := NewScaleDown(&context, clusterStateRegistry)
// Test no superfluous nodes
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1000, n2000},
[]*apiv1.Node{n1000, n2000}, []*apiv1.Pod{p500, p700, p1200}, time.Now().Add(-5*time.Minute), nil)
errors := scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 0, getCountOfChan(deletedNodes))
assert.Equal(t, 0, getCountOfChan(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n1000))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n2000))
// Test one unneeded node
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1000, n2000},
[]*apiv1.Node{n1000, n2000}, []*apiv1.Pod{p500, p1200}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 0, getCountOfChan(deletedNodes))
assert.Equal(t, n1000.Name, getStringFromChanImmediately(updatedNodes))
assert.Equal(t, n1000.Name, getStringFromChanImmediately(taintedNodes))
assert.True(t, deletetaint.HasDeletionCandidateTaint(n1000))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n2000))
// Test remove soft taint
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1000, n2000},
[]*apiv1.Node{n1000, n2000}, []*apiv1.Pod{p500, p700, p1200}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, n1000.Name, getStringFromChanImmediately(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n1000))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n2000))
// Test bulk update taint limit
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1000, n2000},
[]*apiv1.Node{n1000, n2000}, []*apiv1.Pod{}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 1, getCountOfChan(updatedNodes))
assert.Equal(t, 1, getCountOfChan(taintedNodes))
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 1, getCountOfChan(updatedNodes))
assert.Equal(t, 1, getCountOfChan(taintedNodes))
assert.True(t, deletetaint.HasDeletionCandidateTaint(n1000))
assert.True(t, deletetaint.HasDeletionCandidateTaint(n2000))
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 0, getCountOfChan(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
// Test bulk update untaint limit
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1000, n2000},
[]*apiv1.Node{n1000, n2000}, []*apiv1.Pod{p500, p700, p1200}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 1, getCountOfChan(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 1, getCountOfChan(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n1000))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n2000))
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1000, n2000})
assert.Empty(t, errors)
assert.Equal(t, 0, getCountOfChan(updatedNodes))
assert.Equal(t, 0, getCountOfChan(taintedNodes))
}
func TestSoftTaintTimeLimit(t *testing.T) {
updatedNodes := make(chan string, 10)
fakeClient := &fake.Clientset{}
job := batchv1.Job{
ObjectMeta: metav1.ObjectMeta{
Name: "job",
Namespace: "default",
SelfLink: "/apivs/batch/v1/namespaces/default/jobs/job",
},
}
n1 := BuildTestNode("n1", 1000, 1000)
SetNodeReadyState(n1, true, time.Time{})
n2 := BuildTestNode("n2", 1000, 1000)
SetNodeReadyState(n2, true, time.Time{})
p1 := BuildTestPod("p1", 1000, 0)
p2 := BuildTestPod("p2", 1000, 0)
p1.Spec.NodeName = "n1"
p2.Spec.NodeName = "n2"
currentTime := time.Now()
updateTime := time.Millisecond
maxSoftTaintDuration := 1 * time.Second
now = func() time.Time {
return currentTime
}
fakeClient.Fake.AddReactor("get", "nodes", func(action core.Action) (bool, runtime.Object, error) {
getAction := action.(core.GetAction)
switch getAction.GetName() {
case n1.Name:
return true, n1, nil
case n2.Name:
return true, n2, nil
}
return true, nil, fmt.Errorf("Wrong node: %v", getAction.GetName())
})
fakeClient.Fake.AddReactor("update", "nodes", func(action core.Action) (bool, runtime.Object, error) {
currentTime = currentTime.Add(updateTime)
update := action.(core.UpdateAction)
obj := update.GetObject().(*apiv1.Node)
updatedNodes <- obj.Name
return true, obj, nil
})
provider := testprovider.NewTestCloudProvider(nil, func(nodeGroup string, node string) error {
return nil
})
provider.AddNodeGroup("ng1", 1, 10, 2)
provider.AddNode("ng1", n1)
provider.AddNode("ng1", n2)
assert.NotNil(t, provider)
options := config.AutoscalingOptions{
ScaleDownUtilizationThreshold: 0.5,
ScaleDownUnneededTime: 10 * time.Minute,
MaxGracefulTerminationSec: 60,
MaxBulkSoftTaintCount: 10,
MaxBulkSoftTaintTime: maxSoftTaintDuration,
}
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
assert.NoError(t, err)
registry := kube_util.NewListerRegistry(nil, nil, nil, nil, nil, nil, nil, jobLister, nil, nil)
context := NewScaleTestAutoscalingContext(options, fakeClient, registry, provider)
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
scaleDown := NewScaleDown(&context, clusterStateRegistry)
// Test bulk taint
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1, n2},
[]*apiv1.Node{n1, n2}, []*apiv1.Pod{}, time.Now().Add(-5*time.Minute), nil)
errors := scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1, n2})
assert.Empty(t, errors)
assert.Equal(t, 2, getCountOfChan(updatedNodes))
assert.True(t, deletetaint.HasDeletionCandidateTaint(n1))
assert.True(t, deletetaint.HasDeletionCandidateTaint(n2))
// Test bulk untaint
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1, n2},
[]*apiv1.Node{n1, n2}, []*apiv1.Pod{p1, p2}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1, n2})
assert.Empty(t, errors)
assert.Equal(t, 2, getCountOfChan(updatedNodes))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n1))
assert.False(t, deletetaint.HasDeletionCandidateTaint(n2))
// Test duration limit of bulk taint
updateTime = maxSoftTaintDuration
scaleDown.UpdateUnneededNodes([]*apiv1.Node{n1, n2},
[]*apiv1.Node{n1, n2}, []*apiv1.Pod{}, time.Now().Add(-5*time.Minute), nil)
errors = scaleDown.SoftTaintUnneededNodes([]*apiv1.Node{n1, n2})
assert.Empty(t, errors)
assert.Equal(t, 1, getCountOfChan(updatedNodes))
// Clean up
now = time.Now
}

View File

@ -34,6 +34,7 @@ import (
"k8s.io/autoscaler/cluster-autoscaler/processors/status"
"k8s.io/autoscaler/cluster-autoscaler/simulator"
"k8s.io/autoscaler/cluster-autoscaler/utils/backoff"
"k8s.io/autoscaler/cluster-autoscaler/utils/deletetaint"
"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
"k8s.io/autoscaler/cluster-autoscaler/utils/tpu"
@ -112,7 +113,11 @@ func (a *StaticAutoscaler) cleanUpIfRequired() {
if readyNodes, err := a.ReadyNodeLister().List(); err != nil {
klog.Errorf("Failed to list ready nodes, not cleaning up taints: %v", err)
} else {
cleanToBeDeleted(readyNodes, a.AutoscalingContext.ClientSet, a.Recorder)
deletetaint.CleanAllToBeDeleted(readyNodes, a.AutoscalingContext.ClientSet, a.Recorder)
if a.AutoscalingContext.AutoscalingOptions.MaxBulkSoftTaintCount == 0 {
// Clean old taints if soft taints handling is disabled
deletetaint.CleanAllDeletionCandidates(readyNodes, a.AutoscalingContext.ClientSet, a.Recorder)
}
}
a.initialized = true
}
@ -379,6 +384,10 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) errors.AutoscalerError
} else {
klog.V(4).Infof("Starting scale down")
if a.AutoscalingContext.AutoscalingOptions.MaxBulkSoftTaintCount != 0 {
scaleDown.SoftTaintUnneededNodes(allNodes)
}
// We want to delete unneeded Node Groups only if there was no recent scale up,
// and there is no current delete in progress and there was no recent errors.
a.processors.NodeGroupManager.RemoveUnneededNodeGroups(autoscalingContext)

View File

@ -17,12 +17,13 @@ limitations under the License.
package core
import (
"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
"reflect"
"strings"
"testing"
"time"
"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
mockprovider "k8s.io/autoscaler/cluster-autoscaler/cloudprovider/mocks"
testprovider "k8s.io/autoscaler/cluster-autoscaler/cloudprovider/test"
"k8s.io/autoscaler/cluster-autoscaler/clusterstate"

View File

@ -380,6 +380,8 @@ func sanitizeTemplateNode(node *apiv1.Node, nodeGroup string) (*apiv1.Node, erro
klog.V(4).Infof("Removing rescheduler taint when creating template from node %s", node.Name)
case deletetaint.ToBeDeletedTaint:
klog.V(4).Infof("Removing autoscaler taint when creating template from node %s", node.Name)
case deletetaint.DeletionCandidateTaint:
klog.V(4).Infof("Removing autoscaler soft taint when creating template from node %s", node.Name)
default:
newTaints = append(newTaints, taint)
}

View File

@ -118,6 +118,8 @@ var (
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, ",")+"]")
maxBulkSoftTaintCount = flag.Int("max-bulk-soft-taint-count", 10, "Maximum number of nodes that can be tainted/untainted PreferNoSchedule at the same time. Set to 0 to turn off such tainting.")
maxBulkSoftTaintTime = flag.Duration("max-bulk-soft-taint-time", 3*time.Second, "Maximum duration of tainting/untainting nodes as PreferNoSchedule at the same time.")
maxEmptyBulkDeleteFlag = flag.Int("max-empty-bulk-delete", 10, "Maximum number of empty nodes that can be deleted at the same time.")
maxGracefulTerminationFlag = flag.Int("max-graceful-termination-sec", 10*60, "Maximum number of seconds CA waits for pod termination when trying to scale down a node.")
maxTotalUnreadyPercentage = flag.Float64("max-total-unready-percentage", 45, "Maximum percentage of unready nodes in the cluster. After this is exceeded, CA halts operations")
@ -186,6 +188,8 @@ func createAutoscalingOptions() config.AutoscalingOptions {
ExpanderName: *expanderFlag,
IgnoreDaemonSetsUtilization: *ignoreDaemonSetsUtilization,
IgnoreMirrorPodsUtilization: *ignoreMirrorPodsUtilization,
MaxBulkSoftTaintCount: *maxBulkSoftTaintCount,
MaxBulkSoftTaintTime: *maxBulkSoftTaintTime,
MaxEmptyBulkDelete: *maxEmptyBulkDeleteFlag,
MaxGracefulTerminationSec: *maxGracefulTerminationFlag,
MaxNodeProvisionTime: *maxNodeProvisionTime,

View File

@ -79,6 +79,7 @@ const (
ScaleDownNodeDeletion FunctionLabel = "scaleDown:nodeDeletion"
ScaleDownFindNodesToRemove FunctionLabel = "scaleDown:findNodesToRemove"
ScaleDownMiscOperations FunctionLabel = "scaleDown:miscOperations"
ScaleDownSoftTaintUnneeded FunctionLabel = "scaleDown:softTaintUnneeded"
ScaleUp FunctionLabel = "scaleUp"
FindUnneeded FunctionLabel = "findUnneeded"
UpdateState FunctionLabel = "updateClusterState"

View File

@ -25,6 +25,7 @@ import (
"k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
kube_client "k8s.io/client-go/kubernetes"
kube_record "k8s.io/client-go/tools/record"
"k8s.io/klog"
)
@ -32,24 +33,47 @@ import (
const (
// ToBeDeletedTaint is a taint used to make the node unschedulable.
ToBeDeletedTaint = "ToBeDeletedByClusterAutoscaler"
// DeletionCandidateTaint is a taint used to mark unneeded node as preferably unschedulable.
DeletionCandidateTaint = "DeletionCandidateOfClusterAutoscaler"
maxRetryDeadline = 5 * time.Second
conflictRetryInterval = 750 * time.Millisecond
)
// getKeyShortName converts taint key to short name for logging
func getKeyShortName(key string) string {
switch key {
case ToBeDeletedTaint:
return "ToBeDeletedTaint"
case DeletionCandidateTaint:
return "DeletionCandidateTaint"
default:
return key
}
}
// MarkToBeDeleted sets a taint that makes the node unschedulable.
func MarkToBeDeleted(node *apiv1.Node, client kube_client.Interface) error {
return addTaint(node, client, ToBeDeletedTaint, apiv1.TaintEffectNoSchedule)
}
// MarkDeletionCandidate sets a soft taint that makes the node preferably unschedulable.
func MarkDeletionCandidate(node *apiv1.Node, client kube_client.Interface) error {
return addTaint(node, client, DeletionCandidateTaint, apiv1.TaintEffectPreferNoSchedule)
}
func addTaint(node *apiv1.Node, client kube_client.Interface, taintKey string, effect apiv1.TaintEffect) error {
retryDeadline := time.Now().Add(maxRetryDeadline)
for {
// Get the newest version of the node.
freshNode, err := client.CoreV1().Nodes().Get(node.Name, metav1.GetOptions{})
if err != nil || freshNode == nil {
klog.Warningf("Error while adding %v taint on node %v: %v", getKeyShortName(taintKey), node.Name, err)
return fmt.Errorf("failed to get node %v: %v", node.Name, err)
}
added, err := addToBeDeletedTaint(freshNode)
if added == false {
return err
if !addTaintToSpec(freshNode, taintKey, effect) {
return nil
}
_, err = client.CoreV1().Nodes().Update(freshNode)
if err != nil && errors.IsConflict(err) && time.Now().Before(retryDeadline) {
@ -58,33 +82,42 @@ func MarkToBeDeleted(node *apiv1.Node, client kube_client.Interface) error {
}
if err != nil {
klog.Warningf("Error while adding taints on node %v: %v", node.Name, err)
klog.Warningf("Error while adding %v taint on node %v: %v", getKeyShortName(taintKey), node.Name, err)
return err
}
klog.V(1).Infof("Successfully added toBeDeletedTaint on node %v", node.Name)
klog.V(1).Infof("Successfully added %v on node %v", getKeyShortName(taintKey), node.Name)
return nil
}
}
func addToBeDeletedTaint(node *apiv1.Node) (bool, error) {
func addTaintToSpec(node *apiv1.Node, taintKey string, effect apiv1.TaintEffect) bool {
for _, taint := range node.Spec.Taints {
if taint.Key == ToBeDeletedTaint {
klog.V(2).Infof("ToBeDeletedTaint already present on node %v, taint: %v", node.Name, taint)
return false, nil
if taint.Key == taintKey {
klog.V(2).Infof("%v already present on node %v, taint: %v", taintKey, node.Name, taint)
return false
}
}
node.Spec.Taints = append(node.Spec.Taints, apiv1.Taint{
Key: ToBeDeletedTaint,
Key: taintKey,
Value: fmt.Sprint(time.Now().Unix()),
Effect: apiv1.TaintEffectNoSchedule,
Effect: effect,
})
return true, nil
return true
}
// HasToBeDeletedTaint returns true if ToBeDeleted taint is applied on the node.
func HasToBeDeletedTaint(node *apiv1.Node) bool {
return hasTaint(node, ToBeDeletedTaint)
}
// HasDeletionCandidateTaint returns true if DeletionCandidate taint is applied on the node.
func HasDeletionCandidateTaint(node *apiv1.Node) bool {
return hasTaint(node, DeletionCandidateTaint)
}
func hasTaint(node *apiv1.Node, taintKey string) bool {
for _, taint := range node.Spec.Taints {
if taint.Key == ToBeDeletedTaint {
if taint.Key == taintKey {
return true
}
}
@ -93,8 +126,17 @@ func HasToBeDeletedTaint(node *apiv1.Node) bool {
// GetToBeDeletedTime returns the date when the node was marked by CA as for delete.
func GetToBeDeletedTime(node *apiv1.Node) (*time.Time, error) {
return getTaintTime(node, ToBeDeletedTaint)
}
// GetDeletionCandidateTime returns the date when the node was marked by CA as for delete.
func GetDeletionCandidateTime(node *apiv1.Node) (*time.Time, error) {
return getTaintTime(node, DeletionCandidateTaint)
}
func getTaintTime(node *apiv1.Node, taintKey string) (*time.Time, error) {
for _, taint := range node.Spec.Taints {
if taint.Key == ToBeDeletedTaint {
if taint.Key == taintKey {
resultTimestamp, err := strconv.ParseInt(taint.Value, 10, 64)
if err != nil {
return nil, err
@ -106,17 +148,27 @@ func GetToBeDeletedTime(node *apiv1.Node) (*time.Time, error) {
return nil, nil
}
// CleanToBeDeleted cleans ToBeDeleted taint.
// CleanToBeDeleted cleans CA's NoSchedule taint from a node.
func CleanToBeDeleted(node *apiv1.Node, client kube_client.Interface) (bool, error) {
return cleanTaint(node, client, ToBeDeletedTaint)
}
// CleanDeletionCandidate cleans CA's soft NoSchedule taint from a node.
func CleanDeletionCandidate(node *apiv1.Node, client kube_client.Interface) (bool, error) {
return cleanTaint(node, client, DeletionCandidateTaint)
}
func cleanTaint(node *apiv1.Node, client kube_client.Interface, taintKey string) (bool, error) {
retryDeadline := time.Now().Add(maxRetryDeadline)
for {
freshNode, err := client.CoreV1().Nodes().Get(node.Name, metav1.GetOptions{})
if err != nil || freshNode == nil {
klog.Warningf("Error while releasing %v taint on node %v: %v", getKeyShortName(taintKey), node.Name, err)
return false, fmt.Errorf("failed to get node %v: %v", node.Name, err)
}
newTaints := make([]apiv1.Taint, 0)
for _, taint := range freshNode.Spec.Taints {
if taint.Key == ToBeDeletedTaint {
if taint.Key == taintKey {
klog.V(1).Infof("Releasing taint %+v on node %v", taint, node.Name)
} else {
newTaints = append(newTaints, taint)
@ -133,12 +185,38 @@ func CleanToBeDeleted(node *apiv1.Node, client kube_client.Interface) (bool, err
}
if err != nil {
klog.Warningf("Error while releasing taints on node %v: %v", node.Name, err)
klog.Warningf("Error while releasing %v taint on node %v: %v", getKeyShortName(taintKey), node.Name, err)
return false, err
}
klog.V(1).Infof("Successfully released toBeDeletedTaint on node %v", node.Name)
klog.V(1).Infof("Successfully released %v on node %v", getKeyShortName(taintKey), node.Name)
return true, nil
}
return false, nil
}
}
// CleanAllToBeDeleted cleans ToBeDeleted taints from given nodes.
func CleanAllToBeDeleted(nodes []*apiv1.Node, client kube_client.Interface, recorder kube_record.EventRecorder) {
cleanAllTaints(nodes, client, recorder, ToBeDeletedTaint)
}
// CleanAllDeletionCandidates cleans DeletionCandidate taints from given nodes.
func CleanAllDeletionCandidates(nodes []*apiv1.Node, client kube_client.Interface, recorder kube_record.EventRecorder) {
cleanAllTaints(nodes, client, recorder, DeletionCandidateTaint)
}
func cleanAllTaints(nodes []*apiv1.Node, client kube_client.Interface, recorder kube_record.EventRecorder, taintKey string) {
for _, node := range nodes {
if !hasTaint(node, taintKey) {
continue
}
cleaned, err := cleanTaint(node, client, taintKey)
if err != nil {
recorder.Eventf(node, apiv1.EventTypeWarning, "ClusterAutoscalerCleanup",
"failed to clean %v on node %v: %v", getKeyShortName(taintKey), node.Name, err)
} else if cleaned {
recorder.Eventf(node, apiv1.EventTypeNormal, "ClusterAutoscalerCleanup",
"removed %v taint from node %v", getKeyShortName(taintKey), node.Name)
}
}
}

View File

@ -18,6 +18,7 @@ package deletetaint
import (
"fmt"
"strconv"
"sync/atomic"
"testing"
"time"
@ -28,6 +29,7 @@ import (
"k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
"k8s.io/client-go/kubernetes/fake"
core "k8s.io/client-go/testing"
@ -43,9 +45,44 @@ func TestMarkNodes(t *testing.T) {
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.True(t, HasToBeDeletedTaint(updatedNode))
assert.False(t, HasDeletionCandidateTaint(updatedNode))
}
func TestSoftMarkNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
fakeClient := buildFakeClient(t, node)
err := MarkDeletionCandidate(node, fakeClient)
assert.NoError(t, err)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.False(t, HasToBeDeletedTaint(updatedNode))
assert.True(t, HasDeletionCandidateTaint(updatedNode))
}
func TestCheckNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
addTaintToSpec(node, ToBeDeletedTaint, apiv1.TaintEffectNoSchedule)
fakeClient := buildFakeClient(t, node)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.True(t, HasToBeDeletedTaint(updatedNode))
assert.False(t, HasDeletionCandidateTaint(updatedNode))
}
func TestSoftCheckNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
addTaintToSpec(node, DeletionCandidateTaint, apiv1.TaintEffectPreferNoSchedule)
fakeClient := buildFakeClient(t, node)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.False(t, HasToBeDeletedTaint(updatedNode))
assert.True(t, HasDeletionCandidateTaint(updatedNode))
}
func TestQueryNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
fakeClient := buildFakeClient(t, node)
err := MarkToBeDeleted(node, fakeClient)
@ -60,20 +97,131 @@ func TestCheckNodes(t *testing.T) {
assert.True(t, time.Now().Sub(*val) < 10*time.Second)
}
func TestSoftQueryNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
fakeClient := buildFakeClient(t, node)
err := MarkDeletionCandidate(node, fakeClient)
assert.NoError(t, err)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.True(t, HasDeletionCandidateTaint(updatedNode))
val, err := GetDeletionCandidateTime(updatedNode)
assert.NoError(t, err)
assert.True(t, time.Now().Sub(*val) < 10*time.Second)
}
func TestCleanNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
addToBeDeletedTaint(node)
addTaintToSpec(node, ToBeDeletedTaint, apiv1.TaintEffectNoSchedule)
fakeClient := buildFakeClient(t, node)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.True(t, HasToBeDeletedTaint(updatedNode))
cleaned, err := CleanToBeDeleted(node, fakeClient)
assert.True(t, cleaned)
assert.NoError(t, err)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
updatedNode, err = fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.False(t, HasToBeDeletedTaint(updatedNode))
}
func TestSoftCleanNodes(t *testing.T) {
node := BuildTestNode("node", 1000, 1000)
addTaintToSpec(node, DeletionCandidateTaint, apiv1.TaintEffectPreferNoSchedule)
fakeClient := buildFakeClient(t, node)
updatedNode, err := fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.True(t, HasDeletionCandidateTaint(updatedNode))
cleaned, err := CleanDeletionCandidate(node, fakeClient)
assert.True(t, cleaned)
assert.NoError(t, err)
updatedNode, err = fakeClient.Core().Nodes().Get("node", metav1.GetOptions{})
assert.NoError(t, err)
assert.False(t, HasDeletionCandidateTaint(updatedNode))
}
func TestCleanAllToBeDeleted(t *testing.T) {
n1 := BuildTestNode("n1", 1000, 10)
n2 := BuildTestNode("n2", 1000, 10)
n2.Spec.Taints = []apiv1.Taint{{Key: ToBeDeletedTaint, Value: strconv.FormatInt(time.Now().Unix()-301, 10)}}
fakeClient := &fake.Clientset{}
fakeClient.Fake.AddReactor("get", "nodes", func(action core.Action) (bool, runtime.Object, error) {
getAction := action.(core.GetAction)
switch getAction.GetName() {
case n1.Name:
return true, n1, nil
case n2.Name:
return true, n2, nil
}
return true, nil, fmt.Errorf("Wrong node: %v", getAction.GetName())
})
fakeClient.Fake.AddReactor("update", "nodes", func(action core.Action) (bool, runtime.Object, error) {
update := action.(core.UpdateAction)
obj := update.GetObject().(*apiv1.Node)
switch obj.Name {
case n1.Name:
n1 = obj
case n2.Name:
n2 = obj
}
return true, obj, nil
})
fakeRecorder := kube_util.CreateEventRecorder(fakeClient)
assert.Equal(t, 1, len(n2.Spec.Taints))
CleanAllToBeDeleted([]*apiv1.Node{n1, n2}, fakeClient, fakeRecorder)
assert.Equal(t, 0, len(n1.Spec.Taints))
assert.Equal(t, 0, len(n2.Spec.Taints))
}
func TestCleanAllDeletionCandidates(t *testing.T) {
n1 := BuildTestNode("n1", 1000, 10)
n2 := BuildTestNode("n2", 1000, 10)
n2.Spec.Taints = []apiv1.Taint{{Key: DeletionCandidateTaint, Value: strconv.FormatInt(time.Now().Unix()-301, 10)}}
fakeClient := &fake.Clientset{}
fakeClient.Fake.AddReactor("get", "nodes", func(action core.Action) (bool, runtime.Object, error) {
getAction := action.(core.GetAction)
switch getAction.GetName() {
case n1.Name:
return true, n1, nil
case n2.Name:
return true, n2, nil
}
return true, nil, fmt.Errorf("Wrong node: %v", getAction.GetName())
})
fakeClient.Fake.AddReactor("update", "nodes", func(action core.Action) (bool, runtime.Object, error) {
update := action.(core.UpdateAction)
obj := update.GetObject().(*apiv1.Node)
switch obj.Name {
case n1.Name:
n1 = obj
case n2.Name:
n2 = obj
}
return true, obj, nil
})
fakeRecorder := kube_util.CreateEventRecorder(fakeClient)
assert.Equal(t, 1, len(n2.Spec.Taints))
CleanAllDeletionCandidates([]*apiv1.Node{n1, n2}, fakeClient, fakeRecorder)
assert.Equal(t, 0, len(n1.Spec.Taints))
assert.Equal(t, 0, len(n2.Spec.Taints))
}
func buildFakeClient(t *testing.T, node *apiv1.Node) *fake.Clientset {
fakeClient := fake.NewSimpleClientset()