From 27913df0c3b59f88a9370a042eaabe18ab6b925e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 08:10:45 +0000 Subject: [PATCH 01/21] feat(planner): merge planner.go and refinedplanner.go --- .../scheduler/framework/plugins/rsp/rsp.go | 22 +- pkg/controllers/scheduler/framework/types.go | 5 +- pkg/controllers/util/planner/planner.go | 404 +++--- pkg/controllers/util/planner/planner_test.go | 1243 +++++++++++++---- .../util/refinedplanner/planner.go | 299 ---- .../util/refinedplanner/planner_test.go | 314 ----- 6 files changed, 1190 insertions(+), 1097 deletions(-) delete mode 100644 pkg/controllers/util/refinedplanner/planner.go delete mode 100644 pkg/controllers/util/refinedplanner/planner_test.go diff --git a/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go b/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go index 93bbe9cd..e82ba7af 100644 --- a/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go +++ b/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go @@ -35,7 +35,6 @@ import ( fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" "github.com/kubewharf/kubeadmiral/pkg/controllers/scheduler/framework" "github.com/kubewharf/kubeadmiral/pkg/controllers/util/planner" - "github.com/kubewharf/kubeadmiral/pkg/controllers/util/refinedplanner" ) const ( @@ -121,13 +120,6 @@ func (pl *ClusterCapacityWeight) ReplicaScheduling( if su.DesiredReplicas != nil { totalReplicas = *su.DesiredReplicas } - clusterPreference := &refinedplanner.ReplicaSchedulingPreference{ - TotalReplicas: totalReplicas, - Clusters: clusterPreferences, - Rebalance: true, - } - - planner := refinedplanner.NewPlanner(clusterPreference) currentReplicas := map[string]int64{} for cluster, replicas := range su.CurrentClusters { @@ -135,14 +127,20 @@ func (pl *ClusterCapacityWeight) ReplicaScheduling( currentReplicas[cluster] = *replicas continue } - currentReplicas[cluster] = *su.DesiredReplicas + currentReplicas[cluster] = totalReplicas } scheduleResult, overflow, err := planner.Plan( + &planner.ReplicaSchedulingPreference{ + Clusters: clusterPreferences, + }, + totalReplicas, ExtractClusterNames(clusters), currentReplicas, - map[string]int64{}, - "", + nil, + su.Key(), + false, + true, ) if err != nil { return clusterReplicasList, framework.NewResult(framework.Error) @@ -150,7 +148,7 @@ func (pl *ClusterCapacityWeight) ReplicaScheduling( klog.V(4).Infof( "[scheduling] for %q clusterPreferences: %s, currentReplicas: %s, result: %s", - su.Key(), spew.Sprint(clusterPreference), spew.Sprint(currentReplicas), spew.Sprint(scheduleResult), + su.Key(), spew.Sprint(clusterPreferences), spew.Sprint(currentReplicas), spew.Sprint(scheduleResult), ) // TODO: Check if we really need to place the federated type in clusters diff --git a/pkg/controllers/scheduler/framework/types.go b/pkg/controllers/scheduler/framework/types.go index bb8bba23..0051fcf9 100644 --- a/pkg/controllers/scheduler/framework/types.go +++ b/pkg/controllers/scheduler/framework/types.go @@ -104,7 +104,10 @@ type PreferredSchedulingTerm struct { } func (s *SchedulingUnit) Key() string { - return s.Namespace + "/" + s.Name + if len(s.Namespace) > 0 { + return s.Namespace + "/" + s.Name + } + return s.Name } type ClusterScore struct { diff --git a/pkg/controllers/util/planner/planner.go b/pkg/controllers/util/planner/planner.go index 2692d926..d2a0bb40 100644 --- a/pkg/controllers/util/planner/planner.go +++ b/pkg/controllers/util/planner/planner.go @@ -25,28 +25,6 @@ import ( "sort" ) -type ReplicaSchedulingPreference struct { - - // Total number of pods desired across federated clusters. - // Replicas specified in the spec for target deployment template or replicaset - // template will be discarded/overridden when scheduling preferences are - // specified. - TotalReplicas int64 - - // If set to true then already scheduled and running replicas may be moved to other clusters - // in order to match current state to the specified preferences. Otherwise, if set to false, - // up and running replicas will not be moved. - // +optional - Rebalance bool - - // A mapping between cluster names and preferences regarding a local workload object (dep, rs, .. ) in - // these clusters. - // "*" (if provided) applies to all clusters if an explicit mapping is not provided. - // If omitted, clusters without explicit preferences should not have any replicas scheduled. - // +optional - Clusters map[string]ClusterPreferences -} - // Preferences regarding number of replicas assigned to a cluster workload object (dep, rs, ..) within // a federated workload object. type ClusterPreferences struct { @@ -62,10 +40,12 @@ type ClusterPreferences struct { Weight int64 } -// Planner decides how many out of the given replicas should be placed in each of the -// federated clusters. -type Planner struct { - preferences *ReplicaSchedulingPreference +type ReplicaSchedulingPreference struct { + // A mapping between cluster names and preferences regarding a local workload object (dep, rs, .. ) in + // these clusters. + // "*" (if provided) applies to all clusters if an explicit mapping is not provided. + // If omitted, clusters without explicit preferences should not have any replicas scheduled. + Clusters map[string]ClusterPreferences } type namedClusterPreferences struct { @@ -85,12 +65,6 @@ func (a byWeight) Less(i, j int) bool { return (a[i].Weight > a[j].Weight) || (a[i].Weight == a[j].Weight && a[i].hash < a[j].hash) } -func NewPlanner(preferences *ReplicaSchedulingPreference) *Planner { - return &Planner{ - preferences: preferences, - } -} - // Distribute the desired number of replicas among the given cluster according to the planner preferences. // The function tries its best to assign each cluster the preferred number of replicas, however if // sum of MinReplicas for all cluster is bigger than replicasToDistribute (TotalReplicas) then some cluster @@ -103,15 +77,108 @@ func NewPlanner(preferences *ReplicaSchedulingPreference) *Planner { // - a map that contains information how many replicas will be possible to run in a cluster. // - a map that contains information how many extra replicas would be nice to schedule in a cluster so, // if by chance, they are scheduled we will be closer to the desired replicas layout. -func (p *Planner) Plan(availableClusters []string, currentReplicaCount map[string]int64, - estimatedCapacity map[string]int64, replicaSetKey string) (map[string]int64, map[string]int64, error) { +func Plan( + rsp *ReplicaSchedulingPreference, + totalReplicas int64, + availableClusters []string, + currentReplicaCount map[string]int64, + estimatedCapacity map[string]int64, + replicaSetKey string, + avoidDisruption bool, + keepUnschedulableReplicas bool, +) (map[string]int64, map[string]int64, error) { + preferences := make(map[string]*ClusterPreferences, len(availableClusters)) - preferences := make([]*namedClusterPreferences, 0, len(availableClusters)) - plan := make(map[string]int64, len(preferences)) - overflow := make(map[string]int64, len(preferences)) + for _, cluster := range availableClusters { + if preference, found := rsp.Clusters[cluster]; found { + preferences[cluster] = &preference + } else if preference, found := rsp.Clusters["*"]; found { + preferences[cluster] = &preference + } + } + + namedPreferences, err := getNamedPreferences(preferences, replicaSetKey) + if err != nil { + return nil, nil, err + } + + // If keepUnschedulableReplicas is false, + // the resultant plan will likely violate the preferences + // if any cluster has limited capacity. + // If avoidDisruption is also false, a subsequent reschedule will restore + // the replica distribution to the state before we moved the unschedulable + // replicas. This leads to a infinite reschedule loop and is undesirable. + // Therefore we default to keeping the unschedulable replicas if avoidDisruption + // is false. + if !avoidDisruption { + keepUnschedulableReplicas = true + } + + desiredPlan, desiredOverflow := getDesiredPlan( + namedPreferences, + estimatedCapacity, + totalReplicas, + keepUnschedulableReplicas, + ) + + // If we don't want to avoid migration, just return the plan computed from preferences + if !avoidDisruption { + return desiredPlan, desiredOverflow, nil + } + + // Try to avoid instance migration between clusters + + var currentTotalOkReplicas int64 + // currentPlan should only contain clusters in availableClusters + currentPlan := make(map[string]int64, len(namedPreferences)) + for _, preference := range namedPreferences { + replicas := currentReplicaCount[preference.clusterName] + if capacity, exists := estimatedCapacity[preference.clusterName]; exists && capacity < replicas { + replicas = capacity + } + currentPlan[preference.clusterName] = replicas + + currentTotalOkReplicas += replicas + } + + var desiredTotalReplicas int64 + for _, replicas := range desiredPlan { + desiredTotalReplicas += replicas + } + + switch { + case currentTotalOkReplicas == desiredTotalReplicas: + return currentPlan, desiredOverflow, nil + case currentTotalOkReplicas > desiredTotalReplicas: + plan, err := scaleDown( + currentPlan, desiredPlan, + currentTotalOkReplicas-desiredTotalReplicas, + replicaSetKey, + ) + if err != nil { + return nil, nil, err + } + return plan, desiredOverflow, nil + default: + plan, err := scaleUp( + rsp, + currentPlan, desiredPlan, + desiredTotalReplicas-currentTotalOkReplicas, + replicaSetKey, + ) + if err != nil { + return nil, nil, err + } + return plan, desiredOverflow, nil + } +} - named := func(name string, pref ClusterPreferences) (*namedClusterPreferences, error) { - // Seems to work better than addler for our case. +func getNamedPreferences( + preferences map[string]*ClusterPreferences, + replicaSetKey string, +) ([]*namedClusterPreferences, error) { + namedPreferences := make([]*namedClusterPreferences, 0, len(preferences)) + named := func(name string, pref *ClusterPreferences) (*namedClusterPreferences, error) { hasher := fnv.New32() if _, err := hasher.Write([]byte(name)); err != nil { return nil, err @@ -123,165 +190,176 @@ func (p *Planner) Plan(availableClusters []string, currentReplicaCount map[strin return &namedClusterPreferences{ clusterName: name, hash: hasher.Sum32(), - ClusterPreferences: pref, + ClusterPreferences: *pref, }, nil } - for _, cluster := range availableClusters { - if localRSP, found := p.preferences.Clusters[cluster]; found { - preference, err := named(cluster, localRSP) - if err != nil { - return nil, nil, err - } - - preferences = append(preferences, preference) - } else { - if localRSP, found := p.preferences.Clusters["*"]; found { - preference, err := named(cluster, localRSP) - if err != nil { - return nil, nil, err - } - - preferences = append(preferences, preference) - } else { - plan[cluster] = int64(0) - } + for name, preference := range preferences { + namedPreference, err := named(name, preference) + if err != nil { + return nil, err } + namedPreferences = append(namedPreferences, namedPreference) } - sort.Sort(byWeight(preferences)) + sort.Sort(byWeight(namedPreferences)) + return namedPreferences, nil +} - // This is the requested total replicas in preferences - remainingReplicas := int64(p.preferences.TotalReplicas) +func getDesiredPlan( + preferences []*namedClusterPreferences, + estimatedCapacity map[string]int64, + totalReplicas int64, + keepUnschedulableReplicas bool, +) (map[string]int64, map[string]int64) { + remainingReplicas := totalReplicas + plan := make(map[string]int64, len(preferences)) + overflow := make(map[string]int64, len(preferences)) // Assign each cluster the minimum number of replicas it requested. for _, preference := range preferences { min := minInt64(preference.MinReplicas, remainingReplicas) - if capacity, hasCapacity := estimatedCapacity[preference.clusterName]; hasCapacity { - min = minInt64(min, capacity) + if capacity, hasCapacity := estimatedCapacity[preference.clusterName]; hasCapacity && capacity < min { + overflow[preference.clusterName] = min - capacity + min = capacity } remainingReplicas -= min plan[preference.clusterName] = min } - // This map contains information how many replicas were assigned to - // the cluster based only on the current replica count and - // rebalance=false preference. It will be later used in remaining replica - // distribution code. - preallocated := make(map[string]int64) - - if !p.preferences.Rebalance { - for _, preference := range preferences { - planned := plan[preference.clusterName] - count, hasSome := currentReplicaCount[preference.clusterName] - if hasSome && count > planned { - target := count - if preference.MaxReplicas != nil { - target = minInt64(*preference.MaxReplicas, target) - } - if capacity, hasCapacity := estimatedCapacity[preference.clusterName]; hasCapacity { - target = minInt64(capacity, target) - } - extra := minInt64(target-planned, remainingReplicas) - if extra < 0 { - extra = 0 - } - remainingReplicas -= extra - preallocated[preference.clusterName] = extra - plan[preference.clusterName] = extra + planned - } - } - } - modified := true - // It is possible single pass of the loop is not enough to distribute all replicas among clusters due // to weight, max and rounding corner cases. In such case we iterate until either - // there is no replicas or no cluster gets any more replicas or the number - // of attempts is less than available cluster count. If there is no preallocated pods - // every loop either distributes all remainingReplicas or maxes out at least one cluster. - // If there are preallocated then the replica spreading may take longer. - // We reduce the number of pending preallocated replicas by at least half with each iteration so - // we may need log(replicasAtStart) iterations. - // TODO: Prove that clusterCount * log(replicas) iterations solves the problem or adjust the number. - // TODO: This algorithm is O(clusterCount^2 * log(replicas)) which is good for up to 100 clusters. - // Find something faster. - for trial := 0; modified && remainingReplicas > 0; trial++ { - + // there is no replicas or no cluster gets any more replicas. Every loop either distributes all remainingReplicas + // or maxes out at least one cluster. + for modified && remainingReplicas > 0 { modified = false weightSum := int64(0) for _, preference := range preferences { weightSum += preference.Weight } + if weightSum <= 0 { + break + } newPreferences := make([]*namedClusterPreferences, 0, len(preferences)) distributeInThisLoop := remainingReplicas - for _, preference := range preferences { - if weightSum > 0 { - start := plan[preference.clusterName] - // Distribute the remaining replicas, rounding fractions always up. - extra := (distributeInThisLoop*preference.Weight + weightSum - 1) / weightSum - extra = minInt64(extra, remainingReplicas) - - // Account preallocated. - prealloc := preallocated[preference.clusterName] - usedPrealloc := minInt64(extra, prealloc) - preallocated[preference.clusterName] = prealloc - usedPrealloc - extra = extra - usedPrealloc - if usedPrealloc > 0 { - modified = true - } - - // In total there should be the amount that was there at start plus whatever is due - // in this iteration - total := start + extra - - // Check if we don't overflow the cluster, and if yes don't consider this cluster - // in any of the following iterations. - full := false - if preference.MaxReplicas != nil && total > *preference.MaxReplicas { - total = *preference.MaxReplicas - full = true - } - if capacity, hasCapacity := estimatedCapacity[preference.clusterName]; hasCapacity && total > capacity { - overflow[preference.clusterName] = total - capacity - total = capacity - full = true - } - - if !full { - newPreferences = append(newPreferences, preference) - } - - // Only total-start replicas were actually taken. - remainingReplicas -= (total - start) - plan[preference.clusterName] = total - - // Something extra got scheduled on this cluster. - if total > start { - modified = true - } - } else { - break + start := plan[preference.clusterName] + // Distribute the remaining replicas, rounding fractions always up. + extra := (distributeInThisLoop*preference.Weight + weightSum - 1) / weightSum + extra = minInt64(extra, remainingReplicas) + + // In total there should be the amount that was there at start plus whatever is due + // in this iteration + total := start + extra + + // Check if we don't overflow the cluster, and if yes don't consider this cluster + // in any of the following iterations. + full := false + if preference.MaxReplicas != nil && total > *preference.MaxReplicas { + total = *preference.MaxReplicas + full = true + } + if capacity, hasCapacity := estimatedCapacity[preference.clusterName]; hasCapacity && total > capacity { + overflow[preference.clusterName] += total - capacity + total = capacity + full = true + } + if !full { + newPreferences = append(newPreferences, preference) + } + + // Only total-start replicas were actually taken. + remainingReplicas -= total - start + plan[preference.clusterName] = total + + // Something extra got scheduled on this cluster. + if total > start { + modified = true } } preferences = newPreferences } - if p.preferences.Rebalance { - return plan, overflow, nil - } else { - // If rebalance = false then overflow is trimmed at the level - // of replicas that it failed to place somewhere. - newOverflow := make(map[string]int64) - for key, value := range overflow { - value = minInt64(value, remainingReplicas) - if value > 0 { - newOverflow[key] = value + // If we want to keep the unschedulable replicas in their original + // clusters, we return the overflow (which contains these + // unschedulable replicas) as is. + if keepUnschedulableReplicas { + return plan, overflow + } + + // Otherwise, trim overflow at the level + // of replicas that the algorithm failed to place anywhere. + newOverflow := make(map[string]int64) + for key, value := range overflow { + value = minInt64(value, remainingReplicas) + if value > 0 { + newOverflow[key] = value + } + } + return plan, newOverflow +} + +func scaleUp( + rsp *ReplicaSchedulingPreference, + currentReplicaCount, desiredReplicaCount map[string]int64, + scaleUpCount int64, + replicaSetKey string, +) (map[string]int64, error) { + preferences := make(map[string]*ClusterPreferences, len(desiredReplicaCount)) + for cluster, desired := range desiredReplicaCount { + // only pick clusters which have less replicas than desired to sale up, thus replica migration between clusters can be avoid + current := currentReplicaCount[cluster] + if desired > current { + preferences[cluster] = &ClusterPreferences{ + Weight: desired - current, + } + if rsp.Clusters[cluster].MaxReplicas != nil { + // note that max is always positive because MaxReplicas >= desired > current + max := *rsp.Clusters[cluster].MaxReplicas - current + preferences[cluster].MaxReplicas = &max } } - return plan, newOverflow, nil } + + named, err := getNamedPreferences(preferences, replicaSetKey) + if err != nil { + return nil, err + } + // no estimatedCapacity and hence no overflow + replicasToScaleUp, _ := getDesiredPlan(named, nil, scaleUpCount, false) + for cluster, count := range replicasToScaleUp { + currentReplicaCount[cluster] += count + } + return currentReplicaCount, nil +} + +func scaleDown( + currentReplicaCount, desiredReplicaCount map[string]int64, + scaleDownCount int64, + replicaSetKey string, +) (map[string]int64, error) { + preferences := make(map[string]*ClusterPreferences, len(desiredReplicaCount)) + for cluster, desired := range desiredReplicaCount { + // only pick clusters which have more replicas than desired to scale down, thus replica migration between clusters can be avoid + current := currentReplicaCount[cluster] + if desired < current { + preferences[cluster] = &ClusterPreferences{ + Weight: current - desired, + MaxReplicas: ¤t, + } + } + } + named, err := getNamedPreferences(preferences, replicaSetKey) + if err != nil { + return nil, err + } + // no estimatedCapacity and hence no overflow + replicasToScaleDown, _ := getDesiredPlan(named, nil, scaleDownCount, false) + for cluster, count := range replicasToScaleDown { + currentReplicaCount[cluster] -= count + } + return currentReplicaCount, nil } func minInt64(a int64, b int64) int64 { diff --git a/pkg/controllers/util/planner/planner_test.go b/pkg/controllers/util/planner/planner_test.go index d7908a47..ba9605e9 100644 --- a/pkg/controllers/util/planner/planner_test.go +++ b/pkg/controllers/util/planner/planner_test.go @@ -21,336 +21,963 @@ are Copyright 2023 The KubeAdmiral Authors. package planner import ( + "reflect" "testing" "github.com/stretchr/testify/assert" + "k8s.io/utils/pointer" ) -func doCheck(t *testing.T, pref map[string]ClusterPreferences, replicas int64, clusters []string, expected map[string]int64) { - planer := NewPlanner(&ReplicaSchedulingPreference{ - Clusters: pref, - TotalReplicas: replicas, - }) - plan, overflow, err := planer.Plan(clusters, map[string]int64{}, map[string]int64{}, "") - assert.Nil(t, err) - assert.EqualValues(t, expected, plan) - assert.Equal(t, 0, len(overflow)) +type testCase struct { + rsp map[string]ClusterPreferences + replicas int64 + clusters []string + existing map[string]int64 + capacity map[string]int64 } -func doCheckWithExisting(t *testing.T, pref map[string]ClusterPreferences, replicas int64, clusters []string, - existing map[string]int64, expected map[string]int64) { - planer := NewPlanner(&ReplicaSchedulingPreference{ - Clusters: pref, - TotalReplicas: replicas, - }) - plan, overflow, err := planer.Plan(clusters, existing, map[string]int64{}, "") - assert.Nil(t, err) - assert.Equal(t, 0, len(overflow)) - assert.EqualValues(t, expected, plan) +type expectedResult struct { + plan map[string]int64 + overflow map[string]int64 } -func doCheckWithExistingAndCapacity(t *testing.T, rebalance bool, pref map[string]ClusterPreferences, replicas int64, clusters []string, - existing map[string]int64, - capacity map[string]int64, - expected map[string]int64, - expectedOverflow map[string]int64) { - planer := NewPlanner(&ReplicaSchedulingPreference{ - Rebalance: rebalance, - Clusters: pref, - TotalReplicas: replicas, - }) - plan, overflow, err := planer.Plan(clusters, existing, capacity, "") - assert.Nil(t, err) - assert.EqualValues(t, expected, plan) - assert.Equal(t, expectedOverflow, overflow) -} +func estimateCapacity(currentReplicas, actualCapacity map[string]int64) map[string]int64 { + estimatedCapacity := make(map[string]int64, len(actualCapacity)) + for cluster, c := range actualCapacity { + if currentReplicas[cluster] > c { + estimatedCapacity[cluster] = c + } + } -func pint(val int64) *int64 { - return &val + return estimatedCapacity } -func TestEqual(t *testing.T) { - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - // hash dependent - map[string]int64{"A": 16, "B": 17, "C": 17}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 25, "B": 25}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B"}, - // hash dependent - map[string]int64{"A": 0, "B": 1}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B", "C", "D"}, - // hash dependent - map[string]int64{"A": 0, "B": 0, "C": 0, "D": 1}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A"}, - map[string]int64{"A": 1}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{}, - map[string]int64{}) +func doCheck( + t *testing.T, + tc *testCase, + avoidDisruption bool, + keepUnschedulableReplicas bool, + expected *expectedResult, +) { + t.Helper() + assert := assert.New(t) + + existing := tc.existing + var plan, overflow, lastPlan, lastOverflow map[string]int64 + var err error + + converged := false + const maxConvergenceSteps = 3 + for i := 0; i < maxConvergenceSteps; i++ { + estimatedCapacity := estimateCapacity(existing, tc.capacity) + plan, overflow, err = Plan( + &ReplicaSchedulingPreference{ + Clusters: tc.rsp, + }, + tc.replicas, tc.clusters, + existing, estimatedCapacity, "", + avoidDisruption, keepUnschedulableReplicas, + ) + + assert.Nil(err) + t.Logf("Step %v: avoidDisruption=%v keepUnschedulableReplicas=%v pref=%+v existing=%v estimatedCapacity=%v plan=%v overflow=%v\n", + i, avoidDisruption, keepUnschedulableReplicas, tc.rsp, existing, estimatedCapacity, plan, overflow) + + // nil and empty map should be treated as equal + planConverged := len(plan) == 0 && len(lastPlan) == 0 || reflect.DeepEqual(plan, lastPlan) + overflowConverged := len(overflow) == 0 && len(lastOverflow) == 0 || reflect.DeepEqual(overflow, lastOverflow) + converged = planConverged && overflowConverged + if converged { + // Break out of the loop if converged + break + } + + // Not converged yet, do another round + existing = make(map[string]int64, len(plan)) + for cluster, replicas := range plan { + existing[cluster] += replicas + } + for cluster, replicas := range overflow { + existing[cluster] += replicas + } + + lastPlan, lastOverflow = plan, overflow + } + + if !converged { + t.Errorf("did not converge after %v steps", maxConvergenceSteps) + } + + if len(plan) != 0 || len(expected.plan) != 0 { + assert.Equal(expected.plan, plan) + } + if len(overflow) != 0 || len(expected.overflow) != 0 { + assert.Equal(expected.overflow, overflow) + } } -func TestEqualWithExisting(t *testing.T) { - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"C": 30}, - map[string]int64{"A": 10, "B": 10, "C": 30}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 30}, - map[string]int64{"A": 30, "B": 20}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 0, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 1, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 4, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 5, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 6, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 7, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 500000, []string{"A", "B"}, - map[string]int64{"A": 300000}, - map[string]int64{"A": 300000, "B": 200000}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10}, - map[string]int64{"A": 25, "B": 25}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10, "B": 70}, - // hash dependent - // TODO: Should be 10:40, update algorithm. Issue: #31816 - map[string]int64{"A": 0, "B": 50}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B"}, - map[string]int64{"A": 30}, - map[string]int64{"A": 1, "B": 0}) - - doCheckWithExisting(t, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10, "B": 20}, - map[string]int64{"A": 25, "B": 25}) +func doCheckWithoutExisting( + t *testing.T, + tc *testCase, + expected *expectedResult, +) { + // The replica distribution should be the same regardless of + // avoidDisruption and keepUnschedulableReplicas. + + t.Helper() + doCheck(t, tc, false, false, expected) + doCheck(t, tc, false, true, expected) + doCheck(t, tc, true, false, expected) + doCheck(t, tc, true, true, expected) } -func TestWithExistingAndCapacity(t *testing.T) { - // desired without capacity: map[string]int64{"A": 17, "B": 17, "C": 16}) - doCheckWithExistingAndCapacity(t, true, map[string]ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{}, - map[string]int64{"C": 10}, - map[string]int64{"A": 20, "B": 20, "C": 10}, - map[string]int64{"C": 7}) - - // desired B:50 C:0 - doCheckWithExistingAndCapacity(t, true, map[string]ClusterPreferences{ - "A": {Weight: 10000}, - "B": {Weight: 1}}, - 50, []string{"B", "C"}, - map[string]int64{}, - map[string]int64{"B": 10}, - map[string]int64{"B": 10, "C": 0}, - map[string]int64{"B": 40}, - ) - - // desired A:20 B:40 - doCheckWithExistingAndCapacity(t, true, map[string]ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 2}}, - 60, []string{"A", "B", "C"}, - map[string]int64{}, - map[string]int64{"B": 10}, - map[string]int64{"A": 50, "B": 10, "C": 0}, - map[string]int64{"B": 30}) +// some results may be hash dependent + +func TestWithoutExisting(t *testing.T) { + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 16, "B": 17, "C": 17}}, + ) - // map[string]int64{"A": 10, "B": 30, "C": 21, "D": 10}) - doCheckWithExistingAndCapacity(t, true, map[string]ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(21)}, - "D": {Weight: 1, MaxReplicas: pint(10)}}, - 71, []string{"A", "B", "C", "D"}, - map[string]int64{}, - map[string]int64{"C": 10}, - map[string]int64{"A": 10, "B": 41, "C": 10, "D": 10}, - map[string]int64{"C": 11}, - ) - - // desired A:20 B:20 - doCheckWithExistingAndCapacity(t, false, map[string]ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 1}}, - 60, []string{"A", "B", "C"}, - map[string]int64{}, - map[string]int64{"A": 10, "B": 10}, - map[string]int64{"A": 10, "B": 10, "C": 0}, - map[string]int64{"A": 20, "B": 20}) - - // desired A:10 B:50 although A:50 B:10 is fully acceptable because rebalance = false - doCheckWithExistingAndCapacity(t, false, map[string]ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 5}}, - 60, []string{"A", "B", "C"}, - map[string]int64{}, - map[string]int64{"B": 10}, - map[string]int64{"A": 50, "B": 10, "C": 0}, - map[string]int64{}) - - doCheckWithExistingAndCapacity(t, false, map[string]ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 0}}, - 50, []string{"A", "B", "C"}, - map[string]int64{}, - map[string]int64{"B": 10}, - map[string]int64{"A": 20, "B": 10, "C": 20}, - map[string]int64{}) - - // Actually we would like to have extra 20 in B but 15 is also good. - doCheckWithExistingAndCapacity(t, true, map[string]ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 1}}, - 60, []string{"A", "B"}, - map[string]int64{}, - map[string]int64{"B": 10}, - map[string]int64{"A": 50, "B": 10}, - map[string]int64{"B": 15}) + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + }, + &expectedResult{plan: map[string]int64{"A": 25, "B": 25}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 1, + clusters: []string{"A", "B"}, + }, + &expectedResult{plan: map[string]int64{"A": 0, "B": 1}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 1, + clusters: []string{"A", "B", "C", "D"}, + }, + &expectedResult{plan: map[string]int64{"A": 0, "B": 0, "C": 0, "D": 1}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 1, + clusters: []string{"A"}, + }, + &expectedResult{plan: map[string]int64{"A": 1}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 1, + clusters: []string{}, + }, + &expectedResult{plan: map[string]int64{}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 2, Weight: 0}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 2, "B": 2, "C": 2}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 20, Weight: 0}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 10, "B": 20, "C": 20}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 20, Weight: 0}, + "A": {MinReplicas: 100, Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 50, "B": 0, "C": 0}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {MinReplicas: 10, Weight: 1}, + "B": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + }, + &expectedResult{plan: map[string]int64{"A": 30, "B": 20}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {MinReplicas: 3, Weight: 2}, + "B": {MinReplicas: 3, Weight: 3}, + "C": {MinReplicas: 3, Weight: 5}, + }, + replicas: 10, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 3, "B": 3, "C": 4}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 10, Weight: 1, MaxReplicas: pointer.Int64(12)}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 12, "B": 12, "C": 12}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1, MaxReplicas: pointer.Int64(2)}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 2, "B": 2, "C": 2}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 0, MaxReplicas: pointer.Int64(2)}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 0, "B": 0, "C": 0}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 1}, + "B": {Weight: 2}, + }, + replicas: 60, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 20, "B": 40}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000}, + "B": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 50, "B": 0}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000}, + "B": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"B", "C"}, + }, + &expectedResult{plan: map[string]int64{"B": 50}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000, MaxReplicas: pointer.Int64(10)}, + "B": {Weight: 1}, + "C": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 10, "B": 20, "C": 20}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000, MaxReplicas: pointer.Int64(10)}, + "B": {Weight: 1}, + "C": {Weight: 1, MaxReplicas: pointer.Int64(10)}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + }, + &expectedResult{plan: map[string]int64{"A": 10, "B": 30, "C": 10}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000, MaxReplicas: pointer.Int64(10)}, + "B": {Weight: 1}, + "C": {Weight: 1, MaxReplicas: pointer.Int64(21)}, + "D": {Weight: 1, MaxReplicas: pointer.Int64(10)}, + }, + replicas: 71, + clusters: []string{"A", "B", "C", "D"}, + }, + &expectedResult{plan: map[string]int64{"A": 10, "B": 30, "C": 21, "D": 10}}, + ) + + doCheckWithoutExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000, MaxReplicas: pointer.Int64(10)}, + "B": {Weight: 1}, + "C": {Weight: 1, MaxReplicas: pointer.Int64(21)}, + "D": {Weight: 1, MaxReplicas: pointer.Int64(10)}, + "E": {Weight: 1}, + }, + replicas: 91, + clusters: []string{"A", "B", "C", "D", "E"}, + }, + &expectedResult{plan: map[string]int64{"A": 10, "B": 25, "C": 21, "D": 10, "E": 25}}, + ) } -func TestMin(t *testing.T) { - doCheck(t, map[string]ClusterPreferences{ - "*": {MinReplicas: 2, Weight: 0}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 2, "B": 2, "C": 2}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 0}}, - 50, []string{"A", "B", "C"}, - // hash dependant. - map[string]int64{"A": 10, "B": 20, "C": 20}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 0}, - "A": {MinReplicas: 100, Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 50, "B": 0, "C": 0}) - - doCheck(t, map[string]ClusterPreferences{ - "*": {MinReplicas: 10, Weight: 1, MaxReplicas: pint(12)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 12, "B": 12, "C": 12}) +func doCheckWithExisting( + t *testing.T, + tc *testCase, + expected [2]*expectedResult, +) { + // With existing, avoidDisruption should affect the distribution + + t.Helper() + doCheck(t, tc, false, false, expected[0]) + doCheck(t, tc, false, true, expected[0]) + doCheck(t, tc, true, false, expected[1]) + doCheck(t, tc, true, true, expected[1]) } -func TestMax(t *testing.T) { - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 1, MaxReplicas: pint(2)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 2, "B": 2, "C": 2}) +func TestWithExisting(t *testing.T) { + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"C": 30}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 16, "B": 17, "C": 17}}, + {plan: map[string]int64{"A": 9, "B": 11, "C": 30}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 30}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 25, "B": 25}}, + {plan: map[string]int64{"A": 30, "B": 20}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 0, "B": 8}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 7, "B": 8}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 1, "B": 8}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 7, "B": 8}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 4, "B": 8}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 7, "B": 8}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 7, "B": 8}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 7, "B": 8}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 15, "B": 0}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 15, "B": 0}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 5, "B": 10}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8}}, + {plan: map[string]int64{"A": 5, "B": 10}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 30}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 25, "B": 25}}, + {plan: map[string]int64{"A": 30, "B": 20}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 10}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 25, "B": 25}}, + {plan: map[string]int64{"A": 25, "B": 25}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 10, "B": 20}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 25, "B": 25}}, + {plan: map[string]int64{"A": 25, "B": 25}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 10, "B": 70}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 25, "B": 25}}, + {plan: map[string]int64{"A": 10, "B": 40}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 1, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 30}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 0, "B": 1}}, + {plan: map[string]int64{"A": 1, "B": 0}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 10, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 50, "B": 30}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 5, "B": 5}}, + {plan: map[string]int64{"A": 5, "B": 5}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 499}, + "B": {Weight: 499}, + "C": {Weight: 1}, + }, + replicas: 15, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 15, "B": 15, "C": 0}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 7, "B": 8, "C": 0}}, + {plan: map[string]int64{"A": 7, "B": 8, "C": 0}}, + }, + ) + + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 18, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 10, "B": 1, "C": 1}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 6, "B": 6, "C": 6}}, + {plan: map[string]int64{"A": 10, "B": 4, "C": 4}}, + }, + ) - doCheck(t, map[string]ClusterPreferences{ - "*": {Weight: 0, MaxReplicas: pint(2)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 0, "B": 0, "C": 0}) + doCheckWithExisting(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 0}, + "B": {Weight: 1}, + "C": {Weight: 1}, + }, + replicas: 18, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 10, "B": 1, "C": 7}, + }, + [2]*expectedResult{ + {plan: map[string]int64{"A": 0, "B": 9, "C": 9}}, + {plan: map[string]int64{"A": 10, "B": 1, "C": 7}}, + }, + ) +} + +func doCheckWithExistingAndCapacity( + t *testing.T, + tc *testCase, + expected [4]*expectedResult, +) { + // With existing, both avoidDisruption should affect the distribution + + t.Helper() + doCheck(t, tc, false, false, expected[0]) + doCheck(t, tc, false, true, expected[1]) + doCheck(t, tc, true, false, expected[2]) + doCheck(t, tc, true, true, expected[3]) } -func TestWeight(t *testing.T) { - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 2}}, - 60, []string{"A", "B", "C"}, - map[string]int64{"A": 20, "B": 40, "C": 0}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000}, - "B": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 50, "B": 0, "C": 0}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000}, - "B": {Weight: 1}}, - 50, []string{"B", "C"}, - map[string]int64{"B": 50, "C": 0}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 20, "C": 20}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(10)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 30, "C": 10}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(21)}, - "D": {Weight: 1, MaxReplicas: pint(10)}}, - 71, []string{"A", "B", "C", "D"}, - map[string]int64{"A": 10, "B": 30, "C": 21, "D": 10}) - - doCheck(t, map[string]ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(21)}, - "D": {Weight: 1, MaxReplicas: pint(10)}, - "E": {Weight: 1}}, - 91, []string{"A", "B", "C", "D", "E"}, - map[string]int64{"A": 10, "B": 25, "C": 21, "D": 10, "E": 25}) +func TestWithExistingAndCapacity(t *testing.T) { + // With existing and capacity, both avoidDisruption and keepUnschedulableReplicas + // should affect the distribution + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 30, "B": 20}, + capacity: map[string]int64{"C": 10}, + }, + // A:16, B:17, C:17 initially, then migrate 7 in C after unschedulable + [4]*expectedResult{ + { + plan: map[string]int64{"A": 20, "B": 20, "C": 10}, + overflow: map[string]int64{"C": 7}, + }, + { + plan: map[string]int64{"A": 20, "B": 20, "C": 10}, + overflow: map[string]int64{"C": 7}, + }, + { + plan: map[string]int64{"A": 30, "B": 20, "C": 0}, + }, + { + plan: map[string]int64{"A": 30, "B": 20, "C": 0}, + }, + }, + ) + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 30, "C": 20}, + capacity: map[string]int64{"C": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 20, "B": 20, "C": 10}, + overflow: map[string]int64{"C": 7}, + }, + { + plan: map[string]int64{"A": 20, "B": 20, "C": 10}, + overflow: map[string]int64{"C": 7}, + }, + { + plan: map[string]int64{"A": 30, "B": 10, "C": 10}, + }, + { + plan: map[string]int64{"A": 30, "B": 10, "C": 10}, + overflow: map[string]int64{"C": 7}, + }, + }, + ) + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000}, + "B": {Weight: 1}, + }, + replicas: 50, + clusters: []string{"B", "C"}, + existing: map[string]int64{"B": 50}, + capacity: map[string]int64{"B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"B": 10}, + overflow: map[string]int64{"B": 40}, + }, + { + plan: map[string]int64{"B": 10}, + overflow: map[string]int64{"B": 40}, + }, + { + plan: map[string]int64{"B": 10}, + overflow: map[string]int64{"B": 40}, + }, + { + plan: map[string]int64{"B": 10}, + overflow: map[string]int64{"B": 40}, + }, + }, + ) + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 1}, + "B": {Weight: 5}, + }, + replicas: 60, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 20, "B": 40}, + capacity: map[string]int64{"B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 40}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 40}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 40}, + }, + }, + ) + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 1}, + "B": {Weight: 2}, + }, + replicas: 60, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 60}, + capacity: map[string]int64{"B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 30}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 30}, + }, + { + plan: map[string]int64{"A": 60, "B": 0}, + }, + { + plan: map[string]int64{"A": 60, "B": 0}, + }, + }, + ) + + // total capacity < desired replicas + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 1}, + "B": {Weight: 1}, + }, + replicas: 60, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 30, "B": 30}, + capacity: map[string]int64{"A": 10, "B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 10, "B": 10}, + overflow: map[string]int64{"A": 20, "B": 20}, + }, + { + plan: map[string]int64{"A": 10, "B": 10}, + overflow: map[string]int64{"A": 20, "B": 20}, + }, + { + plan: map[string]int64{"A": 10, "B": 10}, + overflow: map[string]int64{"A": 20, "B": 20}, + }, + { + plan: map[string]int64{"A": 10, "B": 10}, + overflow: map[string]int64{"A": 20, "B": 20}, + }, + }, + ) + + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 1}, + "B": {Weight: 2}, + }, + replicas: 60, + clusters: []string{"A", "B"}, + existing: map[string]int64{"A": 30, "B": 40}, + capacity: map[string]int64{"A": 25, "B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 25, "B": 10}, + overflow: map[string]int64{"A": 25, "B": 30}, + }, + { + plan: map[string]int64{"A": 25, "B": 10}, + overflow: map[string]int64{"A": 25, "B": 30}, + }, + { + plan: map[string]int64{"A": 25, "B": 10}, + overflow: map[string]int64{"A": 25, "B": 25}, + }, + { + plan: map[string]int64{"A": 25, "B": 10}, + overflow: map[string]int64{"A": 25, "B": 30}, + }, + }, + ) + + // map[string]int64{"A": 10, "B": 30, "C": 21, "D": 10}) + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "A": {Weight: 10000, MaxReplicas: pointer.Int64(10)}, + "B": {Weight: 1}, + "C": {Weight: 1, MaxReplicas: pointer.Int64(21)}, + "D": {Weight: 1, MaxReplicas: pointer.Int64(10)}, + }, + replicas: 71, + clusters: []string{"A", "B", "C", "D"}, + existing: map[string]int64{"A": 20}, + capacity: map[string]int64{"C": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 10, "B": 41, "C": 10, "D": 10}, + overflow: map[string]int64{"C": 11}, + }, + { + plan: map[string]int64{"A": 10, "B": 41, "C": 10, "D": 10}, + overflow: map[string]int64{"C": 11}, + }, + { + plan: map[string]int64{"A": 20, "B": 33, "C": 10, "D": 8}, + }, + { + plan: map[string]int64{"A": 20, "B": 33, "C": 10, "D": 8}, + overflow: map[string]int64{"C": 11}, + }, + }, + ) + + // capacity < minReplicas should also be recorded as overflow to prevent infinite rescheduling loop + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 20, Weight: 0}, + }, + replicas: 50, + clusters: []string{"A", "B", "C"}, + existing: map[string]int64{"A": 24}, + capacity: map[string]int64{"B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 20, "B": 10, "C": 20}, + overflow: map[string]int64{"B": 10}, + }, + { + plan: map[string]int64{"A": 20, "B": 10, "C": 20}, + overflow: map[string]int64{"B": 10}, + }, + { + plan: map[string]int64{"A": 24, "B": 10, "C": 16}, + }, + { + plan: map[string]int64{"A": 24, "B": 10, "C": 16}, + overflow: map[string]int64{"B": 10}, + }, + }, + ) + + // Actually we'd like 20 overflow, but 25 is also fine. + doCheckWithExistingAndCapacity(t, + &testCase{ + rsp: map[string]ClusterPreferences{ + "*": {MinReplicas: 20, Weight: 1}, + }, + replicas: 60, + clusters: []string{"A", "B"}, + existing: map[string]int64{}, + capacity: map[string]int64{"B": 10}, + }, + [4]*expectedResult{ + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 25}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 25}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + }, + { + plan: map[string]int64{"A": 50, "B": 10}, + overflow: map[string]int64{"B": 25}, + }, + }, + ) } diff --git a/pkg/controllers/util/refinedplanner/planner.go b/pkg/controllers/util/refinedplanner/planner.go deleted file mode 100644 index c5d62e63..00000000 --- a/pkg/controllers/util/refinedplanner/planner.go +++ /dev/null @@ -1,299 +0,0 @@ -/* -Copyright 2016 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. - -This file may have been modified by The KubeAdmiral Authors -("KubeAdmiral Modifications"). All KubeAdmiral Modifications -are Copyright 2023 The KubeAdmiral Authors. -*/ - -package refinedplanner - -import ( - "hash/fnv" - "sort" - - "k8s.io/klog/v2" - - "github.com/kubewharf/kubeadmiral/pkg/controllers/util/planner" -) - -type ReplicaSchedulingPreference struct { - - // Total number of pods desired across federated clusters. - // Replicas specified in the spec for target deployment template or replicaset - // template will be discarded/overridden when scheduling preferences are - // specified. - TotalReplicas int64 - - // If set to true then already scheduled and running replicas may be moved to other clusters - // in order to match current state to the specified preferences. Otherwise, if set to false, - // up and running replicas will not be moved. - // +optional - Rebalance bool - - // A mapping between cluster names and preferences regarding a local workload object (dep, rs, .. ) in - // these clusters. - // "*" (if provided) applies to all clusters if an explicit mapping is not provided. - // If omitted, clusters without explicit preferences should not have any replicas scheduled. - // +optional - Clusters map[string]planner.ClusterPreferences -} - -// Planner decides how many out of the given replicas should be placed in each of the -// federated clusters. -type Planner struct { - preferences *ReplicaSchedulingPreference -} - -type namedClusterPreferences struct { - clusterName string - hash uint32 - planner.ClusterPreferences -} - -type byWeight []*namedClusterPreferences - -func (a byWeight) Len() int { return len(a) } -func (a byWeight) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// Preferences are sorted according by decreasing weight and increasing hash (built on top of cluster name and rs name). -// Sorting is made by a hash to avoid assigning single-replica rs to the alphabetically smallest cluster. -func (a byWeight) Less(i, j int) bool { - return (a[i].Weight > a[j].Weight) || (a[i].Weight == a[j].Weight && a[i].hash < a[j].hash) -} - -func NewPlanner(preferences *ReplicaSchedulingPreference) *Planner { - return &Planner{ - preferences: preferences, - } -} - -// Distribute the desired number of replicas among the given cluster according to the planner preferences. -// The function tries its best to assign each cluster the preferred number of replicas, however if -// sum of MinReplicas for all cluster is bigger than replicasToDistribute (TotalReplicas) then some cluster -// will not have all of the replicas assigned. In such case a cluster with higher weight has priority over -// cluster with lower weight (or with lexicographically smaller name in case of draw). -// It can also use the current replica count and estimated capacity to provide better planning and -// adhere to rebalance policy. To avoid prioritization of clusters with smaller lexicographical names -// a semi-random string (like replica set name) can be provided. -// Two maps are returned: -// - a map that contains information how many replicas will be possible to run in a cluster. -// - a map that contains information how many extra replicas would be nice to schedule in a cluster so, -// if by chance, they are scheduled we will be closer to the desired replicas layout. -func (p *Planner) Plan(availableClusters []string, currentReplicaCount map[string]int64, - estimatedCapacity map[string]int64, replicaSetKey string) (map[string]int64, map[string]int64, error) { - preferences := make(map[string]*planner.ClusterPreferences, len(availableClusters)) - - for _, cluster := range availableClusters { - if preference, found := p.preferences.Clusters[cluster]; found { - preferences[cluster] = &preference - } else if preference, found := p.preferences.Clusters["*"]; found { - preferences[cluster] = &preference - } - } - - namedPreferences, err := p.getNamedPreferences(preferences, replicaSetKey) - if err != nil { - return nil, nil, err - } - - desiredReplicaCount := p.getDesiredPlan(namedPreferences, p.preferences.TotalReplicas) - if p.preferences.Rebalance { - return desiredReplicaCount, nil, nil - } - klog.Infof("desiredReplicaCount: %v\n", desiredReplicaCount) - - // when rebalance is disabled, try to avoid instance migration between clusters - var currentTotalCount int64 - plan := make(map[string]int64, len(namedPreferences)) - for _, preference := range namedPreferences { - count := currentReplicaCount[preference.clusterName] - plan[preference.clusterName] = count - currentTotalCount += count - } - var desiredTotalCount int64 - for _, count := range desiredReplicaCount { - desiredTotalCount += count - } - - if currentTotalCount == desiredTotalCount { - return plan, nil, nil - } else if currentTotalCount > desiredTotalCount { - return p.scaleDown(plan, desiredReplicaCount, currentTotalCount-desiredTotalCount, replicaSetKey) - } else { - return p.scaleUp(plan, desiredReplicaCount, desiredTotalCount-currentTotalCount, replicaSetKey) - } -} - -func (p *Planner) getNamedPreferences( - preferences map[string]*planner.ClusterPreferences, - replicaSetKey string, -) ([]*namedClusterPreferences, error) { - namedPreferences := make([]*namedClusterPreferences, 0, len(preferences)) - named := func(name string, pref *planner.ClusterPreferences) (*namedClusterPreferences, error) { - hasher := fnv.New32() - if _, err := hasher.Write([]byte(name)); err != nil { - return nil, err - } - if _, err := hasher.Write([]byte(replicaSetKey)); err != nil { - return nil, err - } - - return &namedClusterPreferences{ - clusterName: name, - hash: hasher.Sum32(), - ClusterPreferences: *pref, - }, nil - } - - for name, preference := range preferences { - namedPreference, err := named(name, preference) - if err != nil { - return nil, err - } - namedPreferences = append(namedPreferences, namedPreference) - } - sort.Sort(byWeight(namedPreferences)) - return namedPreferences, nil -} - -func (p *Planner) getDesiredPlan(preferences []*namedClusterPreferences, totalReplicas int64) map[string]int64 { - remainingReplicas := totalReplicas - plan := make(map[string]int64, len(preferences)) - - // Assign each cluster the minimum number of replicas it requested. - for _, preference := range preferences { - min := minInt64(preference.MinReplicas, remainingReplicas) - remainingReplicas -= min - plan[preference.clusterName] = min - } - - modified := true - // It is possible single pass of the loop is not enough to distribute all replicas among clusters due - // to weight, max and rounding corner cases. In such case we iterate until either - // there is no replicas or no cluster gets any more replicas. Every loop either distributes all remainingReplicas - // or maxes out at least one cluster. - for modified && remainingReplicas > 0 { - modified = false - weightSum := int64(0) - for _, preference := range preferences { - weightSum += preference.Weight - } - if weightSum <= 0 { - return plan - } - newPreferences := make([]*namedClusterPreferences, 0, len(preferences)) - - distributeInThisLoop := remainingReplicas - for _, preference := range preferences { - start := plan[preference.clusterName] - // Distribute the remaining replicas, rounding fractions always up. - extra := (distributeInThisLoop*preference.Weight + weightSum - 1) / weightSum - extra = minInt64(extra, remainingReplicas) - - // In total there should be the amount that was there at start plus whatever is due - // in this iteration - total := start + extra - - // Check if we don't overflow the cluster, and if yes don't consider this cluster - // in any of the following iterations. - full := false - if preference.MaxReplicas != nil && total > *preference.MaxReplicas { - total = *preference.MaxReplicas - full = true - } - if !full { - newPreferences = append(newPreferences, preference) - } - - // Only total-start replicas were actually taken. - remainingReplicas -= total - start - plan[preference.clusterName] = total - - // Something extra got scheduled on this cluster. - if total > start { - modified = true - } - } - preferences = newPreferences - } - return plan -} - -func (p *Planner) scaleUp( - currentReplicaCount, desiredReplicaCount map[string]int64, - scaleUpCount int64, - replicaSetKey string, -) (map[string]int64, map[string]int64, error) { - preferences := make(map[string]*planner.ClusterPreferences, len(desiredReplicaCount)) - for cluster, desired := range desiredReplicaCount { - // only pick clusters which have less replicas than desired to sale up, thus replica migration between clusters can be avoid - current := currentReplicaCount[cluster] - if desired > current { - preferences[cluster] = &planner.ClusterPreferences{ - Weight: desired - current, - } - if p.preferences.Clusters[cluster].MaxReplicas != nil { - // note that max is always positive because MaxReplicas >= desired > current - max := *p.preferences.Clusters[cluster].MaxReplicas - current - preferences[cluster].MaxReplicas = &max - } - } - } - - named, err := p.getNamedPreferences(preferences, replicaSetKey) - if err != nil { - return nil, nil, err - } - replicasToScaleUp := p.getDesiredPlan(named, scaleUpCount) - for cluster, count := range replicasToScaleUp { - currentReplicaCount[cluster] += count - } - return currentReplicaCount, nil, nil -} - -func (p *Planner) scaleDown( - currentReplicaCount, desiredReplicaCount map[string]int64, - scaleDownCount int64, - replicaSetKey string, -) (map[string]int64, map[string]int64, error) { - preferences := make(map[string]*planner.ClusterPreferences, len(desiredReplicaCount)) - for cluster, desired := range desiredReplicaCount { - // only pick clusters which have more replicas than desired to scale down, thus replica migration between clusters can be avoid - current := currentReplicaCount[cluster] - if desired < current { - preferences[cluster] = &planner.ClusterPreferences{ - Weight: current - desired, - MaxReplicas: ¤t, - } - } - } - named, err := p.getNamedPreferences(preferences, replicaSetKey) - if err != nil { - return nil, nil, err - } - replicasToScaleDown := p.getDesiredPlan(named, scaleDownCount) - for cluster, count := range replicasToScaleDown { - currentReplicaCount[cluster] -= count - } - return currentReplicaCount, nil, nil -} - -func minInt64(a int64, b int64) int64 { - if a < b { - return a - } - return b -} diff --git a/pkg/controllers/util/refinedplanner/planner_test.go b/pkg/controllers/util/refinedplanner/planner_test.go deleted file mode 100644 index 589cc214..00000000 --- a/pkg/controllers/util/refinedplanner/planner_test.go +++ /dev/null @@ -1,314 +0,0 @@ -/* -Copyright 2016 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. - -This file may have been modified by The KubeAdmiral Authors -("KubeAdmiral Modifications"). All KubeAdmiral Modifications -are Copyright 2023 The KubeAdmiral Authors. -*/ - -package refinedplanner - -import ( - "testing" - - "github.com/stretchr/testify/assert" - - "github.com/kubewharf/kubeadmiral/pkg/controllers/util/planner" -) - -func doCheck(t *testing.T, pref map[string]planner.ClusterPreferences, replicas int64, clusters []string, expected map[string]int64) { - planer := NewPlanner(&ReplicaSchedulingPreference{ - Clusters: pref, - TotalReplicas: replicas, - }) - plan, overflow, err := planer.Plan(clusters, map[string]int64{}, map[string]int64{}, "") - assert.Nil(t, err) - assert.EqualValues(t, expected, plan) - assert.Equal(t, 0, len(overflow)) -} - -func doCheckWithExisting(t *testing.T, rebalance bool, pref map[string]planner.ClusterPreferences, replicas int64, clusters []string, - existing map[string]int64, expected map[string]int64) { - planer := NewPlanner(&ReplicaSchedulingPreference{ - Rebalance: rebalance, - Clusters: pref, - TotalReplicas: replicas, - }) - plan, overflow, err := planer.Plan(clusters, existing, map[string]int64{}, "") - assert.Nil(t, err) - assert.Equal(t, 0, len(overflow)) - assert.EqualValues(t, expected, plan) -} - -func pint(val int64) *int64 { - return &val -} - -func TestEqual(t *testing.T) { - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - // hash dependent - map[string]int64{"A": 16, "B": 17, "C": 17}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 25, "B": 25}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B"}, - // hash dependent - map[string]int64{"A": 0, "B": 1}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B", "C", "D"}, - // hash dependent - map[string]int64{"A": 0, "B": 0, "C": 0, "D": 1}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A"}, - map[string]int64{"A": 1}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{}, - map[string]int64{}) -} - -func TestEqualWithExisting(t *testing.T) { - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"C": 30}, - map[string]int64{"A": 9, "B": 11, "C": 30}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 30}, - map[string]int64{"A": 30, "B": 20}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 0, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 1, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 4, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 5, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 6, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 15, []string{"A", "B"}, - map[string]int64{"A": 7, "B": 8}, - map[string]int64{"A": 7, "B": 8}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 500000, []string{"A", "B"}, - map[string]int64{"A": 300000}, - map[string]int64{"A": 300000, "B": 200000}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10}, - map[string]int64{"A": 25, "B": 25}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10, "B": 70}, - // hash dependent - map[string]int64{"A": 10, "B": 40}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 1, []string{"A", "B"}, - map[string]int64{"A": 30}, - map[string]int64{"A": 1, "B": 0}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "*": {Weight: 1}}, - 50, []string{"A", "B"}, - map[string]int64{"A": 10, "B": 20}, - map[string]int64{"A": 25, "B": 25}) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "A": {Weight: 499}, - "B": {Weight: 499}, - "C": {Weight: 1}}, - 15, []string{"A", "B", "C"}, - map[string]int64{"A": 15, "B": 15, "C": 0}, - map[string]int64{"A": 7, "B": 8, "C": 0}, - ) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 18, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 1, "C": 1}, - map[string]int64{"A": 10, "B": 4, "C": 4}, - ) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 18, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 1, "C": 1}, - map[string]int64{"A": 10, "B": 4, "C": 4}, - ) - - doCheckWithExisting(t, true, map[string]planner.ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 18, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 1, "C": 1}, - map[string]int64{"A": 6, "B": 6, "C": 6}, - ) - - doCheckWithExisting(t, false, map[string]planner.ClusterPreferences{ - "A": {Weight: 0}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 18, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 1, "C": 7}, - map[string]int64{"A": 10, "B": 1, "C": 7}, - ) - - doCheckWithExisting(t, true, map[string]planner.ClusterPreferences{ - "A": {Weight: 0}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 18, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 1, "C": 7}, - map[string]int64{"A": 0, "B": 9, "C": 9}, - ) -} - -func TestMin(t *testing.T) { - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {MinReplicas: 2, Weight: 0}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 2, "B": 2, "C": 2}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 0}}, - 50, []string{"A", "B", "C"}, - // hash dependant. - map[string]int64{"A": 10, "B": 20, "C": 20}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {MinReplicas: 20, Weight: 0}, - "A": {MinReplicas: 100, Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 50, "B": 0, "C": 0}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {MinReplicas: 10, Weight: 1, MaxReplicas: pint(12)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 12, "B": 12, "C": 12}) -} - -func TestMax(t *testing.T) { - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 1, MaxReplicas: pint(2)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 2, "B": 2, "C": 2}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "*": {Weight: 0, MaxReplicas: pint(2)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 0, "B": 0, "C": 0}) -} - -func TestWeight(t *testing.T) { - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 1}, - "B": {Weight: 2}}, - 60, []string{"A", "B", "C"}, - map[string]int64{"A": 20, "B": 40}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000}, - "B": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 50, "B": 0}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000}, - "B": {Weight: 1}}, - 50, []string{"B", "C"}, - map[string]int64{"B": 50}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 20, "C": 20}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(10)}}, - 50, []string{"A", "B", "C"}, - map[string]int64{"A": 10, "B": 30, "C": 10}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(21)}, - "D": {Weight: 1, MaxReplicas: pint(10)}}, - 71, []string{"A", "B", "C", "D"}, - map[string]int64{"A": 10, "B": 30, "C": 21, "D": 10}) - - doCheck(t, map[string]planner.ClusterPreferences{ - "A": {Weight: 10000, MaxReplicas: pint(10)}, - "B": {Weight: 1}, - "C": {Weight: 1, MaxReplicas: pint(21)}, - "D": {Weight: 1, MaxReplicas: pint(10)}, - "E": {Weight: 1}}, - 91, []string{"A", "B", "C", "D", "E"}, - map[string]int64{"A": 10, "B": 25, "C": 21, "D": 10, "E": 25}) -} From 38b41f4e3171361b65a3aadfe4577a312720f5fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 08:41:56 +0000 Subject: [PATCH 02/21] fix(scheduler): sort taints --- pkg/controllers/scheduler/schedulingtriggers.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 0327150e..133ffd8b 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -240,7 +240,17 @@ func getClusterTaints(clusters []*fedcorev1a1.FederatedCluster) SortableMap[stri // NOTE: we must sort the taint slice before inserting to ensure deterministic hashing sort.Slice(taints, func(i, j int) bool { - return taints[i].Key < taints[j].Key || taints[i].Value < taints[j].Value || taints[i].Effect < taints[j].Effect + lhs, rhs := taints[i], taints[j] + switch { + case lhs.Key != rhs.Key: + return lhs.Key < rhs.Key + case lhs.Value != rhs.Value: + return lhs.Value < rhs.Value + case lhs.Effect != rhs.Effect: + return lhs.Value < rhs.Value + default: + return false + } }) ret.Put(cluster.Name, taints) } From 26795d6004fd470a863913bc5063c231c7cfbe87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 08:42:25 +0000 Subject: [PATCH 03/21] refactor(scheduler): replace SortableMap with regular maps + sorting --- .../scheduler/schedulingtriggers.go | 94 +++++++------------ 1 file changed, 33 insertions(+), 61 deletions(-) diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 133ffd8b..7a74513a 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -71,53 +71,34 @@ type keyValue[K any, V any] struct { Value V `json:"value"` } -// SortableMap is a simple implementation of map that can be to be used for hashing. The native golang map cannot be used for hashing as it -// is unordered and can produce non-deterministic hashes. -type SortableMap[K constraints.Ordered, V any] struct { - KeyValues []keyValue[K, V] `json:"keyValues"` -} - -func (m *SortableMap[K, V]) Put(key K, value V) { - found := false - for _, kv := range m.KeyValues { - if kv.Key == key { - kv.Value = value - found = true - break - } - } - - if !found { - m.KeyValues = append(m.KeyValues, keyValue[K, V]{Key: key, Value: value}) +func sortMap[K constraints.Ordered, V any](m map[K]V) []keyValue[K, V] { + ret := make([]keyValue[K, V], 0, len(m)) + for k, v := range m { + ret = append(ret, keyValue[K, V]{Key: k, Value: v}) } -} -func (m *SortableMap[K, V]) Sort() { - sort.Slice(m.KeyValues, func(i, j int) bool { - return m.KeyValues[i].Key < m.KeyValues[j].Key + sort.Slice(ret, func(i, j int) bool { + return ret[i].Key < ret[j].Key }) -} - -func NewSortableMap[K constraints.Ordered, V any]() SortableMap[K, V] { - return SortableMap[K, V]{ - KeyValues: []keyValue[K, V]{}, - } + return ret } type schedulingTriggers struct { - SchedulingAnnotations SortableMap[string, string] `json:"schedulingAnnotations"` - ReplicaCount int64 `json:"replicaCount"` - ResourceRequest framework.Resource `json:"resourceRequest"` + // NOTE: Use slices instead of maps for deterministic iteration order + + SchedulingAnnotations []keyValue[string, string] `json:"schedulingAnnotations"` + ReplicaCount int64 `json:"replicaCount"` + ResourceRequest framework.Resource `json:"resourceRequest"` PolicyName string `json:"policyName"` PolicyGeneration int64 `json:"policyGeneration"` // a map from each cluster to its ready condition - ClusterReady SortableMap[string, bool] `json:"clusterReady"` + ClusterReady []keyValue[string, bool] `json:"clusterReady"` // a map from each cluster to its labels - ClusterLabels SortableMap[string, SortableMap[string, string]] `json:"clusterLabels"` + ClusterLabels []keyValue[string, []keyValue[string, string]] `json:"clusterLabels"` // a map from each cluster to its taints - ClusterTaints SortableMap[string, []corev1.Taint] `json:"clusterTaints"` + ClusterTaints []keyValue[string, []corev1.Taint] `json:"clusterTaints"` } func (s *Scheduler) computeSchedulingTriggerHash( @@ -169,15 +150,14 @@ var knownSchedulingAnnotations = sets.New( FollowsObjectAnnotation, ) -func getSchedulingAnnotations(fedObject *unstructured.Unstructured) SortableMap[string, string] { - ret := NewSortableMap[string, string]() - for k, v := range fedObject.GetAnnotations() { - if knownSchedulingAnnotations.Has(k) { - ret.Put(k, v) +func getSchedulingAnnotations(fedObject *unstructured.Unstructured) []keyValue[string, string] { + annotations := fedObject.GetAnnotations() // this is a deep copy + for k := range annotations { + if !knownSchedulingAnnotations.Has(k) { + delete(annotations, k) } } - ret.Sort() - return ret + return sortMap(annotations) } func getReplicaCount(typeConfig *fedcorev1a1.FederatedTypeConfig, fedObject *unstructured.Unstructured) (int64, error) { @@ -202,31 +182,24 @@ func getResourceRequest(fedObject *unstructured.Unstructured) framework.Resource return framework.Resource{} } -func getClusterReady(clusters []*fedcorev1a1.FederatedCluster) SortableMap[string, bool] { - ret := NewSortableMap[string, bool]() +func getClusterReady(clusters []*fedcorev1a1.FederatedCluster) []keyValue[string, bool] { + ret := make(map[string]bool, len(clusters)) for _, cluster := range clusters { - ret.Put(cluster.Name, util.IsClusterReady(&cluster.Status)) + ret[cluster.Name] = util.IsClusterReady(&cluster.Status) } - ret.Sort() - return ret + return sortMap(ret) } -func getClusterLabels(clusters []*fedcorev1a1.FederatedCluster) SortableMap[string, SortableMap[string, string]] { - ret := NewSortableMap[string, SortableMap[string, string]]() +func getClusterLabels(clusters []*fedcorev1a1.FederatedCluster) []keyValue[string, []keyValue[string, string]] { + ret := make(map[string][]keyValue[string, string], len(clusters)) for _, cluster := range clusters { - labelsMap := NewSortableMap[string, string]() - for k, v := range cluster.GetLabels() { - labelsMap.Put(k, v) - } - labelsMap.Sort() - ret.Put(cluster.Name, labelsMap) + ret[cluster.Name] = sortMap(cluster.GetLabels()) } - ret.Sort() - return ret + return sortMap(ret) } -func getClusterTaints(clusters []*fedcorev1a1.FederatedCluster) SortableMap[string, []corev1.Taint] { - ret := NewSortableMap[string, []corev1.Taint]() +func getClusterTaints(clusters []*fedcorev1a1.FederatedCluster) []keyValue[string, []corev1.Taint] { + ret := make(map[string][]corev1.Taint, len(clusters)) for _, cluster := range clusters { taints := make([]corev1.Taint, len(cluster.Spec.Taints)) for i, t := range cluster.Spec.Taints { @@ -252,10 +225,9 @@ func getClusterTaints(clusters []*fedcorev1a1.FederatedCluster) SortableMap[stri return false } }) - ret.Put(cluster.Name, taints) + ret[cluster.Name] = taints } - ret.Sort() - return ret + return sortMap(ret) } // enqueueFederatedObjectsForPolicy enqueues federated objects which match the policy From 9408ef0401d9c1f95ef6df0a0797c51364a74de6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 09:41:32 +0000 Subject: [PATCH 04/21] feat(scheduler): add auto migration to trigger and unit --- pkg/controllers/common/constants.go | 3 +++ pkg/controllers/federate/util.go | 1 + .../scheduler/framework/plugins/rsp/rsp.go | 26 +++++++++++-------- pkg/controllers/scheduler/framework/types.go | 17 ++++++++++-- .../scheduler/schedulingtriggers.go | 8 ++++++ pkg/controllers/scheduler/schedulingunit.go | 26 +++++++++++++++++++ 6 files changed, 68 insertions(+), 13 deletions(-) diff --git a/pkg/controllers/common/constants.go b/pkg/controllers/common/constants.go index d4914ffc..81ce5d92 100644 --- a/pkg/controllers/common/constants.go +++ b/pkg/controllers/common/constants.go @@ -121,6 +121,9 @@ const ( FollowersAnnotation = DefaultPrefix + "followers" // EnableFollowerSchedulingAnnotation indicates whether follower scheduling should be enabled for the leader object. EnableFollowerSchedulingAnnotation = InternalPrefix + "enable-follower-scheduling" + + // AutoMigrationAnnotation contains auto migration information. + AutoMigrationAnnotation = DefaultPrefix + "auto-migration" ) // The following consts are keys used to store information in the federated cluster secret diff --git a/pkg/controllers/federate/util.go b/pkg/controllers/federate/util.go index 5d173c34..c68e97ed 100644 --- a/pkg/controllers/federate/util.go +++ b/pkg/controllers/federate/util.go @@ -203,6 +203,7 @@ var ( common.FollowersAnnotation, ) + // TODO: Do we need to specify the internal annotations here? // List of annotations that should be ignored on the source object ignoredAnnotations = sets.New( RetainReplicasAnnotation, diff --git a/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go b/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go index e82ba7af..7bf8f073 100644 --- a/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go +++ b/pkg/controllers/scheduler/framework/plugins/rsp/rsp.go @@ -130,6 +130,15 @@ func (pl *ClusterCapacityWeight) ReplicaScheduling( currentReplicas[cluster] = totalReplicas } + var estimatedCapacity map[string]int64 + keepUnschedulableReplicas := false + if autoMigration := su.AutoMigration; autoMigration != nil { + keepUnschedulableReplicas = autoMigration.KeepUnschedulableReplicas + if info := autoMigration.Info; info != nil { + estimatedCapacity = info.EstimatedCapacity + } + } + scheduleResult, overflow, err := planner.Plan( &planner.ReplicaSchedulingPreference{ Clusters: clusterPreferences, @@ -137,33 +146,28 @@ func (pl *ClusterCapacityWeight) ReplicaScheduling( totalReplicas, ExtractClusterNames(clusters), currentReplicas, - nil, + estimatedCapacity, su.Key(), - false, - true, + su.AvoidDisruption, + keepUnschedulableReplicas, ) if err != nil { return clusterReplicasList, framework.NewResult(framework.Error) } klog.V(4).Infof( - "[scheduling] for %q clusterPreferences: %s, currentReplicas: %s, result: %s", - su.Key(), spew.Sprint(clusterPreferences), spew.Sprint(currentReplicas), spew.Sprint(scheduleResult), + "[scheduling] for %q clusterPreferences: %s, estimatedCapacity: %v, currentReplicas: %v, result: %v", + su.Key(), spew.Sprint(clusterPreferences), estimatedCapacity, currentReplicas, scheduleResult, ) - // TODO: Check if we really need to place the federated type in clusters - // with 0 replicas. Override replicas would be set to 0 in this case. result := make(map[string]int64) - // for clusterName := range currentReplicasPerCluster { - // result[clusterName] = 0 - // } - for clusterName, replicas := range scheduleResult { result[clusterName] = replicas } for clusterName, replicas := range overflow { result[clusterName] += replicas } + for _, cluster := range clusters { replicas, ok := result[cluster.Name] if !ok || replicas == 0 { diff --git a/pkg/controllers/scheduler/framework/types.go b/pkg/controllers/scheduler/framework/types.go index 0051fcf9..b814aa91 100644 --- a/pkg/controllers/scheduler/framework/types.go +++ b/pkg/controllers/scheduler/framework/types.go @@ -46,10 +46,12 @@ type SchedulingUnit struct { // Describes the current scheduling state CurrentClusters map[string]*int64 + AutoMigration *AutoMigrationConfig // Controls the scheduling behavior - SchedulingMode fedcorev1a1.SchedulingMode - StickyCluster bool + SchedulingMode fedcorev1a1.SchedulingMode + StickyCluster bool + AvoidDisruption bool // Used to filter/select clusters ClusterSelector map[string]string @@ -62,6 +64,17 @@ type SchedulingUnit struct { Weights map[string]int64 } +type AutoMigrationConfig struct { + Info *AutoMigrationInfo + KeepUnschedulableReplicas bool +} + +// AutoMigrationInfo contains information related to auto migration. +type AutoMigrationInfo struct { + // Describes the estimated max number of replicas a cluster can accommodate. + EstimatedCapacity map[string]int64 `json:"estimatedCapacity,omitempty"` +} + // Affinity is a group of affinity scheduling rules. type Affinity struct { // Describes cluster affinity scheduling rules for the scheduling unit. diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 7a74513a..1a1dbcc3 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -90,6 +90,8 @@ type schedulingTriggers struct { ReplicaCount int64 `json:"replicaCount"` ResourceRequest framework.Resource `json:"resourceRequest"` + AutoMigrationInfo *string `json:"autoMigrationInfo,omitempty"` + PolicyName string `json:"policyName"` PolicyGeneration int64 `json:"policyGeneration"` @@ -119,6 +121,12 @@ func (s *Scheduler) computeSchedulingTriggerHash( if policy != nil { trigger.PolicyName = policy.GetName() trigger.PolicyGeneration = policy.GetGeneration() + if policy.GetSpec().AutoMigration != nil { + // Only consider auto-migration annotation when auto-migration is enabled in the policy. + if value, exists := fedObject.GetAnnotations()[common.AutoMigrationAnnotation]; exists { + trigger.AutoMigrationInfo = &value + } + } } trigger.ClusterReady = getClusterReady(clusters) diff --git a/pkg/controllers/scheduler/schedulingunit.go b/pkg/controllers/scheduler/schedulingunit.go index f8385de4..b55f04ac 100644 --- a/pkg/controllers/scheduler/schedulingunit.go +++ b/pkg/controllers/scheduler/schedulingunit.go @@ -81,6 +81,18 @@ func (s *Scheduler) schedulingUnitForFedObject( DesiredReplicas: desiredReplicasOption, CurrentClusters: currentReplicas, + AvoidDisruption: policy.GetSpec().ReplicaRescheduling.AvoidDisruption, + } + + if autoMigration := policy.GetSpec().AutoMigration; autoMigration != nil { + info, err := getAutoMigrationInfo(fedObject) + if err != nil { + return nil, err + } + schedulingUnit.AutoMigration = &framework.AutoMigrationConfig{ + Info: info, + KeepUnschedulableReplicas: autoMigration.KeepUnschedulableReplicas, + } } schedulingUnit.SchedulingMode = schedulingMode @@ -222,11 +234,25 @@ func getSchedulingModeFromObject(object *unstructured.Unstructured) (fedcorev1a1 return "", false } +func getAutoMigrationInfo(fedObject *unstructured.Unstructured) (*framework.AutoMigrationInfo, error) { + value, exists := fedObject.GetAnnotations()[common.AutoMigrationAnnotation] + if !exists { + return nil, nil + } + + autoMigration := new(framework.AutoMigrationInfo) + if err := json.Unmarshal([]byte(value), autoMigration); err != nil { + return nil, err + } + return autoMigration, nil +} + func getIsStickyClusterFromPolicy(policy fedcorev1a1.GenericPropagationPolicy) bool { return policy.GetSpec().StickyCluster } func getIsStickyClusterFromObject(object *unstructured.Unstructured) (bool, bool) { + // TODO: consider passing in the annotations directly to prevent incurring a deep copy for each call annotations := object.GetAnnotations() if annotations == nil { return false, false From f057944b7de7c09f7b0286e7cca0e4453dbd7ccf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 15:24:34 +0000 Subject: [PATCH 05/21] feat(codegen): support post crd gen patches --- hack/generate-groups.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/hack/generate-groups.sh b/hack/generate-groups.sh index cf09d670..f90aeee4 100644 --- a/hack/generate-groups.sh +++ b/hack/generate-groups.sh @@ -25,6 +25,8 @@ set -o pipefail CODEGEN_VERSION=${CODEGEN_VERSION:-"v0.19.0"} CONTROLLERGEN_VERSION=${CONTROLLERGEN_VERSION:-"v0.11.1"} +YQ_VERSION=${YQ_VERSION:-"v4.33.1"} + MODULE_NAME=${MODULE_NAME:-"github.com/kubewharf/kubeadmiral"} groups=( core/v1alpha1 @@ -34,6 +36,7 @@ groups=( # install code-generator binaries go install k8s.io/code-generator/cmd/{client-gen,lister-gen,informer-gen,deepcopy-gen}@${CODEGEN_VERSION} go install sigs.k8s.io/controller-tools/cmd/controller-gen@${CONTROLLERGEN_VERSION} +go install github.com/mikefarah/yq/v4@${YQ_VERSION} # define variables GOBIN="$(go env GOBIN)" @@ -55,6 +58,16 @@ function codegen::join() { local IFS="$1"; shift; echo "$*"; } # generate manifests echo "Generating manifests" ${GOBIN}/controller-gen crd paths=$(codegen::join ";" "${INPUT_DIRS[@]}") output:crd:artifacts:config=config/crds +# apply CRD patches +for patch_file in config/crds/patches/*.yaml; do + crd_file="config/crds/$(basename "${patch_file}")" + if [[ ! -f "$crd_file" ]]; then + echo "CRD patch file $patch_file does not have a corresponding CRD file" >&2 + exit 1 + fi + # the patch file should be an array of yq assignment commands + "${GOBIN}"/yq eval '.[]' "$patch_file" | xargs -I{} "${GOBIN}"/yq -i '{}' "$crd_file" +done # generate deepcopy echo "Generating deepcopy funcs" From 8373b382829ec52097d1479e91ff1ae3744fa33d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 29 Mar 2023 10:50:43 +0000 Subject: [PATCH 06/21] feat(auto-migrate): add fields to propagation policy spec --- ...admiral.io_clusterpropagationpolicies.yaml | 438 +++++++++--------- ...re.kubeadmiral.io_propagationpolicies.yaml | 437 +++++++++-------- ...admiral.io_clusterpropagationpolicies.yaml | 1 + ...re.kubeadmiral.io_propagationpolicies.yaml | 3 + .../core/v1alpha1/types_propagationpolicy.go | 45 ++ .../core/v1alpha1/zz_generated.deepcopy.go | 75 ++- 6 files changed, 535 insertions(+), 464 deletions(-) create mode 120000 config/crds/patches/core.kubeadmiral.io_clusterpropagationpolicies.yaml create mode 100644 config/crds/patches/core.kubeadmiral.io_propagationpolicies.yaml diff --git a/config/crds/core.kubeadmiral.io_clusterpropagationpolicies.yaml b/config/crds/core.kubeadmiral.io_clusterpropagationpolicies.yaml index 279a681e..032c2665 100644 --- a/config/crds/core.kubeadmiral.io_clusterpropagationpolicies.yaml +++ b/config/crds/core.kubeadmiral.io_clusterpropagationpolicies.yaml @@ -13,243 +13,221 @@ spec: listKind: ClusterPropagationPolicyList plural: clusterpropagationpolicies shortNames: - - cpp + - cpp singular: clusterpropagationpolicy scope: Cluster versions: - - name: v1alpha1 - schema: - openAPIV3Schema: - description: ClusterPropagationPolicy describes the scheduling rules for a - resource. - properties: - apiVersion: - description: 'APIVersion defines the versioned schema of this representation - of an object. Servers should convert recognized schemas to the latest - internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' - type: string - kind: - description: 'Kind is a string value representing the REST resource this - object represents. Servers may infer this from the endpoint the client - submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' - type: string - metadata: - type: object - spec: - properties: - clusterAffinity: - description: ClusterAffinity is a list of cluster selector terms, - the terms are ORed. A empty or nil ClusterAffinity selects everything. - items: + - name: v1alpha1 + schema: + openAPIV3Schema: + description: ClusterPropagationPolicy describes the scheduling rules for a resource. + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + properties: + autoMigration: + description: Configures behaviors related to auto migration. If absent, auto migration will be disabled. properties: - matchExpressions: - description: A list of cluster selector requirements by cluster - labels. - items: - description: ClusterSelectorRequirement is a selector that - contains values, a key, and an operator that relates the - values and keys - properties: - key: - type: string - operator: - description: ClusterSelectorOperator is the set of operators - that can be used in a cluster selector requirement. - enum: - - In - - NotIn - - Exists - - DoesNotExist - - Gt - - Lt - type: string - values: - items: - type: string - type: array - required: - - key - - operator - - values - type: object - type: array - matchFields: - description: A list of cluster selector requirements by cluster - fields. - items: - description: ClusterSelectorRequirement is a selector that - contains values, a key, and an operator that relates the - values and keys - properties: - key: - type: string - operator: - description: ClusterSelectorOperator is the set of operators - that can be used in a cluster selector requirement. - enum: - - In - - NotIn - - Exists - - DoesNotExist - - Gt - - Lt - type: string - values: - items: - type: string - type: array - required: - - key - - operator - - values - type: object - type: array - type: object - type: array - clusterSelector: - additionalProperties: - type: string - description: ClusterSelector is a label query over clusters to consider - for scheduling. An empty or nil ClusterSelector selects everything. - type: object - disableFollowerScheduling: - description: DisableFollowerScheduling is a boolean that determines - if follower scheduling is disabled. Resources that depend on other - resources (e.g. deployments) are called leaders, and resources that - are depended on (e.g. configmaps and secrets) are called followers. - If a leader enables follower scheduling, its followers will additionally - be scheduled to clusters where the leader is scheduled. - type: boolean - maxClusters: - description: MaxClusters is the maximum number of replicas that the - federated object can be propagated to The maximum number of clusters - is unbounded if no value is provided. - format: int64 - type: integer - placement: - description: Placement is an explicit list of clusters used to select - member clusters to propagate resources - items: - description: Placement describes a cluster that a federated object - can be propagated to and its propagation preferences. - properties: - clusterName: - description: ClusterName is the name of the cluster to propgate - to. - type: string - preferences: - description: Preferences contains the cluster's propagation - preferences. + keepUnschedulableReplicas: + default: false + description: Besides starting new replicas in other cluster(s), whether to keep the unschedulable replicas in the original cluster so we can go back to the desired state when the cluster recovers. + type: boolean + when: + default: + podUnschedulableFor: 1m + description: When a replica should be subject to auto migration. + minProperties: 1 properties: - maxReplicas: - description: Maximum number of replicas that should be assigned - to this cluster workload object. Unbounded if no value - provided (default). - format: int64 - minimum: 0 - type: integer - minReplicas: - description: Minimum number of replicas that should be assigned - to this cluster workload object. 0 by default. - format: int64 - minimum: 0 - type: integer - weight: - description: A number expressing the preference to put an - additional replica to this cluster workload object. - format: int64 - minimum: 0 - type: integer + podUnschedulableFor: + description: A pod will be subject to auto migration if it remains unschedulable beyond this duration. Duration should be specified in a format that can be parsed by Go's time.ParseDuration. + format: duration + type: string type: object - required: - - clusterName type: object - type: array - schedulingMode: - description: SchedulingMode determines the mode used for scheduling. - enum: - - Duplicate - - Divide - type: string - schedulingProfile: - description: Profile determines the scheduling profile to be used - for scheduling - type: string - stickyCluster: - description: StickyCluster determines if a federated object can be - rescheduled. - type: boolean - tolerations: - description: Tolerations describe a set of cluster taints that the - policy tolerates - items: - description: The pod this Toleration is attached to tolerates any - taint that matches the triple using the matching - operator . - properties: - effect: - description: Effect indicates the taint effect to match. Empty - means match all taint effects. When specified, allowed values - are NoSchedule, PreferNoSchedule and NoExecute. - type: string - key: - description: Key is the taint key that the toleration applies - to. Empty means match all taint keys. If the key is empty, - operator must be Exists; this combination means to match all - values and all keys. - type: string - operator: - description: Operator represents a key's relationship to the - value. Valid operators are Exists and Equal. Defaults to Equal. - Exists is equivalent to wildcard for value, so that a pod - can tolerate all taints of a particular category. - type: string - tolerationSeconds: - description: TolerationSeconds represents the period of time - the toleration (which must be of effect NoExecute, otherwise - this field is ignored) tolerates the taint. By default, it - is not set, which means tolerate the taint forever (do not - evict). Zero and negative values will be treated as 0 (evict - immediately) by the system. - format: int64 - type: integer - value: - description: Value is the taint value the toleration matches - to. If the operator is Exists, the value should be empty, - otherwise just a regular string. - type: string + clusterAffinity: + description: ClusterAffinity is a list of cluster selector terms, the terms are ORed. A empty or nil ClusterAffinity selects everything. + items: + properties: + matchExpressions: + description: A list of cluster selector requirements by cluster labels. + items: + description: ClusterSelectorRequirement is a selector that contains values, a key, and an operator that relates the values and keys + properties: + key: + type: string + operator: + description: ClusterSelectorOperator is the set of operators that can be used in a cluster selector requirement. + enum: + - In + - NotIn + - Exists + - DoesNotExist + - Gt + - Lt + type: string + values: + items: + type: string + type: array + required: + - key + - operator + - values + type: object + type: array + matchFields: + description: A list of cluster selector requirements by cluster fields. + items: + description: ClusterSelectorRequirement is a selector that contains values, a key, and an operator that relates the values and keys + properties: + key: + type: string + operator: + description: ClusterSelectorOperator is the set of operators that can be used in a cluster selector requirement. + enum: + - In + - NotIn + - Exists + - DoesNotExist + - Gt + - Lt + type: string + values: + items: + type: string + type: array + required: + - key + - operator + - values + type: object + type: array + type: object + type: array + clusterSelector: + additionalProperties: + type: string + description: ClusterSelector is a label query over clusters to consider for scheduling. An empty or nil ClusterSelector selects everything. type: object - type: array - required: - - schedulingMode - type: object - status: - properties: - refCount: - format: int64 - minimum: 0 - type: integer - typedRefCount: - items: + disableFollowerScheduling: + description: DisableFollowerScheduling is a boolean that determines if follower scheduling is disabled. Resources that depend on other resources (e.g. deployments) are called leaders, and resources that are depended on (e.g. configmaps and secrets) are called followers. If a leader enables follower scheduling, its followers will additionally be scheduled to clusters where the leader is scheduled. + type: boolean + maxClusters: + description: MaxClusters is the maximum number of replicas that the federated object can be propagated to The maximum number of clusters is unbounded if no value is provided. + format: int64 + type: integer + placement: + description: Placement is an explicit list of clusters used to select member clusters to propagate resources + items: + description: Placement describes a cluster that a federated object can be propagated to and its propagation preferences. + properties: + clusterName: + description: ClusterName is the name of the cluster to propgate to. + type: string + preferences: + description: Preferences contains the cluster's propagation preferences. + properties: + maxReplicas: + description: Maximum number of replicas that should be assigned to this cluster workload object. Unbounded if no value provided (default). + format: int64 + minimum: 0 + type: integer + minReplicas: + description: Minimum number of replicas that should be assigned to this cluster workload object. 0 by default. + format: int64 + minimum: 0 + type: integer + weight: + description: A number expressing the preference to put an additional replica to this cluster workload object. + format: int64 + minimum: 0 + type: integer + type: object + required: + - clusterName + type: object + type: array + replicaRescheduling: + description: Configures behaviors related to replica rescheduling. Default set via a post-generation patch. See patch file for details. properties: - count: - format: int64 - minimum: 0 - type: integer - group: - type: string - resource: - type: string - required: - - count - - resource + avoidDisruption: + default: true + description: If set to true, the scheduler will attempt to prevent migrating existing replicas during rescheduling. In order to do so, replica scheduling preferences might not be fully respected. If set to false, the scheduler will always rebalance the replicas based on the specified preferences, which might cause temporary service disruption. + type: boolean type: object - type: array - type: object - required: - - spec - type: object - served: true - storage: true - subresources: - status: {} + default: {} + schedulingMode: + description: SchedulingMode determines the mode used for scheduling. + enum: + - Duplicate + - Divide + type: string + schedulingProfile: + description: Profile determines the scheduling profile to be used for scheduling + type: string + stickyCluster: + description: StickyCluster determines if a federated object can be rescheduled. + type: boolean + tolerations: + description: Tolerations describe a set of cluster taints that the policy tolerates + items: + description: The pod this Toleration is attached to tolerates any taint that matches the triple using the matching operator . + properties: + effect: + description: Effect indicates the taint effect to match. Empty means match all taint effects. When specified, allowed values are NoSchedule, PreferNoSchedule and NoExecute. + type: string + key: + description: Key is the taint key that the toleration applies to. Empty means match all taint keys. If the key is empty, operator must be Exists; this combination means to match all values and all keys. + type: string + operator: + description: Operator represents a key's relationship to the value. Valid operators are Exists and Equal. Defaults to Equal. Exists is equivalent to wildcard for value, so that a pod can tolerate all taints of a particular category. + type: string + tolerationSeconds: + description: TolerationSeconds represents the period of time the toleration (which must be of effect NoExecute, otherwise this field is ignored) tolerates the taint. By default, it is not set, which means tolerate the taint forever (do not evict). Zero and negative values will be treated as 0 (evict immediately) by the system. + format: int64 + type: integer + value: + description: Value is the taint value the toleration matches to. If the operator is Exists, the value should be empty, otherwise just a regular string. + type: string + type: object + type: array + required: + - schedulingMode + type: object + status: + properties: + refCount: + format: int64 + minimum: 0 + type: integer + typedRefCount: + items: + properties: + count: + format: int64 + minimum: 0 + type: integer + group: + type: string + resource: + type: string + required: + - count + - resource + type: object + type: array + type: object + required: + - spec + type: object + served: true + storage: true + subresources: + status: {} diff --git a/config/crds/core.kubeadmiral.io_propagationpolicies.yaml b/config/crds/core.kubeadmiral.io_propagationpolicies.yaml index 2e48e3b9..59c95edd 100644 --- a/config/crds/core.kubeadmiral.io_propagationpolicies.yaml +++ b/config/crds/core.kubeadmiral.io_propagationpolicies.yaml @@ -13,242 +13,221 @@ spec: listKind: PropagationPolicyList plural: propagationpolicies shortNames: - - pp + - pp singular: propagationpolicy scope: Namespaced versions: - - name: v1alpha1 - schema: - openAPIV3Schema: - description: PropagationPolicy describes the scheduling rules for a resource. - properties: - apiVersion: - description: 'APIVersion defines the versioned schema of this representation - of an object. Servers should convert recognized schemas to the latest - internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' - type: string - kind: - description: 'Kind is a string value representing the REST resource this - object represents. Servers may infer this from the endpoint the client - submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' - type: string - metadata: - type: object - spec: - properties: - clusterAffinity: - description: ClusterAffinity is a list of cluster selector terms, - the terms are ORed. A empty or nil ClusterAffinity selects everything. - items: + - name: v1alpha1 + schema: + openAPIV3Schema: + description: PropagationPolicy describes the scheduling rules for a resource. + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + properties: + autoMigration: + description: Configures behaviors related to auto migration. If absent, auto migration will be disabled. properties: - matchExpressions: - description: A list of cluster selector requirements by cluster - labels. - items: - description: ClusterSelectorRequirement is a selector that - contains values, a key, and an operator that relates the - values and keys - properties: - key: - type: string - operator: - description: ClusterSelectorOperator is the set of operators - that can be used in a cluster selector requirement. - enum: - - In - - NotIn - - Exists - - DoesNotExist - - Gt - - Lt - type: string - values: - items: - type: string - type: array - required: - - key - - operator - - values - type: object - type: array - matchFields: - description: A list of cluster selector requirements by cluster - fields. - items: - description: ClusterSelectorRequirement is a selector that - contains values, a key, and an operator that relates the - values and keys - properties: - key: - type: string - operator: - description: ClusterSelectorOperator is the set of operators - that can be used in a cluster selector requirement. - enum: - - In - - NotIn - - Exists - - DoesNotExist - - Gt - - Lt - type: string - values: - items: - type: string - type: array - required: - - key - - operator - - values - type: object - type: array - type: object - type: array - clusterSelector: - additionalProperties: - type: string - description: ClusterSelector is a label query over clusters to consider - for scheduling. An empty or nil ClusterSelector selects everything. - type: object - disableFollowerScheduling: - description: DisableFollowerScheduling is a boolean that determines - if follower scheduling is disabled. Resources that depend on other - resources (e.g. deployments) are called leaders, and resources that - are depended on (e.g. configmaps and secrets) are called followers. - If a leader enables follower scheduling, its followers will additionally - be scheduled to clusters where the leader is scheduled. - type: boolean - maxClusters: - description: MaxClusters is the maximum number of replicas that the - federated object can be propagated to The maximum number of clusters - is unbounded if no value is provided. - format: int64 - type: integer - placement: - description: Placement is an explicit list of clusters used to select - member clusters to propagate resources - items: - description: Placement describes a cluster that a federated object - can be propagated to and its propagation preferences. - properties: - clusterName: - description: ClusterName is the name of the cluster to propgate - to. - type: string - preferences: - description: Preferences contains the cluster's propagation - preferences. + keepUnschedulableReplicas: + default: false + description: Besides starting new replicas in other cluster(s), whether to keep the unschedulable replicas in the original cluster so we can go back to the desired state when the cluster recovers. + type: boolean + when: + default: + podUnschedulableFor: 1m + description: When a replica should be subject to auto migration. + minProperties: 1 properties: - maxReplicas: - description: Maximum number of replicas that should be assigned - to this cluster workload object. Unbounded if no value - provided (default). - format: int64 - minimum: 0 - type: integer - minReplicas: - description: Minimum number of replicas that should be assigned - to this cluster workload object. 0 by default. - format: int64 - minimum: 0 - type: integer - weight: - description: A number expressing the preference to put an - additional replica to this cluster workload object. - format: int64 - minimum: 0 - type: integer + podUnschedulableFor: + description: A pod will be subject to auto migration if it remains unschedulable beyond this duration. Duration should be specified in a format that can be parsed by Go's time.ParseDuration. + format: duration + type: string type: object - required: - - clusterName type: object - type: array - schedulingMode: - description: SchedulingMode determines the mode used for scheduling. - enum: - - Duplicate - - Divide - type: string - schedulingProfile: - description: Profile determines the scheduling profile to be used - for scheduling - type: string - stickyCluster: - description: StickyCluster determines if a federated object can be - rescheduled. - type: boolean - tolerations: - description: Tolerations describe a set of cluster taints that the - policy tolerates - items: - description: The pod this Toleration is attached to tolerates any - taint that matches the triple using the matching - operator . - properties: - effect: - description: Effect indicates the taint effect to match. Empty - means match all taint effects. When specified, allowed values - are NoSchedule, PreferNoSchedule and NoExecute. - type: string - key: - description: Key is the taint key that the toleration applies - to. Empty means match all taint keys. If the key is empty, - operator must be Exists; this combination means to match all - values and all keys. - type: string - operator: - description: Operator represents a key's relationship to the - value. Valid operators are Exists and Equal. Defaults to Equal. - Exists is equivalent to wildcard for value, so that a pod - can tolerate all taints of a particular category. - type: string - tolerationSeconds: - description: TolerationSeconds represents the period of time - the toleration (which must be of effect NoExecute, otherwise - this field is ignored) tolerates the taint. By default, it - is not set, which means tolerate the taint forever (do not - evict). Zero and negative values will be treated as 0 (evict - immediately) by the system. - format: int64 - type: integer - value: - description: Value is the taint value the toleration matches - to. If the operator is Exists, the value should be empty, - otherwise just a regular string. - type: string + clusterAffinity: + description: ClusterAffinity is a list of cluster selector terms, the terms are ORed. A empty or nil ClusterAffinity selects everything. + items: + properties: + matchExpressions: + description: A list of cluster selector requirements by cluster labels. + items: + description: ClusterSelectorRequirement is a selector that contains values, a key, and an operator that relates the values and keys + properties: + key: + type: string + operator: + description: ClusterSelectorOperator is the set of operators that can be used in a cluster selector requirement. + enum: + - In + - NotIn + - Exists + - DoesNotExist + - Gt + - Lt + type: string + values: + items: + type: string + type: array + required: + - key + - operator + - values + type: object + type: array + matchFields: + description: A list of cluster selector requirements by cluster fields. + items: + description: ClusterSelectorRequirement is a selector that contains values, a key, and an operator that relates the values and keys + properties: + key: + type: string + operator: + description: ClusterSelectorOperator is the set of operators that can be used in a cluster selector requirement. + enum: + - In + - NotIn + - Exists + - DoesNotExist + - Gt + - Lt + type: string + values: + items: + type: string + type: array + required: + - key + - operator + - values + type: object + type: array + type: object + type: array + clusterSelector: + additionalProperties: + type: string + description: ClusterSelector is a label query over clusters to consider for scheduling. An empty or nil ClusterSelector selects everything. type: object - type: array - required: - - schedulingMode - type: object - status: - properties: - refCount: - format: int64 - minimum: 0 - type: integer - typedRefCount: - items: + disableFollowerScheduling: + description: DisableFollowerScheduling is a boolean that determines if follower scheduling is disabled. Resources that depend on other resources (e.g. deployments) are called leaders, and resources that are depended on (e.g. configmaps and secrets) are called followers. If a leader enables follower scheduling, its followers will additionally be scheduled to clusters where the leader is scheduled. + type: boolean + maxClusters: + description: MaxClusters is the maximum number of replicas that the federated object can be propagated to The maximum number of clusters is unbounded if no value is provided. + format: int64 + type: integer + placement: + description: Placement is an explicit list of clusters used to select member clusters to propagate resources + items: + description: Placement describes a cluster that a federated object can be propagated to and its propagation preferences. + properties: + clusterName: + description: ClusterName is the name of the cluster to propgate to. + type: string + preferences: + description: Preferences contains the cluster's propagation preferences. + properties: + maxReplicas: + description: Maximum number of replicas that should be assigned to this cluster workload object. Unbounded if no value provided (default). + format: int64 + minimum: 0 + type: integer + minReplicas: + description: Minimum number of replicas that should be assigned to this cluster workload object. 0 by default. + format: int64 + minimum: 0 + type: integer + weight: + description: A number expressing the preference to put an additional replica to this cluster workload object. + format: int64 + minimum: 0 + type: integer + type: object + required: + - clusterName + type: object + type: array + replicaRescheduling: + description: Configures behaviors related to replica rescheduling. Default set via a post-generation patch. See patch file for details. properties: - count: - format: int64 - minimum: 0 - type: integer - group: - type: string - resource: - type: string - required: - - count - - resource + avoidDisruption: + default: true + description: If set to true, the scheduler will attempt to prevent migrating existing replicas during rescheduling. In order to do so, replica scheduling preferences might not be fully respected. If set to false, the scheduler will always rebalance the replicas based on the specified preferences, which might cause temporary service disruption. + type: boolean type: object - type: array - type: object - required: - - spec - type: object - served: true - storage: true - subresources: - status: {} + default: {} + schedulingMode: + description: SchedulingMode determines the mode used for scheduling. + enum: + - Duplicate + - Divide + type: string + schedulingProfile: + description: Profile determines the scheduling profile to be used for scheduling + type: string + stickyCluster: + description: StickyCluster determines if a federated object can be rescheduled. + type: boolean + tolerations: + description: Tolerations describe a set of cluster taints that the policy tolerates + items: + description: The pod this Toleration is attached to tolerates any taint that matches the triple using the matching operator . + properties: + effect: + description: Effect indicates the taint effect to match. Empty means match all taint effects. When specified, allowed values are NoSchedule, PreferNoSchedule and NoExecute. + type: string + key: + description: Key is the taint key that the toleration applies to. Empty means match all taint keys. If the key is empty, operator must be Exists; this combination means to match all values and all keys. + type: string + operator: + description: Operator represents a key's relationship to the value. Valid operators are Exists and Equal. Defaults to Equal. Exists is equivalent to wildcard for value, so that a pod can tolerate all taints of a particular category. + type: string + tolerationSeconds: + description: TolerationSeconds represents the period of time the toleration (which must be of effect NoExecute, otherwise this field is ignored) tolerates the taint. By default, it is not set, which means tolerate the taint forever (do not evict). Zero and negative values will be treated as 0 (evict immediately) by the system. + format: int64 + type: integer + value: + description: Value is the taint value the toleration matches to. If the operator is Exists, the value should be empty, otherwise just a regular string. + type: string + type: object + type: array + required: + - schedulingMode + type: object + status: + properties: + refCount: + format: int64 + minimum: 0 + type: integer + typedRefCount: + items: + properties: + count: + format: int64 + minimum: 0 + type: integer + group: + type: string + resource: + type: string + required: + - count + - resource + type: object + type: array + type: object + required: + - spec + type: object + served: true + storage: true + subresources: + status: {} diff --git a/config/crds/patches/core.kubeadmiral.io_clusterpropagationpolicies.yaml b/config/crds/patches/core.kubeadmiral.io_clusterpropagationpolicies.yaml new file mode 120000 index 00000000..921c216a --- /dev/null +++ b/config/crds/patches/core.kubeadmiral.io_clusterpropagationpolicies.yaml @@ -0,0 +1 @@ +core.kubeadmiral.io_propagationpolicies.yaml \ No newline at end of file diff --git a/config/crds/patches/core.kubeadmiral.io_propagationpolicies.yaml b/config/crds/patches/core.kubeadmiral.io_propagationpolicies.yaml new file mode 100644 index 00000000..3af34af9 --- /dev/null +++ b/config/crds/patches/core.kubeadmiral.io_propagationpolicies.yaml @@ -0,0 +1,3 @@ +# controller-gen does not respect {} as default value for a struct field +# issue: https://github.com/kubernetes-sigs/controller-tools/issues/622 +- .spec.versions[0].schema.openAPIV3Schema.properties.spec.properties.replicaRescheduling.default = {} diff --git a/pkg/apis/core/v1alpha1/types_propagationpolicy.go b/pkg/apis/core/v1alpha1/types_propagationpolicy.go index 56d3b203..d463bdc4 100644 --- a/pkg/apis/core/v1alpha1/types_propagationpolicy.go +++ b/pkg/apis/core/v1alpha1/types_propagationpolicy.go @@ -97,6 +97,16 @@ type PropagationPolicySpec struct { // to clusters where the leader is scheduled. // +optional DisableFollowerScheduling bool `json:"disableFollowerScheduling,omitempty"` + + // Configures behaviors related to auto migration. If absent, auto migration will be disabled. + // +optional + AutoMigration *AutoMigration `json:"autoMigration,omitempty"` + + // Configures behaviors related to replica rescheduling. + // +optional + // Default set via a post-generation patch. + // See patch file for details. + ReplicaRescheduling *ReplicaRescheduling `json:"replicaRescheduling,omitempty"` } type PropagationPolicyStatus struct { @@ -142,3 +152,38 @@ type Preferences struct { // +kubebuilder:validation:Minimum=0 Weight *int64 `json:"weight,omitempty"` } + +// Preferences regarding auto migration. +type AutoMigration struct { + // When a replica should be subject to auto migration. + // +optional + // +kubebuilder:default:={podUnschedulableFor:"1m"} + Trigger AutoMigrationTrigger `json:"when"` + + // Besides starting new replicas in other cluster(s), whether to keep the unschedulable replicas + // in the original cluster so we can go back to the desired state when the cluster recovers. + // +optional + // +kubebuilder:default:=false + KeepUnschedulableReplicas bool `json:"keepUnschedulableReplicas"` +} + +// Criteria for determining when a replica is subject to auto migration. +// +kubebuilder:validation:MinProperties:=1 +type AutoMigrationTrigger struct { + // A pod will be subject to auto migration if it remains unschedulable beyond this duration. + // Duration should be specified in a format that can be parsed by Go's time.ParseDuration. + // +optional + // +kubebuilder:validation:Format:=duration + PodUnschedulableDuration *metav1.Duration `json:"podUnschedulableFor,omitempty"` +} + +// Preferences regarding replica rescheduling. +type ReplicaRescheduling struct { + // If set to true, the scheduler will attempt to prevent migrating existing replicas during rescheduling. + // In order to do so, replica scheduling preferences might not be fully respected. + // If set to false, the scheduler will always rebalance the replicas based on the specified preferences, which might + // cause temporary service disruption. + // +optional + // +kubebuilder:default:=true + AvoidDisruption bool `json:"avoidDisruption"` +} diff --git a/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go index 853fed33..15a4e113 100644 --- a/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go @@ -6,8 +6,9 @@ package v1alpha1 import ( - v1 "k8s.io/api/core/v1" + corev1 "k8s.io/api/core/v1" apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" runtime "k8s.io/apimachinery/pkg/runtime" ) @@ -27,6 +28,44 @@ func (in *APIResource) DeepCopy() *APIResource { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AutoMigration) DeepCopyInto(out *AutoMigration) { + *out = *in + in.Trigger.DeepCopyInto(&out.Trigger) + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AutoMigration. +func (in *AutoMigration) DeepCopy() *AutoMigration { + if in == nil { + return nil + } + out := new(AutoMigration) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AutoMigrationTrigger) DeepCopyInto(out *AutoMigrationTrigger) { + *out = *in + if in.PodUnschedulableDuration != nil { + in, out := &in.PodUnschedulableDuration, &out.PodUnschedulableDuration + *out = new(v1.Duration) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AutoMigrationTrigger. +func (in *AutoMigrationTrigger) DeepCopy() *AutoMigrationTrigger { + if in == nil { + return nil + } + out := new(AutoMigrationTrigger) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *ClusterCondition) DeepCopyInto(out *ClusterCondition) { *out = *in @@ -374,7 +413,7 @@ func (in *FederatedClusterSpec) DeepCopyInto(out *FederatedClusterSpec) { out.SecretRef = in.SecretRef if in.Taints != nil { in, out := &in.Taints, &out.Taints - *out = make([]v1.Taint, len(*in)) + *out = make([]corev1.Taint, len(*in)) for i := range *in { (*in)[i].DeepCopyInto(&(*out)[i]) } @@ -982,7 +1021,7 @@ func (in *PropagationPolicySpec) DeepCopyInto(out *PropagationPolicySpec) { } if in.Tolerations != nil { in, out := &in.Tolerations, &out.Tolerations - *out = make([]v1.Toleration, len(*in)) + *out = make([]corev1.Toleration, len(*in)) for i := range *in { (*in)[i].DeepCopyInto(&(*out)[i]) } @@ -999,6 +1038,16 @@ func (in *PropagationPolicySpec) DeepCopyInto(out *PropagationPolicySpec) { (*in)[i].DeepCopyInto(&(*out)[i]) } } + if in.AutoMigration != nil { + in, out := &in.AutoMigration, &out.AutoMigration + *out = new(AutoMigration) + (*in).DeepCopyInto(*out) + } + if in.ReplicaRescheduling != nil { + in, out := &in.ReplicaRescheduling, &out.ReplicaRescheduling + *out = new(ReplicaRescheduling) + **out = **in + } return } @@ -1029,6 +1078,22 @@ func (in *PropagationPolicyStatus) DeepCopy() *PropagationPolicyStatus { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ReplicaRescheduling) DeepCopyInto(out *ReplicaRescheduling) { + *out = *in + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ReplicaRescheduling. +func (in *ReplicaRescheduling) DeepCopy() *ReplicaRescheduling { + if in == nil { + return nil + } + out := new(ReplicaRescheduling) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Resources) DeepCopyInto(out *Resources) { *out = *in @@ -1039,14 +1104,14 @@ func (in *Resources) DeepCopyInto(out *Resources) { } if in.Allocatable != nil { in, out := &in.Allocatable, &out.Allocatable - *out = make(v1.ResourceList, len(*in)) + *out = make(corev1.ResourceList, len(*in)) for key, val := range *in { (*out)[key] = val.DeepCopy() } } if in.Available != nil { in, out := &in.Available, &out.Available - *out = make(v1.ResourceList, len(*in)) + *out = make(corev1.ResourceList, len(*in)) for key, val := range *in { (*out)[key] = val.DeepCopy() } From 3831af7206707c9a1c8527bf93bf72daa1fafd0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:03:13 +0000 Subject: [PATCH 07/21] feat(auto-migrate): add auto-migration switch to FTC --- ...e.kubeadmiral.io_federatedtypeconfigs.yaml | 9 ++++++++ config/sample/host/01-ftc.yaml | 2 ++ .../v1alpha1/types_federatedtypeconfig.go | 10 +++++++++ .../core/v1alpha1/zz_generated.deepcopy.go | 21 +++++++++++++++++++ 4 files changed, 42 insertions(+) diff --git a/config/crds/core.kubeadmiral.io_federatedtypeconfigs.yaml b/config/crds/core.kubeadmiral.io_federatedtypeconfigs.yaml index 12ccadba..44ed05d5 100644 --- a/config/crds/core.kubeadmiral.io_federatedtypeconfigs.yaml +++ b/config/crds/core.kubeadmiral.io_federatedtypeconfigs.yaml @@ -37,6 +37,15 @@ spec: type: object spec: properties: + autoMigration: + description: Configurations for auto migration. + properties: + enabled: + description: Whether or not to enable auto migration. + type: boolean + required: + - enabled + type: object controllers: description: The controllers that must run before the resource can be propagated to member clusters. Each inner slice specifies a step. diff --git a/config/sample/host/01-ftc.yaml b/config/sample/host/01-ftc.yaml index bfb2b940..9d5e4ef3 100644 --- a/config/sample/host/01-ftc.yaml +++ b/config/sample/host/01-ftc.yaml @@ -89,6 +89,8 @@ spec: version: v1alpha1 statusAggregation: Enabled revisionHistory: Enabled + autoMigration: + enabled: true controllers: - - kubeadmiral.io/global-scheduler - - kubeadmiral.io/overridepolicy-controller diff --git a/pkg/apis/core/v1alpha1/types_federatedtypeconfig.go b/pkg/apis/core/v1alpha1/types_federatedtypeconfig.go index bd729684..5847bff8 100644 --- a/pkg/apis/core/v1alpha1/types_federatedtypeconfig.go +++ b/pkg/apis/core/v1alpha1/types_federatedtypeconfig.go @@ -82,6 +82,7 @@ type FederatedTypeConfigSpec struct { // resource. // +optional StatusType *APIResource `json:"statusType,omitempty"` + // Whether or not Status object should be populated. // +optional StatusCollection *StatusCollection `json:"statusCollection,omitempty"` @@ -92,6 +93,10 @@ type FederatedTypeConfigSpec struct { // Whether or not to plan the rollout process // +optional RolloutPlan *RolloutPlanMode `json:"rolloutPlan,omitempty"` + // Configurations for auto migration. + // +optional + AutoMigration *AutoMigrationConfig `json:"autoMigration,omitempty"` + // The controllers that must run before the resource can be propagated to member clusters. // Each inner slice specifies a step. Step T must complete before step T+1 can commence. // Controllers within each step can execute in parallel. @@ -151,6 +156,11 @@ type RevisionHistoryMode string type RolloutPlanMode string +type AutoMigrationConfig struct { + // Whether or not to enable auto migration. + Enabled bool `json:"enabled"` +} + // APIResource defines how to configure the dynamic client for an API resource. type APIResource struct { // metav1.GroupVersion is not used since the json annotation of diff --git a/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go index 15a4e113..37dbc191 100644 --- a/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/core/v1alpha1/zz_generated.deepcopy.go @@ -45,6 +45,22 @@ func (in *AutoMigration) DeepCopy() *AutoMigration { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AutoMigrationConfig) DeepCopyInto(out *AutoMigrationConfig) { + *out = *in + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AutoMigrationConfig. +func (in *AutoMigrationConfig) DeepCopy() *AutoMigrationConfig { + if in == nil { + return nil + } + out := new(AutoMigrationConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *AutoMigrationTrigger) DeepCopyInto(out *AutoMigrationTrigger) { *out = *in @@ -556,6 +572,11 @@ func (in *FederatedTypeConfigSpec) DeepCopyInto(out *FederatedTypeConfigSpec) { *out = new(RolloutPlanMode) **out = **in } + if in.AutoMigration != nil { + in, out := &in.AutoMigration, &out.AutoMigration + *out = new(AutoMigrationConfig) + **out = **in + } if in.Controllers != nil { in, out := &in.Controllers, &out.Controllers *out = make([][]string, len(*in)) From e2268d1aa4cbf88a99a93276e5208ef3673415e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:05:36 +0000 Subject: [PATCH 08/21] feat(auto-migrate): count unschedulable pods --- pkg/controllers/automigration/util.go | 62 ++++++++++ pkg/controllers/automigration/util_test.go | 111 ++++++++++++++++++ .../util/podanalyzer/pod_helper.go | 67 ----------- .../util/podanalyzer/pod_helper_test.go | 94 --------------- 4 files changed, 173 insertions(+), 161 deletions(-) create mode 100644 pkg/controllers/automigration/util.go create mode 100644 pkg/controllers/automigration/util_test.go delete mode 100644 pkg/controllers/util/podanalyzer/pod_helper.go delete mode 100644 pkg/controllers/util/podanalyzer/pod_helper_test.go diff --git a/pkg/controllers/automigration/util.go b/pkg/controllers/automigration/util.go new file mode 100644 index 00000000..25ae95e7 --- /dev/null +++ b/pkg/controllers/automigration/util.go @@ -0,0 +1,62 @@ +/* +Copyright 2023 The KubeAdmiral 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 automigration + +import ( + "time" + + corev1 "k8s.io/api/core/v1" +) + +// Returns the number of unschedulable pods that remain +// unschedulable for more than unschedulableThreshold, +// and a time.Duration representing the time from now +// when the new unschedulable pod will cross the threshold, if any. +func countUnschedulablePods( + podList *corev1.PodList, + currentTime time.Time, + unschedulableThreshold time.Duration, +) (unschedulableCount int, nextCrossIn *time.Duration) { + for i := range podList.Items { + pod := &podList.Items[i] + + var scheduledCondition *corev1.PodCondition + for i := range pod.Status.Conditions { + condition := &pod.Status.Conditions[i] + if condition.Type == corev1.PodScheduled { + scheduledCondition = condition + break + } + } + if scheduledCondition == nil || + scheduledCondition.Status != corev1.ConditionFalse || + scheduledCondition.Reason != corev1.PodReasonUnschedulable { + continue + } + + timeBecameUnschedulable := scheduledCondition.LastTransitionTime + timeCrossingThreshold := timeBecameUnschedulable.Add(unschedulableThreshold) + crossingThresholdIn := timeCrossingThreshold.Sub(currentTime) + if crossingThresholdIn <= 0 { + unschedulableCount++ + } else if nextCrossIn == nil || *nextCrossIn > crossingThresholdIn { + nextCrossIn = &crossingThresholdIn + } + } + + return +} diff --git a/pkg/controllers/automigration/util_test.go b/pkg/controllers/automigration/util_test.go new file mode 100644 index 00000000..87bc8786 --- /dev/null +++ b/pkg/controllers/automigration/util_test.go @@ -0,0 +1,111 @@ +/* +Copyright 2023 The KubeAdmiral 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 automigration + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/utils/pointer" +) + +func doCheck( + t *testing.T, + now time.Time, + threshold time.Duration, + pods []corev1.Pod, + expectedUnschedulable int, + expectedNextCrossIn *time.Duration, +) { + t.Helper() + assert := assert.New(t) + + unschedulableCount, nextCrossIn := countUnschedulablePods( + &corev1.PodList{Items: pods}, now, threshold, + ) + assert.Equal(expectedUnschedulable, unschedulableCount) + assert.Equal(expectedNextCrossIn, nextCrossIn) +} + +func TestCountUnschedulablePods(t *testing.T) { + now := time.Now() + threshold := time.Minute + + okPod := newPod(true, now) + unschedulablePod := newPod(false, now.Add(-2*threshold)) + crossingIn10s := newPod(false, now.Add(10*time.Second-threshold)) + crossingIn20s := newPod(false, now.Add(20*time.Second-threshold)) + + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *okPod, + }, 0, nil) + + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *unschedulablePod, + }, 1, nil) + + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *crossingIn10s, + }, 0, pointer.Duration(10*time.Second)) + + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *unschedulablePod, + *crossingIn20s, + }, 1, pointer.Duration(20*time.Second)) + + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *unschedulablePod, + *unschedulablePod, + *crossingIn10s, + *crossingIn20s, + }, 2, pointer.Duration(10*time.Second)) + +} + +func newPod(schedulable bool, lastTransitionTimestamp time.Time) *corev1.Pod { + condition := corev1.PodCondition{ + Type: corev1.PodScheduled, + Status: corev1.ConditionTrue, + LastTransitionTime: metav1.Time{Time: lastTransitionTimestamp}, + } + if !schedulable { + condition.Status = corev1.ConditionFalse + condition.Reason = corev1.PodReasonUnschedulable + } + return &corev1.Pod{ + TypeMeta: metav1.TypeMeta{ + Kind: "pod", + APIVersion: "v1", + }, + Status: corev1.PodStatus{ + Conditions: []corev1.PodCondition{condition}, + }, + } +} diff --git a/pkg/controllers/util/podanalyzer/pod_helper.go b/pkg/controllers/util/podanalyzer/pod_helper.go deleted file mode 100644 index 27399299..00000000 --- a/pkg/controllers/util/podanalyzer/pod_helper.go +++ /dev/null @@ -1,67 +0,0 @@ -/* -Copyright 2016 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. - -This file may have been modified by The KubeAdmiral Authors -("KubeAdmiral Modifications"). All KubeAdmiral Modifications -are Copyright 2023 The KubeAdmiral Authors. -*/ - -package podanalyzer - -import ( - "time" - - corev1 "k8s.io/api/core/v1" -) - -type PodAnalysisResult struct { - // Total number of pods created. - Total int - // Number of pods that are running and ready. - RunningAndReady int - // Number of pods that have been in unschedulable state for UnshedulableThreshold seconds. - Unschedulable int - - // TODO: Handle other scenarios like pod waiting too long for scheduler etc. -} - -const ( - // TODO: make it configurable - UnschedulableThreshold = 60 * time.Second -) - -// AnalyzePods calculates how many pods from the list are in one of -// the meaningful (from the replica set perspective) states. This function is -// a temporary workaround against the current lack of ownerRef in pods. -func AnalyzePods(podList *corev1.PodList, currentTime time.Time) PodAnalysisResult { - result := PodAnalysisResult{} - for _, pod := range podList.Items { - result.Total++ - for _, condition := range pod.Status.Conditions { - if pod.Status.Phase == corev1.PodRunning { - if condition.Type == corev1.PodReady { - result.RunningAndReady++ - } - } else if condition.Type == corev1.PodScheduled && - condition.Status == corev1.ConditionFalse && - condition.Reason == corev1.PodReasonUnschedulable && - condition.LastTransitionTime.Add(UnschedulableThreshold).Before(currentTime) { - - result.Unschedulable++ - } - } - } - return result -} diff --git a/pkg/controllers/util/podanalyzer/pod_helper_test.go b/pkg/controllers/util/podanalyzer/pod_helper_test.go deleted file mode 100644 index a13b2ad4..00000000 --- a/pkg/controllers/util/podanalyzer/pod_helper_test.go +++ /dev/null @@ -1,94 +0,0 @@ -/* -Copyright 2016 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. - -This file may have been modified by The KubeAdmiral Authors -("KubeAdmiral Modifications"). All KubeAdmiral Modifications -are Copyright 2023 The KubeAdmiral Authors. -*/ - -package podanalyzer - -import ( - "testing" - "time" - - "github.com/stretchr/testify/assert" - corev1 "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" -) - -func TestAnalyze(t *testing.T) { - now := time.Now() - podRunning := newPod(t, "p1", - corev1.PodStatus{ - Phase: corev1.PodRunning, - Conditions: []corev1.PodCondition{ - { - Type: corev1.PodReady, - Status: corev1.ConditionTrue, - }, - }, - }) - podUnschedulable := newPod(t, "pU", - corev1.PodStatus{ - Phase: corev1.PodPending, - Conditions: []corev1.PodCondition{ - { - Type: corev1.PodScheduled, - Status: corev1.ConditionFalse, - Reason: corev1.PodReasonUnschedulable, - LastTransitionTime: metav1.Time{Time: now.Add(-10 * time.Minute)}, - }, - }, - }) - podOther := newPod(t, "pO", - corev1.PodStatus{ - Phase: corev1.PodPending, - Conditions: []corev1.PodCondition{}, - }) - - result := AnalyzePods( - &corev1.PodList{ - Items: []corev1.Pod{*podRunning, *podRunning, *podRunning, *podUnschedulable, *podUnschedulable}, - }, - now, - ) - assert.Equal(t, PodAnalysisResult{ - Total: 5, - RunningAndReady: 3, - Unschedulable: 2, - }, result) - - result = AnalyzePods(&corev1.PodList{Items: []corev1.Pod{*podOther}}, now) - assert.Equal(t, PodAnalysisResult{ - Total: 1, - RunningAndReady: 0, - Unschedulable: 0, - }, result) -} - -func newPod(t *testing.T, name string, status corev1.PodStatus) *corev1.Pod { - return &corev1.Pod{ - TypeMeta: metav1.TypeMeta{ - Kind: "pod", - APIVersion: "v1", - }, - ObjectMeta: metav1.ObjectMeta{ - Name: name, - Namespace: metav1.NamespaceDefault, - }, - Status: status, - } -} From c3bbf0e86a2fe7e1a5d9132240716e1fae9ef82a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:06:50 +0000 Subject: [PATCH 09/21] fix(scheduler): only enqueue objects for cluster when labels or taints changed --- pkg/controllers/scheduler/scheduler.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/controllers/scheduler/scheduler.go b/pkg/controllers/scheduler/scheduler.go index 2feb36df..cdaaa139 100644 --- a/pkg/controllers/scheduler/scheduler.go +++ b/pkg/controllers/scheduler/scheduler.go @@ -24,6 +24,7 @@ import ( "time" corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/equality" apierrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" @@ -141,7 +142,17 @@ func NewScheduler( s.clusterLister = clusterInformer.Lister() s.clusterSynced = clusterInformer.Informer().HasSynced - clusterInformer.Informer().AddEventHandler(util.NewTriggerOnAllChanges(s.enqueueFederatedObjectsForCluster)) + clusterInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: func(obj interface{}) { s.enqueueFederatedObjectsForCluster(obj.(pkgruntime.Object)) }, + DeleteFunc: func(obj interface{}) { s.enqueueFederatedObjectsForCluster(obj.(pkgruntime.Object)) }, + UpdateFunc: func(oldUntyped, newUntyped interface{}) { + oldCluster, newCluster := oldUntyped.(*fedcorev1a1.FederatedCluster), newUntyped.(*fedcorev1a1.FederatedCluster) + if !equality.Semantic.DeepEqual(oldCluster.Labels, newCluster.Labels) || + !equality.Semantic.DeepEqual(oldCluster.Spec.Taints, newCluster.Spec.Taints) { + s.enqueueFederatedObjectsForCluster(newCluster) + } + }, + }) s.schedulingProfileLister = schedulingProfileInformer.Lister() s.schedulingProfileSynced = schedulingProfileInformer.Informer().HasSynced From ad6e264adffc79e8619d84d60aa04cc43ac3b329 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:13:14 +0000 Subject: [PATCH 10/21] chore(scheduler): standardise logging levels --- pkg/controllers/scheduler/scheduler.go | 40 ++++++++++++------- .../scheduler/schedulingtriggers.go | 6 +-- 2 files changed, 29 insertions(+), 17 deletions(-) diff --git a/pkg/controllers/scheduler/scheduler.go b/pkg/controllers/scheduler/scheduler.go index cdaaa139..edd62b30 100644 --- a/pkg/controllers/scheduler/scheduler.go +++ b/pkg/controllers/scheduler/scheduler.go @@ -190,10 +190,10 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker keyedLogger := s.logger.WithValues("control-loop", "reconcile", "key", key) startTime := time.Now() - keyedLogger.Info("Start reconcile") + keyedLogger.V(3).Info("Start reconcile") defer func() { s.metrics.Duration(fmt.Sprintf("%s.latency", s.name), startTime) - keyedLogger.WithValues("duration", time.Since(startTime), "status", status.String()).Info("Finished reconcile") + keyedLogger.V(3).WithValues("duration", time.Since(startTime), "status", status.String()).Info("Finished reconcile") }() fedObject, err := s.federatedObjectFromStore(qualifiedName) @@ -202,7 +202,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker return worker.StatusError } if apierrors.IsNotFound(err) || fedObject.GetDeletionTimestamp() != nil { - keyedLogger.Info("Observed object deletion") + keyedLogger.V(3).Info("Observed object deletion") return worker.StatusAllOK } @@ -212,7 +212,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker keyedLogger.Error(err, "Failed to check controller dependencies") return worker.StatusError } else if !ok { - keyedLogger.Info("Controller dependencies not fulfilled") + keyedLogger.V(3).Info("Controller dependencies not fulfilled") return worker.StatusAllOK } @@ -231,7 +231,6 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker if hasSchedulingPolicy { if policy, err = s.policyFromStore(policyKey); err != nil { - keyedLogger.WithValues("policy", policyKey.String()).Error(err, "Failed to find matched policy") if apierrors.IsNotFound(err) { // do not retry since the object will be reenqueued after the policy is subsequently created // emit an event to warn users that the assigned propagation policy does not exist @@ -244,6 +243,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker ) return worker.Result{Success: false, RequeueAfter: nil} } + keyedLogger.WithValues("policy", policyKey.String()).Error(err, "Failed to find matched policy") return worker.StatusError } } @@ -264,11 +264,11 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker if !triggersChanged { // scheduling triggers have not changed, skip scheduling shouldSkipScheduling = true - keyedLogger.Info("Scheduling triggers not changed, skip scheduling") + keyedLogger.V(3).Info("Scheduling triggers not changed, skip scheduling") } else if len(fedObject.GetAnnotations()[common.NoSchedulingAnnotation]) > 0 { // skip scheduling if no-scheduling annotation is found shouldSkipScheduling = true - keyedLogger.Info("No-scheduling annotation found, skip scheduling") + keyedLogger.V(3).Info("No-scheduling annotation found, skip scheduling") s.eventRecorder.Eventf( fedObject, corev1.EventTypeNormal, @@ -304,7 +304,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker if !hasSchedulingPolicy { // deschedule the federated object if there is no policy attached - keyedLogger.Info("No policy specified, scheduling to no clusters") + keyedLogger.V(3).Info("No policy specified, scheduling to no clusters") s.eventRecorder.Eventf( fedObject, corev1.EventTypeNormal, @@ -317,7 +317,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker } } else { // schedule according to matched policy - keyedLogger.WithValues("policy", policyKey.String()).Info("Matched policy found, start scheduling") + keyedLogger.WithValues("policy", policyKey.String()).V(3).Info("Matched policy found, start scheduling") s.eventRecorder.Eventf( fedObject, corev1.EventTypeNormal, @@ -362,9 +362,8 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker ) return worker.StatusError } - - keyedLogger.Info(fmt.Sprintf("Scheduling result obtained: %s", result.String())) } + keyedLogger.V(3).Info("Scheduling result obtained", "result", result.String()) var followerSchedulingEnabled bool if policy != nil { @@ -394,12 +393,26 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker ) return worker.StatusError } + + updateLogger := keyedLogger.WithValues( + "result", result.String(), + "policy", policyKey.String(), + "trigger-hash", triggerHash, + "enableFollowerScheduling", auxInfo.enableFollowerScheduling, + ) + if auxInfo.unschedulableThreshold != nil { + updateLogger = updateLogger.WithValues("unschedulableThreshold", auxInfo.unschedulableThreshold.String()) + } else { + updateLogger = updateLogger.WithValues("unschedulableThreshold", "nil") + } + + updateLogger.V(1).Info("Updating federated object") if _, err := s.federatedObjectClient.Namespace(qualifiedName.Namespace).Update( context.TODO(), fedObject, metav1.UpdateOptions{}, ); err != nil { - keyedLogger.Error(err, "Failed to update federated object") + updateLogger.Error(err, "Failed to update federated object") s.eventRecorder.Eventf( fedObject, corev1.EventTypeWarning, @@ -413,8 +426,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker return worker.StatusError } - keyedLogger.WithValues("result", result.String(), "policy", policyKey.String(), "trigger-hash", triggerHash). - Info("Scheduling success") + updateLogger.V(1).Info("Updated federated object") s.eventRecorder.Eventf( fedObject, corev1.EventTypeNormal, diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 1a1dbcc3..62cd32fe 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -246,7 +246,7 @@ func (s *Scheduler) enqueueFederatedObjectsForPolicy(policy pkgruntime.Object) { return } - s.logger.WithValues("policy", policyAccessor.GetName()).Info("Enqueue federated objects for policy") + s.logger.WithValues("policy", policyAccessor.GetName()).V(2).Info("Enqueue federated objects for policy") fedObjects, err := s.federatedObjectLister.List(labels.Everything()) if err != nil { @@ -271,11 +271,11 @@ func (s *Scheduler) enqueueFederatedObjectsForPolicy(policy pkgruntime.Object) { func (s *Scheduler) enqueueFederatedObjectsForCluster(cluster pkgruntime.Object) { clusterObj := cluster.(*fedcorev1a1.FederatedCluster) if !util.IsClusterJoined(&clusterObj.Status) { - s.logger.WithValues("cluster", clusterObj.Name).Info("Skip enqueue federated objects for cluster, cluster not joined") + s.logger.WithValues("cluster", clusterObj.Name).V(3).Info("Skip enqueue federated objects for cluster, cluster not joined") return } - s.logger.WithValues("cluster", clusterObj.Name).Info("Enqueue federated objects for cluster") + s.logger.WithValues("cluster", clusterObj.Name).V(2).Info("Enqueue federated objects for cluster") fedObjects, err := s.federatedObjectLister.List(labels.Everything()) if err != nil { From b5b4ff32b621a3cec793573a129ffcfb05427407 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:40:24 +0000 Subject: [PATCH 11/21] fix(scheduler): remove cluster ready conditions from scheduling trigger --- pkg/controllers/scheduler/schedulingtriggers.go | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 62cd32fe..6fc72505 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -95,8 +95,6 @@ type schedulingTriggers struct { PolicyName string `json:"policyName"` PolicyGeneration int64 `json:"policyGeneration"` - // a map from each cluster to its ready condition - ClusterReady []keyValue[string, bool] `json:"clusterReady"` // a map from each cluster to its labels ClusterLabels []keyValue[string, []keyValue[string, string]] `json:"clusterLabels"` // a map from each cluster to its taints @@ -129,7 +127,6 @@ func (s *Scheduler) computeSchedulingTriggerHash( } } - trigger.ClusterReady = getClusterReady(clusters) trigger.ClusterLabels = getClusterLabels(clusters) trigger.ClusterTaints = getClusterTaints(clusters) @@ -190,14 +187,6 @@ func getResourceRequest(fedObject *unstructured.Unstructured) framework.Resource return framework.Resource{} } -func getClusterReady(clusters []*fedcorev1a1.FederatedCluster) []keyValue[string, bool] { - ret := make(map[string]bool, len(clusters)) - for _, cluster := range clusters { - ret[cluster.Name] = util.IsClusterReady(&cluster.Status) - } - return sortMap(ret) -} - func getClusterLabels(clusters []*fedcorev1a1.FederatedCluster) []keyValue[string, []keyValue[string, string]] { ret := make(map[string][]keyValue[string, string], len(clusters)) for _, cluster := range clusters { From ec9e73aad9130ef3595ac7640eaa5597b6b93e97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:46:17 +0000 Subject: [PATCH 12/21] fix(scheduler): fix handling of pending controllers --- pkg/controllers/scheduler/scheduler.go | 40 +++++++++++++++----------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/pkg/controllers/scheduler/scheduler.go b/pkg/controllers/scheduler/scheduler.go index edd62b30..81958ac6 100644 --- a/pkg/controllers/scheduler/scheduler.go +++ b/pkg/controllers/scheduler/scheduler.go @@ -36,6 +36,7 @@ import ( "k8s.io/client-go/tools/cache" "k8s.io/client-go/tools/record" "k8s.io/klog/v2" + "k8s.io/utils/pointer" fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" fedclient "github.com/kubewharf/kubeadmiral/pkg/client/clientset/versioned" @@ -278,21 +279,21 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker } if shouldSkipScheduling { - if err := s.updatePendingControllers(fedObject, false); err != nil { + if updated, err := s.updatePendingControllers(fedObject, false); err != nil { keyedLogger.Error(err, "Failed to update pending controllers") return worker.StatusError - } - - if _, err := s.federatedObjectClient.Namespace(qualifiedName.Namespace).Update( - context.TODO(), - fedObject, - metav1.UpdateOptions{}, - ); err != nil { - keyedLogger.Error(err, "Failed to update pending controllers") - if apierrors.IsConflict(err) { - return worker.StatusConflict + } else if updated { + if _, err := s.federatedObjectClient.Namespace(qualifiedName.Namespace).Update( + context.TODO(), + fedObject, + metav1.UpdateOptions{}, + ); err != nil { + keyedLogger.Error(err, "Failed to update pending controllers") + if apierrors.IsConflict(err) { + return worker.StatusConflict + } + return worker.StatusError } - return worker.StatusError } return worker.StatusAllOK @@ -370,7 +371,7 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker followerSchedulingEnabled = !policy.GetSpec().DisableFollowerScheduling } - updated, err := s.applySchedulingResult(fedObject, result, followerSchedulingEnabled) + schedulingResultsChanged, err := s.applySchedulingResult(fedObject, result, followerSchedulingEnabled) if err != nil { keyedLogger.Error(err, "Failed to apply scheduling result") s.eventRecorder.Eventf( @@ -382,7 +383,8 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker ) return worker.StatusError } - if err := s.updatePendingControllers(fedObject, updated); err != nil { + pendingControllersChanged, err := s.updatePendingControllers(fedObject, schedulingResultsChanged) + if err != nil { keyedLogger.Error(err, "Failed to update pending controllers") s.eventRecorder.Eventf( fedObject, @@ -394,6 +396,11 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker return worker.StatusError } + needsUpdate := schedulingResultsChanged || pendingControllersChanged + if !needsUpdate { + return worker.StatusAllOK + } + updateLogger := keyedLogger.WithValues( "result", result.String(), "policy", policyKey.String(), @@ -464,9 +471,8 @@ func (s *Scheduler) policyFromStore(qualifiedName common.QualifiedName) (fedcore // updatePendingControllers removes the scheduler from the object's pending controller annotation. If wasModified is true (the scheduling // result was not modified), it will additionally set the downstream processors to notify them to reconcile the changes made by the // scheduler. -func (s *Scheduler) updatePendingControllers(fedObject *unstructured.Unstructured, wasModified bool) error { - // we ignore the first value, since the trigger hash is always updated when this method is called and an update will be needed - _, err := pendingcontrollers.UpdatePendingControllers( +func (s *Scheduler) updatePendingControllers(fedObject *unstructured.Unstructured, wasModified bool) (bool, error) { + return pendingcontrollers.UpdatePendingControllers( fedObject, PrefixedGlobalSchedulerName, wasModified, From d14d092e4db9268954930df355eb0e3206c09785 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:50:06 +0000 Subject: [PATCH 13/21] feat(scheduler): pass unschedulable threshold in annotation --- pkg/controllers/common/constants.go | 2 + pkg/controllers/scheduler/scheduler.go | 60 +++++++++++++++++++------- 2 files changed, 47 insertions(+), 15 deletions(-) diff --git a/pkg/controllers/common/constants.go b/pkg/controllers/common/constants.go index 81ce5d92..97145732 100644 --- a/pkg/controllers/common/constants.go +++ b/pkg/controllers/common/constants.go @@ -122,6 +122,8 @@ const ( // EnableFollowerSchedulingAnnotation indicates whether follower scheduling should be enabled for the leader object. EnableFollowerSchedulingAnnotation = InternalPrefix + "enable-follower-scheduling" + // When a pod remains unschedulable beyond this threshold, it becomes eligible for automatic migration. + PodUnschedulableThresholdAnnotation = InternalPrefix + "pod-unschedulable-threshold" // AutoMigrationAnnotation contains auto migration information. AutoMigrationAnnotation = DefaultPrefix + "auto-migration" ) diff --git a/pkg/controllers/scheduler/scheduler.go b/pkg/controllers/scheduler/scheduler.go index 81958ac6..34de5603 100644 --- a/pkg/controllers/scheduler/scheduler.go +++ b/pkg/controllers/scheduler/scheduler.go @@ -366,12 +366,19 @@ func (s *Scheduler) reconcile(qualifiedName common.QualifiedName) (status worker } keyedLogger.V(3).Info("Scheduling result obtained", "result", result.String()) - var followerSchedulingEnabled bool + auxInfo := &auxiliarySchedulingInformation{ + enableFollowerScheduling: false, + unschedulableThreshold: nil, + } if policy != nil { - followerSchedulingEnabled = !policy.GetSpec().DisableFollowerScheduling + spec := policy.GetSpec() + auxInfo.enableFollowerScheduling = !spec.DisableFollowerScheduling + if autoMigration := spec.AutoMigration; autoMigration != nil { + auxInfo.unschedulableThreshold = pointer.Duration(autoMigration.Trigger.PodUnschedulableDuration.Duration) + } } - schedulingResultsChanged, err := s.applySchedulingResult(fedObject, result, followerSchedulingEnabled) + schedulingResultsChanged, err := s.applySchedulingResult(fedObject, result, auxInfo) if err != nil { keyedLogger.Error(err, "Failed to apply scheduling result") s.eventRecorder.Eventf( @@ -478,7 +485,11 @@ func (s *Scheduler) updatePendingControllers(fedObject *unstructured.Unstructure wasModified, s.typeConfig.GetControllers(), ) - return err +} + +type auxiliarySchedulingInformation struct { + enableFollowerScheduling bool + unschedulableThreshold *time.Duration } // applySchedulingResult updates the federated object with the scheduling result and the enableFollowerScheduling annotation, it returns a @@ -486,7 +497,7 @@ func (s *Scheduler) updatePendingControllers(fedObject *unstructured.Unstructure func (s *Scheduler) applySchedulingResult( fedObject *unstructured.Unstructured, result core.ScheduleResult, - enableFollowerScheduling bool, + auxInfo *auxiliarySchedulingInformation, ) (bool, error) { objectModified := false clusterSet := result.ClusterSet() @@ -511,20 +522,39 @@ func (s *Scheduler) applySchedulingResult( } objectModified = objectModified || overridesUpdated - // set enableFollowerScheduling annotation + // set annotations + annotations := fedObject.GetAnnotations() + if annotations == nil { + annotations = make(map[string]string, 2) + } + annotationsModified := false + enableFollowerSchedulingAnnotationValue := common.AnnotationValueTrue - if !enableFollowerScheduling { + if !auxInfo.enableFollowerScheduling { enableFollowerSchedulingAnnotationValue = common.AnnotationValueFalse } - annotationsUpdated, err := annotationutil.AddAnnotation( - fedObject, - common.EnableFollowerSchedulingAnnotation, - enableFollowerSchedulingAnnotationValue, - ) - if err != nil { - return false, err + if annotations[common.EnableFollowerSchedulingAnnotation] != enableFollowerSchedulingAnnotationValue { + annotations[common.EnableFollowerSchedulingAnnotation] = enableFollowerSchedulingAnnotationValue + annotationsModified = true + } + + if auxInfo.unschedulableThreshold == nil { + if _, ok := annotations[common.PodUnschedulableThresholdAnnotation]; ok { + delete(annotations, common.PodUnschedulableThresholdAnnotation) + annotationsModified = true + } + } else { + unschedulableThresholdAnnotationValue := auxInfo.unschedulableThreshold.String() + if annotations[common.PodUnschedulableThresholdAnnotation] != unschedulableThresholdAnnotationValue { + annotations[common.PodUnschedulableThresholdAnnotation] = unschedulableThresholdAnnotationValue + annotationsModified = true + } + } + + if annotationsModified { + fedObject.SetAnnotations(annotations) + objectModified = true } - objectModified = objectModified || annotationsUpdated return objectModified, nil } From 1b561ccd8fc32d72c599fef07f9cef43493c6524 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Fri, 31 Mar 2023 10:51:27 +0000 Subject: [PATCH 14/21] feat(auto-migrate): auto-migration controller --- cmd/controller-manager/app/core.go | 35 ++ cmd/controller-manager/app/ftcmanager.go | 11 +- pkg/controllers/automigration/controller.go | 410 ++++++++++++++++++++ pkg/controllers/util/store.go | 33 ++ 4 files changed, 485 insertions(+), 4 deletions(-) create mode 100644 pkg/controllers/automigration/controller.go create mode 100644 pkg/controllers/util/store.go diff --git a/cmd/controller-manager/app/core.go b/cmd/controller-manager/app/core.go index f02cdc24..ffa8d891 100644 --- a/cmd/controller-manager/app/core.go +++ b/cmd/controller-manager/app/core.go @@ -24,6 +24,8 @@ import ( "k8s.io/klog/v2" fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" + "github.com/kubewharf/kubeadmiral/pkg/client/generic" + "github.com/kubewharf/kubeadmiral/pkg/controllers/automigration" controllercontext "github.com/kubewharf/kubeadmiral/pkg/controllers/context" "github.com/kubewharf/kubeadmiral/pkg/controllers/federate" "github.com/kubewharf/kubeadmiral/pkg/controllers/federatedcluster" @@ -190,3 +192,36 @@ func startFederateController( go federateController.Run(ctx) return nil } + +func startAutoMigrationController( + ctx context.Context, + controllerCtx *controllercontext.Context, + typeConfig *fedcorev1a1.FederatedTypeConfig, +) error { + if typeConfig.Spec.AutoMigration == nil || !typeConfig.Spec.AutoMigration.Enabled { + klog.Infof("Auto migration controller disabled for FederatedTypeConfig %s", typeConfig.Name) + return nil + } + + genericClient, err := generic.New(controllerCtx.RestConfig) + if err != nil { + return fmt.Errorf("error creating generic client: %w", err) + } + + federatedAPIResource := typeConfig.GetFederatedType() + federatedGVR := schemautil.APIResourceToGVR(&federatedAPIResource) + + federateController, err := automigration.NewAutoMigrationController( + controllerConfigFromControllerContext(controllerCtx), + typeConfig, + genericClient, + controllerCtx.KubeClientset, + controllerCtx.DynamicClientset.Resource(federatedGVR), + controllerCtx.DynamicInformerFactory.ForResource(federatedGVR), + ) + if err != nil { + return fmt.Errorf("error creating auto-migration controller: %w", err) + } + go federateController.Run(ctx) + return nil +} diff --git a/cmd/controller-manager/app/ftcmanager.go b/cmd/controller-manager/app/ftcmanager.go index ae6c0dda..5c8aa642 100644 --- a/cmd/controller-manager/app/ftcmanager.go +++ b/cmd/controller-manager/app/ftcmanager.go @@ -34,13 +34,15 @@ import ( ) const ( - FederateControllerName = "federate-controller" - GlobalSchedulerName = "global-scheduler" + FederateControllerName = "federate-controller" + GlobalSchedulerName = "global-scheduler" + AutoMigrationControllerName = "auto-migration-controller" ) var knownFTCSubControllers = map[string]StartFTCSubControllerFunc{ - GlobalSchedulerName: startGlobalScheduler, - FederateControllerName: startFederateController, + GlobalSchedulerName: startGlobalScheduler, + FederateControllerName: startFederateController, + AutoMigrationControllerName: startAutoMigrationController, } // StartFTCSubControllerFunc is responsible for constructing and starting a FTC subcontroller. A FTC subcontroller is started/stopped @@ -162,6 +164,7 @@ func (m *FederatedTypeConfigManager) processQueueItem(ctx context.Context) { continue } + // TODO[ftcmanager]: handle controllers that do not need to be started for certain FTCs if err := startFunc(subControllerCtx, m.controllerCtx, typeConfig); err != nil { keyedLogger.Error(err, "Failed to start subcontrolelr") } else { diff --git a/pkg/controllers/automigration/controller.go b/pkg/controllers/automigration/controller.go new file mode 100644 index 00000000..13ee8ce3 --- /dev/null +++ b/pkg/controllers/automigration/controller.go @@ -0,0 +1,410 @@ +/* +Copyright 2023 The KubeAdmiral 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 automigration + +import ( + "context" + "encoding/json" + "fmt" + "time" + + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/equality" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + pkgruntime "k8s.io/apimachinery/pkg/runtime" + dynamicclient "k8s.io/client-go/dynamic" + "k8s.io/client-go/informers" + kubeclient "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/tools/record" + "k8s.io/klog/v2" + runtimeclient "sigs.k8s.io/controller-runtime/pkg/client" + + fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" + "github.com/kubewharf/kubeadmiral/pkg/client/generic" + "github.com/kubewharf/kubeadmiral/pkg/controllers/common" + "github.com/kubewharf/kubeadmiral/pkg/controllers/scheduler/framework" + "github.com/kubewharf/kubeadmiral/pkg/controllers/util" + "github.com/kubewharf/kubeadmiral/pkg/controllers/util/delayingdeliver" + "github.com/kubewharf/kubeadmiral/pkg/controllers/util/eventsink" + utilunstructured "github.com/kubewharf/kubeadmiral/pkg/controllers/util/unstructured" + "github.com/kubewharf/kubeadmiral/pkg/controllers/util/worker" + "github.com/kubewharf/kubeadmiral/pkg/stats" +) + +const ( + EventReasonAutoMigrationInfoUpdated = "AutoMigrationInfoUpdated" +) + +/* +The current implementation would query the member apiserver for pods for every reconcile, which is not ideal. +An easy alternative is to cache all pods from all clusters, but this would significantly +reduce scalability based on past experience. + +One way to prevent both is: +- Watch pods directly (would the cpu cost be ok?), but cache only unschedulable pods in a map from UID to pod labels +- When a pod is deleted, remove it from the map +- When the status of a target resource is updated, find its unschedulable pods and compute latest estimatedCapacity +*/ + +type Controller struct { + typeConfig *fedcorev1a1.FederatedTypeConfig + name string + + federatedObjectClient dynamicclient.NamespaceableResourceInterface + federatedObjectInformer informers.GenericInformer + + federatedInformer util.FederatedInformer + + worker worker.ReconcileWorker + + eventRecorder record.EventRecorder + + metrics stats.Metrics + logger klog.Logger +} + +func NewAutoMigrationController( + controllerConfig *util.ControllerConfig, + typeConfig *fedcorev1a1.FederatedTypeConfig, + genericFedClient generic.Client, + kubeClient kubeclient.Interface, + federatedObjectClient dynamicclient.NamespaceableResourceInterface, + federatedObjectInformer informers.GenericInformer, +) (*Controller, error) { + targetType := typeConfig.GetTargetType() + controllerName := fmt.Sprintf("%s-auto-migration", targetType.Name) + + c := &Controller{ + typeConfig: typeConfig, + name: controllerName, + + federatedObjectClient: federatedObjectClient, + federatedObjectInformer: federatedObjectInformer, + + metrics: controllerConfig.Metrics, + logger: klog.NewKlogr().WithValues("controller", "auto-migration", "ftc", typeConfig.Name), + eventRecorder: eventsink.NewDefederatingRecorderMux(kubeClient, controllerName, 6), + } + + c.worker = worker.NewReconcileWorker( + c.reconcile, + worker.WorkerTiming{}, + controllerConfig.WorkerCount, + controllerConfig.Metrics, + delayingdeliver.NewMetricTags("auto-migration-worker", c.typeConfig.GetFederatedType().Kind), + ) + + federatedObjectInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + // Only need to handle UnschedulableThreshold updates + // Addition and deletion will be triggered by the target resources. + UpdateFunc: func(oldUntyped, newUntyped interface{}) { + oldObj, newObj := oldUntyped.(*unstructured.Unstructured), newUntyped.(*unstructured.Unstructured) + oldThreshold := oldObj.GetAnnotations()[common.PodUnschedulableThresholdAnnotation] + newThreshold := newObj.GetAnnotations()[common.PodUnschedulableThresholdAnnotation] + if oldThreshold != newThreshold { + c.worker.Enqueue(common.NewQualifiedName(newObj)) + } + }, + }) + + var err error + c.federatedInformer, err = util.NewFederatedInformer( + controllerConfig, + genericFedClient, + controllerConfig.KubeConfig, + &targetType, + func(o pkgruntime.Object) { + // enqueue with a delay to simulate a rudimentary rate limiter + c.worker.EnqueueWithDelay(common.NewQualifiedName(o), 10*time.Second) + }, + &util.ClusterLifecycleHandlerFuncs{}, + ) + if err != nil { + return nil, fmt.Errorf("failed to create federated informer: %w", err) + } + + return c, nil +} + +func (c *Controller) Run(ctx context.Context) { + c.logger.Info("Starting controller") + defer c.logger.Info("Stopping controller") + + c.federatedInformer.Start() + defer c.federatedInformer.Stop() + + cachesSynced := []cache.InformerSynced{ + c.federatedObjectInformer.Informer().HasSynced, + c.federatedInformer.ClustersSynced, + func() bool { + clusters, err := c.federatedInformer.GetReadyClusters() + if err != nil { + c.logger.Error(err, "failed to get ready clusters") + return false + } + return c.federatedInformer.GetTargetStore().ClustersSynced(clusters) + }, + } + + if !cache.WaitForNamedCacheSync(c.name, ctx.Done(), cachesSynced...) { + return + } + c.worker.Run(ctx.Done()) + + <-ctx.Done() +} + +func (c *Controller) reconcile(qualifiedName common.QualifiedName) (status worker.Result) { + key := qualifiedName.String() + keyedLogger := c.logger.WithValues("control-loop", "reconcile", "object", key) + ctx := klog.NewContext(context.TODO(), keyedLogger) + + startTime := time.Now() + c.metrics.Rate("auto-migration.throughput", 1) + keyedLogger.V(3).Info("Start reconcile") + defer func() { + c.metrics.Duration(fmt.Sprintf("%s.latency", c.name), startTime) + keyedLogger.V(3).Info("Finished reconcile", "duration", time.Since(startTime), "status", status.String()) + }() + + fedObject, err := util.UnstructuredFromStore(c.federatedObjectInformer.Informer().GetStore(), key) + if err != nil { + keyedLogger.Error(err, "Failed to get object from store") + return worker.StatusError + } + if fedObject == nil || fedObject.GetDeletionTimestamp() != nil { + return worker.StatusAllOK + } + + // PodUnschedulableThresholdAnnotation is set by the scheduler. Its presence determines whether auto migration is enabled. + annotations := fedObject.GetAnnotations() + var unschedulableThreshold *time.Duration + if value, exists := annotations[common.PodUnschedulableThresholdAnnotation]; exists { + if duration, err := time.ParseDuration(value); err != nil { + keyedLogger.Error(err, "Failed to parse PodUnschedulableThresholdAnnotation") + } else { + unschedulableThreshold = &duration + } + } + + // auto-migration controller sets AutoMigrationAnnotation to + // feedback auto-migration information back to the scheduler + + var estimatedCapacity map[string]int64 + var result *worker.Result + needsUpdate := false + if unschedulableThreshold == nil { + // Clean up the annotation if auto migration is disabled. + keyedLogger.V(3).Info("Auto migration is disabled") + _, exists := annotations[common.AutoMigrationAnnotation] + if exists { + delete(annotations, common.AutoMigrationAnnotation) + needsUpdate = true + } + } else { + // Keep the annotation up-to-date if auto migration is enabled. + keyedLogger.V(3).Info("Auto migration is enabled") + clusterObjs, err := c.federatedInformer.GetTargetStore().GetFromAllClusters(key) + if err != nil { + keyedLogger.Error(err, "Failed to get objects from federated informer stores") + return worker.StatusError + } + + estimatedCapacity, result = c.estimateCapacity(ctx, clusterObjs, *unschedulableThreshold) + autoMigration := &framework.AutoMigrationInfo{EstimatedCapacity: estimatedCapacity} + + // Compare with the existing autoMigration annotation + existingAutoMigration := &framework.AutoMigrationInfo{EstimatedCapacity: nil} + if existingAutoMigrationBytes, exists := annotations[common.AutoMigrationAnnotation]; exists { + err := json.Unmarshal([]byte(existingAutoMigrationBytes), existingAutoMigration) + if err != nil { + keyedLogger.Error(err, "Existing auto migration annotation is invalid, ignoring") + // we treat invalid existing annotation as if it doesn't exist + } + } + + if !equality.Semantic.DeepEqual(existingAutoMigration, autoMigration) { + autoMigrationBytes, err := json.Marshal(autoMigration) + if err != nil { + keyedLogger.Error(err, "Failed to marshal auto migration") + return worker.StatusAllOK + } + annotations[common.AutoMigrationAnnotation] = string(autoMigrationBytes) + needsUpdate = true + } + } + + keyedLogger.V(3).Info("Observed migration information", "estimatedCapacity", estimatedCapacity) + if needsUpdate { + fedObject = fedObject.DeepCopy() + fedObject.SetAnnotations(annotations) + + keyedLogger.V(1).Info("Updating federated object with auto migration information", "estimatedCapacity", estimatedCapacity) + _, err = c.federatedObjectClient. + Namespace(qualifiedName.Namespace). + Update(ctx, fedObject, metav1.UpdateOptions{}) + if err != nil { + keyedLogger.Error(err, "Failed to update federated object for auto migration") + if apierrors.IsConflict(err) { + return worker.StatusConflict + } + return worker.StatusError + } + + keyedLogger.V(1).Info("Updated federated object with auto migration information", "estimatedCapacity", estimatedCapacity) + c.eventRecorder.Eventf( + fedObject, + corev1.EventTypeNormal, + EventReasonAutoMigrationInfoUpdated, + "Auto migration information updated: estimatedCapacity=%+v", + estimatedCapacity, + ) + } + + if result == nil { + return worker.StatusAllOK + } else { + return *result + } +} + +func (c *Controller) estimateCapacity( + ctx context.Context, + clusterObjs []util.FederatedObject, + unschedulableThreshold time.Duration, +) (map[string]int64, *worker.Result) { + keyedLogger := klog.FromContext(ctx) + needsBackoff := false + var retryAfter *time.Duration + + estimatedCapacity := make(map[string]int64, len(clusterObjs)) + + for _, clusterObj := range clusterObjs { + unsObj := clusterObj.Object.(*unstructured.Unstructured) + + totalReplicas, readyReplicas, err := c.getTotalAndReadyReplicas(unsObj) + if err != nil { + keyedLogger.Error(err, "Failed to get total and ready replicas from object", "cluster", clusterObj.ClusterName) + continue + } + + if totalReplicas == readyReplicas { + // no unschedulable pods + continue + } + + keyedLogger.V(2).Info("Getting pods from cluster", "cluster", clusterObj.ClusterName) + pods, clusterNeedsBackoff, err := c.getPodsFromCluster(ctx, unsObj, clusterObj.ClusterName) + if err != nil { + keyedLogger.Error(err, "Failed to get pods from cluster", "cluster", clusterObj.ClusterName) + if clusterNeedsBackoff { + needsBackoff = true + } + continue + } + + unschedulable, nextCrossIn := countUnschedulablePods(pods, time.Now(), unschedulableThreshold) + if unschedulable > 0 { + estimatedCapacity[clusterObj.ClusterName] = totalReplicas - int64(unschedulable) + keyedLogger.V(3).Info("Cluster has unschedulable pods", + "cluster", clusterObj.ClusterName, + "total", totalReplicas, + "unschedulable", unschedulable, + ) + } + if nextCrossIn != nil && (retryAfter == nil || *nextCrossIn < *retryAfter) { + retryAfter = nextCrossIn + } + } + + var result *worker.Result + if needsBackoff || retryAfter != nil { + result = &worker.Result{ + Success: true, + RequeueAfter: retryAfter, + Backoff: needsBackoff, + } + } + return estimatedCapacity, result +} + +func (c *Controller) getTotalAndReadyReplicas( + unsObj *unstructured.Unstructured, +) (int64, int64, error) { + // These values might not have been populated by the controller, in which case we default to 0 + + totalReplicas := int64(0) + if replicasPtr, err := utilunstructured.GetInt64FromPath( + unsObj, c.typeConfig.Spec.PathDefinition.ReplicasStatus, nil, + ); err != nil { + return 0, 0, fmt.Errorf("replicas: %w", err) + } else if replicasPtr != nil { + totalReplicas = *replicasPtr + } + + readyReplicas := int64(0) + if readyReplicasPtr, err := utilunstructured.GetInt64FromPath( + unsObj, c.typeConfig.Spec.PathDefinition.ReadyReplicasStatus, nil, + ); err != nil { + return 0, 0, fmt.Errorf("ready replicas: %w", err) + } else if readyReplicasPtr != nil { + readyReplicas = *readyReplicasPtr + } + + return totalReplicas, readyReplicas, nil +} + +func (c *Controller) getPodsFromCluster( + ctx context.Context, + unsObj *unstructured.Unstructured, + clusterName string, +) (*corev1.PodList, bool, error) { + labelSelector, err := utilunstructured.GetLabelSelectorFromPath(unsObj, c.typeConfig.Spec.PathDefinition.LabelSelector, nil) + if err != nil { + return nil, false, fmt.Errorf("failed to get label selector from object: %w", err) + } + if labelSelector == nil { + return nil, false, fmt.Errorf("missing label selector on object") + } + + selector, err := metav1.LabelSelectorAsSelector(labelSelector) + if err != nil { + return nil, false, fmt.Errorf("failed to convert label selector to selector: %w", err) + } + + client, err := c.federatedInformer.GetClientForCluster(clusterName) + if err != nil { + return nil, false, fmt.Errorf("failed to get client for cluster: %w", err) + } + + podList := &corev1.PodList{} + err = client.ListWithOptions( + ctx, + podList, + &runtimeclient.ListOptions{ + Namespace: unsObj.GetNamespace(), + LabelSelector: selector, + }) + if err != nil { + return nil, true, fmt.Errorf("failed to list pods: %w", err) + } + + return podList, false, nil +} diff --git a/pkg/controllers/util/store.go b/pkg/controllers/util/store.go new file mode 100644 index 00000000..df613e46 --- /dev/null +++ b/pkg/controllers/util/store.go @@ -0,0 +1,33 @@ +/* +Copyright 2023 The KubeAdmiral 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 util + +import ( + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/client-go/tools/cache" +) + +func UnstructuredFromStore(store cache.Store, key string) (*unstructured.Unstructured, error) { + obj, exists, err := store.GetByKey(key) + if err != nil { + return nil, err + } + if !exists { + return nil, nil + } + return obj.(*unstructured.Unstructured), nil +} From 153dfd841a2de331f1b45d069751a1e80e07fb92 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Mon, 3 Apr 2023 10:45:29 +0000 Subject: [PATCH 15/21] refactor(util): use metav1.ParseToLabelSelector instead of json unmarshalling --- pkg/controllers/util/unstructured/unstructured.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/controllers/util/unstructured/unstructured.go b/pkg/controllers/util/unstructured/unstructured.go index f907fc44..3d258d21 100644 --- a/pkg/controllers/util/unstructured/unstructured.go +++ b/pkg/controllers/util/unstructured/unstructured.go @@ -17,7 +17,6 @@ limitations under the License. package utilunstructured import ( - "encoding/json" "fmt" "strings" @@ -41,18 +40,19 @@ func GetLabelSelectorFromPath(obj *unstructured.Unstructured, path string, prefi return nil, nil } - labelSelector := metav1.LabelSelector{} switch unsLabelSelector := unsLabelSelector.(type) { case map[string]interface{}: + labelSelector := metav1.LabelSelector{} if err := pkgruntime.DefaultUnstructuredConverter.FromUnstructured(unsLabelSelector, &labelSelector); err != nil { return nil, fmt.Errorf("field value cannot be unmarshalled into metav1.LabelSelector: %w", err) } return &labelSelector, nil case string: - if err := json.Unmarshal([]byte(unsLabelSelector), &labelSelector); err != nil { + if labelSelector, err := metav1.ParseToLabelSelector(unsLabelSelector); err != nil { return nil, fmt.Errorf("field value cannot be unmarshalled into metav1.LabelSelector: %w", err) + } else { + return labelSelector, nil } - return &labelSelector, nil default: return nil, fmt.Errorf("field value is not a string or a map[string]interface{}") } From 7fc903551cd5d151332fb0bbd6faf3a3bfd67fe5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 5 Apr 2023 06:54:12 +0000 Subject: [PATCH 16/21] test(e2e): add AssertForItems util function --- test/e2e/framework/util/util.go | 54 +++++++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/test/e2e/framework/util/util.go b/test/e2e/framework/util/util.go index 165d5731..f4b30366 100644 --- a/test/e2e/framework/util/util.go +++ b/test/e2e/framework/util/util.go @@ -19,9 +19,13 @@ package util import ( "context" "errors" + "fmt" + "strings" "sync" "time" + "github.com/onsi/ginkgo/v2" + "github.com/onsi/gomega" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" errorutil "k8s.io/apimachinery/pkg/util/errors" "k8s.io/apimachinery/pkg/util/wait" @@ -29,6 +33,56 @@ import ( "github.com/kubewharf/kubeadmiral/test/e2e/framework" ) +type failure[T any] struct { + t T + message string +} + +// Runs fn for each item in items and fails the test if any of the fn calls fail. +func AssertForItems[T any]( + items []T, + fn func(g gomega.Gomega, t T), +) { + ginkgo.GinkgoHelper() + + mu := sync.Mutex{} + failures := make([]failure[T], 0, len(items)) + + wg := sync.WaitGroup{} + for _, item := range items { + wg.Add(1) + go func(item T) { + defer wg.Done() + g := gomega.NewWithT(ginkgo.GinkgoT()) + g.ConfigureWithFailHandler(func(message string, callerSkip ...int) { + mu.Lock() + defer mu.Unlock() + failures = append(failures, failure[T]{t: item, message: message}) + }) + fn(g, item) + }(item) + } + + wg.Wait() + + if len(failures) == 0 { + return + } + + buf := strings.Builder{} + for _, failure := range failures { + buf.WriteString("\n") + buf.WriteString(fmt.Sprintf("For item `%v`:", failure.t)) + buf.WriteString("\n") + for _, line := range strings.Split(failure.message, "\n") { + buf.WriteString(" ") + buf.WriteString(line) + buf.WriteString("\n") + } + } + ginkgo.Fail(buf.String()) +} + func PollUntilForItems[T any]( ctx context.Context, items []T, From a9f7cbb12a152ce296528715f2cf487a1b9c13a5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 5 Apr 2023 06:56:17 +0000 Subject: [PATCH 17/21] test(e2e): add utils for using OverridePolicies --- test/e2e/framework/policies/overridepolicy.go | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 test/e2e/framework/policies/overridepolicy.go diff --git a/test/e2e/framework/policies/overridepolicy.go b/test/e2e/framework/policies/overridepolicy.go new file mode 100644 index 00000000..4bf51585 --- /dev/null +++ b/test/e2e/framework/policies/overridepolicy.go @@ -0,0 +1,63 @@ +/* +Copyright 2023 The KubeAdmiral 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 policies + +import ( + "fmt" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/util/rand" + + fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" + "github.com/kubewharf/kubeadmiral/pkg/controllers/override" +) + +func OverridePolicyForClustersWithPatches( + baseName string, + clusterOverriders map[string]fedcorev1a1.Overriders, +) *fedcorev1a1.OverridePolicy { + name := fmt.Sprintf("%s-%s", baseName, rand.String(12)) + policy := &fedcorev1a1.OverridePolicy{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + Spec: fedcorev1a1.GenericOverridePolicySpec{ + OverrideRules: []fedcorev1a1.OverrideRule{}, + }, + } + + for clusterName, overriders := range clusterOverriders { + policy.Spec.OverrideRules = append(policy.Spec.OverrideRules, fedcorev1a1.OverrideRule{ + TargetClusters: &fedcorev1a1.TargetClusters{ + ClusterNames: []string{clusterName}, + }, + Overriders: &overriders, + }) + } + + return policy +} + +func SetOverridePolicy(obj metav1.Object, policyName string) { + labels := obj.GetLabels() + if labels == nil { + labels = map[string]string{} + } + + labels[override.OverridePolicyNameLabel] = policyName + obj.SetLabels(labels) +} From 64759cb161249630743f82bc600bb75f577041bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 5 Apr 2023 06:56:39 +0000 Subject: [PATCH 18/21] test(e2e): auto-migration --- pkg/controllers/automigration/controller.go | 4 +- test/e2e/automigration/automigration.go | 227 ++++++++++++++++++++ test/e2e/e2e_test.go | 2 + 3 files changed, 231 insertions(+), 2 deletions(-) create mode 100644 test/e2e/automigration/automigration.go diff --git a/pkg/controllers/automigration/controller.go b/pkg/controllers/automigration/controller.go index 13ee8ce3..4338d0d0 100644 --- a/pkg/controllers/automigration/controller.go +++ b/pkg/controllers/automigration/controller.go @@ -88,8 +88,7 @@ func NewAutoMigrationController( federatedObjectClient dynamicclient.NamespaceableResourceInterface, federatedObjectInformer informers.GenericInformer, ) (*Controller, error) { - targetType := typeConfig.GetTargetType() - controllerName := fmt.Sprintf("%s-auto-migration", targetType.Name) + controllerName := fmt.Sprintf("%s-auto-migration", typeConfig.Name) c := &Controller{ typeConfig: typeConfig, @@ -125,6 +124,7 @@ func NewAutoMigrationController( }) var err error + targetType := typeConfig.GetTargetType() c.federatedInformer, err = util.NewFederatedInformer( controllerConfig, genericFedClient, diff --git a/test/e2e/automigration/automigration.go b/test/e2e/automigration/automigration.go new file mode 100644 index 00000000..b20d3c04 --- /dev/null +++ b/test/e2e/automigration/automigration.go @@ -0,0 +1,227 @@ +/* +Copyright 2023 The KubeAdmiral 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 automigration + +import ( + "context" + "time" + + "github.com/onsi/ginkgo/v2" + "github.com/onsi/gomega" + appsv1 "k8s.io/api/apps/v1" + apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "k8s.io/apimachinery/pkg/util/intstr" + "k8s.io/utils/pointer" + + fedcorev1a1 "github.com/kubewharf/kubeadmiral/pkg/apis/core/v1alpha1" + "github.com/kubewharf/kubeadmiral/pkg/controllers/override" + "github.com/kubewharf/kubeadmiral/test/e2e/framework" + "github.com/kubewharf/kubeadmiral/test/e2e/framework/policies" + "github.com/kubewharf/kubeadmiral/test/e2e/framework/resources" + "github.com/kubewharf/kubeadmiral/test/e2e/framework/util" +) + +var ( + autoMigrationTestLabel = ginkgo.Label("auto-migration") + + defaultPollingInterval = 10 * time.Millisecond + + resourcePropagationTimeout = 10 * time.Second + replicasReadyTimeout = 30 * time.Second + autoMigrationTimeout = 30 * time.Second +) + +var _ = ginkgo.Describe("auto migration", autoMigrationTestLabel, func() { + f := framework.NewFramework("auto-migration", framework.FrameworkOptions{CreateNamespace: true}) + + ginkgo.It("Should automatically migrate unschedulable pods", func(ctx context.Context) { + var err error + + var clusters []*fedcorev1a1.FederatedCluster + var clusterToMigrateFrom *fedcorev1a1.FederatedCluster + ginkgo.By("Getting clusters", func() { + clusterList, err := f.HostFedClient().CoreV1alpha1().FederatedClusters().List(ctx, metav1.ListOptions{}) + gomega.Expect(err).ToNot(gomega.HaveOccurred(), framework.MessageUnexpectedError) + + candidateClustes := make([]*fedcorev1a1.FederatedCluster, len(clusterList.Items)) + for i := range clusterList.Items { + candidateClustes[i] = &clusterList.Items[i] + } + candidateClustes = util.FilterOutE2ETestObjects(candidateClustes) + gomega.Expect(len(candidateClustes)).To(gomega.BeNumerically(">=", 3), "At least 3 clusters are required for this test") + + clusters = candidateClustes[:3] + clusterToMigrateFrom = candidateClustes[0] + }) + + propPolicy := policies.PropagationPolicyForClustersWithPlacements(f.Name(), clusters) + // Equal weight for all clusters + for i := range propPolicy.Spec.Placements { + propPolicy.Spec.Placements[i].Preferences = fedcorev1a1.Preferences{ + Weight: pointer.Int64(1), + } + } + propPolicy.Spec.SchedulingMode = fedcorev1a1.SchedulingModeDivide + // Enable auto migration; don't keep unschedulable replicas + propPolicy.Spec.AutoMigration = &fedcorev1a1.AutoMigration{ + Trigger: fedcorev1a1.AutoMigrationTrigger{ + PodUnschedulableDuration: &metav1.Duration{Duration: 5 * time.Second}, + }, + KeepUnschedulableReplicas: false, + } + + ginkgo.By("Creating PropagationPolicy", func() { + propPolicy, err = f.HostFedClient().CoreV1alpha1().PropagationPolicies(f.TestNamespace().Name).Create( + ctx, + propPolicy, + metav1.CreateOptions{}, + ) + gomega.Expect(err).ToNot(gomega.HaveOccurred(), framework.MessageUnexpectedError) + }) + + replicasPerCluster := int32(2) + totalReplicas := int32(len(clusters)) * replicasPerCluster + replicasPerClusterAfterMigration := replicasPerCluster + 1 + + dp := resources.GetSimpleDeployment(f.Name()) + dp.Spec.Replicas = pointer.Int32(totalReplicas) + maxSurge := intstr.FromInt(0) + dp.Spec.Strategy = appsv1.DeploymentStrategy{ + Type: appsv1.RollingUpdateDeploymentStrategyType, + RollingUpdate: &appsv1.RollingUpdateDeployment{ + // Prevent upgrade getting stuck + MaxSurge: &maxSurge, + }, + } + policies.SetPropagationPolicy(dp, propPolicy) + + ginkgo.By("Creating Deployment", func() { + dp, err = f.HostKubeClient().AppsV1().Deployments(f.TestNamespace().Name).Create( + ctx, dp, metav1.CreateOptions{}, + ) + gomega.Expect(err).ToNot(gomega.HaveOccurred(), framework.MessageUnexpectedError) + }) + + ginkgo.By("Checking replicas evenly distributed in member clusters", func() { + ctx, cancel := context.WithTimeout(ctx, resourcePropagationTimeout) + defer cancel() + + util.AssertForItems(clusters, func(g gomega.Gomega, c *fedcorev1a1.FederatedCluster) { + g.Eventually(ctx, func(g gomega.Gomega) { + clusterDp, err := f.ClusterKubeClient(ctx, c).AppsV1().Deployments(f.TestNamespace().Name).Get( + ctx, dp.Name, metav1.GetOptions{}, + ) + if err != nil { + gomega.Expect(err).To(gomega.Satisfy(apierrors.IsNotFound)) + } + g.Expect(err).NotTo(gomega.HaveOccurred()) + g.Expect(clusterDp.Spec.Replicas).To(gomega.HaveValue(gomega.Equal(replicasPerCluster))) + }).WithPolling(defaultPollingInterval).Should(gomega.Succeed()) + }) + }) + + ginkgo.By("Checking replicas ready in member clusters", func() { + ctx, cancel := context.WithTimeout(ctx, replicasReadyTimeout) + defer cancel() + + util.AssertForItems(clusters, func(g gomega.Gomega, c *fedcorev1a1.FederatedCluster) { + g.Eventually(ctx, func(g gomega.Gomega) { + clusterDp, err := f.ClusterKubeClient(ctx, c).AppsV1().Deployments(f.TestNamespace().Name).Get( + ctx, dp.Name, metav1.GetOptions{}, + ) + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + g.Expect(clusterDp.Status.ReadyReplicas).To(gomega.Equal(replicasPerCluster)) + }).WithPolling(defaultPollingInterval).Should(gomega.Succeed()) + }) + }) + + overridePolicy := policies.OverridePolicyForClustersWithPatches(f.Name(), map[string]fedcorev1a1.Overriders{ + clusterToMigrateFrom.Name: { + JsonPatch: []fedcorev1a1.JsonPatchOverrider{{ + Operator: "add", + Path: "/spec/template/spec/nodeSelector", + Value: apiextensionsv1.JSON{ + // should select no nodes + Raw: []byte(`{"non-existing-key": "non-existing-value"}`), + }, + }}, + }, + }) + ginkgo.By("Creating OverridePolicy", func() { + overridePolicy, err = f.HostFedClient().CoreV1alpha1().OverridePolicies(f.TestNamespace().Name).Create( + ctx, overridePolicy, metav1.CreateOptions{}, + ) + gomega.Expect(err).ToNot(gomega.HaveOccurred(), framework.MessageUnexpectedError) + }) + + ginkgo.By("Adding OverridePolicy to Deployment", func() { + policies.SetOverridePolicy(dp, overridePolicy.Name) + dp, err = f.HostKubeClient().AppsV1().Deployments(f.TestNamespace().Name).Patch( + ctx, dp.Name, types.MergePatchType, + []byte(`{"metadata":{"labels":{"`+override.OverridePolicyNameLabel+`":"`+overridePolicy.Name+`"}}}`), metav1.PatchOptions{}, + ) + gomega.Expect(err).ToNot(gomega.HaveOccurred(), framework.MessageUnexpectedError) + }) + + ginkgo.By("Check override propagated", func() { + ctx, cancel := context.WithTimeout(ctx, resourcePropagationTimeout) + defer cancel() + + util.AssertForItems(clusters, func(g gomega.Gomega, c *fedcorev1a1.FederatedCluster) { + g.Eventually(ctx, func(g gomega.Gomega) { + clusterDp, err := f.ClusterKubeClient(ctx, c).AppsV1().Deployments(f.TestNamespace().Name).Get( + ctx, dp.Name, metav1.GetOptions{}, + ) + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + if c.Name == clusterToMigrateFrom.Name { + g.Expect(clusterDp.Spec.Template.Spec.NodeSelector).To(gomega.HaveKeyWithValue("non-existing-key", "non-existing-value")) + } else { + gomega.Expect(clusterDp.Spec.Template.Spec.NodeSelector).To(gomega.BeEmpty()) + } + }).WithPolling(defaultPollingInterval).Should(gomega.Succeed()) + }) + }) + + ginkgo.By("Check pods migrated", func() { + ctx, cancel := context.WithTimeout(ctx, autoMigrationTimeout) + defer cancel() + + gomega.Eventually(ctx, func(g gomega.Gomega) { + _, err := f.ClusterKubeClient(ctx, clusterToMigrateFrom).AppsV1().Deployments(f.TestNamespace().Name).Get( + ctx, dp.Name, metav1.GetOptions{}, + ) + g.Expect(err).To(gomega.Satisfy(apierrors.IsNotFound)) + }).WithPolling(defaultPollingInterval).Should(gomega.Succeed()) + + for _, cluster := range clusters { + if cluster.Name == clusterToMigrateFrom.Name { + continue + } + gomega.Eventually(ctx, func(g gomega.Gomega) { + clusterDp, err := f.ClusterKubeClient(ctx, cluster).AppsV1().Deployments(f.TestNamespace().Name).Get( + ctx, dp.Name, metav1.GetOptions{}, + ) + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + g.Expect(clusterDp.Spec.Replicas).To(gomega.HaveValue(gomega.Equal(replicasPerClusterAfterMigration))) + }).WithPolling(defaultPollingInterval).Should(gomega.Succeed()) + } + }) + }) +}) diff --git a/test/e2e/e2e_test.go b/test/e2e/e2e_test.go index ee842439..894e7d79 100644 --- a/test/e2e/e2e_test.go +++ b/test/e2e/e2e_test.go @@ -24,6 +24,8 @@ import ( "github.com/onsi/ginkgo/v2" "github.com/onsi/gomega" + // Remember to import the package for tests to be discovered by ginkgo + _ "github.com/kubewharf/kubeadmiral/test/e2e/automigration" _ "github.com/kubewharf/kubeadmiral/test/e2e/federatedcluster" _ "github.com/kubewharf/kubeadmiral/test/e2e/resourcepropagation" // _ "github.com/kubewharf/kubeadmiral/test/e2e/example" From c65353d67e3848d31a85572f7888bf3528b4a263 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Wed, 5 Apr 2023 11:28:52 +0000 Subject: [PATCH 19/21] fix(auto-migrate): don't consider terminating unschedulable pods --- pkg/controllers/automigration/controller.go | 12 ++++++---- pkg/controllers/automigration/util.go | 6 ++++- pkg/controllers/automigration/util_test.go | 26 ++++++++++++++++----- 3 files changed, 32 insertions(+), 12 deletions(-) diff --git a/pkg/controllers/automigration/controller.go b/pkg/controllers/automigration/controller.go index 4338d0d0..7bad72a4 100644 --- a/pkg/controllers/automigration/controller.go +++ b/pkg/controllers/automigration/controller.go @@ -299,6 +299,7 @@ func (c *Controller) estimateCapacity( for _, clusterObj := range clusterObjs { unsObj := clusterObj.Object.(*unstructured.Unstructured) + // NOTE: these should follow Deployment's status semantics and should not include terminating pods totalReplicas, readyReplicas, err := c.getTotalAndReadyReplicas(unsObj) if err != nil { keyedLogger.Error(err, "Failed to get total and ready replicas from object", "cluster", clusterObj.ClusterName) @@ -321,13 +322,14 @@ func (c *Controller) estimateCapacity( } unschedulable, nextCrossIn := countUnschedulablePods(pods, time.Now(), unschedulableThreshold) + keyedLogger.V(3).Info("Analyzed pods", + "cluster", clusterObj.ClusterName, + "total", totalReplicas, + "ready", readyReplicas, + "unschedulable", unschedulable, + ) if unschedulable > 0 { estimatedCapacity[clusterObj.ClusterName] = totalReplicas - int64(unschedulable) - keyedLogger.V(3).Info("Cluster has unschedulable pods", - "cluster", clusterObj.ClusterName, - "total", totalReplicas, - "unschedulable", unschedulable, - ) } if nextCrossIn != nil && (retryAfter == nil || *nextCrossIn < *retryAfter) { retryAfter = nextCrossIn diff --git a/pkg/controllers/automigration/util.go b/pkg/controllers/automigration/util.go index 25ae95e7..fc366c53 100644 --- a/pkg/controllers/automigration/util.go +++ b/pkg/controllers/automigration/util.go @@ -34,6 +34,10 @@ func countUnschedulablePods( for i := range podList.Items { pod := &podList.Items[i] + if pod.GetDeletionTimestamp() != nil { + continue + } + var scheduledCondition *corev1.PodCondition for i := range pod.Status.Conditions { condition := &pod.Status.Conditions[i] @@ -58,5 +62,5 @@ func countUnschedulablePods( } } - return + return unschedulableCount, nextCrossIn } diff --git a/pkg/controllers/automigration/util_test.go b/pkg/controllers/automigration/util_test.go index 87bc8786..ffe8172d 100644 --- a/pkg/controllers/automigration/util_test.go +++ b/pkg/controllers/automigration/util_test.go @@ -48,10 +48,11 @@ func TestCountUnschedulablePods(t *testing.T) { now := time.Now() threshold := time.Minute - okPod := newPod(true, now) - unschedulablePod := newPod(false, now.Add(-2*threshold)) - crossingIn10s := newPod(false, now.Add(10*time.Second-threshold)) - crossingIn20s := newPod(false, now.Add(20*time.Second-threshold)) + okPod := newPod(false, true, now) + unschedulablePod := newPod(false, false, now.Add(-2*threshold)) + unschedulableTerminatingPod := newPod(true, false, now.Add(-2*threshold)) + crossingIn10s := newPod(false, false, now.Add(10*time.Second-threshold)) + crossingIn20s := newPod(false, false, now.Add(20*time.Second-threshold)) doCheck(t, now, time.Minute, []corev1.Pod{ *okPod, @@ -87,9 +88,18 @@ func TestCountUnschedulablePods(t *testing.T) { *crossingIn20s, }, 2, pointer.Duration(10*time.Second)) + doCheck(t, now, time.Minute, []corev1.Pod{ + *okPod, + *okPod, + *unschedulablePod, + *unschedulableTerminatingPod, + *crossingIn10s, + *crossingIn20s, + }, 1, pointer.Duration(10*time.Second)) + } -func newPod(schedulable bool, lastTransitionTimestamp time.Time) *corev1.Pod { +func newPod(terminating bool, schedulable bool, lastTransitionTimestamp time.Time) *corev1.Pod { condition := corev1.PodCondition{ Type: corev1.PodScheduled, Status: corev1.ConditionTrue, @@ -99,7 +109,7 @@ func newPod(schedulable bool, lastTransitionTimestamp time.Time) *corev1.Pod { condition.Status = corev1.ConditionFalse condition.Reason = corev1.PodReasonUnschedulable } - return &corev1.Pod{ + pod := &corev1.Pod{ TypeMeta: metav1.TypeMeta{ Kind: "pod", APIVersion: "v1", @@ -108,4 +118,8 @@ func newPod(schedulable bool, lastTransitionTimestamp time.Time) *corev1.Pod { Conditions: []corev1.PodCondition{condition}, }, } + if terminating { + pod.DeletionTimestamp = &metav1.Time{Time: time.Now()} + } + return pod } From 05aedc01704209318bdc74e11386a2530f0e383b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Mon, 10 Apr 2023 07:36:58 +0000 Subject: [PATCH 20/21] fix(auto-migrate): list pods with RV=0 --- pkg/controllers/automigration/controller.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/controllers/automigration/controller.go b/pkg/controllers/automigration/controller.go index 7bad72a4..75028b9c 100644 --- a/pkg/controllers/automigration/controller.go +++ b/pkg/controllers/automigration/controller.go @@ -403,6 +403,9 @@ func (c *Controller) getPodsFromCluster( &runtimeclient.ListOptions{ Namespace: unsObj.GetNamespace(), LabelSelector: selector, + Raw: &metav1.ListOptions{ + ResourceVersion: "0", + }, }) if err != nil { return nil, true, fmt.Errorf("failed to list pods: %w", err) From 8c245520e14ab5e5fd739590851808146039bd8a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gary=20Liu=20=28=E5=88=98=E5=B9=BF=E6=BA=90=29?= Date: Mon, 10 Apr 2023 07:48:22 +0000 Subject: [PATCH 21/21] refactor(auto-migrate): standardise info annotation --- pkg/controllers/automigration/controller.go | 18 +++++++++--------- pkg/controllers/common/constants.go | 4 ++-- pkg/controllers/scheduler/framework/types.go | 4 ++-- .../scheduler/schedulingtriggers.go | 2 +- pkg/controllers/scheduler/schedulingunit.go | 10 +++++----- 5 files changed, 19 insertions(+), 19 deletions(-) diff --git a/pkg/controllers/automigration/controller.go b/pkg/controllers/automigration/controller.go index 75028b9c..2498d7a5 100644 --- a/pkg/controllers/automigration/controller.go +++ b/pkg/controllers/automigration/controller.go @@ -213,9 +213,9 @@ func (c *Controller) reconcile(qualifiedName common.QualifiedName) (status worke if unschedulableThreshold == nil { // Clean up the annotation if auto migration is disabled. keyedLogger.V(3).Info("Auto migration is disabled") - _, exists := annotations[common.AutoMigrationAnnotation] + _, exists := annotations[common.AutoMigrationInfoAnnotation] if exists { - delete(annotations, common.AutoMigrationAnnotation) + delete(annotations, common.AutoMigrationInfoAnnotation) needsUpdate = true } } else { @@ -228,25 +228,25 @@ func (c *Controller) reconcile(qualifiedName common.QualifiedName) (status worke } estimatedCapacity, result = c.estimateCapacity(ctx, clusterObjs, *unschedulableThreshold) - autoMigration := &framework.AutoMigrationInfo{EstimatedCapacity: estimatedCapacity} + autoMigrationInfo := &framework.AutoMigrationInfo{EstimatedCapacity: estimatedCapacity} // Compare with the existing autoMigration annotation - existingAutoMigration := &framework.AutoMigrationInfo{EstimatedCapacity: nil} - if existingAutoMigrationBytes, exists := annotations[common.AutoMigrationAnnotation]; exists { - err := json.Unmarshal([]byte(existingAutoMigrationBytes), existingAutoMigration) + existingAutoMigrationInfo := &framework.AutoMigrationInfo{EstimatedCapacity: nil} + if existingAutoMigrationInfoBytes, exists := annotations[common.AutoMigrationInfoAnnotation]; exists { + err := json.Unmarshal([]byte(existingAutoMigrationInfoBytes), existingAutoMigrationInfo) if err != nil { keyedLogger.Error(err, "Existing auto migration annotation is invalid, ignoring") // we treat invalid existing annotation as if it doesn't exist } } - if !equality.Semantic.DeepEqual(existingAutoMigration, autoMigration) { - autoMigrationBytes, err := json.Marshal(autoMigration) + if !equality.Semantic.DeepEqual(existingAutoMigrationInfo, autoMigrationInfo) { + autoMigrationInfoBytes, err := json.Marshal(autoMigrationInfo) if err != nil { keyedLogger.Error(err, "Failed to marshal auto migration") return worker.StatusAllOK } - annotations[common.AutoMigrationAnnotation] = string(autoMigrationBytes) + annotations[common.AutoMigrationInfoAnnotation] = string(autoMigrationInfoBytes) needsUpdate = true } } diff --git a/pkg/controllers/common/constants.go b/pkg/controllers/common/constants.go index 97145732..a3e6d522 100644 --- a/pkg/controllers/common/constants.go +++ b/pkg/controllers/common/constants.go @@ -124,8 +124,8 @@ const ( // When a pod remains unschedulable beyond this threshold, it becomes eligible for automatic migration. PodUnschedulableThresholdAnnotation = InternalPrefix + "pod-unschedulable-threshold" - // AutoMigrationAnnotation contains auto migration information. - AutoMigrationAnnotation = DefaultPrefix + "auto-migration" + // AutoMigrationInfoAnnotation contains auto migration information. + AutoMigrationInfoAnnotation = DefaultPrefix + "auto-migration-info" ) // The following consts are keys used to store information in the federated cluster secret diff --git a/pkg/controllers/scheduler/framework/types.go b/pkg/controllers/scheduler/framework/types.go index b814aa91..e032655a 100644 --- a/pkg/controllers/scheduler/framework/types.go +++ b/pkg/controllers/scheduler/framework/types.go @@ -46,7 +46,7 @@ type SchedulingUnit struct { // Describes the current scheduling state CurrentClusters map[string]*int64 - AutoMigration *AutoMigrationConfig + AutoMigration *AutoMigrationSpec // Controls the scheduling behavior SchedulingMode fedcorev1a1.SchedulingMode @@ -64,7 +64,7 @@ type SchedulingUnit struct { Weights map[string]int64 } -type AutoMigrationConfig struct { +type AutoMigrationSpec struct { Info *AutoMigrationInfo KeepUnschedulableReplicas bool } diff --git a/pkg/controllers/scheduler/schedulingtriggers.go b/pkg/controllers/scheduler/schedulingtriggers.go index 6fc72505..b4a13969 100644 --- a/pkg/controllers/scheduler/schedulingtriggers.go +++ b/pkg/controllers/scheduler/schedulingtriggers.go @@ -121,7 +121,7 @@ func (s *Scheduler) computeSchedulingTriggerHash( trigger.PolicyGeneration = policy.GetGeneration() if policy.GetSpec().AutoMigration != nil { // Only consider auto-migration annotation when auto-migration is enabled in the policy. - if value, exists := fedObject.GetAnnotations()[common.AutoMigrationAnnotation]; exists { + if value, exists := fedObject.GetAnnotations()[common.AutoMigrationInfoAnnotation]; exists { trigger.AutoMigrationInfo = &value } } diff --git a/pkg/controllers/scheduler/schedulingunit.go b/pkg/controllers/scheduler/schedulingunit.go index b55f04ac..7e156187 100644 --- a/pkg/controllers/scheduler/schedulingunit.go +++ b/pkg/controllers/scheduler/schedulingunit.go @@ -89,7 +89,7 @@ func (s *Scheduler) schedulingUnitForFedObject( if err != nil { return nil, err } - schedulingUnit.AutoMigration = &framework.AutoMigrationConfig{ + schedulingUnit.AutoMigration = &framework.AutoMigrationSpec{ Info: info, KeepUnschedulableReplicas: autoMigration.KeepUnschedulableReplicas, } @@ -235,16 +235,16 @@ func getSchedulingModeFromObject(object *unstructured.Unstructured) (fedcorev1a1 } func getAutoMigrationInfo(fedObject *unstructured.Unstructured) (*framework.AutoMigrationInfo, error) { - value, exists := fedObject.GetAnnotations()[common.AutoMigrationAnnotation] + value, exists := fedObject.GetAnnotations()[common.AutoMigrationInfoAnnotation] if !exists { return nil, nil } - autoMigration := new(framework.AutoMigrationInfo) - if err := json.Unmarshal([]byte(value), autoMigration); err != nil { + autoMigrationInfo := new(framework.AutoMigrationInfo) + if err := json.Unmarshal([]byte(value), autoMigrationInfo); err != nil { return nil, err } - return autoMigration, nil + return autoMigrationInfo, nil } func getIsStickyClusterFromPolicy(policy fedcorev1a1.GenericPropagationPolicy) bool {