Skip to content

Commit

Permalink
Add error details to autoscaling backoff.
Browse files Browse the repository at this point in the history
Change-Id: I3b5c62ba13c2e048ce2d7170016af07182c11eee
  • Loading branch information
walidghallab committed Dec 13, 2023
1 parent 8f75e9c commit e5ef663
Show file tree
Hide file tree
Showing 7 changed files with 168 additions and 71 deletions.
62 changes: 44 additions & 18 deletions cluster-autoscaler/clusterstate/clusterstate.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,13 @@ type ClusterStateRegistry struct {
scaleUpFailures map[string][]ScaleUpFailure
}

// NodeGroupScalingSafety contains information about the safety of the node group to scale up/down.
type NodeGroupScalingSafety struct {
SafeToScale bool
Healthy bool
BackoffStatus *backoff.Status
}

// NewClusterStateRegistry creates new ClusterStateRegistry.
func NewClusterStateRegistry(cloudProvider cloudprovider.CloudProvider, config ClusterStateRegistryConfig, logRecorder *utils.LogEventRecorder, backoff backoff.Backoff, nodeGroupConfigProcessor nodegroupconfig.NodeGroupConfigProcessor) *ClusterStateRegistry {
emptyStatus := &api.ClusterAutoscalerStatus{
Expand Down Expand Up @@ -276,7 +283,11 @@ func (csr *ClusterStateRegistry) updateScaleRequests(currentTime time.Time) {
} else {
gpuResource, gpuType = gpu.GetGpuInfoForMetrics(csr.cloudProvider.GetNodeGpuConfig(nodeInfo.Node()), availableGPUTypes, nodeInfo.Node(), scaleUpRequest.NodeGroup)
}
csr.registerFailedScaleUpNoLock(scaleUpRequest.NodeGroup, metrics.Timeout, cloudprovider.OtherErrorClass, "timeout", gpuResource, gpuType, currentTime)
csr.registerFailedScaleUpNoLock(scaleUpRequest.NodeGroup, metrics.Timeout, cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: "timeout",
ErrorMessage: fmt.Sprintf("Scale-up timed out for node group %v after %v", nodeGroupName, currentTime.Sub(scaleUpRequest.Time)),
}, gpuResource, gpuType, currentTime)
delete(csr.scaleUpRequests, nodeGroupName)
}
}
Expand All @@ -291,25 +302,29 @@ func (csr *ClusterStateRegistry) updateScaleRequests(currentTime time.Time) {
}

// To be executed under a lock.
func (csr *ClusterStateRegistry) backoffNodeGroup(nodeGroup cloudprovider.NodeGroup, errorClass cloudprovider.InstanceErrorClass, errorCode string, currentTime time.Time) {
func (csr *ClusterStateRegistry) backoffNodeGroup(nodeGroup cloudprovider.NodeGroup, errorInfo cloudprovider.InstanceErrorInfo, currentTime time.Time) {
nodeGroupInfo := csr.nodeInfosForGroups[nodeGroup.Id()]
backoffUntil := csr.backoff.Backoff(nodeGroup, nodeGroupInfo, errorClass, errorCode, currentTime)
klog.Warningf("Disabling scale-up for node group %v until %v; errorClass=%v; errorCode=%v", nodeGroup.Id(), backoffUntil, errorClass, errorCode)
backoffUntil := csr.backoff.Backoff(nodeGroup, nodeGroupInfo, errorInfo, currentTime)
klog.Warningf("Disabling scale-up for node group %v until %v; errorClass=%v; errorCode=%v", nodeGroup.Id(), backoffUntil, errorInfo.ErrorClass, errorInfo.ErrorCode)
}

// RegisterFailedScaleUp should be called after getting error from cloudprovider
// when trying to scale-up node group. It will mark this group as not safe to autoscale
// for some time.
func (csr *ClusterStateRegistry) RegisterFailedScaleUp(nodeGroup cloudprovider.NodeGroup, reason metrics.FailedScaleUpReason, gpuResourceName, gpuType string, currentTime time.Time) {
func (csr *ClusterStateRegistry) RegisterFailedScaleUp(nodeGroup cloudprovider.NodeGroup, reason metrics.FailedScaleUpReason, errorMessage, gpuResourceName, gpuType string, currentTime time.Time) {
csr.Lock()
defer csr.Unlock()
csr.registerFailedScaleUpNoLock(nodeGroup, reason, cloudprovider.OtherErrorClass, string(reason), gpuResourceName, gpuType, currentTime)
csr.registerFailedScaleUpNoLock(nodeGroup, reason, cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: string(reason),
ErrorMessage: errorMessage,
}, gpuResourceName, gpuType, currentTime)
}

func (csr *ClusterStateRegistry) registerFailedScaleUpNoLock(nodeGroup cloudprovider.NodeGroup, reason metrics.FailedScaleUpReason, errorClass cloudprovider.InstanceErrorClass, errorCode string, gpuResourceName, gpuType string, currentTime time.Time) {
func (csr *ClusterStateRegistry) registerFailedScaleUpNoLock(nodeGroup cloudprovider.NodeGroup, reason metrics.FailedScaleUpReason, errorInfo cloudprovider.InstanceErrorInfo, gpuResourceName, gpuType string, currentTime time.Time) {
csr.scaleUpFailures[nodeGroup.Id()] = append(csr.scaleUpFailures[nodeGroup.Id()], ScaleUpFailure{NodeGroup: nodeGroup, Reason: reason, Time: currentTime})
metrics.RegisterFailedScaleUp(reason, gpuResourceName, gpuType)
csr.backoffNodeGroup(nodeGroup, errorClass, errorCode, currentTime)
csr.backoffNodeGroup(nodeGroup, errorInfo, currentTime)
}

// UpdateNodes updates the state of the nodes in the ClusterStateRegistry and recalculates the stats
Expand Down Expand Up @@ -414,7 +429,6 @@ func (csr *ClusterStateRegistry) IsNodeGroupHealthy(nodeGroupName string) bool {
unjustifiedUnready += acceptable.MinNodes - len(readiness.Ready)
}
// TODO: verify against max nodes as well.

if unjustifiedUnready > csr.config.OkTotalUnreadyCount &&
float64(unjustifiedUnready) > csr.config.MaxTotalUnreadyPercentage/100.0*
float64(len(readiness.Ready)+len(readiness.Unready)+len(readiness.NotStarted)) {
Expand All @@ -441,12 +455,16 @@ func (csr *ClusterStateRegistry) updateNodeGroupMetrics() {
metrics.UpdateNodeGroupsCount(autoscaled, autoprovisioned)
}

// IsNodeGroupSafeToScaleUp returns true if node group can be scaled up now.
func (csr *ClusterStateRegistry) IsNodeGroupSafeToScaleUp(nodeGroup cloudprovider.NodeGroup, now time.Time) bool {
// IsNodeGroupSafeToScaleUp returns information about node group safety to be scaled up now.
func (csr *ClusterStateRegistry) IsNodeGroupSafeToScaleUp(nodeGroup cloudprovider.NodeGroup, now time.Time) NodeGroupScalingSafety {
if !csr.IsNodeGroupHealthy(nodeGroup.Id()) {
return false
return NodeGroupScalingSafety{SafeToScale: false, Healthy: false, BackoffStatus: nil}
}
return !csr.backoff.IsBackedOff(nodeGroup, csr.nodeInfosForGroups[nodeGroup.Id()], now)
backoffStatus := csr.backoff.BackoffStatus(nodeGroup, csr.nodeInfosForGroups[nodeGroup.Id()], now)
if !backoffStatus.IsBackedOff {
return NodeGroupScalingSafety{SafeToScale: !backoffStatus.IsBackedOff, Healthy: true}
}
return NodeGroupScalingSafety{SafeToScale: !backoffStatus.IsBackedOff, Healthy: true, BackoffStatus: &backoffStatus}
}

func (csr *ClusterStateRegistry) getProvisionedAndTargetSizesForNodeGroup(nodeGroupName string) (provisioned, target int, ok bool) {
Expand Down Expand Up @@ -788,7 +806,7 @@ func (csr *ClusterStateRegistry) GetClusterReadiness() Readiness {
return csr.totalReadiness
}

func buildHealthStatusNodeGroup(isReady bool, readiness Readiness, acceptable AcceptableRange, minSize, maxSize int) api.ClusterAutoscalerCondition {
func buildHealthStatusNodeGroup(isHealthy bool, readiness Readiness, acceptable AcceptableRange, minSize, maxSize int) api.ClusterAutoscalerCondition {
condition := api.ClusterAutoscalerCondition{
Type: api.ClusterAutoscalerHealth,
Message: fmt.Sprintf("ready=%d unready=%d (resourceUnready=%d) notStarted=%d longNotStarted=0 registered=%d longUnregistered=%d cloudProviderTarget=%d (minSize=%d, maxSize=%d)",
Expand All @@ -803,15 +821,15 @@ func buildHealthStatusNodeGroup(isReady bool, readiness Readiness, acceptable Ac
maxSize),
LastProbeTime: metav1.Time{Time: readiness.Time},
}
if isReady {
if isHealthy {
condition.Status = api.ClusterAutoscalerHealthy
} else {
condition.Status = api.ClusterAutoscalerUnhealthy
}
return condition
}

func buildScaleUpStatusNodeGroup(isScaleUpInProgress bool, isSafeToScaleUp bool, readiness Readiness, acceptable AcceptableRange) api.ClusterAutoscalerCondition {
func buildScaleUpStatusNodeGroup(isScaleUpInProgress bool, scaleUpSafety NodeGroupScalingSafety, readiness Readiness, acceptable AcceptableRange) api.ClusterAutoscalerCondition {
condition := api.ClusterAutoscalerCondition{
Type: api.ClusterAutoscalerScaleUp,
Message: fmt.Sprintf("ready=%d cloudProviderTarget=%d",
Expand All @@ -821,8 +839,11 @@ func buildScaleUpStatusNodeGroup(isScaleUpInProgress bool, isSafeToScaleUp bool,
}
if isScaleUpInProgress {
condition.Status = api.ClusterAutoscalerInProgress
} else if !isSafeToScaleUp {
} else if !scaleUpSafety.Healthy {
condition.Status = api.ClusterAutoscalerUnhealthy
} else if !scaleUpSafety.SafeToScale {
condition.Status = api.ClusterAutoscalerBackoff
condition.Message = scaleUpSafety.BackoffStatus.ErrorInfo.ErrorMessage
} else {
condition.Status = api.ClusterAutoscalerNoActivity
}
Expand Down Expand Up @@ -1123,7 +1144,12 @@ func (csr *ClusterStateRegistry) handleInstanceCreationErrorsForNodeGroup(
}
// Decrease the scale up request by the number of deleted nodes
csr.registerOrUpdateScaleUpNoLock(nodeGroup, -len(unseenInstanceIds), currentTime)
csr.registerFailedScaleUpNoLock(nodeGroup, metrics.FailedScaleUpReason(errorCode.code), errorCode.class, errorCode.code, gpuResource, gpuType, currentTime)

csr.registerFailedScaleUpNoLock(nodeGroup, metrics.FailedScaleUpReason(errorCode.code), cloudprovider.InstanceErrorInfo{
ErrorClass: errorCode.class,
ErrorCode: errorCode.code,
ErrorMessage: csr.buildErrorMessageEventString(currentUniqueErrorMessagesForErrorCode[errorCode]),
}, gpuResource, gpuType, currentTime)
}
}
}
Expand Down
58 changes: 49 additions & 9 deletions cluster-autoscaler/clusterstate/clusterstate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -923,13 +923,31 @@ func TestScaleUpBackoff(t *testing.T) {
assert.NoError(t, err)
assert.True(t, clusterstate.IsClusterHealthy())
assert.True(t, clusterstate.IsNodeGroupHealthy("ng1"))
assert.False(t, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, NodeGroupScalingSafety{
SafeToScale: false,
Healthy: true,
BackoffStatus: &backoff.Status{
IsBackedOff: true,
ErrorInfo: &cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: "timeout",
ErrorMessage: "Scale-up timed out for node group ng1 after 3m0s",
},
},
}, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, backoff.Status{
IsBackedOff: true,
ErrorInfo: &cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: "timeout",
ErrorMessage: "Scale-up timed out for node group ng1 after 3m0s",
}}, clusterstate.backoff.BackoffStatus(ng1, nil, now))

// Backoff should expire after timeout
now = now.Add(5 * time.Minute /*InitialNodeGroupBackoffDuration*/).Add(time.Second)
assert.True(t, clusterstate.IsClusterHealthy())
assert.True(t, clusterstate.IsNodeGroupHealthy("ng1"))
assert.True(t, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, NodeGroupScalingSafety{SafeToScale: true, Healthy: true}, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))

// Another failed scale up should cause longer backoff
clusterstate.RegisterOrUpdateScaleUp(provider.GetNodeGroup("ng1"), 1, now.Add(-121*time.Second))
Expand All @@ -938,10 +956,32 @@ func TestScaleUpBackoff(t *testing.T) {
assert.NoError(t, err)
assert.True(t, clusterstate.IsClusterHealthy())
assert.True(t, clusterstate.IsNodeGroupHealthy("ng1"))
assert.False(t, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, NodeGroupScalingSafety{
SafeToScale: false,
Healthy: true,
BackoffStatus: &backoff.Status{
IsBackedOff: true,
ErrorInfo: &cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: "timeout",
ErrorMessage: "Scale-up timed out for node group ng1 after 2m1s",
},
},
}, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))

now = now.Add(5 * time.Minute /*InitialNodeGroupBackoffDuration*/).Add(time.Second)
assert.False(t, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, NodeGroupScalingSafety{
SafeToScale: false,
Healthy: true,
BackoffStatus: &backoff.Status{
IsBackedOff: true,
ErrorInfo: &cloudprovider.InstanceErrorInfo{
ErrorClass: cloudprovider.OtherErrorClass,
ErrorCode: "timeout",
ErrorMessage: "Scale-up timed out for node group ng1 after 2m1s",
},
},
}, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))

// The backoff should be cleared after a successful scale-up
clusterstate.RegisterOrUpdateScaleUp(provider.GetNodeGroup("ng1"), 1, now)
Expand All @@ -952,8 +992,8 @@ func TestScaleUpBackoff(t *testing.T) {
assert.NoError(t, err)
assert.True(t, clusterstate.IsClusterHealthy())
assert.True(t, clusterstate.IsNodeGroupHealthy("ng1"))
assert.True(t, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.False(t, clusterstate.backoff.IsBackedOff(ng1, nil, now))
assert.Equal(t, NodeGroupScalingSafety{SafeToScale: true, Healthy: true}, clusterstate.IsNodeGroupSafeToScaleUp(ng1, now))
assert.Equal(t, backoff.Status{IsBackedOff: false}, clusterstate.backoff.BackoffStatus(ng1, nil, now))
}

func TestGetClusterSize(t *testing.T) {
Expand Down Expand Up @@ -1070,9 +1110,9 @@ func TestScaleUpFailures(t *testing.T) {
fakeLogRecorder, _ := utils.NewStatusMapRecorder(fakeClient, "kube-system", kube_record.NewFakeRecorder(5), false, "my-cool-configmap")
clusterstate := NewClusterStateRegistry(provider, ClusterStateRegistryConfig{}, fakeLogRecorder, newBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(config.NodeGroupAutoscalingOptions{MaxNodeProvisionTime: 15 * time.Minute}))

clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng1"), metrics.Timeout, "", "", now)
clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng2"), metrics.Timeout, "", "", now)
clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng1"), metrics.APIError, "", "", now.Add(time.Minute))
clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng1"), metrics.Timeout, "", "", "", now)
clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng2"), metrics.Timeout, "", "", "", now)
clusterstate.RegisterFailedScaleUp(provider.GetNodeGroup("ng1"), metrics.APIError, "", "", "", now.Add(time.Minute))

failures := clusterstate.GetScaleUpFailures()
assert.Equal(t, map[string][]ScaleUpFailure{
Expand Down
2 changes: 1 addition & 1 deletion cluster-autoscaler/core/scaleup/orchestrator/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ func (e *scaleUpExecutor) executeScaleUp(
if err := info.Group.IncreaseSize(increase); err != nil {
e.autoscalingContext.LogRecorder.Eventf(apiv1.EventTypeWarning, "FailedToScaleUpGroup", "Scale-up failed for group %s: %v", info.Group.Id(), err)
aerr := errors.ToAutoscalerError(errors.CloudProviderError, err).AddPrefix("failed to increase node group size: ")
e.clusterStateRegistry.RegisterFailedScaleUp(info.Group, metrics.FailedScaleUpReason(string(aerr.Type())), gpuResourceName, gpuType, now)
e.clusterStateRegistry.RegisterFailedScaleUp(info.Group, metrics.FailedScaleUpReason(string(aerr.Type())), aerr.Error(), gpuResourceName, gpuType, now)
return aerr
}
e.clusterStateRegistry.RegisterOrUpdateScaleUp(
Expand Down
10 changes: 6 additions & 4 deletions cluster-autoscaler/core/scaleup/orchestrator/orchestrator.go
Original file line number Diff line number Diff line change
Expand Up @@ -569,9 +569,11 @@ func (o *ScaleUpOrchestrator) UpcomingNodes(nodeInfos map[string]*schedulerframe
// IsNodeGroupReadyToScaleUp returns nil if node group is ready to be scaled up, otherwise a reason is provided.
func (o *ScaleUpOrchestrator) IsNodeGroupReadyToScaleUp(nodeGroup cloudprovider.NodeGroup, now time.Time) *SkippedReasons {
// Non-existing node groups are created later so skip check for them.
if nodeGroup.Exist() && !o.clusterStateRegistry.IsNodeGroupSafeToScaleUp(nodeGroup, now) {
// Hack that depends on internals of IsNodeGroupSafeToScaleUp.
if !o.clusterStateRegistry.IsNodeGroupHealthy(nodeGroup.Id()) {
if !nodeGroup.Exist() {
return nil
}
if scaleUpSafety := o.clusterStateRegistry.IsNodeGroupSafeToScaleUp(nodeGroup, now); !scaleUpSafety.SafeToScale {
if !scaleUpSafety.SafeToScale {
klog.Warningf("Node group %s is not ready for scaleup - unhealthy", nodeGroup.Id())
return NotReadyReason
}
Expand Down Expand Up @@ -658,7 +660,7 @@ func (o *ScaleUpOrchestrator) ComputeSimilarNodeGroups(
var validSimilarNodeGroups []cloudprovider.NodeGroup
for _, ng := range similarNodeGroups {
// Non-existing node groups are created later so skip check for them.
if ng.Exist() && !o.clusterStateRegistry.IsNodeGroupSafeToScaleUp(ng, now) {
if ng.Exist() && !o.clusterStateRegistry.IsNodeGroupSafeToScaleUp(ng, now).SafeToScale {
klog.V(2).Infof("Ignoring node group %s when balancing: group is not ready for scaleup", ng.Id())
} else if similarSchedulablePods, found := schedulablePods[ng.Id()]; found && matchingSchedulablePods(groupSchedulablePods, similarSchedulablePods) {
validSimilarNodeGroups = append(validSimilarNodeGroups, ng)
Expand Down
12 changes: 9 additions & 3 deletions cluster-autoscaler/utils/backoff/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,18 @@ import (
schedulerframework "k8s.io/kubernetes/pkg/scheduler/framework"
)

// Status contains information about back off status.
type Status struct {
IsBackedOff bool
ErrorInfo *cloudprovider.InstanceErrorInfo
}

// Backoff allows time-based backing off of node groups considered in scale up algorithm
type Backoff interface {
// Backoff execution for the given node group. Returns time till execution is backed off.
Backoff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, errorClass cloudprovider.InstanceErrorClass, errorCode string, currentTime time.Time) time.Time
// IsBackedOff returns true if execution is backed off for the given node group.
IsBackedOff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, currentTime time.Time) bool
Backoff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, errorInfo cloudprovider.InstanceErrorInfo, currentTime time.Time) time.Time
// BackoffStatus returns whether the execution is backed off for the given node group and error info when the node group is backed off.
BackoffStatus(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, currentTime time.Time) Status
// RemoveBackoff removes backoff data for the given node group.
RemoveBackoff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo)
// RemoveStaleBackoffData removes stale backoff data.
Expand Down
16 changes: 12 additions & 4 deletions cluster-autoscaler/utils/backoff/exponential_backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ type exponentialBackoffInfo struct {
duration time.Duration
backoffUntil time.Time
lastFailedExecution time.Time
errorInfo cloudprovider.InstanceErrorInfo
}

// NewExponentialBackoff creates an instance of exponential backoff.
Expand Down Expand Up @@ -66,7 +67,7 @@ func NewIdBasedExponentialBackoff(initialBackoffDuration time.Duration, maxBacko
}

// Backoff execution for the given node group. Returns time till execution is backed off.
func (b *exponentialBackoff) Backoff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, errorClass cloudprovider.InstanceErrorClass, errorCode string, currentTime time.Time) time.Time {
func (b *exponentialBackoff) Backoff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, errorInfo cloudprovider.InstanceErrorInfo, currentTime time.Time) time.Time {
duration := b.initialBackoffDuration
key := b.nodeGroupKey(nodeGroup)
if backoffInfo, found := b.backoffInfo[key]; found {
Expand All @@ -87,14 +88,21 @@ func (b *exponentialBackoff) Backoff(nodeGroup cloudprovider.NodeGroup, nodeInfo
duration: duration,
backoffUntil: backoffUntil,
lastFailedExecution: currentTime,
errorInfo: errorInfo,
}
return backoffUntil
}

// IsBackedOff returns true if execution is backed off for the given node group.
func (b *exponentialBackoff) IsBackedOff(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, currentTime time.Time) bool {
// BackoffStatus returns whether the execution is backed off for the given node group and error info when the node group is backed off.
func (b *exponentialBackoff) BackoffStatus(nodeGroup cloudprovider.NodeGroup, nodeInfo *schedulerframework.NodeInfo, currentTime time.Time) Status {
backoffInfo, found := b.backoffInfo[b.nodeGroupKey(nodeGroup)]
return found && backoffInfo.backoffUntil.After(currentTime)
if !found || backoffInfo.backoffUntil.Before(currentTime) {
return Status{IsBackedOff: false}
}
return Status{
IsBackedOff: true,
ErrorInfo: &backoffInfo.errorInfo,
}
}

// RemoveBackoff removes backoff data for the given node group.
Expand Down
Loading

0 comments on commit e5ef663

Please sign in to comment.