Optimize internal data structure of EvenPodsSpread

- Rename 'topologyPairsPodSpreadMap' to 'podSpreadCache'
- New struct `criticalPaths criticalPaths`
- Add unified method `*criticalPaths.update()` for:
    - regular update
    - addPod in preemption case
    - remotePod in preemption case
This commit is contained in:
Wei Huang 2019-08-16 09:20:11 -07:00
parent 7f1a3965fd
commit 8f559ea53b
No known key found for this signature in database
GPG Key ID: BE5E9752F8B6E005
6 changed files with 462 additions and 613 deletions

View File

@ -39,7 +39,7 @@ import (
type PredicateMetadata interface {
ShallowCopy() PredicateMetadata
AddPod(addedPod *v1.Pod, nodeInfo *schedulernodeinfo.NodeInfo) error
RemovePod(deletedPod *v1.Pod) error
RemovePod(deletedPod *v1.Pod, node *v1.Node) error
}
// PredicateMetadataProducer is a function that computes predicate metadata for a given pod.
@ -67,17 +67,67 @@ type topologyPairsMaps struct {
podToTopologyPairs map[string]topologyPairSet
}
// topologyPairsPodSpreadMap combines topologyKeyToMinPodsMap and topologyPairsMaps
type criticalPath struct {
// topologyValue denotes the topology value mapping to topology key.
topologyValue string
// matchNum denotes the number of matching pods.
matchNum int32
}
// CAVEAT: the reason that `[2]criticalPath` can work is based on the implementation of current
// preemption algorithm, in particular the following 2 facts:
// Fact 1: we only preempt pods on the same node, instead of pods on multiple nodes.
// Fact 2: each node is evaluated on a separate copy of the metadata during its preemption cycle.
// If we plan to turn to a more complex algorithm like "arbitrary pods on multiple nodes", this
// structure needs to be revisited.
type criticalPaths [2]criticalPath
func newCriticalPaths() *criticalPaths {
return &criticalPaths{{matchNum: math.MaxInt32}, {matchNum: math.MaxInt32}}
}
func (paths *criticalPaths) update(tpVal string, num int32) {
// first verify if `tpVal` exists or not
i := -1
if tpVal == paths[0].topologyValue {
i = 0
} else if tpVal == paths[1].topologyValue {
i = 1
}
if i >= 0 {
// `tpVal` exists
paths[i].matchNum = num
if paths[0].matchNum > paths[1].matchNum {
// swap paths[0] and paths[1]
paths[0], paths[1] = paths[1], paths[0]
}
} else {
// `tpVal` doesn't exist
if num < paths[0].matchNum {
// update paths[1] with paths[0]
paths[1] = paths[0]
// update paths[0]
paths[0].topologyValue, paths[0].matchNum = tpVal, num
} else if num < paths[1].matchNum {
// update paths[1]
paths[1].topologyValue, paths[1].matchNum = tpVal, num
}
}
}
// podSpreadCache combines tpKeyToCriticalPaths and tpPairToMatchNum
// to represent:
// (1) minimum number of pods matched on the spread constraints.
// (2) how existing pods match incoming pod on its spread constraints.
type topologyPairsPodSpreadMap struct {
// This map is keyed with a topology key, and valued with minimum number
// of pods matched on that topology domain.
// TODO(Huang-Wei): refactor to {tpKey->tpValSet(or tpValSlice)}
topologyKeyToMinPodsMap map[string]int32
// TODO(Huang-Wei): refactor to {tpPair->count, podName->tpPairSet(optional)}
*topologyPairsMaps
// (1) critical paths where the least pods are matched on each spread constraint.
// (2) number of pods matched on each spread constraint.
type podSpreadCache struct {
// We record 2 critical paths instead of all critical paths here.
// criticalPaths[0].matchNum always holds the minimum matching number.
// criticalPaths[1].matchNum is always greater or equal to criticalPaths[0].matchNum, but
// it's not guaranteed to be the 2nd minimum match number.
tpKeyToCriticalPaths map[string]*criticalPaths
// tpPairToMatchNum is keyed with topologyPair, and valued with the number of matching pods.
tpPairToMatchNum map[topologyPair]int32
}
// NOTE: When new fields are added/removed or logic is changed, please make sure that
@ -105,9 +155,9 @@ type predicateMetadata struct {
// which should be accounted only by the extenders. This set is synthesized
// from scheduler extender configuration and does not change per pod.
ignoredExtendedResources sets.String
// Similar to the map for pod (anti-)affinity, but imposes additional min matches info
// to describe minimum match number on each topology spread constraint.
topologyPairsPodSpreadMap *topologyPairsPodSpreadMap
// podSpreadCache holds info of the minimum match number on each topology spread constraint,
// and the match number of all valid topology pairs.
podSpreadCache *podSpreadCache
}
// Ensure that predicateMetadata implements algorithm.PredicateMetadata.
@ -154,9 +204,9 @@ func (pfactory *PredicateMetadataFactory) GetMetadata(pod *v1.Pod, nodeNameToInf
if pod == nil {
return nil
}
// existingPodSpreadConstraintsMap represents how existing pods match "pod"
// existingPodSpreadCache represents how existing pods match "pod"
// on its spread constraints
existingPodSpreadConstraintsMap, err := getTPMapMatchingSpreadConstraints(pod, nodeNameToInfoMap)
existingPodSpreadCache, err := getExistingPodSpreadCache(pod, nodeNameToInfoMap)
if err != nil {
klog.Errorf("Error calculating spreadConstraintsMap: %v", err)
return nil
@ -182,7 +232,7 @@ func (pfactory *PredicateMetadataFactory) GetMetadata(pod *v1.Pod, nodeNameToInf
topologyPairsPotentialAffinityPods: incomingPodAffinityMap,
topologyPairsPotentialAntiAffinityPods: incomingPodAntiAffinityMap,
topologyPairsAntiAffinityPodsMap: existingPodAntiAffinityMap,
topologyPairsPodSpreadMap: existingPodSpreadConstraintsMap,
podSpreadCache: existingPodSpreadCache,
}
for predicateName, precomputeFunc := range predicateMetadataProducers {
klog.V(10).Infof("Precompute: %v", predicateName)
@ -191,7 +241,7 @@ func (pfactory *PredicateMetadataFactory) GetMetadata(pod *v1.Pod, nodeNameToInf
return predicateMetadata
}
func getTPMapMatchingSpreadConstraints(pod *v1.Pod, nodeInfoMap map[string]*schedulernodeinfo.NodeInfo) (*topologyPairsPodSpreadMap, error) {
func getExistingPodSpreadCache(pod *v1.Pod, nodeInfoMap map[string]*schedulernodeinfo.NodeInfo) (*podSpreadCache, error) {
// We have feature gating in APIServer to strip the spec
// so don't need to re-check feature gate, just check length of constraints.
constraints := getHardTopologySpreadConstraints(pod)
@ -207,14 +257,15 @@ func getTPMapMatchingSpreadConstraints(pod *v1.Pod, nodeInfoMap map[string]*sche
errCh := schedutil.NewErrorChannel()
var lock sync.Mutex
topologyPairsPodSpreadMap := &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap will be initialized with proper size later.
topologyPairsMaps: newTopologyPairsMaps(),
// TODO(Huang-Wei): It might be possible to use "make(map[topologyPair]*int32)".
// In that case, need to consider how to init each tpPairToCount[pair] in an atomic fashion.
m := podSpreadCache{
tpKeyToCriticalPaths: make(map[string]*criticalPaths, len(constraints)),
tpPairToMatchNum: make(map[topologyPair]int32),
}
appendTopologyPairsMaps := func(toAppend *topologyPairsMaps) {
addTopologyPairMatchNum := func(pair topologyPair, num int32) {
lock.Lock()
topologyPairsPodSpreadMap.appendMaps(toAppend)
m.tpPairToMatchNum[pair] += num
lock.Unlock()
}
@ -237,9 +288,8 @@ func getTPMapMatchingSpreadConstraints(pod *v1.Pod, nodeInfoMap map[string]*sche
if !NodeLabelsMatchSpreadConstraints(node.Labels, constraints) {
return
}
nodeTopologyMaps := newTopologyPairsMaps()
for _, constraint := range constraints {
pairAdded := false
matchTotal := int32(0)
// nodeInfo.Pods() can be empty; or all pods don't fit
for _, existingPod := range nodeInfo.Pods() {
if existingPod.Namespace != pod.Namespace {
@ -251,26 +301,12 @@ func getTPMapMatchingSpreadConstraints(pod *v1.Pod, nodeInfoMap map[string]*sche
return
}
if ok {
// constraint.TopologyKey is already guaranteed to be present
pair := topologyPair{key: constraint.TopologyKey, value: node.Labels[constraint.TopologyKey]}
nodeTopologyMaps.addTopologyPair(pair, existingPod)
pairAdded = true
matchTotal++
}
}
// If needed, append topology pair without entry of pods.
// For example, on node-x, there is no pod matching spread constraints,
// but node-x should be also considered as a match (with match number 0)
// i.e. <node: node-x>: {}
if !pairAdded {
pair := topologyPair{
key: constraint.TopologyKey,
value: node.Labels[constraint.TopologyKey],
}
nodeTopologyMaps.addTopologyPairWithoutPods(pair)
}
pair := topologyPair{key: constraint.TopologyKey, value: node.Labels[constraint.TopologyKey]}
addTopologyPairMatchNum(pair, matchTotal)
}
appendTopologyPairsMaps(nodeTopologyMaps)
}
workqueue.ParallelizeUntil(ctx, 16, len(allNodeNames), processNode)
@ -279,18 +315,15 @@ func getTPMapMatchingSpreadConstraints(pod *v1.Pod, nodeInfoMap map[string]*sche
}
// calculate min match for each topology pair
topologyPairsPodSpreadMap.topologyKeyToMinPodsMap = make(map[string]int32, len(constraints))
for _, constraint := range constraints {
topologyPairsPodSpreadMap.topologyKeyToMinPodsMap[constraint.TopologyKey] = math.MaxInt32
for i := 0; i < len(constraints); i++ {
key := constraints[i].TopologyKey
m.tpKeyToCriticalPaths[key] = newCriticalPaths()
}
for pair, podSet := range topologyPairsPodSpreadMap.topologyPairToPods {
// TODO(Huang-Wei): short circuit unvisited portions of <topologyKey: any value>
// if we already see 0 as min match of that topologyKey.
if l := int32(len(podSet)); l < topologyPairsPodSpreadMap.topologyKeyToMinPodsMap[pair.key] {
topologyPairsPodSpreadMap.topologyKeyToMinPodsMap[pair.key] = l
}
for pair, num := range m.tpPairToMatchNum {
m.tpKeyToCriticalPaths[pair.key].update(pair.value, num)
}
return topologyPairsPodSpreadMap, nil
return &m, nil
}
func getHardTopologySpreadConstraints(pod *v1.Pod) (constraints []v1.TopologySpreadConstraint) {
@ -337,7 +370,9 @@ func newTopologyPairsMaps() *topologyPairsMaps {
func (m *topologyPairsMaps) addTopologyPair(pair topologyPair, pod *v1.Pod) {
podFullName := schedutil.GetPodFullName(pod)
m.addTopologyPairWithoutPods(pair)
if m.topologyPairToPods[pair] == nil {
m.topologyPairToPods[pair] = make(map[*v1.Pod]struct{})
}
m.topologyPairToPods[pair][pod] = struct{}{}
if m.podToTopologyPairs[podFullName] == nil {
m.podToTopologyPairs[podFullName] = make(map[topologyPair]struct{})
@ -345,13 +380,6 @@ func (m *topologyPairsMaps) addTopologyPair(pair topologyPair, pod *v1.Pod) {
m.podToTopologyPairs[podFullName][pair] = struct{}{}
}
// add a topology pair holder if needed
func (m *topologyPairsMaps) addTopologyPairWithoutPods(pair topologyPair) {
if m.topologyPairToPods[pair] == nil {
m.topologyPairToPods[pair] = make(map[*v1.Pod]struct{})
}
}
func (m *topologyPairsMaps) removePod(deletedPod *v1.Pod) {
deletedPodFullName := schedutil.GetPodFullName(deletedPod)
for pair := range m.podToTopologyPairs[deletedPodFullName] {
@ -368,12 +396,8 @@ func (m *topologyPairsMaps) appendMaps(toAppend *topologyPairsMaps) {
return
}
for pair := range toAppend.topologyPairToPods {
if podSet := toAppend.topologyPairToPods[pair]; len(podSet) == 0 {
m.addTopologyPairWithoutPods(pair)
} else {
for pod := range podSet {
m.addTopologyPair(pair, pod)
}
for pod := range toAppend.topologyPairToPods[pair] {
m.addTopologyPair(pair, pod)
}
}
}
@ -384,8 +408,16 @@ func (m *topologyPairsMaps) clone() *topologyPairsMaps {
return copy
}
func (m *topologyPairsPodSpreadMap) addPod(addedPod, preemptorPod *v1.Pod, node *v1.Node) error {
if addedPod.Namespace != preemptorPod.Namespace {
func (c *podSpreadCache) addPod(addedPod, preemptorPod *v1.Pod, node *v1.Node) error {
return c.updatePod(addedPod, preemptorPod, node, 1)
}
func (c *podSpreadCache) removePod(deletedPod, preemptorPod *v1.Pod, node *v1.Node) {
c.updatePod(deletedPod, preemptorPod, node, -1)
}
func (c *podSpreadCache) updatePod(updatedPod, preemptorPod *v1.Pod, node *v1.Node, delta int32) error {
if updatedPod.Namespace != preemptorPod.Namespace || node == nil {
return nil
}
constraints := getHardTopologySpreadConstraints(preemptorPod)
@ -393,98 +425,45 @@ func (m *topologyPairsPodSpreadMap) addPod(addedPod, preemptorPod *v1.Pod, node
return nil
}
// records which topology key(s) needs to be updated
minMatchNeedingUpdate := make(map[string]struct{})
podLabelSet := labels.Set(addedPod.Labels)
podLabelSet := labels.Set(updatedPod.Labels)
for _, constraint := range constraints {
if match, err := PodMatchesSpreadConstraint(podLabelSet, constraint); err != nil {
return err
} else if !match {
continue
}
pair := topologyPair{
key: constraint.TopologyKey,
value: node.Labels[constraint.TopologyKey],
}
// it means current node is one of the critical paths of topologyKeyToMinPodsMap[TopologyKey]
if int32(len(m.topologyPairToPods[pair])) == m.topologyKeyToMinPodsMap[pair.key] {
minMatchNeedingUpdate[pair.key] = struct{}{}
}
m.addTopologyPair(pair, addedPod)
}
// no need to addTopologyPairWithoutPods b/c if a pair without pods must be present,
// it should have already been created earlier in removePod() phase
// In most cases, min match map doesn't need to be updated.
// But it's required to be updated when current node is the ONLY critical path which impacts
// the min match. With that said, in this case min match needs to be updated to min match + 1
if len(minMatchNeedingUpdate) != 0 {
// TODO(Huang-Wei): performance can be optimized.
// A possible solution is to record number of critical paths which co-impact the min match.
tempMinMatchMap := make(map[string]int32, len(minMatchNeedingUpdate))
for key := range minMatchNeedingUpdate {
tempMinMatchMap[key] = math.MaxInt32
}
for pair, podSet := range m.topologyPairToPods {
if _, ok := minMatchNeedingUpdate[pair.key]; !ok {
continue
}
if l := int32(len(podSet)); l < tempMinMatchMap[pair.key] {
tempMinMatchMap[pair.key] = l
}
}
for key, tempMin := range tempMinMatchMap {
if tempMin == m.topologyKeyToMinPodsMap[key]+1 {
m.topologyKeyToMinPodsMap[key] = tempMin
}
}
k, v := constraint.TopologyKey, node.Labels[constraint.TopologyKey]
pair := topologyPair{key: k, value: v}
c.tpPairToMatchNum[pair] = c.tpPairToMatchNum[pair] + delta
c.tpKeyToCriticalPaths[k].update(v, c.tpPairToMatchNum[pair])
}
return nil
}
func (m *topologyPairsPodSpreadMap) removePod(deletedPod *v1.Pod) {
if m == nil || deletedPod == nil {
return
}
deletedPodFullName := schedutil.GetPodFullName(deletedPod)
pairSet, ok := m.podToTopologyPairs[deletedPodFullName]
if !ok {
return
}
topologyPairToPods := m.topologyPairToPods
for pair := range pairSet {
delete(topologyPairToPods[pair], deletedPod)
// if topologyPairToPods[pair] is empty after deletion
// don't clean it up as that topology counts as a match now
// removal of the deletedPod would probably genereate a smaller matching number
// so re-calculate minMatch to a smaller value if possible
if l := int32(len(topologyPairToPods[pair])); l < m.topologyKeyToMinPodsMap[pair.key] {
m.topologyKeyToMinPodsMap[pair.key] = l
}
}
delete(m.podToTopologyPairs, deletedPodFullName)
}
func (m *topologyPairsPodSpreadMap) clone() *topologyPairsPodSpreadMap {
// m could be nil when EvenPodsSpread feature is disabled
if m == nil {
func (c *podSpreadCache) clone() *podSpreadCache {
// c could be nil when EvenPodsSpread feature is disabled
if c == nil {
return nil
}
copy := &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: make(map[string]int32),
topologyPairsMaps: m.topologyPairsMaps.clone(),
copy := podSpreadCache{
tpKeyToCriticalPaths: make(map[string]*criticalPaths),
tpPairToMatchNum: make(map[topologyPair]int32),
}
for key, minMatched := range m.topologyKeyToMinPodsMap {
copy.topologyKeyToMinPodsMap[key] = minMatched
for tpKey, paths := range c.tpKeyToCriticalPaths {
copy.tpKeyToCriticalPaths[tpKey] = &criticalPaths{paths[0], paths[1]}
}
return copy
for tpPair, matchNum := range c.tpPairToMatchNum {
copyPair := topologyPair{key: tpPair.key, value: tpPair.value}
copy.tpPairToMatchNum[copyPair] = matchNum
}
return &copy
}
// RemovePod changes predicateMetadata assuming that the given `deletedPod` is
// deleted from the system.
func (meta *predicateMetadata) RemovePod(deletedPod *v1.Pod) error {
func (meta *predicateMetadata) RemovePod(deletedPod *v1.Pod, node *v1.Node) error {
deletedPodFullName := schedutil.GetPodFullName(deletedPod)
if deletedPodFullName == schedutil.GetPodFullName(meta.pod) {
return fmt.Errorf("deletedPod and meta.pod must not be the same")
@ -494,7 +473,7 @@ func (meta *predicateMetadata) RemovePod(deletedPod *v1.Pod) error {
meta.topologyPairsPotentialAffinityPods.removePod(deletedPod)
meta.topologyPairsPotentialAntiAffinityPods.removePod(deletedPod)
// Delete pod from the pod spread topology maps.
meta.topologyPairsPodSpreadMap.removePod(deletedPod)
meta.podSpreadCache.removePod(deletedPod, meta.pod, node)
// All pods in the serviceAffinityMatchingPodList are in the same namespace.
// So, if the namespace of the first one is not the same as the namespace of the
// deletedPod, we don't need to check the list, as deletedPod isn't in the list.
@ -556,9 +535,9 @@ func (meta *predicateMetadata) AddPod(addedPod *v1.Pod, nodeInfo *schedulernodei
}
}
}
// Update meta.topologyPairsPodSpreadMap if meta.pod has hard spread constraints
// Update meta.podSpreadCache if meta.pod has hard spread constraints
// and addedPod matches that
if err := meta.topologyPairsPodSpreadMap.addPod(addedPod, meta.pod, nodeInfo.Node()); err != nil {
if err := meta.podSpreadCache.addPod(addedPod, meta.pod, nodeInfo.Node()); err != nil {
return err
}
@ -588,7 +567,7 @@ func (meta *predicateMetadata) ShallowCopy() PredicateMetadata {
newPredMeta.topologyPairsPotentialAffinityPods = meta.topologyPairsPotentialAffinityPods.clone()
newPredMeta.topologyPairsPotentialAntiAffinityPods = meta.topologyPairsPotentialAntiAffinityPods.clone()
newPredMeta.topologyPairsAntiAffinityPodsMap = meta.topologyPairsAntiAffinityPodsMap.clone()
newPredMeta.topologyPairsPodSpreadMap = meta.topologyPairsPodSpreadMap.clone()
newPredMeta.podSpreadCache = meta.podSpreadCache.clone()
newPredMeta.serviceAffinityMatchingPodServices = append([]*v1.Service(nil),
meta.serviceAffinityMatchingPodServices...)
newPredMeta.serviceAffinityMatchingPodList = append([]*v1.Pod(nil),

View File

@ -385,7 +385,7 @@ func TestPredicateMetadata_AddRemovePod(t *testing.T) {
// Remove the added pod and from existingPodsMeta1 an make sure it is equal
// to meta generated for existing pods.
existingPodsMeta2, _ := getMeta(st.FakePodLister(test.existingPods))
if err := existingPodsMeta1.RemovePod(test.addedPod); err != nil {
if err := existingPodsMeta1.RemovePod(test.addedPod, nil); err != nil {
t.Errorf("error removing pod from meta: %v", err)
}
if err := predicateMetadataEquivalent(existingPodsMeta1, existingPodsMeta2); err != nil {
@ -512,37 +512,13 @@ func TestPredicateMetadata_ShallowCopy(t *testing.T) {
},
},
},
topologyPairsPodSpreadMap: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"name": 1},
topologyPairsMaps: &topologyPairsMaps{
topologyPairToPods: map[topologyPair]podSet{
{key: "name", value: "nodeA"}: {
&v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "p1", Labels: selector1},
Spec: v1.PodSpec{NodeName: "nodeA"},
}: struct{}{},
},
{key: "name", value: "nodeC"}: {
&v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "p2"},
Spec: v1.PodSpec{
NodeName: "nodeC",
},
}: struct{}{},
&v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "p6", Labels: selector1},
Spec: v1.PodSpec{NodeName: "nodeC"},
}: struct{}{},
},
},
podToTopologyPairs: map[string]topologyPairSet{
"p1_": {
topologyPair{key: "name", value: "nodeA"}: struct{}{},
},
"p2_": {
topologyPair{key: "name", value: "nodeC"}: struct{}{},
},
"p6_": {
topologyPair{key: "name", value: "nodeC"}: struct{}{},
},
},
podSpreadCache: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"name": {{"nodeA", 1}, {"nodeC", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "name", value: "nodeA"}: 1,
{key: "name", value: "nodeC"}: 2,
},
},
serviceAffinityInUse: true,
@ -916,15 +892,12 @@ func TestPodMatchesSpreadConstraint(t *testing.T) {
}
func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
// we need to inject the exact pod pointers to want.topologyPairsMaps.topologyPairToPods
// otherwise, *pod (as key of a map) will always fail in reflect.DeepEqual()
tests := []struct {
name string
pod *v1.Pod
nodes []*v1.Node
existingPods []*v1.Pod
injectPodPointers map[topologyPair][]int
want *topologyPairsPodSpreadMap
name string
pod *v1.Pod
nodes []*v1.Node
existingPods []*v1.Pod
want *podSpreadCache
}{
{
name: "clean cluster with one spreadConstraint",
@ -937,16 +910,13 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
st.MakeNode().Name("node-y").Label("zone", "zone2").Label("node", "node-y").Obj(),
},
injectPodPointers: map[topologyPair][]int{
// denotes no existing pod is matched on this zone pair, but still needed to be
// calculated if incoming pod matches its own spread constraints
{key: "zone", value: "zone1"}: {},
{key: "zone", value: "zone2"}: {},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: make(map[string]topologyPairSet),
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 0}, {"zone2", 0}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 0,
{key: "zone", value: "zone2"}: 0,
},
},
},
@ -968,22 +938,44 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y1").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y2").Node("node-y").Label("foo", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
// denotes existingPods[0,1,2]
{key: "zone", value: "zone1"}: {0, 1, 2},
// denotes existingPods[3,4]
{key: "zone", value: "zone2"}: {3, 4},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 2}, {"zone1", 3}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 2,
},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 2},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-a2_": newPairSet("zone", "zone1"),
"p-b1_": newPairSet("zone", "zone1"),
"p-y1_": newPairSet("zone", "zone2"),
"p-y2_": newPairSet("zone", "zone2"),
},
},
{
name: "normal case with one spreadConstraint, on a 3-zone cluster",
pod: st.MakePod().Name("p").Label("foo", "").SpreadConstraint(
1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj(),
).Obj(),
nodes: []*v1.Node{
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
st.MakeNode().Name("node-y").Label("zone", "zone2").Label("node", "node-y").Obj(),
st.MakeNode().Name("node-o").Label("zone", "zone3").Label("node", "node-o").Obj(),
st.MakeNode().Name("node-p").Label("zone", "zone3").Label("node", "node-p").Obj(),
},
existingPods: []*v1.Pod{
st.MakePod().Name("p-a1").Node("node-a").Label("foo", "").Obj(),
st.MakePod().Name("p-a2").Node("node-a").Label("foo", "").Obj(),
st.MakePod().Name("p-b1").Node("node-b").Label("foo", "").Obj(),
st.MakePod().Name("p-y1").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y2").Node("node-y").Label("foo", "").Obj(),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone3", 0}, {"zone2", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 2,
{key: "zone", value: "zone3"}: 0,
},
},
},
@ -1005,18 +997,13 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y1").Namespace("ns2").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y2").Node("node-y").Label("foo", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 2},
{key: "zone", value: "zone2"}: {4},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-b1_": newPairSet("zone", "zone1"),
"p-y2_": newPairSet("zone", "zone2"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 1}, {"zone1", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 2,
{key: "zone", value: "zone2"}: 1,
},
},
},
@ -1041,26 +1028,18 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y3").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y4").Node("node-y").Label("foo", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1, 2},
{key: "zone", value: "zone2"}: {3, 4, 5, 6},
{key: "node", value: "node-a"}: {0, 1},
{key: "node", value: "node-b"}: {2},
{key: "node", value: "node-x"}: {},
{key: "node", value: "node-y"}: {3, 4, 5, 6},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 3, "node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-a2_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-y1_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y2_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y3_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y4_": newPairSet("zone", "zone2", "node", "node-y"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 3}, {"zone2", 4}},
"node": {{"node-x", 0}, {"node-b", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 4,
{key: "node", value: "node-a"}: 2,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-x"}: 0,
{key: "node", value: "node-y"}: 4,
},
},
},
@ -1086,25 +1065,17 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y3").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y4").Node("node-y").Label("foo", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1, 2},
{key: "zone", value: "zone2"}: {3, 4, 5, 6},
{key: "node", value: "node-a"}: {0, 1},
{key: "node", value: "node-b"}: {2},
{key: "node", value: "node-y"}: {3, 4, 5, 6},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 3, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-a2_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-y1_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y2_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y3_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y4_": newPairSet("zone", "zone2", "node", "node-y"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 3}, {"zone2", 4}},
"node": {{"node-b", 1}, {"node-a", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 4,
{key: "node", value: "node-a"}: 2,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-y"}: 4,
},
},
},
@ -1121,20 +1092,19 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
},
existingPods: []*v1.Pod{
st.MakePod().Name("p-a").Node("node-a").Label("foo", "").Obj(),
st.MakePod().Name("p-b").Node("node-b").Label("bar", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0},
{key: "zone", value: "zone2"}: {},
{key: "node", value: "node-a"}: {},
{key: "node", value: "node-b"}: {},
{key: "node", value: "node-y"}: {},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 0, "node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a_": newPairSet("zone", "zone1"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 0}, {"zone1", 1}},
"node": {{"node-a", 0}, {"node-y", 0}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 0,
{key: "node", value: "node-a"}: 0,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-y"}: 0,
},
},
},
@ -1158,25 +1128,17 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y3").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y4").Node("node-y").Label("foo", "").Label("bar", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1, 2},
{key: "zone", value: "zone2"}: {3, 4, 5, 6},
{key: "node", value: "node-a"}: {1},
{key: "node", value: "node-b"}: {},
{key: "node", value: "node-y"}: {4, 6},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 3, "node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-a2_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1"),
"p-y1_": newPairSet("zone", "zone2"),
"p-y2_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y3_": newPairSet("zone", "zone2"),
"p-y4_": newPairSet("zone", "zone2", "node", "node-y"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 3}, {"zone2", 4}},
"node": {{"node-b", 0}, {"node-a", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 4,
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-b"}: 0,
{key: "node", value: "node-y"}: 2,
},
},
},
@ -1202,61 +1164,46 @@ func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
st.MakePod().Name("p-y3").Node("node-y").Label("foo", "").Obj(),
st.MakePod().Name("p-y4").Node("node-y").Label("foo", "").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1, 2},
{key: "zone", value: "zone2"}: {3, 4, 5, 6},
{key: "node", value: "node-a"}: {0, 1},
{key: "node", value: "node-b"}: {2},
{key: "node", value: "node-y"}: {3, 4, 5, 6},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 3, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-a2_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-y1_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y2_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y3_": newPairSet("zone", "zone2", "node", "node-y"),
"p-y4_": newPairSet("zone", "zone2", "node", "node-y"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 3}, {"zone2", 4}},
"node": {{"node-b", 1}, {"node-a", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 4,
{key: "node", value: "node-a"}: 2,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-y"}: 4,
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tt.want.topologyPairToPods = make(map[topologyPair]podSet)
for pair, indexes := range tt.injectPodPointers {
pSet := make(podSet)
for _, i := range indexes {
pSet[tt.existingPods[i]] = struct{}{}
}
tt.want.topologyPairToPods[pair] = pSet
}
nodeInfoMap := schedulernodeinfo.CreateNodeNameToInfoMap(tt.existingPods, tt.nodes)
if got, _ := getTPMapMatchingSpreadConstraints(tt.pod, nodeInfoMap); !reflect.DeepEqual(got, tt.want) {
t.Errorf("getTPMapMatchingSpreadConstraints() = %v, want %v", got, tt.want)
got, _ := getExistingPodSpreadCache(tt.pod, nodeInfoMap)
got.sortCriticalPaths()
if !reflect.DeepEqual(got, tt.want) {
t.Errorf("getExistingPodSpreadCache() = %v, want %v", *got, *tt.want)
}
})
}
}
func TestPodSpreadMap_addPod(t *testing.T) {
func TestPodSpreadCache_addPod(t *testing.T) {
tests := []struct {
name string
preemptorPod *v1.Pod
addedPod *v1.Pod
existingPods []*v1.Pod
nodeIdx int // denotes which node 'addedPod' belongs to
nodes []*v1.Node
injectPodPointers map[topologyPair][]int // non-negative index refers to existingPods[i], negative index refers to addedPod
want *topologyPairsPodSpreadMap
name string
preemptor *v1.Pod
addedPod *v1.Pod
existingPods []*v1.Pod
nodeIdx int // denotes which node 'addedPod' belongs to
nodes []*v1.Node
want *podSpreadCache
}{
{
name: "node a and b both impact current min match",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-a1").Node("node-a").Label("foo", "").Obj(),
@ -1266,24 +1213,19 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-b"}: {},
},
want: &topologyPairsPodSpreadMap{
// min match map shouldn't be changed b/c node-b is still on the critical path
// determining min match
topologyKeyToMinPodsMap: map[string]int32{"node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("node", "node-a"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"node": {{"node-b", 0}, {"node-a", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-b"}: 0,
},
},
},
{
name: "only node a impacts current min match",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-a1").Node("node-a").Label("foo", "").Obj(),
@ -1295,24 +1237,19 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-b"}: {0},
},
want: &topologyPairsPodSpreadMap{
// min match should be changed from 0 to 1
topologyKeyToMinPodsMap: map[string]int32{"node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("node", "node-a"),
"p-b1_": newPairSet("node", "node-b"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"node": {{"node-a", 1}, {"node-b", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-b"}: 1,
},
},
},
{
name: "add a pod with mis-matched namespace doesn't change topologyKeyToMinPodsMap",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-a1").Namespace("ns1").Node("node-a").Label("foo", "").Obj(),
@ -1324,24 +1261,19 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "node", value: "node-a"}: {},
{key: "node", value: "node-b"}: {0},
},
want: &topologyPairsPodSpreadMap{
// min match remains the same
topologyKeyToMinPodsMap: map[string]int32{"node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
// "p-a1_": newPairSet("node", "node-a") shouldn't exist
"p-b1_": newPairSet("node", "node-b"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"node": {{"node-a", 0}, {"node-b", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "node", value: "node-a"}: 0,
{key: "node", value: "node-b"}: 1,
},
},
},
{
name: "add pod on non-critical node won't trigger re-calculation",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-b2").Node("node-b").Label("foo", "").Obj(),
@ -1353,23 +1285,19 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "node", value: "node-a"}: {},
{key: "node", value: "node-b"}: {-1, 0},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-b1_": newPairSet("node", "node-b"),
"p-b2_": newPairSet("node", "node-b"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"node": {{"node-a", 0}, {"node-b", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "node", value: "node-a"}: 0,
{key: "node", value: "node-b"}: 2,
},
},
},
{
name: "node a and x both impact topologyKeyToMinPodsMap on zone and node",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
@ -1380,24 +1308,22 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {-1},
{key: "zone", value: "zone2"}: {},
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-x"}: {},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 0, "node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 0}, {"zone1", 1}},
"node": {{"node-x", 0}, {"node-a", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 0,
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-x"}: 0,
},
},
},
{
name: "only node a impacts topologyKeyToMinPodsMap on zone and node",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
@ -1410,25 +1336,22 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-a").Label("zone", "zone1").Label("node", "node-a").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {-1},
{key: "zone", value: "zone2"}: {0},
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-x"}: {0},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 1, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-x1_": newPairSet("zone", "zone2", "node", "node-x"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 1}, {"zone2", 1}},
"node": {{"node-a", 1}, {"node-x", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 1,
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-x"}: 1,
},
},
},
{
name: "node a impacts topologyKeyToMinPodsMap on node, node x impacts topologyKeyToMinPodsMap on zone",
preemptorPod: st.MakePod().Name("p").Label("foo", "").
preemptor: st.MakePod().Name("p").Label("foo", "").
SpreadConstraint(1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
Obj(),
@ -1444,35 +1367,30 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {-1, 0, 1},
{key: "zone", value: "zone2"}: {2},
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-b"}: {0, 1},
{key: "node", value: "node-x"}: {2},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 1, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-b2_": newPairSet("zone", "zone1", "node", "node-b"),
"p-x1_": newPairSet("zone", "zone2", "node", "node-x"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 1}, {"zone1", 3}},
"node": {{"node-a", 1}, {"node-x", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 1,
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-b"}: 2,
{key: "node", value: "node-x"}: 1,
},
},
},
{
name: "constraints hold different labelSelectors, node a impacts topologyKeyToMinPodsMap on node",
preemptorPod: st.MakePod().Name("p").Label("foo", "").Label("bar", "").
name: "constraints hold different labelSelectors, node a impacts topologyKeyToMinPodsMap on zone",
preemptor: st.MakePod().Name("p").Label("foo", "").Label("bar", "").
SpreadConstraint(1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("bar").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-a1").Node("node-a").Label("foo", "").Obj(),
existingPods: []*v1.Pod{
st.MakePod().Name("p-b1").Node("node-b").Label("foo", "").Label("bar", "").Obj(),
st.MakePod().Name("p-x1").Node("node-x").Label("foo", "").Obj(),
st.MakePod().Name("p-x1").Node("node-x").Label("foo", "").Label("bar", "").Obj(),
st.MakePod().Name("p-x2").Node("node-x").Label("bar", "").Obj(),
},
nodeIdx: 0,
@ -1481,35 +1399,30 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {-1, 0},
{key: "zone", value: "zone2"}: {1},
{key: "node", value: "node-a"}: {},
{key: "node", value: "node-b"}: {0},
{key: "node", value: "node-x"}: {2},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 1, "node": 0},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-x1_": newPairSet("zone", "zone2"),
"p-x2_": newPairSet("node", "node-x"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 1}, {"zone1", 2}},
"node": {{"node-a", 0}, {"node-b", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 2,
{key: "zone", value: "zone2"}: 1,
{key: "node", value: "node-a"}: 0,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-x"}: 2,
},
},
},
{
name: "constraints hold different labelSelectors, node a impacts topologyKeyToMinPodsMap on both zone and node",
preemptorPod: st.MakePod().Name("p").Label("foo", "").Label("bar", "").
preemptor: st.MakePod().Name("p").Label("foo", "").Label("bar", "").
SpreadConstraint(1, "zone", hardSpread, st.MakeLabelSelector().Exists("foo").Obj()).
SpreadConstraint(1, "node", hardSpread, st.MakeLabelSelector().Exists("bar").Obj()).
Obj(),
addedPod: st.MakePod().Name("p-a1").Node("node-a").Label("foo", "").Label("bar", "").Obj(),
existingPods: []*v1.Pod{
st.MakePod().Name("p-b1").Node("node-b").Label("bar", "").Obj(),
st.MakePod().Name("p-x1").Node("node-x").Label("foo", "").Obj(),
st.MakePod().Name("p-x1").Node("node-x").Label("foo", "").Label("bar", "").Obj(),
st.MakePod().Name("p-x2").Node("node-x").Label("bar", "").Obj(),
},
nodeIdx: 0,
@ -1518,62 +1431,45 @@ func TestPodSpreadMap_addPod(t *testing.T) {
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {-1},
{key: "zone", value: "zone2"}: {1},
{key: "node", value: "node-a"}: {-1},
{key: "node", value: "node-b"}: {0},
{key: "node", value: "node-x"}: {2},
},
want: &topologyPairsPodSpreadMap{
topologyKeyToMinPodsMap: map[string]int32{"zone": 1, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("node", "node-b"),
"p-x1_": newPairSet("zone", "zone2"),
"p-x2_": newPairSet("node", "node-x"),
},
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 1}, {"zone2", 1}},
"node": {{"node-a", 1}, {"node-b", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 1,
{key: "node", value: "node-a"}: 1,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-x"}: 2,
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tt.want.topologyPairToPods = make(map[topologyPair]podSet)
for pair, indexes := range tt.injectPodPointers {
pSet := make(podSet)
for _, i := range indexes {
if i >= 0 {
pSet[tt.existingPods[i]] = struct{}{}
} else {
pSet[tt.addedPod] = struct{}{}
}
}
tt.want.topologyPairToPods[pair] = pSet
}
nodeInfoMap := schedulernodeinfo.CreateNodeNameToInfoMap(tt.existingPods, tt.nodes)
podSpreadMap, _ := getTPMapMatchingSpreadConstraints(tt.preemptorPod, nodeInfoMap)
podSpreadCache, _ := getExistingPodSpreadCache(tt.preemptor, nodeInfoMap)
podSpreadMap.addPod(tt.addedPod, tt.preemptorPod, tt.nodes[tt.nodeIdx])
if !reflect.DeepEqual(podSpreadMap, tt.want) {
t.Errorf("podSpreadMap#addPod() = %v, want %v", podSpreadMap, tt.want)
podSpreadCache.addPod(tt.addedPod, tt.preemptor, tt.nodes[tt.nodeIdx])
podSpreadCache.sortCriticalPaths()
if !reflect.DeepEqual(podSpreadCache, tt.want) {
t.Errorf("podSpreadCache#addPod() = %v, want %v", podSpreadCache, tt.want)
}
})
}
}
func TestPodSpreadMap_removePod(t *testing.T) {
func TestPodSpreadCache_removePod(t *testing.T) {
tests := []struct {
name string
preemptor *v1.Pod // preemptor pod
nodes []*v1.Node
existingPods []*v1.Pod
deletedPodIdx int // need to reuse *Pod of existingPods[i]
deletedPod *v1.Pod // if deletedPodIdx is invalid, this field is bypassed
injectPodPointers map[topologyPair][]int
want *topologyPairsPodSpreadMap
name string
preemptor *v1.Pod // preemptor pod
nodes []*v1.Node
existingPods []*v1.Pod
deletedPodIdx int // need to reuse *Pod of existingPods[i]
deletedPod *v1.Pod // this field is used only when deletedPodIdx is -1
nodeIdx int // denotes which node "deletedPod" belongs to
want *podSpreadCache
}{
{
// A high priority pod may not be scheduled due to node taints or resource shortage.
@ -1593,18 +1489,14 @@ func TestPodSpreadMap_removePod(t *testing.T) {
st.MakePod().Name("p-x1").Node("node-x").Label("foo", "").Obj(),
},
deletedPodIdx: 0, // remove pod "p-a1"
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {1},
{key: "zone", value: "zone2"}: {2},
},
want: &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap actually doesn't change
topologyKeyToMinPodsMap: map[string]int32{"zone": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-b1_": newPairSet("zone", "zone1"),
"p-x1_": newPairSet("zone", "zone2"),
},
nodeIdx: 0, // node-a
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 1}, {"zone2", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 1,
},
},
},
@ -1626,20 +1518,14 @@ func TestPodSpreadMap_removePod(t *testing.T) {
st.MakePod().Name("p-y1").Node("node-y").Label("foo", "").Obj(),
},
deletedPodIdx: 0, // remove pod "p-a1"
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {1},
{key: "zone", value: "zone2"}: {2, 3},
},
want: &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap is expected to be re-calculated from {"zone": 2}
// to {"zone": 1}
topologyKeyToMinPodsMap: map[string]int32{"zone": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-b1_": newPairSet("zone", "zone1"),
"p-x1_": newPairSet("zone", "zone2"),
"p-y1_": newPairSet("zone", "zone2"),
},
nodeIdx: 0, // node-a
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 1}, {"zone2", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 1,
{key: "zone", value: "zone2"}: 2,
},
},
},
@ -1662,20 +1548,14 @@ func TestPodSpreadMap_removePod(t *testing.T) {
st.MakePod().Name("p-y1").Node("node-y").Label("foo", "").Obj(),
},
deletedPodIdx: 0, // remove pod "p-a0"
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {1, 2},
{key: "zone", value: "zone2"}: {3, 4},
},
want: &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap is unchanged
topologyKeyToMinPodsMap: map[string]int32{"zone": 2},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-b1_": newPairSet("zone", "zone1"),
"p-x1_": newPairSet("zone", "zone2"),
"p-y1_": newPairSet("zone", "zone2"),
},
nodeIdx: 0, // node-a
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 2}, {"zone2", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 2,
{key: "zone", value: "zone2"}: 2,
},
},
},
@ -1698,20 +1578,14 @@ func TestPodSpreadMap_removePod(t *testing.T) {
},
deletedPodIdx: -1,
deletedPod: st.MakePod().Name("p-a0").Node("node-a").Label("bar", "").Obj(),
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1},
{key: "zone", value: "zone2"}: {2, 3},
},
want: &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap is unchanged
topologyKeyToMinPodsMap: map[string]int32{"zone": 2},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1"),
"p-b1_": newPairSet("zone", "zone1"),
"p-x1_": newPairSet("zone", "zone2"),
"p-y1_": newPairSet("zone", "zone2"),
},
nodeIdx: 0, // node-a
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone1", 2}, {"zone2", 2}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 2,
{key: "zone", value: "zone2"}: 2,
},
},
},
@ -1734,41 +1608,26 @@ func TestPodSpreadMap_removePod(t *testing.T) {
st.MakePod().Name("p-x2").Node("node-x").Label("foo", "").Obj(),
},
deletedPodIdx: 3, // remove pod "p-x1"
injectPodPointers: map[topologyPair][]int{
{key: "zone", value: "zone1"}: {0, 1, 2},
{key: "zone", value: "zone2"}: {4},
{key: "node", value: "node-a"}: {0, 1},
{key: "node", value: "node-b"}: {2},
{key: "node", value: "node-x"}: {4},
},
want: &topologyPairsPodSpreadMap{
// topologyKeyToMinPodsMap is expected to be re-calculated from {"zone": 2, "node": 1}
// to {"zone": 1, "node": 1}
topologyKeyToMinPodsMap: map[string]int32{"zone": 1, "node": 1},
topologyPairsMaps: &topologyPairsMaps{
podToTopologyPairs: map[string]topologyPairSet{
"p-a1_": newPairSet("zone", "zone1", "node", "node-a"),
"p-a2_": newPairSet("zone", "zone1", "node", "node-a"),
"p-b1_": newPairSet("zone", "zone1", "node", "node-b"),
"p-x2_": newPairSet("zone", "zone2", "node", "node-x"),
},
nodeIdx: 2, // node-x
want: &podSpreadCache{
tpKeyToCriticalPaths: map[string]*criticalPaths{
"zone": {{"zone2", 1}, {"zone1", 3}},
"node": {{"node-b", 1}, {"node-x", 1}},
},
tpPairToMatchNum: map[topologyPair]int32{
{key: "zone", value: "zone1"}: 3,
{key: "zone", value: "zone2"}: 1,
{key: "node", value: "node-a"}: 2,
{key: "node", value: "node-b"}: 1,
{key: "node", value: "node-x"}: 1,
},
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tt.want.topologyPairToPods = make(map[topologyPair]podSet)
for pair, indexes := range tt.injectPodPointers {
pSet := make(podSet)
for _, i := range indexes {
pSet[tt.existingPods[i]] = struct{}{}
}
tt.want.topologyPairToPods[pair] = pSet
}
nodeInfoMap := schedulernodeinfo.CreateNodeNameToInfoMap(tt.existingPods, tt.nodes)
podSpreadMap, _ := getTPMapMatchingSpreadConstraints(tt.preemptor, nodeInfoMap)
podSpreadCache, _ := getExistingPodSpreadCache(tt.preemptor, nodeInfoMap)
var deletedPod *v1.Pod
if tt.deletedPodIdx < len(tt.existingPods) && tt.deletedPodIdx >= 0 {
@ -1776,9 +1635,10 @@ func TestPodSpreadMap_removePod(t *testing.T) {
} else {
deletedPod = tt.deletedPod
}
podSpreadMap.removePod(deletedPod)
if !reflect.DeepEqual(podSpreadMap, tt.want) {
t.Errorf("podSpreadMap#removePod() = %v, want %v", podSpreadMap, tt.want)
podSpreadCache.removePod(deletedPod, tt.preemptor, tt.nodes[tt.nodeIdx])
podSpreadCache.sortCriticalPaths()
if !reflect.DeepEqual(podSpreadCache, tt.want) {
t.Errorf("podSpreadCache#removePod() = %v, want %v", podSpreadCache, tt.want)
}
})
}
@ -1827,7 +1687,7 @@ func BenchmarkTestGetTPMapMatchingSpreadConstraints(b *testing.B) {
nodeNameToInfo := schedulernodeinfo.CreateNodeNameToInfoMap(existingPods, allNodes)
b.ResetTimer()
for i := 0; i < b.N; i++ {
getTPMapMatchingSpreadConstraints(tt.pod, nodeNameToInfo)
getExistingPodSpreadCache(tt.pod, nodeNameToInfo)
}
})
}
@ -1846,3 +1706,14 @@ func newPairSet(kv ...string) topologyPairSet {
}
return result
}
// sortCriticalPaths is only served for testing purpose.
func (c *podSpreadCache) sortCriticalPaths() {
for _, paths := range c.tpKeyToCriticalPaths {
// If two paths both hold minimum matching number, and topologyValue is unordered.
if paths[0].matchNum == paths[1].matchNum && paths[0].topologyValue > paths[1].topologyValue {
// Swap topologyValue to make them sorted alphabetically.
paths[0].topologyValue, paths[1].topologyValue = paths[1].topologyValue, paths[0].topologyValue
}
}
}

View File

@ -1796,15 +1796,15 @@ func EvenPodsSpreadPredicate(pod *v1.Pod, meta PredicateMetadata, nodeInfo *sche
return true, nil, nil
}
var topologyPairsPodSpreadMap *topologyPairsPodSpreadMap
var podSpreadCache *podSpreadCache
if predicateMeta, ok := meta.(*predicateMetadata); ok {
topologyPairsPodSpreadMap = predicateMeta.topologyPairsPodSpreadMap
podSpreadCache = predicateMeta.podSpreadCache
} else { // We don't have precomputed metadata. We have to follow a slow path to check spread constraints.
// TODO(Huang-Wei): get it implemented
// TODO(autoscaler): get it implemented
return false, nil, errors.New("metadata not pre-computed for EvenPodsSpreadPredicate")
}
if topologyPairsPodSpreadMap == nil || len(topologyPairsPodSpreadMap.topologyKeyToMinPodsMap) == 0 {
if podSpreadCache == nil || len(podSpreadCache.tpPairToMatchNum) == 0 {
return true, nil, nil
}
@ -1821,25 +1821,24 @@ func EvenPodsSpreadPredicate(pod *v1.Pod, meta PredicateMetadata, nodeInfo *sche
if err != nil {
return false, nil, err
}
selfMatchNum := 0
selfMatchNum := int32(0)
if selfMatch {
selfMatchNum = 1
}
pair := topologyPair{key: tpKey, value: tpVal}
minMatchNum, ok := topologyPairsPodSpreadMap.topologyKeyToMinPodsMap[tpKey]
paths, ok := podSpreadCache.tpKeyToCriticalPaths[tpKey]
if !ok {
// error which should not happen
klog.Errorf("internal error: get minMatchNum from key %q of %#v", tpKey, topologyPairsPodSpreadMap.topologyKeyToMinPodsMap)
klog.Errorf("internal error: get paths from key %q of %#v", tpKey, podSpreadCache.tpKeyToCriticalPaths)
continue
}
// judging criteria:
// 'existing matching num' + 'if self-match (1 or 0)' - 'global min matching num' <= 'maxSkew'
matchNum := len(topologyPairsPodSpreadMap.topologyPairToPods[pair])
// cast to int to avoid potential overflow.
skew := matchNum + selfMatchNum - int(minMatchNum)
if skew > int(constraint.MaxSkew) {
minMatchNum := paths[0].matchNum
matchNum := podSpreadCache.tpPairToMatchNum[pair]
skew := matchNum + selfMatchNum - minMatchNum
if skew > constraint.MaxSkew {
klog.V(5).Infof("node '%s' failed spreadConstraint[%s]: matchNum(%d) + selfMatchNum(%d) - minMatchNum(%d) > maxSkew(%d)", node.Name, tpKey, matchNum, selfMatchNum, minMatchNum, constraint.MaxSkew)
return false, []PredicateFailureReason{ErrTopologySpreadConstraintsNotMatch}, nil
}

View File

@ -38,15 +38,15 @@ type topologyPair struct {
type topologySpreadConstraintsMap struct {
// nodeNameToPodCounts is keyed with node name, and valued with the number of matching pods.
nodeNameToPodCounts map[string]int64
nodeNameToPodCounts map[string]int32
// topologyPairToPodCounts is keyed with topologyPair, and valued with the number of matching pods.
topologyPairToPodCounts map[topologyPair]*int64
topologyPairToPodCounts map[topologyPair]*int32
}
func newTopologySpreadConstraintsMap() *topologySpreadConstraintsMap {
return &topologySpreadConstraintsMap{
nodeNameToPodCounts: make(map[string]int64),
topologyPairToPodCounts: make(map[topologyPair]*int64),
nodeNameToPodCounts: make(map[string]int32),
topologyPairToPodCounts: make(map[topologyPair]*int32),
}
}
@ -54,7 +54,7 @@ func newTopologySpreadConstraintsMap() *topologySpreadConstraintsMap {
// This function iterates <nodes> to filter out the nodes which don't have required topologyKey(s),
// and initialize two maps:
// 1) t.topologyPairToPodCounts: keyed with both eligible topology pair and node names.
// 2) t.nodeNameToPodCounts: keyed with node name, and valued with a *int64 pointer for eligible node only.
// 2) t.nodeNameToPodCounts: keyed with node name, and valued with a *int32 pointer for eligible node only.
func (t *topologySpreadConstraintsMap) initialize(pod *v1.Pod, nodes []*v1.Node) {
constraints := getSoftTopologySpreadConstraints(pod)
for _, node := range nodes {
@ -64,7 +64,7 @@ func (t *topologySpreadConstraintsMap) initialize(pod *v1.Pod, nodes []*v1.Node)
for _, constraint := range constraints {
pair := topologyPair{key: constraint.TopologyKey, value: node.Labels[constraint.TopologyKey]}
if t.topologyPairToPodCounts[pair] == nil {
t.topologyPairToPodCounts[pair] = new(int64)
t.topologyPairToPodCounts[pair] = new(int32)
}
}
t.nodeNameToPodCounts[node.Name] = 0
@ -122,7 +122,7 @@ func CalculateEvenPodsSpreadPriority(pod *v1.Pod, nodeNameToInfo map[string]*sch
}
// <matchSum> indicates how many pods (on current node) match the <constraint>.
matchSum := int64(0)
matchSum := int32(0)
for _, existingPod := range nodeInfo.Pods() {
match, err := predicates.PodMatchesSpreadConstraint(existingPod.Labels, constraint)
if err != nil {
@ -133,7 +133,7 @@ func CalculateEvenPodsSpreadPriority(pod *v1.Pod, nodeNameToInfo map[string]*sch
matchSum++
}
}
atomic.AddInt64(t.topologyPairToPodCounts[pair], matchSum)
atomic.AddInt32(t.topologyPairToPodCounts[pair], matchSum)
}
}
workqueue.ParallelizeUntil(ctx, 16, len(allNodeNames), processAllNode)
@ -141,9 +141,9 @@ func CalculateEvenPodsSpreadPriority(pod *v1.Pod, nodeNameToInfo map[string]*sch
return nil, err
}
var minCount int64 = math.MaxInt64
var minCount int32 = math.MaxInt32
// <total> sums up the number of matching pods on each qualified topology pair
var total int64
var total int32
for _, node := range nodes {
if _, ok := t.nodeNameToPodCounts[node.Name]; !ok {
continue

View File

@ -31,8 +31,8 @@ func Test_topologySpreadConstraintsMap_initialize(t *testing.T) {
name string
pod *v1.Pod
nodes []*v1.Node
wantNodeNameMap map[string]int64
wantTopologyPairMap map[topologyPair]*int64
wantNodeNameMap map[string]int32
wantTopologyPairMap map[topologyPair]*int32
}{
{
name: "normal case",
@ -45,17 +45,17 @@ func Test_topologySpreadConstraintsMap_initialize(t *testing.T) {
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("zone", "zone2").Label("node", "node-x").Obj(),
},
wantNodeNameMap: map[string]int64{
wantNodeNameMap: map[string]int32{
"node-a": 0,
"node-b": 0,
"node-x": 0,
},
wantTopologyPairMap: map[topologyPair]*int64{
{key: "zone", value: "zone1"}: new(int64),
{key: "zone", value: "zone2"}: new(int64),
{key: "node", value: "node-a"}: new(int64),
{key: "node", value: "node-b"}: new(int64),
{key: "node", value: "node-x"}: new(int64),
wantTopologyPairMap: map[topologyPair]*int32{
{key: "zone", value: "zone1"}: new(int32),
{key: "zone", value: "zone2"}: new(int32),
{key: "node", value: "node-a"}: new(int32),
{key: "node", value: "node-b"}: new(int32),
{key: "node", value: "node-x"}: new(int32),
},
},
{
@ -69,14 +69,14 @@ func Test_topologySpreadConstraintsMap_initialize(t *testing.T) {
st.MakeNode().Name("node-b").Label("zone", "zone1").Label("node", "node-b").Obj(),
st.MakeNode().Name("node-x").Label("node", "node-x").Obj(),
},
wantNodeNameMap: map[string]int64{
wantNodeNameMap: map[string]int32{
"node-a": 0,
"node-b": 0,
},
wantTopologyPairMap: map[topologyPair]*int64{
{key: "zone", value: "zone1"}: new(int64),
{key: "node", value: "node-a"}: new(int64),
{key: "node", value: "node-b"}: new(int64),
wantTopologyPairMap: map[topologyPair]*int32{
{key: "zone", value: "zone1"}: new(int32),
{key: "node", value: "node-a"}: new(int32),
{key: "node", value: "node-b"}: new(int32),
},
},
}

View File

@ -1107,7 +1107,7 @@ func selectVictimsOnNode(
removePod := func(rp *v1.Pod) {
nodeInfoCopy.RemovePod(rp)
if meta != nil {
meta.RemovePod(rp)
meta.RemovePod(rp, nodeInfoCopy.Node())
}
}
addPod := func(ap *v1.Pod) {