Limit refresh rate of GCE MIG instances.
This commit is contained in:
parent
ad86bdd641
commit
656f1919a8
|
|
@ -120,8 +120,10 @@ func main() {
|
|||
NodeGroupAutoDiscovery: *nodeGroupAutoDiscoveryFlag,
|
||||
NodeGroups: *nodeGroupsFlag,
|
||||
ClusterName: *clusterName,
|
||||
ConcurrentGceRefreshes: 1,
|
||||
UserAgent: "user-agent",
|
||||
GCEOptions: config.GCEOptions{
|
||||
ConcurrentRefreshes: 1,
|
||||
},
|
||||
UserAgent: "user-agent",
|
||||
}
|
||||
cloudProvider := cloudBuilder.NewCloudProvider(autoscalingOptions)
|
||||
srv := wrapper.NewCloudProviderGrpcWrapper(cloudProvider)
|
||||
|
|
|
|||
|
|
@ -369,12 +369,12 @@ func BuildGCE(opts config.AutoscalingOptions, do cloudprovider.NodeGroupDiscover
|
|||
defer config.Close()
|
||||
}
|
||||
|
||||
manager, err := CreateGceManager(config, do, opts.Regional, opts.ConcurrentGceRefreshes, opts.UserAgent)
|
||||
manager, err := CreateGceManager(config, do, opts.Regional, opts.GCEOptions.ConcurrentRefreshes, opts.UserAgent, opts.GCEOptions.MigInstancesMinRefreshWaitTime)
|
||||
if err != nil {
|
||||
klog.Fatalf("Failed to create GCE Manager: %v", err)
|
||||
}
|
||||
|
||||
pricingModel := NewGcePriceModel(NewGcePriceInfo(), opts.GceExpanderEphemeralStorageSupport)
|
||||
pricingModel := NewGcePriceModel(NewGcePriceInfo(), opts.GCEOptions.ExpanderEphemeralStorageSupport)
|
||||
provider, err := BuildGceCloudProvider(manager, rl, pricingModel)
|
||||
if err != nil {
|
||||
klog.Fatalf("Failed to create GCE cloud provider: %v", err)
|
||||
|
|
|
|||
|
|
@ -123,7 +123,7 @@ type gceManagerImpl struct {
|
|||
}
|
||||
|
||||
// CreateGceManager constructs GceManager object.
|
||||
func CreateGceManager(configReader io.Reader, discoveryOpts cloudprovider.NodeGroupDiscoveryOptions, regional bool, concurrentGceRefreshes int, userAgent string) (GceManager, error) {
|
||||
func CreateGceManager(configReader io.Reader, discoveryOpts cloudprovider.NodeGroupDiscoveryOptions, regional bool, concurrentGceRefreshes int, userAgent string, migInstancesMinRefreshWaitTime time.Duration) (GceManager, error) {
|
||||
// Create Google Compute Engine token.
|
||||
var err error
|
||||
tokenSource := google.ComputeTokenSource("")
|
||||
|
|
@ -183,7 +183,7 @@ func CreateGceManager(configReader io.Reader, discoveryOpts cloudprovider.NodeGr
|
|||
cache: cache,
|
||||
GceService: gceService,
|
||||
migLister: migLister,
|
||||
migInfoProvider: NewCachingMigInfoProvider(cache, migLister, gceService, projectId, concurrentGceRefreshes),
|
||||
migInfoProvider: NewCachingMigInfoProvider(cache, migLister, gceService, projectId, concurrentGceRefreshes, migInstancesMinRefreshWaitTime),
|
||||
location: location,
|
||||
regional: regional,
|
||||
projectId: projectId,
|
||||
|
|
|
|||
|
|
@ -351,7 +351,7 @@ func newTestGceManager(t *testing.T, testServerURL string, regional bool) *gceMa
|
|||
manager := &gceManagerImpl{
|
||||
cache: cache,
|
||||
migLister: migLister,
|
||||
migInfoProvider: NewCachingMigInfoProvider(cache, migLister, gceService, projectId, 1),
|
||||
migInfoProvider: NewCachingMigInfoProvider(cache, migLister, gceService, projectId, 1, 0*time.Second),
|
||||
GceService: gceService,
|
||||
projectId: projectId,
|
||||
regional: regional,
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import (
|
|||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
gce "google.golang.org/api/compute/v1"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
|
||||
|
|
@ -52,24 +53,40 @@ type MigInfoProvider interface {
|
|||
GetMigMachineType(migRef GceRef) (MachineType, error)
|
||||
}
|
||||
|
||||
type timeProvider interface {
|
||||
Now() time.Time
|
||||
}
|
||||
|
||||
type cachingMigInfoProvider struct {
|
||||
migInfoMutex sync.Mutex
|
||||
cache *GceCache
|
||||
migLister MigLister
|
||||
gceClient AutoscalingGceClient
|
||||
projectId string
|
||||
concurrentGceRefreshes int
|
||||
migInstanceMutex sync.Mutex
|
||||
migInfoMutex sync.Mutex
|
||||
cache *GceCache
|
||||
migLister MigLister
|
||||
gceClient AutoscalingGceClient
|
||||
projectId string
|
||||
concurrentGceRefreshes int
|
||||
migInstanceMutex sync.Mutex
|
||||
migInstancesMinRefreshWaitTime time.Duration
|
||||
migInstancesLastRefreshedInfo map[string]time.Time
|
||||
timeProvider timeProvider
|
||||
}
|
||||
|
||||
type realTime struct{}
|
||||
|
||||
func (r *realTime) Now() time.Time {
|
||||
return time.Now()
|
||||
}
|
||||
|
||||
// NewCachingMigInfoProvider creates an instance of caching MigInfoProvider
|
||||
func NewCachingMigInfoProvider(cache *GceCache, migLister MigLister, gceClient AutoscalingGceClient, projectId string, concurrentGceRefreshes int) MigInfoProvider {
|
||||
func NewCachingMigInfoProvider(cache *GceCache, migLister MigLister, gceClient AutoscalingGceClient, projectId string, concurrentGceRefreshes int, migInstancesMinRefreshWaitTime time.Duration) MigInfoProvider {
|
||||
return &cachingMigInfoProvider{
|
||||
cache: cache,
|
||||
migLister: migLister,
|
||||
gceClient: gceClient,
|
||||
projectId: projectId,
|
||||
concurrentGceRefreshes: concurrentGceRefreshes,
|
||||
cache: cache,
|
||||
migLister: migLister,
|
||||
gceClient: gceClient,
|
||||
projectId: projectId,
|
||||
concurrentGceRefreshes: concurrentGceRefreshes,
|
||||
migInstancesMinRefreshWaitTime: migInstancesMinRefreshWaitTime,
|
||||
migInstancesLastRefreshedInfo: make(map[string]time.Time),
|
||||
timeProvider: &realTime{},
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -158,12 +175,21 @@ func (c *cachingMigInfoProvider) findMigWithMatchingBasename(instanceRef GceRef)
|
|||
}
|
||||
|
||||
func (c *cachingMigInfoProvider) fillMigInstances(migRef GceRef) error {
|
||||
if val, ok := c.migInstancesLastRefreshedInfo[migRef.String()]; ok {
|
||||
// do not regenerate MIG instances cache if last refresh happened recently.
|
||||
if c.timeProvider.Now().Sub(val) < c.migInstancesMinRefreshWaitTime {
|
||||
klog.V(4).Infof("Not regenerating MIG instances cache for %s, as it was refreshed in last MinRefreshWaitTime (%s).", migRef.String(), c.migInstancesMinRefreshWaitTime)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
klog.V(4).Infof("Regenerating MIG instances cache for %s", migRef.String())
|
||||
instances, err := c.gceClient.FetchMigInstances(migRef)
|
||||
if err != nil {
|
||||
c.migLister.HandleMigIssue(migRef, err)
|
||||
return err
|
||||
}
|
||||
// only save information for successful calls, given the errors above may be transient.
|
||||
c.migInstancesLastRefreshedInfo[migRef.String()] = c.timeProvider.Now()
|
||||
return c.cache.SetMigInstances(migRef, instances)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import (
|
|||
"fmt"
|
||||
"regexp"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
gce "google.golang.org/api/compute/v1"
|
||||
|
|
@ -235,7 +236,7 @@ func TestMigInfoProviderGetMigForInstance(t *testing.T) {
|
|||
fetchMigs: fetchMigsConst(nil),
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
|
||||
mig, err := provider.GetMigForInstance(instanceRef)
|
||||
|
||||
|
|
@ -307,7 +308,7 @@ func TestGetMigInstances(t *testing.T) {
|
|||
fetchMigInstances: tc.fetchMigInstances,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
instances, err := provider.GetMigInstances(mig.GceRef())
|
||||
cachedInstances, cached := tc.cache.GetMigInstances(mig.GceRef())
|
||||
|
||||
|
|
@ -471,7 +472,7 @@ func TestRegenerateMigInstancesCache(t *testing.T) {
|
|||
fetchMigInstances: tc.fetchMigInstances,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
err := provider.RegenerateMigInstancesCache()
|
||||
|
||||
assert.Equal(t, tc.expectedErr, err)
|
||||
|
|
@ -550,7 +551,7 @@ func TestGetMigTargetSize(t *testing.T) {
|
|||
fetchMigTargetSize: tc.fetchMigTargetSize,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
|
||||
targetSize, err := provider.GetMigTargetSize(mig.GceRef())
|
||||
cachedTargetSize, found := tc.cache.GetMigTargetSize(mig.GceRef())
|
||||
|
|
@ -632,7 +633,7 @@ func TestGetMigBasename(t *testing.T) {
|
|||
fetchMigBasename: tc.fetchMigBasename,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
|
||||
basename, err := provider.GetMigBasename(mig.GceRef())
|
||||
cachedBasename, found := tc.cache.GetMigBasename(mig.GceRef())
|
||||
|
|
@ -714,7 +715,7 @@ func TestGetMigInstanceTemplateName(t *testing.T) {
|
|||
fetchMigTemplateName: tc.fetchMigTemplateName,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
|
||||
templateName, err := provider.GetMigInstanceTemplateName(mig.GceRef())
|
||||
cachedTemplateName, found := tc.cache.GetMigInstanceTemplateName(mig.GceRef())
|
||||
|
|
@ -820,7 +821,7 @@ func TestGetMigInstanceTemplate(t *testing.T) {
|
|||
fetchMigTemplate: tc.fetchMigTemplate,
|
||||
}
|
||||
migLister := NewMigLister(tc.cache)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(tc.cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
|
||||
template, err := provider.GetMigInstanceTemplate(mig.GceRef())
|
||||
cachedTemplate, found := tc.cache.GetMigInstanceTemplate(mig.GceRef())
|
||||
|
|
@ -915,7 +916,7 @@ func TestGetMigMachineType(t *testing.T) {
|
|||
fetchMachineType: tc.fetchMachineType,
|
||||
}
|
||||
migLister := NewMigLister(cache)
|
||||
provider := NewCachingMigInfoProvider(cache, migLister, client, mig.GceRef().Project, 1)
|
||||
provider := NewCachingMigInfoProvider(cache, migLister, client, mig.GceRef().Project, 1, 0*time.Second)
|
||||
machine, err := provider.GetMigMachineType(mig.GceRef())
|
||||
if tc.expectError {
|
||||
assert.Error(t, err)
|
||||
|
|
@ -928,6 +929,91 @@ func TestGetMigMachineType(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestMultipleGetMigInstanceCallsLimited(t *testing.T) {
|
||||
mig := &gceMig{
|
||||
gceRef: GceRef{
|
||||
Project: "project",
|
||||
Zone: "zone",
|
||||
Name: "base-instance-name",
|
||||
},
|
||||
}
|
||||
instance := cloudprovider.Instance{
|
||||
Id: "gce://project/zone/base-instance-name-abcd",
|
||||
}
|
||||
instanceRef, err := GceRefFromProviderId(instance.Id)
|
||||
assert.Nil(t, err)
|
||||
instance2 := cloudprovider.Instance{
|
||||
Id: "gce://project/zone/base-instance-name-abcd2",
|
||||
}
|
||||
instanceRef2, err := GceRefFromProviderId(instance2.Id)
|
||||
assert.Nil(t, err)
|
||||
now := time.Now()
|
||||
for name, tc := range map[string]struct {
|
||||
refreshRateDuration time.Duration
|
||||
firstCallTime time.Time
|
||||
secondCallTime time.Time
|
||||
expectedCallsToFetchMigInstances int
|
||||
}{
|
||||
"0s refresh rate duration, refetch expected": {
|
||||
refreshRateDuration: 0 * time.Second,
|
||||
firstCallTime: now,
|
||||
secondCallTime: now,
|
||||
expectedCallsToFetchMigInstances: 2,
|
||||
},
|
||||
"5s refresh rate duration, 0.01s between calls, no refetch expected": {
|
||||
refreshRateDuration: 5 * time.Second,
|
||||
firstCallTime: now,
|
||||
secondCallTime: now.Add(10 * time.Millisecond),
|
||||
expectedCallsToFetchMigInstances: 1,
|
||||
},
|
||||
"0.01s refresh rate duration, 0.01s between calls, refetch expected": {
|
||||
refreshRateDuration: 10 * time.Millisecond,
|
||||
firstCallTime: now,
|
||||
secondCallTime: now.Add(11 * time.Millisecond),
|
||||
expectedCallsToFetchMigInstances: 2,
|
||||
},
|
||||
} {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
cache := emptyCache()
|
||||
cache.migs = map[GceRef]Mig{
|
||||
mig.gceRef: mig,
|
||||
}
|
||||
cache.migBaseNameCache = map[GceRef]string{mig.GceRef(): "base-instance-name"}
|
||||
callCounter := make(map[GceRef]int)
|
||||
client := &mockAutoscalingGceClient{
|
||||
fetchMigInstances: fetchMigInstancesWithCounter(nil, callCounter),
|
||||
}
|
||||
migLister := NewMigLister(cache)
|
||||
ft := &fakeTime{}
|
||||
provider := &cachingMigInfoProvider{
|
||||
cache: cache,
|
||||
migLister: migLister,
|
||||
gceClient: client,
|
||||
projectId: projectId,
|
||||
concurrentGceRefreshes: 1,
|
||||
migInstancesMinRefreshWaitTime: tc.refreshRateDuration,
|
||||
migInstancesLastRefreshedInfo: make(map[string]time.Time),
|
||||
timeProvider: ft,
|
||||
}
|
||||
ft.now = tc.firstCallTime
|
||||
_, err = provider.GetMigForInstance(instanceRef)
|
||||
assert.NoError(t, err)
|
||||
ft.now = tc.secondCallTime
|
||||
_, err = provider.GetMigForInstance(instanceRef2)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, tc.expectedCallsToFetchMigInstances, callCounter[mig.GceRef()])
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type fakeTime struct {
|
||||
now time.Time
|
||||
}
|
||||
|
||||
func (f *fakeTime) Now() time.Time {
|
||||
return f.now
|
||||
}
|
||||
|
||||
func emptyCache() *GceCache {
|
||||
return &GceCache{
|
||||
migs: map[GceRef]Mig{mig.GceRef(): mig},
|
||||
|
|
@ -936,6 +1022,7 @@ func emptyCache() *GceCache {
|
|||
migBaseNameCache: make(map[GceRef]string),
|
||||
instanceTemplateNameCache: make(map[GceRef]string),
|
||||
instanceTemplatesCache: make(map[GceRef]*gce.InstanceTemplate),
|
||||
instancesFromUnknownMig: make(map[GceRef]bool),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -959,6 +1046,13 @@ func fetchMigInstancesConst(instances []cloudprovider.Instance) func(GceRef) ([]
|
|||
}
|
||||
}
|
||||
|
||||
func fetchMigInstancesWithCounter(instances []cloudprovider.Instance, migCounter map[GceRef]int) func(GceRef) ([]cloudprovider.Instance, error) {
|
||||
return func(ref GceRef) ([]cloudprovider.Instance, error) {
|
||||
migCounter[ref] = migCounter[ref] + 1
|
||||
return instances, nil
|
||||
}
|
||||
}
|
||||
|
||||
func fetchMigInstancesMapping(instancesMapping map[GceRef][]cloudprovider.Instance) func(GceRef) ([]cloudprovider.Instance, error) {
|
||||
return func(migRef GceRef) ([]cloudprovider.Instance, error) {
|
||||
return instancesMapping[migRef], nil
|
||||
|
|
|
|||
|
|
@ -46,6 +46,16 @@ type NodeGroupAutoscalingOptions struct {
|
|||
ScaleDownUnreadyTime time.Duration
|
||||
}
|
||||
|
||||
// GCEOptions contain autoscaling options specific to GCE cloud provider.
|
||||
type GCEOptions struct {
|
||||
// ConcurrentRefreshes is the maximum number of concurrently refreshed instance groups or instance templates.
|
||||
ConcurrentRefreshes int
|
||||
// MigInstancesMinRefreshWaitTime is the minimum time which needs to pass before GCE MIG instances from a given MIG can be refreshed.
|
||||
MigInstancesMinRefreshWaitTime time.Duration
|
||||
// ExpanderEphemeralStorageSupport is whether scale-up takes ephemeral storage resources into account.
|
||||
ExpanderEphemeralStorageSupport bool
|
||||
}
|
||||
|
||||
const (
|
||||
// DefaultMaxAllocatableDifferenceRatio describes how Node.Status.Allocatable can differ between groups in the same NodeGroupSet
|
||||
DefaultMaxAllocatableDifferenceRatio = 0.05
|
||||
|
|
@ -194,8 +204,8 @@ type AutoscalingOptions struct {
|
|||
BalancingLabels []string
|
||||
// AWSUseStaticInstanceList tells if AWS cloud provider use static instance type list or dynamically fetch from remote APIs.
|
||||
AWSUseStaticInstanceList bool
|
||||
// ConcurrentGceRefreshes is the maximum number of concurrently refreshed instance groups or instance templates.
|
||||
ConcurrentGceRefreshes int
|
||||
// GCEOptions contain autoscaling options specific to GCE cloud provider.
|
||||
GCEOptions GCEOptions
|
||||
// Path to kube configuration if available
|
||||
KubeConfigPath string
|
||||
// Burst setting for kubernetes client
|
||||
|
|
@ -223,8 +233,6 @@ type AutoscalingOptions struct {
|
|||
MaxScaleDownParallelism int
|
||||
// MaxDrainParallelism is the maximum number of nodes needing drain, that can be drained and deleted in parallel.
|
||||
MaxDrainParallelism int
|
||||
// GceExpanderEphemeralStorageSupport is whether scale-up takes ephemeral storage resources into account.
|
||||
GceExpanderEphemeralStorageSupport bool
|
||||
// RecordDuplicatedEvents controls whether events should be duplicated within a 5 minute window.
|
||||
RecordDuplicatedEvents bool
|
||||
// MaxNodesPerScaleUp controls how many nodes can be added in a single scale-up.
|
||||
|
|
|
|||
|
|
@ -187,11 +187,16 @@ var (
|
|||
regional = flag.Bool("regional", false, "Cluster is regional.")
|
||||
newPodScaleUpDelay = flag.Duration("new-pod-scale-up-delay", 0*time.Second, "Pods less than this old will not be considered for scale-up. Can be increased for individual pods through annotation 'cluster-autoscaler.kubernetes.io/pod-scale-up-delay'.")
|
||||
|
||||
ignoreTaintsFlag = multiStringFlag("ignore-taint", "Specifies a taint to ignore in node templates when considering to scale a node group")
|
||||
balancingIgnoreLabelsFlag = multiStringFlag("balancing-ignore-label", "Specifies a label to ignore in addition to the basic and cloud-provider set of labels when comparing if two node groups are similar")
|
||||
balancingLabelsFlag = multiStringFlag("balancing-label", "Specifies a label to use for comparing if two node groups are similar, rather than the built in heuristics. Setting this flag disables all other comparison logic, and cannot be combined with --balancing-ignore-label.")
|
||||
awsUseStaticInstanceList = flag.Bool("aws-use-static-instance-list", false, "Should CA fetch instance types in runtime or use a static list. AWS only")
|
||||
ignoreTaintsFlag = multiStringFlag("ignore-taint", "Specifies a taint to ignore in node templates when considering to scale a node group")
|
||||
balancingIgnoreLabelsFlag = multiStringFlag("balancing-ignore-label", "Specifies a label to ignore in addition to the basic and cloud-provider set of labels when comparing if two node groups are similar")
|
||||
balancingLabelsFlag = multiStringFlag("balancing-label", "Specifies a label to use for comparing if two node groups are similar, rather than the built in heuristics. Setting this flag disables all other comparison logic, and cannot be combined with --balancing-ignore-label.")
|
||||
awsUseStaticInstanceList = flag.Bool("aws-use-static-instance-list", false, "Should CA fetch instance types in runtime or use a static list. AWS only")
|
||||
|
||||
// GCE specific flags
|
||||
concurrentGceRefreshes = flag.Int("gce-concurrent-refreshes", 1, "Maximum number of concurrent refreshes per cloud object type.")
|
||||
gceMigInstancesMinRefreshWaitTime = flag.Duration("gce-mig-instances-min-refresh-wait-time", 5*time.Second, "The minimum time which needs to pass before GCE MIG instances from a given MIG can be refreshed.")
|
||||
gceExpanderEphemeralStorageSupport = flag.Bool("gce-expander-ephemeral-storage-support", false, "Whether scale-up takes ephemeral storage resources into account for GCE cloud provider")
|
||||
|
||||
enableProfiling = flag.Bool("profiling", false, "Is debug/pprof endpoint enabled")
|
||||
clusterAPICloudConfigAuthoritative = flag.Bool("clusterapi-cloud-config-authoritative", false, "Treat the cloud-config flag authoritatively (do not fallback to using kubeconfig flag). ClusterAPI only")
|
||||
cordonNodeBeforeTerminate = flag.Bool("cordon-node-before-terminating", false, "Should CA cordon nodes before terminating during downscale process")
|
||||
|
|
@ -208,23 +213,22 @@ var (
|
|||
"maxNodeGroupBackoffDuration is the maximum backoff duration for a NodeGroup after new nodes failed to start.")
|
||||
nodeGroupBackoffResetTimeout = flag.Duration("node-group-backoff-reset-timeout", 3*time.Hour,
|
||||
"nodeGroupBackoffResetTimeout is the time after last failed scale-up when the backoff duration is reset.")
|
||||
maxScaleDownParallelismFlag = flag.Int("max-scale-down-parallelism", 10, "Maximum number of nodes (both empty and needing drain) that can be deleted in parallel.")
|
||||
maxDrainParallelismFlag = flag.Int("max-drain-parallelism", 1, "Maximum number of nodes needing drain, that can be drained and deleted in parallel.")
|
||||
gceExpanderEphemeralStorageSupport = flag.Bool("gce-expander-ephemeral-storage-support", false, "Whether scale-up takes ephemeral storage resources into account for GCE cloud provider")
|
||||
recordDuplicatedEvents = flag.Bool("record-duplicated-events", false, "enable duplication of similar events within a 5 minute window.")
|
||||
maxNodesPerScaleUp = flag.Int("max-nodes-per-scaleup", 1000, "Max nodes added in a single scale-up. This is intended strictly for optimizing CA algorithm latency and not a tool to rate-limit scale-up throughput.")
|
||||
maxNodeGroupBinpackingDuration = flag.Duration("max-nodegroup-binpacking-duration", 10*time.Second, "Maximum time that will be spent in binpacking simulation for each NodeGroup.")
|
||||
skipNodesWithSystemPods = flag.Bool("skip-nodes-with-system-pods", true, "If true cluster autoscaler will never delete nodes with pods from kube-system (except for DaemonSet or mirror pods)")
|
||||
skipNodesWithLocalStorage = flag.Bool("skip-nodes-with-local-storage", true, "If true cluster autoscaler will never delete nodes with pods with local storage, e.g. EmptyDir or HostPath")
|
||||
skipNodesWithCustomControllerPods = flag.Bool("skip-nodes-with-custom-controller-pods", true, "If true cluster autoscaler will never delete nodes with pods owned by custom controllers")
|
||||
minReplicaCount = flag.Int("min-replica-count", 0, "Minimum number or replicas that a replica set or replication controller should have to allow their pods deletion in scale down")
|
||||
nodeDeleteDelayAfterTaint = flag.Duration("node-delete-delay-after-taint", 5*time.Second, "How long to wait before deleting a node after tainting it")
|
||||
scaleDownSimulationTimeout = flag.Duration("scale-down-simulation-timeout", 30*time.Second, "How long should we run scale down simulation.")
|
||||
parallelDrain = flag.Bool("parallel-drain", false, "Whether to allow parallel drain of nodes.")
|
||||
maxCapacityMemoryDifferenceRatio = flag.Float64("memory-difference-ratio", config.DefaultMaxCapacityMemoryDifferenceRatio, "Maximum difference in memory capacity between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's memory capacity.")
|
||||
maxFreeDifferenceRatio = flag.Float64("max-free-difference-ratio", config.DefaultMaxFreeDifferenceRatio, "Maximum difference in free resources between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's free resource.")
|
||||
maxAllocatableDifferenceRatio = flag.Float64("max-allocatable-difference-ratio", config.DefaultMaxAllocatableDifferenceRatio, "Maximum difference in allocatable resources between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's allocatable resource.")
|
||||
forceDaemonSets = flag.Bool("force-ds", false, "Blocks scale-up of node groups too small for all suitable Daemon Sets pods.")
|
||||
maxScaleDownParallelismFlag = flag.Int("max-scale-down-parallelism", 10, "Maximum number of nodes (both empty and needing drain) that can be deleted in parallel.")
|
||||
maxDrainParallelismFlag = flag.Int("max-drain-parallelism", 1, "Maximum number of nodes needing drain, that can be drained and deleted in parallel.")
|
||||
recordDuplicatedEvents = flag.Bool("record-duplicated-events", false, "enable duplication of similar events within a 5 minute window.")
|
||||
maxNodesPerScaleUp = flag.Int("max-nodes-per-scaleup", 1000, "Max nodes added in a single scale-up. This is intended strictly for optimizing CA algorithm latency and not a tool to rate-limit scale-up throughput.")
|
||||
maxNodeGroupBinpackingDuration = flag.Duration("max-nodegroup-binpacking-duration", 10*time.Second, "Maximum time that will be spent in binpacking simulation for each NodeGroup.")
|
||||
skipNodesWithSystemPods = flag.Bool("skip-nodes-with-system-pods", true, "If true cluster autoscaler will never delete nodes with pods from kube-system (except for DaemonSet or mirror pods)")
|
||||
skipNodesWithLocalStorage = flag.Bool("skip-nodes-with-local-storage", true, "If true cluster autoscaler will never delete nodes with pods with local storage, e.g. EmptyDir or HostPath")
|
||||
skipNodesWithCustomControllerPods = flag.Bool("skip-nodes-with-custom-controller-pods", true, "If true cluster autoscaler will never delete nodes with pods owned by custom controllers")
|
||||
minReplicaCount = flag.Int("min-replica-count", 0, "Minimum number or replicas that a replica set or replication controller should have to allow their pods deletion in scale down")
|
||||
nodeDeleteDelayAfterTaint = flag.Duration("node-delete-delay-after-taint", 5*time.Second, "How long to wait before deleting a node after tainting it")
|
||||
scaleDownSimulationTimeout = flag.Duration("scale-down-simulation-timeout", 30*time.Second, "How long should we run scale down simulation.")
|
||||
parallelDrain = flag.Bool("parallel-drain", false, "Whether to allow parallel drain of nodes.")
|
||||
maxCapacityMemoryDifferenceRatio = flag.Float64("memory-difference-ratio", config.DefaultMaxCapacityMemoryDifferenceRatio, "Maximum difference in memory capacity between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's memory capacity.")
|
||||
maxFreeDifferenceRatio = flag.Float64("max-free-difference-ratio", config.DefaultMaxFreeDifferenceRatio, "Maximum difference in free resources between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's free resource.")
|
||||
maxAllocatableDifferenceRatio = flag.Float64("max-allocatable-difference-ratio", config.DefaultMaxAllocatableDifferenceRatio, "Maximum difference in allocatable resources between two similar node groups to be considered for balancing. Value is a ratio of the smaller node group's allocatable resource.")
|
||||
forceDaemonSets = flag.Bool("force-ds", false, "Blocks scale-up of node groups too small for all suitable Daemon Sets pods.")
|
||||
)
|
||||
|
||||
func createAutoscalingOptions() config.AutoscalingOptions {
|
||||
|
|
@ -254,60 +258,64 @@ func createAutoscalingOptions() config.AutoscalingOptions {
|
|||
ScaleDownUnneededTime: *scaleDownUnneededTime,
|
||||
ScaleDownUnreadyTime: *scaleDownUnreadyTime,
|
||||
},
|
||||
CloudConfig: *cloudConfig,
|
||||
CloudProviderName: *cloudProviderFlag,
|
||||
NodeGroupAutoDiscovery: *nodeGroupAutoDiscoveryFlag,
|
||||
MaxTotalUnreadyPercentage: *maxTotalUnreadyPercentage,
|
||||
OkTotalUnreadyCount: *okTotalUnreadyCount,
|
||||
ScaleUpFromZero: *scaleUpFromZero,
|
||||
EstimatorName: *estimatorFlag,
|
||||
ExpanderNames: *expanderFlag,
|
||||
GRPCExpanderCert: *grpcExpanderCert,
|
||||
GRPCExpanderURL: *grpcExpanderURL,
|
||||
IgnoreDaemonSetsUtilization: *ignoreDaemonSetsUtilization,
|
||||
IgnoreMirrorPodsUtilization: *ignoreMirrorPodsUtilization,
|
||||
MaxBulkSoftTaintCount: *maxBulkSoftTaintCount,
|
||||
MaxBulkSoftTaintTime: *maxBulkSoftTaintTime,
|
||||
MaxEmptyBulkDelete: *maxEmptyBulkDeleteFlag,
|
||||
MaxGracefulTerminationSec: *maxGracefulTerminationFlag,
|
||||
MaxNodeProvisionTime: *maxNodeProvisionTime,
|
||||
MaxPodEvictionTime: *maxPodEvictionTime,
|
||||
MaxNodesTotal: *maxNodesTotal,
|
||||
MaxCoresTotal: maxCoresTotal,
|
||||
MinCoresTotal: minCoresTotal,
|
||||
MaxMemoryTotal: maxMemoryTotal,
|
||||
MinMemoryTotal: minMemoryTotal,
|
||||
GpuTotal: parsedGpuTotal,
|
||||
NodeGroups: *nodeGroupsFlag,
|
||||
EnforceNodeGroupMinSize: *enforceNodeGroupMinSize,
|
||||
ScaleDownDelayAfterAdd: *scaleDownDelayAfterAdd,
|
||||
ScaleDownDelayAfterDelete: *scaleDownDelayAfterDelete,
|
||||
ScaleDownDelayAfterFailure: *scaleDownDelayAfterFailure,
|
||||
ScaleDownEnabled: *scaleDownEnabled,
|
||||
ScaleDownUnreadyEnabled: *scaleDownUnreadyEnabled,
|
||||
ScaleDownNonEmptyCandidatesCount: *scaleDownNonEmptyCandidatesCount,
|
||||
ScaleDownCandidatesPoolRatio: *scaleDownCandidatesPoolRatio,
|
||||
ScaleDownCandidatesPoolMinCount: *scaleDownCandidatesPoolMinCount,
|
||||
WriteStatusConfigMap: *writeStatusConfigMapFlag,
|
||||
StatusConfigMapName: *statusConfigMapName,
|
||||
BalanceSimilarNodeGroups: *balanceSimilarNodeGroupsFlag,
|
||||
ConfigNamespace: *namespace,
|
||||
ClusterName: *clusterName,
|
||||
NodeAutoprovisioningEnabled: *nodeAutoprovisioningEnabled,
|
||||
MaxAutoprovisionedNodeGroupCount: *maxAutoprovisionedNodeGroupCount,
|
||||
UnremovableNodeRecheckTimeout: *unremovableNodeRecheckTimeout,
|
||||
ExpendablePodsPriorityCutoff: *expendablePodsPriorityCutoff,
|
||||
Regional: *regional,
|
||||
NewPodScaleUpDelay: *newPodScaleUpDelay,
|
||||
IgnoredTaints: *ignoreTaintsFlag,
|
||||
BalancingExtraIgnoredLabels: *balancingIgnoreLabelsFlag,
|
||||
BalancingLabels: *balancingLabelsFlag,
|
||||
KubeConfigPath: *kubeConfigFile,
|
||||
KubeClientBurst: *kubeClientBurst,
|
||||
KubeClientQPS: *kubeClientQPS,
|
||||
NodeDeletionDelayTimeout: *nodeDeletionDelayTimeout,
|
||||
AWSUseStaticInstanceList: *awsUseStaticInstanceList,
|
||||
ConcurrentGceRefreshes: *concurrentGceRefreshes,
|
||||
CloudConfig: *cloudConfig,
|
||||
CloudProviderName: *cloudProviderFlag,
|
||||
NodeGroupAutoDiscovery: *nodeGroupAutoDiscoveryFlag,
|
||||
MaxTotalUnreadyPercentage: *maxTotalUnreadyPercentage,
|
||||
OkTotalUnreadyCount: *okTotalUnreadyCount,
|
||||
ScaleUpFromZero: *scaleUpFromZero,
|
||||
EstimatorName: *estimatorFlag,
|
||||
ExpanderNames: *expanderFlag,
|
||||
GRPCExpanderCert: *grpcExpanderCert,
|
||||
GRPCExpanderURL: *grpcExpanderURL,
|
||||
IgnoreDaemonSetsUtilization: *ignoreDaemonSetsUtilization,
|
||||
IgnoreMirrorPodsUtilization: *ignoreMirrorPodsUtilization,
|
||||
MaxBulkSoftTaintCount: *maxBulkSoftTaintCount,
|
||||
MaxBulkSoftTaintTime: *maxBulkSoftTaintTime,
|
||||
MaxEmptyBulkDelete: *maxEmptyBulkDeleteFlag,
|
||||
MaxGracefulTerminationSec: *maxGracefulTerminationFlag,
|
||||
MaxNodeProvisionTime: *maxNodeProvisionTime,
|
||||
MaxPodEvictionTime: *maxPodEvictionTime,
|
||||
MaxNodesTotal: *maxNodesTotal,
|
||||
MaxCoresTotal: maxCoresTotal,
|
||||
MinCoresTotal: minCoresTotal,
|
||||
MaxMemoryTotal: maxMemoryTotal,
|
||||
MinMemoryTotal: minMemoryTotal,
|
||||
GpuTotal: parsedGpuTotal,
|
||||
NodeGroups: *nodeGroupsFlag,
|
||||
EnforceNodeGroupMinSize: *enforceNodeGroupMinSize,
|
||||
ScaleDownDelayAfterAdd: *scaleDownDelayAfterAdd,
|
||||
ScaleDownDelayAfterDelete: *scaleDownDelayAfterDelete,
|
||||
ScaleDownDelayAfterFailure: *scaleDownDelayAfterFailure,
|
||||
ScaleDownEnabled: *scaleDownEnabled,
|
||||
ScaleDownUnreadyEnabled: *scaleDownUnreadyEnabled,
|
||||
ScaleDownNonEmptyCandidatesCount: *scaleDownNonEmptyCandidatesCount,
|
||||
ScaleDownCandidatesPoolRatio: *scaleDownCandidatesPoolRatio,
|
||||
ScaleDownCandidatesPoolMinCount: *scaleDownCandidatesPoolMinCount,
|
||||
WriteStatusConfigMap: *writeStatusConfigMapFlag,
|
||||
StatusConfigMapName: *statusConfigMapName,
|
||||
BalanceSimilarNodeGroups: *balanceSimilarNodeGroupsFlag,
|
||||
ConfigNamespace: *namespace,
|
||||
ClusterName: *clusterName,
|
||||
NodeAutoprovisioningEnabled: *nodeAutoprovisioningEnabled,
|
||||
MaxAutoprovisionedNodeGroupCount: *maxAutoprovisionedNodeGroupCount,
|
||||
UnremovableNodeRecheckTimeout: *unremovableNodeRecheckTimeout,
|
||||
ExpendablePodsPriorityCutoff: *expendablePodsPriorityCutoff,
|
||||
Regional: *regional,
|
||||
NewPodScaleUpDelay: *newPodScaleUpDelay,
|
||||
IgnoredTaints: *ignoreTaintsFlag,
|
||||
BalancingExtraIgnoredLabels: *balancingIgnoreLabelsFlag,
|
||||
BalancingLabels: *balancingLabelsFlag,
|
||||
KubeConfigPath: *kubeConfigFile,
|
||||
KubeClientBurst: *kubeClientBurst,
|
||||
KubeClientQPS: *kubeClientQPS,
|
||||
NodeDeletionDelayTimeout: *nodeDeletionDelayTimeout,
|
||||
AWSUseStaticInstanceList: *awsUseStaticInstanceList,
|
||||
GCEOptions: config.GCEOptions{
|
||||
ConcurrentRefreshes: *concurrentGceRefreshes,
|
||||
MigInstancesMinRefreshWaitTime: *gceMigInstancesMinRefreshWaitTime,
|
||||
ExpanderEphemeralStorageSupport: *gceExpanderEphemeralStorageSupport,
|
||||
},
|
||||
ClusterAPICloudConfigAuthoritative: *clusterAPICloudConfigAuthoritative,
|
||||
CordonNodeBeforeTerminate: *cordonNodeBeforeTerminate,
|
||||
DaemonSetEvictionForEmptyNodes: *daemonSetEvictionForEmptyNodes,
|
||||
|
|
@ -318,7 +326,6 @@ func createAutoscalingOptions() config.AutoscalingOptions {
|
|||
NodeGroupBackoffResetTimeout: *nodeGroupBackoffResetTimeout,
|
||||
MaxScaleDownParallelism: *maxScaleDownParallelismFlag,
|
||||
MaxDrainParallelism: *maxDrainParallelismFlag,
|
||||
GceExpanderEphemeralStorageSupport: *gceExpanderEphemeralStorageSupport,
|
||||
RecordDuplicatedEvents: *recordDuplicatedEvents,
|
||||
MaxNodesPerScaleUp: *maxNodesPerScaleUp,
|
||||
MaxNodeGroupBinpackingDuration: *maxNodeGroupBinpackingDuration,
|
||||
|
|
|
|||
Loading…
Reference in New Issue