move inter pod affinity predicate logic to its Filter plugin

This commit is contained in:
Abdullah Gharaibeh 2019-12-19 20:35:57 -05:00
parent 641d0290e4
commit 429448cb40
14 changed files with 1686 additions and 1713 deletions

View File

@ -20,8 +20,6 @@ go_library(
"//pkg/apis/core/v1/helper:go_default_library",
"//pkg/features:go_default_library",
"//pkg/scheduler/algorithm:go_default_library",
"//pkg/scheduler/algorithm/priorities/util:go_default_library",
"//pkg/scheduler/listers:go_default_library",
"//pkg/scheduler/nodeinfo:go_default_library",
"//pkg/scheduler/util:go_default_library",
"//pkg/scheduler/volumebinder:go_default_library",

View File

@ -24,29 +24,15 @@ import (
v1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/labels"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/client-go/util/workqueue"
"k8s.io/klog"
priorityutil "k8s.io/kubernetes/pkg/scheduler/algorithm/priorities/util"
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
schedutil "k8s.io/kubernetes/pkg/scheduler/util"
)
// Metadata interface represents anything that can access a predicate metadata.
// DEPRECATED.
type Metadata interface{}
// AntiAffinityTerm's topology key value used in predicate metadata
type topologyPair struct {
key string
value string
}
// TODO(Huang-Wei): It might be possible to use "make(map[topologyPair]*int64)" so that
// we can do atomic additions instead of using a global mutext, however we need to consider
// how to init each topologyToMatchedTermCount.
type topologyToMatchedTermCount map[topologyPair]int64
type criticalPath struct {
// topologyValue denotes the topology value mapping to topology key.
topologyValue string
@ -96,6 +82,11 @@ func (paths *criticalPaths) update(tpVal string, num int32) {
}
}
type topologyPair struct {
key string
value string
}
// PodTopologySpreadMetadata combines tpKeyToCriticalPaths and tpPairToMatchNum
// to represent:
// (1) critical paths where the least pods are matched on each spread constraint.
@ -120,129 +111,6 @@ type topologySpreadConstraint struct {
selector labels.Selector
}
// PodAffinityMetadata pre-computed state for inter-pod affinity predicate.
type PodAffinityMetadata struct {
// A map of topology pairs to the number of existing pods that has anti-affinity terms that match the "pod".
topologyToMatchedExistingAntiAffinityTerms topologyToMatchedTermCount
// A map of topology pairs to the number of existing pods that match the affinity terms of the "pod".
topologyToMatchedAffinityTerms topologyToMatchedTermCount
// A map of topology pairs to the number of existing pods that match the anti-affinity terms of the "pod".
topologyToMatchedAntiAffinityTerms topologyToMatchedTermCount
}
// updateWithAffinityTerms updates the topologyToMatchedTermCount map with the specified value
// for each affinity term if "targetPod" matches ALL terms.
func (m topologyToMatchedTermCount) updateWithAffinityTerms(targetPod *v1.Pod, targetPodNode *v1.Node, affinityTerms []*affinityTermProperties, value int64) {
if podMatchesAllAffinityTermProperties(targetPod, affinityTerms) {
for _, t := range affinityTerms {
if topologyValue, ok := targetPodNode.Labels[t.topologyKey]; ok {
pair := topologyPair{key: t.topologyKey, value: topologyValue}
m[pair] += value
// value could be a negative value, hence we delete the entry if
// the entry is down to zero.
if m[pair] == 0 {
delete(m, pair)
}
}
}
}
}
// updateAntiAffinityTerms updates the topologyToMatchedTermCount map with the specified value
// for each anti-affinity term matched the target pod.
func (m topologyToMatchedTermCount) updateWithAntiAffinityTerms(targetPod *v1.Pod, targetPodNode *v1.Node, antiAffinityTerms []*affinityTermProperties, value int64) {
// Check anti-affinity properties.
for _, a := range antiAffinityTerms {
if priorityutil.PodMatchesTermsNamespaceAndSelector(targetPod, a.namespaces, a.selector) {
if topologyValue, ok := targetPodNode.Labels[a.topologyKey]; ok {
pair := topologyPair{key: a.topologyKey, value: topologyValue}
m[pair] += value
// value could be a negative value, hence we delete the entry if
// the entry is down to zero.
if m[pair] == 0 {
delete(m, pair)
}
}
}
}
}
// UpdateWithPod updates the metadata counters with the (anti)affinity matches for the given pod.
func (m *PodAffinityMetadata) UpdateWithPod(updatedPod, pod *v1.Pod, node *v1.Node, multiplier int64) error {
if m == nil {
return nil
}
// Update matching existing anti-affinity terms.
updatedPodAffinity := updatedPod.Spec.Affinity
if updatedPodAffinity != nil && updatedPodAffinity.PodAntiAffinity != nil {
antiAffinityProperties, err := getAffinityTermProperties(pod, GetPodAntiAffinityTerms(updatedPodAffinity.PodAntiAffinity))
if err != nil {
klog.Errorf("error in getting anti-affinity properties of Pod %v", updatedPod.Name)
return err
}
m.topologyToMatchedExistingAntiAffinityTerms.updateWithAntiAffinityTerms(pod, node, antiAffinityProperties, multiplier)
}
// Update matching incoming pod (anti)affinity terms.
affinity := pod.Spec.Affinity
podNodeName := updatedPod.Spec.NodeName
if affinity != nil && len(podNodeName) > 0 {
if affinity.PodAffinity == nil {
affinityProperties, err := getAffinityTermProperties(pod, GetPodAffinityTerms(affinity.PodAffinity))
if err != nil {
klog.Errorf("error in getting affinity properties of Pod %v", pod.Name)
return err
}
m.topologyToMatchedAffinityTerms.updateWithAffinityTerms(updatedPod, node, affinityProperties, multiplier)
}
if affinity.PodAntiAffinity != nil {
antiAffinityProperties, err := getAffinityTermProperties(pod, GetPodAntiAffinityTerms(affinity.PodAntiAffinity))
if err != nil {
klog.Errorf("error in getting anti-affinity properties of Pod %v", pod.Name)
return err
}
m.topologyToMatchedAntiAffinityTerms.updateWithAntiAffinityTerms(updatedPod, node, antiAffinityProperties, multiplier)
}
}
return nil
}
// Clone makes a deep copy of PodAffinityMetadata.
func (m *PodAffinityMetadata) Clone() *PodAffinityMetadata {
if m == nil {
return nil
}
copy := PodAffinityMetadata{}
copy.topologyToMatchedAffinityTerms = m.topologyToMatchedAffinityTerms.clone()
copy.topologyToMatchedAntiAffinityTerms = m.topologyToMatchedAntiAffinityTerms.clone()
copy.topologyToMatchedExistingAntiAffinityTerms = m.topologyToMatchedExistingAntiAffinityTerms.clone()
return &copy
}
// GetPodAffinityMetadata computes inter-pod affinity metadata.
func GetPodAffinityMetadata(pod *v1.Pod, allNodes []*schedulernodeinfo.NodeInfo, havePodsWithAffinityNodes []*schedulernodeinfo.NodeInfo) (*PodAffinityMetadata, error) {
// existingPodAntiAffinityMap will be used later for efficient check on existing pods' anti-affinity
existingPodAntiAffinityMap, err := getTPMapMatchingExistingAntiAffinity(pod, havePodsWithAffinityNodes)
if err != nil {
return nil, err
}
// incomingPodAffinityMap will be used later for efficient check on incoming pod's affinity
// incomingPodAntiAffinityMap will be used later for efficient check on incoming pod's anti-affinity
incomingPodAffinityMap, incomingPodAntiAffinityMap, err := getTPMapMatchingIncomingAffinityAntiAffinity(pod, allNodes)
if err != nil {
return nil, err
}
return &PodAffinityMetadata{
topologyToMatchedAffinityTerms: incomingPodAffinityMap,
topologyToMatchedAntiAffinityTerms: incomingPodAntiAffinityMap,
topologyToMatchedExistingAntiAffinityTerms: existingPodAntiAffinityMap,
}, nil
}
// GetPodTopologySpreadMetadata computes pod topology spread metadata.
func GetPodTopologySpreadMetadata(pod *v1.Pod, allNodes []*schedulernodeinfo.NodeInfo) (*PodTopologySpreadMetadata, error) {
// We have feature gating in APIServer to strip the spec
@ -344,18 +212,6 @@ func NodeLabelsMatchSpreadConstraints(nodeLabels map[string]string, constraints
return true
}
func (m topologyToMatchedTermCount) appendMaps(toAppend topologyToMatchedTermCount) {
for pair := range toAppend {
m[pair] += toAppend[pair]
}
}
func (m topologyToMatchedTermCount) clone() topologyToMatchedTermCount {
copy := make(topologyToMatchedTermCount, len(m))
copy.appendMaps(m)
return copy
}
// AddPod updates the metadata with addedPod.
func (m *PodTopologySpreadMetadata) AddPod(addedPod, preemptorPod *v1.Pod, node *v1.Node) {
m.updateWithPod(addedPod, preemptorPod, node, 1)
@ -409,163 +265,3 @@ func (m *PodTopologySpreadMetadata) Clone() *PodTopologySpreadMetadata {
}
return &cp
}
// A processed version of v1.PodAffinityTerm.
type affinityTermProperties struct {
namespaces sets.String
selector labels.Selector
topologyKey string
}
// getAffinityTermProperties receives a Pod and affinity terms and returns the namespaces and
// selectors of the terms.
func getAffinityTermProperties(pod *v1.Pod, terms []v1.PodAffinityTerm) (properties []*affinityTermProperties, err error) {
if terms == nil {
return properties, nil
}
for _, term := range terms {
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(pod, &term)
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)
if err != nil {
return nil, err
}
properties = append(properties, &affinityTermProperties{namespaces: namespaces, selector: selector, topologyKey: term.TopologyKey})
}
return properties, nil
}
// podMatchesAllAffinityTermProperties returns true IFF the given pod matches all the given properties.
func podMatchesAllAffinityTermProperties(pod *v1.Pod, properties []*affinityTermProperties) bool {
if len(properties) == 0 {
return false
}
for _, property := range properties {
if !priorityutil.PodMatchesTermsNamespaceAndSelector(pod, property.namespaces, property.selector) {
return false
}
}
return true
}
// getTPMapMatchingExistingAntiAffinity calculates the following for each existing pod on each node:
// (1) Whether it has PodAntiAffinity
// (2) Whether any AffinityTerm matches the incoming pod
func getTPMapMatchingExistingAntiAffinity(pod *v1.Pod, allNodes []*schedulernodeinfo.NodeInfo) (topologyToMatchedTermCount, error) {
errCh := schedutil.NewErrorChannel()
var lock sync.Mutex
topologyMap := make(topologyToMatchedTermCount)
appendResult := func(toAppend topologyToMatchedTermCount) {
lock.Lock()
defer lock.Unlock()
topologyMap.appendMaps(toAppend)
}
ctx, cancel := context.WithCancel(context.Background())
processNode := func(i int) {
nodeInfo := allNodes[i]
node := nodeInfo.Node()
if node == nil {
klog.Error("node not found")
return
}
for _, existingPod := range nodeInfo.PodsWithAffinity() {
existingPodTopologyMaps, err := getMatchingAntiAffinityTopologyPairsOfPod(pod, existingPod, node)
if err != nil {
errCh.SendErrorWithCancel(err, cancel)
return
}
if existingPodTopologyMaps != nil {
appendResult(existingPodTopologyMaps)
}
}
}
workqueue.ParallelizeUntil(ctx, 16, len(allNodes), processNode)
if err := errCh.ReceiveError(); err != nil {
return nil, err
}
return topologyMap, nil
}
// getTPMapMatchingIncomingAffinityAntiAffinity finds existing Pods that match affinity terms of the given "pod".
// It returns a topologyToMatchedTermCount that are checked later by the affinity
// predicate. With this topologyToMatchedTermCount available, the affinity predicate does not
// need to check all the pods in the cluster.
func getTPMapMatchingIncomingAffinityAntiAffinity(pod *v1.Pod, allNodes []*schedulernodeinfo.NodeInfo) (topologyToMatchedTermCount, topologyToMatchedTermCount, error) {
topologyPairsAffinityPodsMap := make(topologyToMatchedTermCount)
topologyToMatchedExistingAntiAffinityTerms := make(topologyToMatchedTermCount)
affinity := pod.Spec.Affinity
if affinity == nil || (affinity.PodAffinity == nil && affinity.PodAntiAffinity == nil) {
return topologyPairsAffinityPodsMap, topologyToMatchedExistingAntiAffinityTerms, nil
}
var lock sync.Mutex
appendResult := func(nodeName string, nodeTopologyPairsAffinityPodsMap, nodeTopologyPairsAntiAffinityPodsMap topologyToMatchedTermCount) {
lock.Lock()
defer lock.Unlock()
if len(nodeTopologyPairsAffinityPodsMap) > 0 {
topologyPairsAffinityPodsMap.appendMaps(nodeTopologyPairsAffinityPodsMap)
}
if len(nodeTopologyPairsAntiAffinityPodsMap) > 0 {
topologyToMatchedExistingAntiAffinityTerms.appendMaps(nodeTopologyPairsAntiAffinityPodsMap)
}
}
affinityTerms := GetPodAffinityTerms(affinity.PodAffinity)
affinityProperties, err := getAffinityTermProperties(pod, affinityTerms)
if err != nil {
return nil, nil, err
}
antiAffinityTerms := GetPodAntiAffinityTerms(affinity.PodAntiAffinity)
antiAffinityProperties, err := getAffinityTermProperties(pod, antiAffinityTerms)
if err != nil {
return nil, nil, err
}
processNode := func(i int) {
nodeInfo := allNodes[i]
node := nodeInfo.Node()
if node == nil {
klog.Error("node not found")
return
}
nodeTopologyPairsAffinityPodsMap := make(topologyToMatchedTermCount)
nodeTopologyPairsAntiAffinityPodsMap := make(topologyToMatchedTermCount)
for _, existingPod := range nodeInfo.Pods() {
// Check affinity properties.
nodeTopologyPairsAffinityPodsMap.updateWithAffinityTerms(existingPod, node, affinityProperties, 1)
// Check anti-affinity properties.
nodeTopologyPairsAntiAffinityPodsMap.updateWithAntiAffinityTerms(existingPod, node, antiAffinityProperties, 1)
}
if len(nodeTopologyPairsAffinityPodsMap) > 0 || len(nodeTopologyPairsAntiAffinityPodsMap) > 0 {
appendResult(node.Name, nodeTopologyPairsAffinityPodsMap, nodeTopologyPairsAntiAffinityPodsMap)
}
}
workqueue.ParallelizeUntil(context.Background(), 16, len(allNodes), processNode)
return topologyPairsAffinityPodsMap, topologyToMatchedExistingAntiAffinityTerms, nil
}
// targetPodMatchesAffinityOfPod returns true if "targetPod" matches ALL affinity terms of
// "pod". This function does not check topology.
// So, whether the targetPod actually matches or not needs further checks for a specific
// node.
func targetPodMatchesAffinityOfPod(pod, targetPod *v1.Pod) bool {
affinity := pod.Spec.Affinity
if affinity == nil || affinity.PodAffinity == nil {
return false
}
affinityProperties, err := getAffinityTermProperties(pod, GetPodAffinityTerms(affinity.PodAffinity))
if err != nil {
klog.Errorf("error in getting affinity properties of Pod %v", pod.Name)
return false
}
return podMatchesAllAffinityTermProperties(targetPod, affinityProperties)
}

View File

@ -27,248 +27,6 @@ import (
st "k8s.io/kubernetes/pkg/scheduler/testing"
)
func TestPodAffinityMetadata_Clone(t *testing.T) {
source := &PodAffinityMetadata{
topologyToMatchedExistingAntiAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "machine1"}: 1,
{key: "name", value: "machine2"}: 1,
},
topologyToMatchedAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "nodeA"}: 1,
{key: "name", value: "nodeC"}: 2,
},
topologyToMatchedAntiAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "nodeN"}: 3,
{key: "name", value: "nodeM"}: 1,
},
}
clone := source.Clone()
if clone == source {
t.Errorf("Clone returned the exact same object!")
}
if !reflect.DeepEqual(clone, source) {
t.Errorf("Copy is not equal to source!")
}
}
// TestGetTPMapMatchingIncomingAffinityAntiAffinity tests against method getTPMapMatchingIncomingAffinityAntiAffinity
// on Anti Affinity cases
func TestGetTPMapMatchingIncomingAffinityAntiAffinity(t *testing.T) {
newPodAffinityTerms := func(keys ...string) []v1.PodAffinityTerm {
var terms []v1.PodAffinityTerm
for _, key := range keys {
terms = append(terms, v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: key,
Operator: metav1.LabelSelectorOpExists,
},
},
},
TopologyKey: "hostname",
})
}
return terms
}
newPod := func(labels ...string) *v1.Pod {
labelMap := make(map[string]string)
for _, l := range labels {
labelMap[l] = ""
}
return &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "normal", Labels: labelMap},
Spec: v1.PodSpec{NodeName: "nodeA"},
}
}
normalPodA := newPod("aaa")
normalPodB := newPod("bbb")
normalPodAB := newPod("aaa", "bbb")
nodeA := &v1.Node{ObjectMeta: metav1.ObjectMeta{Name: "nodeA", Labels: map[string]string{"hostname": "nodeA"}}}
tests := []struct {
name string
existingPods []*v1.Pod
nodes []*v1.Node
pod *v1.Pod
wantAffinityPodsMap topologyToMatchedTermCount
wantAntiAffinityPodsMap topologyToMatchedTermCount
wantErr bool
}{
{
name: "nil test",
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-normal"},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "incoming pod without affinity/anti-affinity causes a no-op",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-normal"},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "no pod has label that violates incoming pod's affinity and anti-affinity",
existingPods: []*v1.Pod{normalPodB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-anti"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "existing pod matches incoming pod's affinity and anti-affinity - single term case",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "existing pod matches incoming pod's affinity and anti-affinity - multiple terms case",
existingPods: []*v1.Pod{normalPodAB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 2, // 2 one for each term.
},
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "existing pod not match incoming pod's affinity but matches anti-affinity",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "incoming pod's anti-affinity has more than one term - existing pod violates partial term - case 1",
existingPods: []*v1.Pod{normalPodAB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "anaffi-antiaffiti"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "ccc"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "ccc"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "incoming pod's anti-affinity has more than one term - existing pod violates partial term - case 2",
existingPods: []*v1.Pod{normalPodB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
s := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(tt.existingPods, tt.nodes))
l, _ := s.NodeInfos().List()
gotAffinityPodsMap, gotAntiAffinityPodsMap, err := getTPMapMatchingIncomingAffinityAntiAffinity(tt.pod, l)
if (err != nil) != tt.wantErr {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() error = %v, wantErr %v", err, tt.wantErr)
return
}
if !reflect.DeepEqual(gotAffinityPodsMap, tt.wantAffinityPodsMap) {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() gotAffinityPodsMap = %#v, want %#v", gotAffinityPodsMap, tt.wantAffinityPodsMap)
}
if !reflect.DeepEqual(gotAntiAffinityPodsMap, tt.wantAntiAffinityPodsMap) {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() gotAntiAffinityPodsMap = %#v, want %#v", gotAntiAffinityPodsMap, tt.wantAntiAffinityPodsMap)
}
})
}
}
func TestGetTPMapMatchingSpreadConstraints(t *testing.T) {
fooSelector := st.MakeLabelSelector().Exists("foo").Obj()
barSelector := st.MakeLabelSelector().Exists("bar").Obj()

View File

@ -27,7 +27,6 @@ import (
v1 "k8s.io/api/core/v1"
storage "k8s.io/api/storage/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/fields"
"k8s.io/apimachinery/pkg/labels"
"k8s.io/apimachinery/pkg/util/rand"
@ -40,8 +39,6 @@ import (
v1helper "k8s.io/kubernetes/pkg/apis/core/v1/helper"
"k8s.io/kubernetes/pkg/features"
"k8s.io/kubernetes/pkg/scheduler/algorithm"
priorityutil "k8s.io/kubernetes/pkg/scheduler/algorithm/priorities/util"
schedulerlisters "k8s.io/kubernetes/pkg/scheduler/listers"
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
schedutil "k8s.io/kubernetes/pkg/scheduler/util"
"k8s.io/kubernetes/pkg/scheduler/volumebinder"
@ -1056,341 +1053,6 @@ func GeneralPredicates(pod *v1.Pod, meta Metadata, nodeInfo *schedulernodeinfo.N
return len(predicateFails) == 0, predicateFails, nil
}
// PodAffinityChecker contains information to check pod affinity.
type PodAffinityChecker struct {
nodeInfoLister schedulerlisters.NodeInfoLister
podLister schedulerlisters.PodLister
}
// NewPodAffinityChecker returns a PodAffinityChecker.
func NewPodAffinityChecker(sharedLister schedulerlisters.SharedLister) *PodAffinityChecker {
return &PodAffinityChecker{
nodeInfoLister: sharedLister.NodeInfos(),
podLister: sharedLister.Pods(),
}
}
// NewPodAffinityPredicate creates a PodAffinityChecker.
func NewPodAffinityPredicate(nodeInfoLister schedulerlisters.NodeInfoLister, podLister schedulerlisters.PodLister) FitPredicate {
return func(pod *v1.Pod, meta Metadata, nodeInfo *schedulernodeinfo.NodeInfo) (bool, []PredicateFailureReason, error) {
return false, nil, fmt.Errorf("This function should never be called")
}
}
// InterPodAffinityMatches checks if a pod can be scheduled on the specified node with pod affinity/anti-affinity configuration.
// First return value indicates whether a pod can be scheduled on the specified node while the second return value indicates the
// predicate failure reasons if the pod cannot be scheduled on the specified node.
func (c *PodAffinityChecker) InterPodAffinityMatches(pod *v1.Pod, meta *PodAffinityMetadata, nodeInfo *schedulernodeinfo.NodeInfo) (bool, []PredicateFailureReason, error) {
node := nodeInfo.Node()
if node == nil {
return false, nil, fmt.Errorf("node not found")
}
if failedPredicates, error := c.satisfiesExistingPodsAntiAffinity(pod, meta, nodeInfo); failedPredicates != nil {
failedPredicates := append([]PredicateFailureReason{ErrPodAffinityNotMatch}, failedPredicates)
return false, failedPredicates, error
}
// Now check if <pod> requirements will be satisfied on this node.
affinity := pod.Spec.Affinity
if affinity == nil || (affinity.PodAffinity == nil && affinity.PodAntiAffinity == nil) {
return true, nil, nil
}
if failedPredicates, error := c.satisfiesPodsAffinityAntiAffinity(pod, meta, nodeInfo, affinity); failedPredicates != nil {
failedPredicates := append([]PredicateFailureReason{ErrPodAffinityNotMatch}, failedPredicates)
return false, failedPredicates, error
}
if klog.V(10) {
// We explicitly don't do klog.V(10).Infof() to avoid computing all the parameters if this is
// not logged. There is visible performance gain from it.
klog.Infof("Schedule Pod %+v on Node %+v is allowed, pod (anti)affinity constraints satisfied",
podName(pod), node.Name)
}
return true, nil, nil
}
// podMatchesPodAffinityTerms checks if the "targetPod" matches the given "terms"
// of the "pod" on the given "nodeInfo".Node(). It returns three values: 1) whether
// targetPod matches all the terms and their topologies, 2) whether targetPod
// matches all the terms label selector and namespaces (AKA term properties),
// 3) any error.
func (c *PodAffinityChecker) podMatchesPodAffinityTerms(pod, targetPod *v1.Pod, nodeInfo *schedulernodeinfo.NodeInfo, terms []v1.PodAffinityTerm) (bool, bool, error) {
if len(terms) == 0 {
return false, false, fmt.Errorf("terms array is empty")
}
props, err := getAffinityTermProperties(pod, terms)
if err != nil {
return false, false, err
}
if !podMatchesAllAffinityTermProperties(targetPod, props) {
return false, false, nil
}
// Namespace and selector of the terms have matched. Now we check topology of the terms.
targetPodNodeInfo, err := c.nodeInfoLister.Get(targetPod.Spec.NodeName)
if err != nil {
return false, false, err
}
for _, term := range terms {
if len(term.TopologyKey) == 0 {
return false, false, fmt.Errorf("empty topologyKey is not allowed except for PreferredDuringScheduling pod anti-affinity")
}
if !priorityutil.NodesHaveSameTopologyKey(nodeInfo.Node(), targetPodNodeInfo.Node(), term.TopologyKey) {
return false, true, nil
}
}
return true, true, nil
}
// GetPodAffinityTerms gets pod affinity terms by a pod affinity object.
func GetPodAffinityTerms(podAffinity *v1.PodAffinity) (terms []v1.PodAffinityTerm) {
if podAffinity != nil {
if len(podAffinity.RequiredDuringSchedulingIgnoredDuringExecution) != 0 {
terms = podAffinity.RequiredDuringSchedulingIgnoredDuringExecution
}
// TODO: Uncomment this block when implement RequiredDuringSchedulingRequiredDuringExecution.
//if len(podAffinity.RequiredDuringSchedulingRequiredDuringExecution) != 0 {
// terms = append(terms, podAffinity.RequiredDuringSchedulingRequiredDuringExecution...)
//}
}
return terms
}
// GetPodAntiAffinityTerms gets pod affinity terms by a pod anti-affinity.
func GetPodAntiAffinityTerms(podAntiAffinity *v1.PodAntiAffinity) (terms []v1.PodAffinityTerm) {
if podAntiAffinity != nil {
if len(podAntiAffinity.RequiredDuringSchedulingIgnoredDuringExecution) != 0 {
terms = podAntiAffinity.RequiredDuringSchedulingIgnoredDuringExecution
}
// TODO: Uncomment this block when implement RequiredDuringSchedulingRequiredDuringExecution.
//if len(podAntiAffinity.RequiredDuringSchedulingRequiredDuringExecution) != 0 {
// terms = append(terms, podAntiAffinity.RequiredDuringSchedulingRequiredDuringExecution...)
//}
}
return terms
}
// getMatchingAntiAffinityTopologyPairs calculates the following for "existingPod" on given node:
// (1) Whether it has PodAntiAffinity
// (2) Whether ANY AffinityTerm matches the incoming pod
func getMatchingAntiAffinityTopologyPairsOfPod(newPod *v1.Pod, existingPod *v1.Pod, node *v1.Node) (topologyToMatchedTermCount, error) {
affinity := existingPod.Spec.Affinity
if affinity == nil || affinity.PodAntiAffinity == nil {
return nil, nil
}
topologyMap := make(topologyToMatchedTermCount)
for _, term := range GetPodAntiAffinityTerms(affinity.PodAntiAffinity) {
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)
if err != nil {
return nil, err
}
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(existingPod, &term)
if priorityutil.PodMatchesTermsNamespaceAndSelector(newPod, namespaces, selector) {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
topologyMap[pair]++
}
}
}
return topologyMap, nil
}
func (c *PodAffinityChecker) getMatchingAntiAffinityTopologyPairsOfPods(pod *v1.Pod, existingPods []*v1.Pod) (topologyToMatchedTermCount, error) {
topologyMaps := make(topologyToMatchedTermCount)
for _, existingPod := range existingPods {
existingPodNodeInfo, err := c.nodeInfoLister.Get(existingPod.Spec.NodeName)
if err != nil {
klog.Errorf("Pod %s has NodeName %q but node is not found", podName(existingPod), existingPod.Spec.NodeName)
continue
}
existingPodTopologyMaps, err := getMatchingAntiAffinityTopologyPairsOfPod(pod, existingPod, existingPodNodeInfo.Node())
if err != nil {
return nil, err
}
topologyMaps.appendMaps(existingPodTopologyMaps)
}
return topologyMaps, nil
}
// Checks if scheduling the pod onto this node would break any anti-affinity
// terms indicated by the existing pods.
func (c *PodAffinityChecker) satisfiesExistingPodsAntiAffinity(pod *v1.Pod, meta *PodAffinityMetadata, nodeInfo *schedulernodeinfo.NodeInfo) (PredicateFailureReason, error) {
node := nodeInfo.Node()
if node == nil {
return ErrExistingPodsAntiAffinityRulesNotMatch, fmt.Errorf("node not found")
}
var topologyMap topologyToMatchedTermCount
if meta != nil {
topologyMap = meta.topologyToMatchedExistingAntiAffinityTerms
} else {
// Filter out pods whose nodeName is equal to nodeInfo.node.Name, but are not
// present in nodeInfo. Pods on other nodes pass the filter.
filteredPods, err := c.podLister.FilteredList(nodeInfo.Filter, labels.Everything())
if err != nil {
errMessage := fmt.Sprintf("Failed to get all pods: %v", err)
klog.Error(errMessage)
return ErrExistingPodsAntiAffinityRulesNotMatch, errors.New(errMessage)
}
if topologyMap, err = c.getMatchingAntiAffinityTopologyPairsOfPods(pod, filteredPods); err != nil {
errMessage := fmt.Sprintf("Failed to get all terms that match pod %s: %v", podName(pod), err)
klog.Error(errMessage)
return ErrExistingPodsAntiAffinityRulesNotMatch, errors.New(errMessage)
}
}
// Iterate over topology pairs to get any of the pods being affected by
// the scheduled pod anti-affinity terms
for topologyKey, topologyValue := range node.Labels {
if topologyMap[topologyPair{key: topologyKey, value: topologyValue}] > 0 {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v", podName(pod), node.Name)
return ErrExistingPodsAntiAffinityRulesNotMatch, nil
}
}
if klog.V(10) {
// We explicitly don't do klog.V(10).Infof() to avoid computing all the parameters if this is
// not logged. There is visible performance gain from it.
klog.Infof("Schedule Pod %+v on Node %+v is allowed, existing pods anti-affinity terms satisfied.",
podName(pod), node.Name)
}
return nil, nil
}
// nodeMatchesAllTopologyTerms checks whether "nodeInfo" matches
// topology of all the "terms" for the given "pod".
func (c *PodAffinityChecker) nodeMatchesAllTopologyTerms(pod *v1.Pod, topologyPairs topologyToMatchedTermCount, nodeInfo *schedulernodeinfo.NodeInfo, terms []v1.PodAffinityTerm) bool {
node := nodeInfo.Node()
for _, term := range terms {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
if topologyPairs[pair] <= 0 {
return false
}
} else {
return false
}
}
return true
}
// nodeMatchesAnyTopologyTerm checks whether "nodeInfo" matches
// topology of any "term" for the given "pod".
func (c *PodAffinityChecker) nodeMatchesAnyTopologyTerm(pod *v1.Pod, topologyPairs topologyToMatchedTermCount, nodeInfo *schedulernodeinfo.NodeInfo, terms []v1.PodAffinityTerm) bool {
node := nodeInfo.Node()
for _, term := range terms {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
if topologyPairs[pair] > 0 {
return true
}
}
}
return false
}
// satisfiesPodsAffinityAntiAffinity checks if scheduling the pod onto this node would break any term of this pod.
func (c *PodAffinityChecker) satisfiesPodsAffinityAntiAffinity(pod *v1.Pod,
predicateMeta *PodAffinityMetadata, nodeInfo *schedulernodeinfo.NodeInfo,
affinity *v1.Affinity) (PredicateFailureReason, error) {
node := nodeInfo.Node()
if node == nil {
return ErrPodAffinityRulesNotMatch, fmt.Errorf("node not found")
}
if predicateMeta != nil {
// Check all affinity terms.
topologyToMatchedAffinityTerms := predicateMeta.topologyToMatchedAffinityTerms
if affinityTerms := GetPodAffinityTerms(affinity.PodAffinity); len(affinityTerms) > 0 {
matchExists := c.nodeMatchesAllTopologyTerms(pod, topologyToMatchedAffinityTerms, nodeInfo, affinityTerms)
if !matchExists {
// This pod may the first pod in a series that have affinity to themselves. In order
// to not leave such pods in pending state forever, we check that if no other pod
// in the cluster matches the namespace and selector of this pod and the pod matches
// its own terms, then we allow the pod to pass the affinity check.
if len(topologyToMatchedAffinityTerms) != 0 || !targetPodMatchesAffinityOfPod(pod, pod) {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v, because of PodAffinity",
podName(pod), node.Name)
return ErrPodAffinityRulesNotMatch, nil
}
}
}
// Check all anti-affinity terms.
topologyToMatchedAntiAffinityTerms := predicateMeta.topologyToMatchedAntiAffinityTerms
if antiAffinityTerms := GetPodAntiAffinityTerms(affinity.PodAntiAffinity); len(antiAffinityTerms) > 0 {
matchExists := c.nodeMatchesAnyTopologyTerm(pod, topologyToMatchedAntiAffinityTerms, nodeInfo, antiAffinityTerms)
if matchExists {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v, because of PodAntiAffinity",
podName(pod), node.Name)
return ErrPodAntiAffinityRulesNotMatch, nil
}
}
} else { // We don't have precomputed metadata. We have to follow a slow path to check affinity terms.
filteredPods, err := c.podLister.FilteredList(nodeInfo.Filter, labels.Everything())
if err != nil {
return ErrPodAffinityRulesNotMatch, err
}
affinityTerms := GetPodAffinityTerms(affinity.PodAffinity)
antiAffinityTerms := GetPodAntiAffinityTerms(affinity.PodAntiAffinity)
matchFound, termsSelectorMatchFound := false, false
for _, targetPod := range filteredPods {
// Check all affinity terms.
if !matchFound && len(affinityTerms) > 0 {
affTermsMatch, termsSelectorMatch, err := c.podMatchesPodAffinityTerms(pod, targetPod, nodeInfo, affinityTerms)
if err != nil {
errMessage := fmt.Sprintf("Cannot schedule pod %s onto node %s, because of PodAffinity: %v", podName(pod), node.Name, err)
klog.Error(errMessage)
return ErrPodAffinityRulesNotMatch, errors.New(errMessage)
}
if termsSelectorMatch {
termsSelectorMatchFound = true
}
if affTermsMatch {
matchFound = true
}
}
// Check all anti-affinity terms.
if len(antiAffinityTerms) > 0 {
antiAffTermsMatch, _, err := c.podMatchesPodAffinityTerms(pod, targetPod, nodeInfo, antiAffinityTerms)
if err != nil || antiAffTermsMatch {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v, because of PodAntiAffinityTerm, err: %v",
podName(pod), node.Name, err)
return ErrPodAntiAffinityRulesNotMatch, nil
}
}
}
if !matchFound && len(affinityTerms) > 0 {
// We have not been able to find any matches for the pod's affinity terms.
// This pod may be the first pod in a series that have affinity to themselves. In order
// to not leave such pods in pending state forever, we check that if no other pod
// in the cluster matches the namespace and selector of this pod and the pod matches
// its own terms, then we allow the pod to pass the affinity check.
if termsSelectorMatchFound {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v, because of PodAffinity",
podName(pod), node.Name)
return ErrPodAffinityRulesNotMatch, nil
}
// Check if pod matches its own affinity properties (namespace and label selector).
if !targetPodMatchesAffinityOfPod(pod, pod) {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v, because of PodAffinity",
podName(pod), node.Name)
return ErrPodAffinityRulesNotMatch, nil
}
}
}
if klog.V(10) {
// We explicitly don't do klog.V(10).Infof() to avoid computing all the parameters if this is
// not logged. There is visible performance gain from it.
klog.Infof("Schedule Pod %+v on Node %+v is allowed, pod affinity/anti-affinity constraints satisfied.",
podName(pod), node.Name)
}
return nil, nil
}
// CheckNodeUnschedulablePredicate checks if a pod can be scheduled on a node with Unschedulable spec.
func CheckNodeUnschedulablePredicate(pod *v1.Pod, meta Metadata, nodeInfo *schedulernodeinfo.NodeInfo) (bool, []PredicateFailureReason, error) {
if nodeInfo == nil || nodeInfo.Node() == nil {

View File

@ -112,7 +112,7 @@ func init() {
scheduler.RegisterFitPredicateFactory(
predicates.MatchInterPodAffinityPred,
func(args scheduler.AlgorithmFactoryArgs) predicates.FitPredicate {
return predicates.NewPodAffinityPredicate(args.SharedLister.NodeInfos(), args.SharedLister.Pods())
return nil
},
)

View File

@ -2,7 +2,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["interpod_affinity.go"],
srcs = [
"filtering.go",
"plugin.go",
"scoring.go",
],
importpath = "k8s.io/kubernetes/pkg/scheduler/framework/plugins/interpodaffinity",
visibility = ["//visibility:public"],
deps = [
@ -25,7 +29,10 @@ go_library(
go_test(
name = "go_default_test",
srcs = ["interpod_affinity_test.go"],
srcs = [
"filtering_test.go",
"scoring_test.go",
],
embed = [":go_default_library"],
deps = [
"//pkg/scheduler/algorithm/predicates:go_default_library",

View File

@ -0,0 +1,638 @@
/*
Copyright 2019 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.
*/
package interpodaffinity
import (
"context"
"fmt"
"sync"
"k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/labels"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/client-go/util/workqueue"
"k8s.io/klog"
"k8s.io/kubernetes/pkg/scheduler/algorithm/predicates"
priorityutil "k8s.io/kubernetes/pkg/scheduler/algorithm/priorities/util"
"k8s.io/kubernetes/pkg/scheduler/framework/plugins/migration"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
"k8s.io/kubernetes/pkg/scheduler/nodeinfo"
schedutil "k8s.io/kubernetes/pkg/scheduler/util"
)
// preFilterStateKey is the key in CycleState to InterPodAffinity pre-computed data for Filtering.
// Using the name of the plugin will likely help us avoid collisions with other plugins.
const preFilterStateKey = "PreFilter" + Name
// preFilterState computed at PreFilter and used at Filter.
type preFilterState struct {
// A map of topology pairs to the number of existing pods that has anti-affinity terms that match the "pod".
topologyToMatchedExistingAntiAffinityTerms topologyToMatchedTermCount
// A map of topology pairs to the number of existing pods that match the affinity terms of the "pod".
topologyToMatchedAffinityTerms topologyToMatchedTermCount
// A map of topology pairs to the number of existing pods that match the anti-affinity terms of the "pod".
topologyToMatchedAntiAffinityTerms topologyToMatchedTermCount
}
// Clone the prefilter state.
func (s *preFilterState) Clone() framework.StateData {
if s == nil {
return nil
}
copy := preFilterState{}
copy.topologyToMatchedAffinityTerms = s.topologyToMatchedAffinityTerms.clone()
copy.topologyToMatchedAntiAffinityTerms = s.topologyToMatchedAntiAffinityTerms.clone()
copy.topologyToMatchedExistingAntiAffinityTerms = s.topologyToMatchedExistingAntiAffinityTerms.clone()
return &copy
}
// updateWithPod updates the preFilterState counters with the (anti)affinity matches for the given pod.
func (s *preFilterState) updateWithPod(updatedPod, pod *v1.Pod, node *v1.Node, multiplier int64) error {
if s == nil {
return nil
}
// Update matching existing anti-affinity terms.
updatedPodAffinity := updatedPod.Spec.Affinity
if updatedPodAffinity != nil && updatedPodAffinity.PodAntiAffinity != nil {
antiAffinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAntiAffinityTerms(updatedPodAffinity.PodAntiAffinity))
if err != nil {
return fmt.Errorf("error in getting anti-affinity terms of Pod %v: %v", updatedPod.Name, err)
}
s.topologyToMatchedExistingAntiAffinityTerms.updateWithAntiAffinityTerms(pod, node, antiAffinityTerms, multiplier)
}
// Update matching incoming pod (anti)affinity terms.
affinity := pod.Spec.Affinity
podNodeName := updatedPod.Spec.NodeName
if affinity != nil && len(podNodeName) > 0 {
if affinity.PodAffinity == nil {
affinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAffinityTerms(affinity.PodAffinity))
if err != nil {
return fmt.Errorf("error in getting affinity terms of Pod %v: %v", pod.Name, err)
}
s.topologyToMatchedAffinityTerms.updateWithAffinityTerms(updatedPod, node, affinityTerms, multiplier)
}
if affinity.PodAntiAffinity != nil {
antiAffinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAntiAffinityTerms(affinity.PodAntiAffinity))
if err != nil {
klog.Errorf("error in getting anti-affinity terms of Pod %v: %v", pod.Name, err)
}
s.topologyToMatchedAntiAffinityTerms.updateWithAntiAffinityTerms(updatedPod, node, antiAffinityTerms, multiplier)
}
}
return nil
}
// TODO(Huang-Wei): It might be possible to use "make(map[topologyPair]*int64)" so that
// we can do atomic additions instead of using a global mutext, however we need to consider
// how to init each topologyToMatchedTermCount.
type topologyPair struct {
key string
value string
}
type topologyToMatchedTermCount map[topologyPair]int64
func (m topologyToMatchedTermCount) append(toAppend topologyToMatchedTermCount) {
for pair := range toAppend {
m[pair] += toAppend[pair]
}
}
func (m topologyToMatchedTermCount) clone() topologyToMatchedTermCount {
copy := make(topologyToMatchedTermCount, len(m))
copy.append(m)
return copy
}
// updateWithAffinityTerms updates the topologyToMatchedTermCount map with the specified value
// for each affinity term if "targetPod" matches ALL terms.
func (m topologyToMatchedTermCount) updateWithAffinityTerms(targetPod *v1.Pod, targetPodNode *v1.Node, affinityTerms []*affinityTerm, value int64) {
if podMatchesAllAffinityTerms(targetPod, affinityTerms) {
for _, t := range affinityTerms {
if topologyValue, ok := targetPodNode.Labels[t.topologyKey]; ok {
pair := topologyPair{key: t.topologyKey, value: topologyValue}
m[pair] += value
// value could be a negative value, hence we delete the entry if
// the entry is down to zero.
if m[pair] == 0 {
delete(m, pair)
}
}
}
}
}
// updateAntiAffinityTerms updates the topologyToMatchedTermCount map with the specified value
// for each anti-affinity term matched the target pod.
func (m topologyToMatchedTermCount) updateWithAntiAffinityTerms(targetPod *v1.Pod, targetPodNode *v1.Node, antiAffinityTerms []*affinityTerm, value int64) {
// Check anti-affinity terms.
for _, a := range antiAffinityTerms {
if priorityutil.PodMatchesTermsNamespaceAndSelector(targetPod, a.namespaces, a.selector) {
if topologyValue, ok := targetPodNode.Labels[a.topologyKey]; ok {
pair := topologyPair{key: a.topologyKey, value: topologyValue}
m[pair] += value
// value could be a negative value, hence we delete the entry if
// the entry is down to zero.
if m[pair] == 0 {
delete(m, pair)
}
}
}
}
}
// A processed version of v1.PodAffinityTerm.
type affinityTerm struct {
namespaces sets.String
selector labels.Selector
topologyKey string
}
// getAffinityTerms receives a Pod and affinity terms and returns the namespaces and
// selectors of the terms.
func getAffinityTerms(pod *v1.Pod, v1Terms []v1.PodAffinityTerm) ([]*affinityTerm, error) {
if v1Terms == nil {
return nil, nil
}
var terms []*affinityTerm
for _, term := range v1Terms {
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(pod, &term)
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)
if err != nil {
return nil, err
}
terms = append(terms, &affinityTerm{namespaces: namespaces, selector: selector, topologyKey: term.TopologyKey})
}
return terms, nil
}
// podMatchesAllAffinityTerms returns true IFF the given pod matches all the given terms.
func podMatchesAllAffinityTerms(pod *v1.Pod, terms []*affinityTerm) bool {
if len(terms) == 0 {
return false
}
for _, term := range terms {
if !priorityutil.PodMatchesTermsNamespaceAndSelector(pod, term.namespaces, term.selector) {
return false
}
}
return true
}
// getTPMapMatchingExistingAntiAffinity calculates the following for each existing pod on each node:
// (1) Whether it has PodAntiAffinity
// (2) Whether any AffinityTerm matches the incoming pod
func getTPMapMatchingExistingAntiAffinity(pod *v1.Pod, allNodes []*nodeinfo.NodeInfo) (topologyToMatchedTermCount, error) {
errCh := schedutil.NewErrorChannel()
var lock sync.Mutex
topologyMap := make(topologyToMatchedTermCount)
appendResult := func(toAppend topologyToMatchedTermCount) {
lock.Lock()
defer lock.Unlock()
topologyMap.append(toAppend)
}
ctx, cancel := context.WithCancel(context.Background())
processNode := func(i int) {
nodeInfo := allNodes[i]
node := nodeInfo.Node()
if node == nil {
klog.Error("node not found")
return
}
for _, existingPod := range nodeInfo.PodsWithAffinity() {
existingPodTopologyMaps, err := getMatchingAntiAffinityTopologyPairsOfPod(pod, existingPod, node)
if err != nil {
errCh.SendErrorWithCancel(err, cancel)
return
}
if existingPodTopologyMaps != nil {
appendResult(existingPodTopologyMaps)
}
}
}
workqueue.ParallelizeUntil(ctx, 16, len(allNodes), processNode)
if err := errCh.ReceiveError(); err != nil {
return nil, err
}
return topologyMap, nil
}
// getTPMapMatchingIncomingAffinityAntiAffinity finds existing Pods that match affinity terms of the given "pod".
// It returns a topologyToMatchedTermCount that are checked later by the affinity
// predicate. With this topologyToMatchedTermCount available, the affinity predicate does not
// need to check all the pods in the cluster.
func getTPMapMatchingIncomingAffinityAntiAffinity(pod *v1.Pod, allNodes []*nodeinfo.NodeInfo) (topologyToMatchedTermCount, topologyToMatchedTermCount, error) {
topologyPairsAffinityPodsMap := make(topologyToMatchedTermCount)
topologyToMatchedExistingAntiAffinityTerms := make(topologyToMatchedTermCount)
affinity := pod.Spec.Affinity
if affinity == nil || (affinity.PodAffinity == nil && affinity.PodAntiAffinity == nil) {
return topologyPairsAffinityPodsMap, topologyToMatchedExistingAntiAffinityTerms, nil
}
var lock sync.Mutex
appendResult := func(nodeName string, nodeTopologyPairsAffinityPodsMap, nodeTopologyPairsAntiAffinityPodsMap topologyToMatchedTermCount) {
lock.Lock()
defer lock.Unlock()
if len(nodeTopologyPairsAffinityPodsMap) > 0 {
topologyPairsAffinityPodsMap.append(nodeTopologyPairsAffinityPodsMap)
}
if len(nodeTopologyPairsAntiAffinityPodsMap) > 0 {
topologyToMatchedExistingAntiAffinityTerms.append(nodeTopologyPairsAntiAffinityPodsMap)
}
}
affinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAffinityTerms(affinity.PodAffinity))
if err != nil {
return nil, nil, err
}
antiAffinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAntiAffinityTerms(affinity.PodAntiAffinity))
if err != nil {
return nil, nil, err
}
processNode := func(i int) {
nodeInfo := allNodes[i]
node := nodeInfo.Node()
if node == nil {
klog.Error("node not found")
return
}
nodeTopologyPairsAffinityPodsMap := make(topologyToMatchedTermCount)
nodeTopologyPairsAntiAffinityPodsMap := make(topologyToMatchedTermCount)
for _, existingPod := range nodeInfo.Pods() {
// Check affinity terms.
nodeTopologyPairsAffinityPodsMap.updateWithAffinityTerms(existingPod, node, affinityTerms, 1)
// Check anti-affinity terms.
nodeTopologyPairsAntiAffinityPodsMap.updateWithAntiAffinityTerms(existingPod, node, antiAffinityTerms, 1)
}
if len(nodeTopologyPairsAffinityPodsMap) > 0 || len(nodeTopologyPairsAntiAffinityPodsMap) > 0 {
appendResult(node.Name, nodeTopologyPairsAffinityPodsMap, nodeTopologyPairsAntiAffinityPodsMap)
}
}
workqueue.ParallelizeUntil(context.Background(), 16, len(allNodes), processNode)
return topologyPairsAffinityPodsMap, topologyToMatchedExistingAntiAffinityTerms, nil
}
// targetPodMatchesAffinityOfPod returns true if "targetPod" matches ALL affinity terms of
// "pod". This function does not check topology.
// So, whether the targetPod actually matches or not needs further checks for a specific
// node.
func targetPodMatchesAffinityOfPod(pod, targetPod *v1.Pod) bool {
affinity := pod.Spec.Affinity
if affinity == nil || affinity.PodAffinity == nil {
return false
}
affinityTerms, err := getAffinityTerms(pod, schedutil.GetPodAffinityTerms(affinity.PodAffinity))
if err != nil {
klog.Errorf("error in getting affinity terms of Pod %v", pod.Name)
return false
}
return podMatchesAllAffinityTerms(targetPod, affinityTerms)
}
// PreFilter invoked at the prefilter extension point.
func (pl *InterPodAffinity) PreFilter(ctx context.Context, cycleState *framework.CycleState, pod *v1.Pod) *framework.Status {
var allNodes []*nodeinfo.NodeInfo
var havePodsWithAffinityNodes []*nodeinfo.NodeInfo
var err error
if allNodes, err = pl.sharedLister.NodeInfos().List(); err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("failed to list NodeInfos: %v", err))
}
if havePodsWithAffinityNodes, err = pl.sharedLister.NodeInfos().HavePodsWithAffinityList(); err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("failed to list NodeInfos with pods with affinity: %v", err))
}
// existingPodAntiAffinityMap will be used later for efficient check on existing pods' anti-affinity
existingPodAntiAffinityMap, err := getTPMapMatchingExistingAntiAffinity(pod, havePodsWithAffinityNodes)
if err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("calculating preFilterState: %v", err))
}
// incomingPodAffinityMap will be used later for efficient check on incoming pod's affinity
// incomingPodAntiAffinityMap will be used later for efficient check on incoming pod's anti-affinity
incomingPodAffinityMap, incomingPodAntiAffinityMap, err := getTPMapMatchingIncomingAffinityAntiAffinity(pod, allNodes)
if err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("calculating preFilterState: %v", err))
}
s := &preFilterState{
topologyToMatchedAffinityTerms: incomingPodAffinityMap,
topologyToMatchedAntiAffinityTerms: incomingPodAntiAffinityMap,
topologyToMatchedExistingAntiAffinityTerms: existingPodAntiAffinityMap,
}
cycleState.Write(preFilterStateKey, s)
return nil
}
// PreFilterExtensions returns prefilter extensions, pod add and remove.
func (pl *InterPodAffinity) PreFilterExtensions() framework.PreFilterExtensions {
return pl
}
// AddPod from pre-computed data in cycleState.
func (pl *InterPodAffinity) AddPod(ctx context.Context, cycleState *framework.CycleState, podToSchedule *v1.Pod, podToAdd *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
state, err := getPreFilterState(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
state.updateWithPod(podToAdd, podToSchedule, nodeInfo.Node(), 1)
return nil
}
// RemovePod from pre-computed data in cycleState.
func (pl *InterPodAffinity) RemovePod(ctx context.Context, cycleState *framework.CycleState, podToSchedule *v1.Pod, podToRemove *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
state, err := getPreFilterState(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
state.updateWithPod(podToRemove, podToSchedule, nodeInfo.Node(), -1)
return nil
}
func getPreFilterState(cycleState *framework.CycleState) (*preFilterState, error) {
c, err := cycleState.Read(preFilterStateKey)
if err != nil {
// The preFilterState wasn't pre-computed in prefilter. We ignore the error for now since
// Filter is able to handle that by computing it again.
klog.V(5).Infof("Error reading %q from cycleState: %v", preFilterStateKey, err)
return nil, nil
}
s, ok := c.(*preFilterState)
if !ok {
return nil, fmt.Errorf("%+v convert to interpodaffinity.state error", c)
}
return s, nil
}
// Checks if scheduling the pod onto this node would break any anti-affinity
// terms indicated by the existing pods.
func (pl *InterPodAffinity) satisfiesExistingPodsAntiAffinity(pod *v1.Pod, state *preFilterState, nodeInfo *nodeinfo.NodeInfo) (bool, error) {
node := nodeInfo.Node()
var topologyMap topologyToMatchedTermCount
if state != nil {
topologyMap = state.topologyToMatchedExistingAntiAffinityTerms
} else {
// Filter out pods whose nodeName is equal to nodeInfo.node.Name, but are not
// present in nodeInfo. Pods on other nodes pass the filter.
filteredPods, err := pl.sharedLister.Pods().FilteredList(nodeInfo.Filter, labels.Everything())
if err != nil {
return false, fmt.Errorf("Failed to get all pods: %v", err)
}
if topologyMap, err = pl.getMatchingAntiAffinityTopologyPairsOfPods(pod, filteredPods); err != nil {
return false, fmt.Errorf("Failed to get all terms that match pod: %v", err)
}
}
// Iterate over topology pairs to get any of the pods being affected by
// the scheduled pod anti-affinity terms
for topologyKey, topologyValue := range node.Labels {
if topologyMap[topologyPair{key: topologyKey, value: topologyValue}] > 0 {
klog.V(10).Infof("Cannot schedule pod %+v onto node %v", pod.Name, node.Name)
return false, nil
}
}
return true, nil
}
// nodeMatchesAllTopologyTerms checks whether "nodeInfo" matches topology of all the "terms" for the given "pod".
func nodeMatchesAllTopologyTerms(pod *v1.Pod, topologyPairs topologyToMatchedTermCount, nodeInfo *nodeinfo.NodeInfo, terms []v1.PodAffinityTerm) bool {
node := nodeInfo.Node()
for _, term := range terms {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
if topologyPairs[pair] <= 0 {
return false
}
} else {
return false
}
}
return true
}
// nodeMatchesAnyTopologyTerm checks whether "nodeInfo" matches
// topology of any "term" for the given "pod".
func nodeMatchesAnyTopologyTerm(pod *v1.Pod, topologyPairs topologyToMatchedTermCount, nodeInfo *nodeinfo.NodeInfo, terms []v1.PodAffinityTerm) bool {
node := nodeInfo.Node()
for _, term := range terms {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
if topologyPairs[pair] > 0 {
return true
}
}
}
return false
}
// podMatchesPodAffinityTerms checks if the "targetPod" matches the given "terms"
// of the "pod" on the given "nodeInfo".Node(). It returns three values: 1) whether
// targetPod matches all the terms and their topologies, 2) whether targetPod
// matches all the terms label selector and namespaces 3) any error.
func (pl *InterPodAffinity) podMatchesPodAffinityTerms(pod, targetPod *v1.Pod, nodeInfo *nodeinfo.NodeInfo, terms []v1.PodAffinityTerm) (bool, bool, error) {
if len(terms) == 0 {
return false, false, fmt.Errorf("terms array is empty")
}
props, err := getAffinityTerms(pod, terms)
if err != nil {
return false, false, err
}
if !podMatchesAllAffinityTerms(targetPod, props) {
return false, false, nil
}
// Namespace and selector of the terms have matched. Now we check topology of the terms.
targetPodNodeInfo, err := pl.sharedLister.NodeInfos().Get(targetPod.Spec.NodeName)
if err != nil {
return false, false, err
}
for _, term := range terms {
if len(term.TopologyKey) == 0 {
return false, false, fmt.Errorf("empty topologyKey is not allowed except for PreferredDuringScheduling pod anti-affinity")
}
if !priorityutil.NodesHaveSameTopologyKey(nodeInfo.Node(), targetPodNodeInfo.Node(), term.TopologyKey) {
return false, true, nil
}
}
return true, true, nil
}
// getMatchingAntiAffinityTopologyPairs calculates the following for "existingPod" on given node:
// (1) Whether it has PodAntiAffinity
// (2) Whether ANY AffinityTerm matches the incoming pod
func getMatchingAntiAffinityTopologyPairsOfPod(newPod *v1.Pod, existingPod *v1.Pod, node *v1.Node) (topologyToMatchedTermCount, error) {
affinity := existingPod.Spec.Affinity
if affinity == nil || affinity.PodAntiAffinity == nil {
return nil, nil
}
topologyMap := make(topologyToMatchedTermCount)
for _, term := range schedutil.GetPodAntiAffinityTerms(affinity.PodAntiAffinity) {
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)
if err != nil {
return nil, err
}
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(existingPod, &term)
if priorityutil.PodMatchesTermsNamespaceAndSelector(newPod, namespaces, selector) {
if topologyValue, ok := node.Labels[term.TopologyKey]; ok {
pair := topologyPair{key: term.TopologyKey, value: topologyValue}
topologyMap[pair]++
}
}
}
return topologyMap, nil
}
func (pl *InterPodAffinity) getMatchingAntiAffinityTopologyPairsOfPods(pod *v1.Pod, existingPods []*v1.Pod) (topologyToMatchedTermCount, error) {
topologyMaps := make(topologyToMatchedTermCount)
for _, existingPod := range existingPods {
existingPodNodeInfo, err := pl.sharedLister.NodeInfos().Get(existingPod.Spec.NodeName)
if err != nil {
klog.Errorf("Pod %s has NodeName %q but node is not found", existingPod.Name, existingPod.Spec.NodeName)
continue
}
existingPodTopologyMaps, err := getMatchingAntiAffinityTopologyPairsOfPod(pod, existingPod, existingPodNodeInfo.Node())
if err != nil {
return nil, err
}
topologyMaps.append(existingPodTopologyMaps)
}
return topologyMaps, nil
}
// satisfiesPodsAffinityAntiAffinity checks if scheduling the pod onto this node would break any term of this pod.
func (pl *InterPodAffinity) satisfiesPodsAffinityAntiAffinity(pod *v1.Pod,
state *preFilterState, nodeInfo *nodeinfo.NodeInfo,
affinity *v1.Affinity) (predicates.PredicateFailureReason, error) {
node := nodeInfo.Node()
if node == nil {
return predicates.ErrPodAffinityRulesNotMatch, fmt.Errorf("node not found")
}
if state != nil {
// Check all affinity terms.
topologyToMatchedAffinityTerms := state.topologyToMatchedAffinityTerms
if affinityTerms := schedutil.GetPodAffinityTerms(affinity.PodAffinity); len(affinityTerms) > 0 {
matchExists := nodeMatchesAllTopologyTerms(pod, topologyToMatchedAffinityTerms, nodeInfo, affinityTerms)
if !matchExists {
// This pod may the first pod in a series that have affinity to themselves. In order
// to not leave such pods in pending state forever, we check that if no other pod
// in the cluster matches the namespace and selector of this pod and the pod matches
// its own terms, then we allow the pod to pass the affinity check.
if len(topologyToMatchedAffinityTerms) != 0 || !targetPodMatchesAffinityOfPod(pod, pod) {
return predicates.ErrPodAffinityRulesNotMatch, nil
}
}
}
// Check all anti-affinity terms.
topologyToMatchedAntiAffinityTerms := state.topologyToMatchedAntiAffinityTerms
if antiAffinityTerms := schedutil.GetPodAntiAffinityTerms(affinity.PodAntiAffinity); len(antiAffinityTerms) > 0 {
matchExists := nodeMatchesAnyTopologyTerm(pod, topologyToMatchedAntiAffinityTerms, nodeInfo, antiAffinityTerms)
if matchExists {
return predicates.ErrPodAntiAffinityRulesNotMatch, nil
}
}
} else { // We don't have precomputed preFilterState. We have to follow a slow path to check affinity terms.
filteredPods, err := pl.sharedLister.Pods().FilteredList(nodeInfo.Filter, labels.Everything())
if err != nil {
return predicates.ErrPodAffinityRulesNotMatch, err
}
affinityTerms := schedutil.GetPodAffinityTerms(affinity.PodAffinity)
antiAffinityTerms := schedutil.GetPodAntiAffinityTerms(affinity.PodAntiAffinity)
matchFound, termsSelectorMatchFound := false, false
for _, targetPod := range filteredPods {
// Check all affinity terms.
if !matchFound && len(affinityTerms) > 0 {
affTermsMatch, termsSelectorMatch, err := pl.podMatchesPodAffinityTerms(pod, targetPod, nodeInfo, affinityTerms)
if err != nil {
return predicates.ErrPodAffinityRulesNotMatch, err
}
if termsSelectorMatch {
termsSelectorMatchFound = true
}
if affTermsMatch {
matchFound = true
}
}
// Check all anti-affinity terms.
if len(antiAffinityTerms) > 0 {
antiAffTermsMatch, _, err := pl.podMatchesPodAffinityTerms(pod, targetPod, nodeInfo, antiAffinityTerms)
if err != nil || antiAffTermsMatch {
return predicates.ErrPodAntiAffinityRulesNotMatch, err
}
}
}
if !matchFound && len(affinityTerms) > 0 {
// We have not been able to find any matches for the pod's affinity terms.
// This pod may be the first pod in a series that have affinity to themselves. In order
// to not leave such pods in pending state forever, we check that if no other pod
// in the cluster matches the namespace and selector of this pod and the pod matches
// its own terms, then we allow the pod to pass the affinity check.
if termsSelectorMatchFound {
return predicates.ErrPodAffinityRulesNotMatch, nil
}
// Check if pod matches its own affinity terms (namespace and label selector).
if !targetPodMatchesAffinityOfPod(pod, pod) {
return predicates.ErrPodAffinityRulesNotMatch, nil
}
}
}
return nil, nil
}
// Filter invoked at the filter extension point.
// It checks if a pod can be scheduled on the specified node with pod affinity/anti-affinity configuration.
func (pl *InterPodAffinity) Filter(ctx context.Context, cycleState *framework.CycleState, pod *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
state, err := getPreFilterState(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
if s, err := pl.satisfiesExistingPodsAntiAffinity(pod, state, nodeInfo); !s || err != nil {
return migration.PredicateResultToFrameworkStatus([]predicates.PredicateFailureReason{predicates.ErrPodAffinityNotMatch, predicates.ErrExistingPodsAntiAffinityRulesNotMatch}, err)
}
// Now check if <pod> requirements will be satisfied on this node.
affinity := pod.Spec.Affinity
if affinity == nil || (affinity.PodAffinity == nil && affinity.PodAntiAffinity == nil) {
return nil
}
if status, err := pl.satisfiesPodsAffinityAntiAffinity(pod, state, nodeInfo, affinity); err != nil || status != nil {
return migration.PredicateResultToFrameworkStatus([]predicates.PredicateFailureReason{predicates.ErrPodAffinityNotMatch, status}, err)
}
return nil
}

View File

@ -18,13 +18,11 @@ package interpodaffinity
import (
"context"
"fmt"
"reflect"
"testing"
"k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/kubernetes/pkg/scheduler/algorithm/predicates"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
nodeinfosnapshot "k8s.io/kubernetes/pkg/scheduler/nodeinfo/snapshot"
@ -781,8 +779,7 @@ func TestRequiredAffinitySingleNode(t *testing.T) {
t.Run(test.name, func(t *testing.T) {
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, []*v1.Node{test.node}))
p := &InterPodAffinity{
sharedLister: snapshot,
podAffinityChecker: predicates.NewPodAffinityChecker(snapshot),
sharedLister: snapshot,
}
state := framework.NewCycleState()
preFilterStatus := p.PreFilter(context.Background(), state, test.pod)
@ -1619,8 +1616,7 @@ func TestRequiredAffinityMultipleNodes(t *testing.T) {
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, test.nodes))
for indexNode, node := range test.nodes {
p := &InterPodAffinity{
sharedLister: snapshot,
podAffinityChecker: predicates.NewPodAffinityChecker(snapshot),
sharedLister: snapshot,
}
state := framework.NewCycleState()
preFilterStatus := p.PreFilter(context.Background(), state, test.pod)
@ -1636,632 +1632,6 @@ func TestRequiredAffinityMultipleNodes(t *testing.T) {
}
}
func TestPreferredAffinity(t *testing.T) {
labelRgChina := map[string]string{
"region": "China",
}
labelRgIndia := map[string]string{
"region": "India",
}
labelAzAz1 := map[string]string{
"az": "az1",
}
labelAzAz2 := map[string]string{
"az": "az2",
}
labelRgChinaAzAz1 := map[string]string{
"region": "China",
"az": "az1",
}
podLabelSecurityS1 := map[string]string{
"security": "S1",
}
podLabelSecurityS2 := map[string]string{
"security": "S2",
}
// considered only preferredDuringSchedulingIgnoredDuringExecution in pod affinity
stayWithS1InRegion := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
},
}
stayWithS2InRegion := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 6,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "region",
},
},
},
},
}
affinity3 := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 8,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpNotIn,
Values: []string{"S1"},
}, {
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "region",
},
}, {
Weight: 2,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpExists,
}, {
Key: "wrongkey",
Operator: metav1.LabelSelectorOpDoesNotExist,
},
},
},
TopologyKey: "region",
},
},
},
},
}
hardAffinity := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1", "value2"},
},
},
},
TopologyKey: "region",
}, {
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpExists,
}, {
Key: "wrongkey",
Operator: metav1.LabelSelectorOpDoesNotExist,
},
},
},
TopologyKey: "region",
},
},
},
}
awayFromS1InAz := &v1.Affinity{
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
// to stay away from security S2 in any az.
awayFromS2InAz := &v1.Affinity{
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
// to stay with security S1 in same region, stay away from security S2 in any az.
stayWithS1InRegionAwayFromS2InAz := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 8,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
},
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
tests := []struct {
pod *v1.Pod
pods []*v1.Pod
nodes []*v1.Node
expectedList framework.NodeScoreList
name string
}{
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "all machines are same priority as Affinity is nil",
},
// the node(machine1) that have the label {"region": "China"} (match the topology key) and that have existing pods that match the labelSelector get high score
// the node(machine3) that don't have the label {"region": "whatever the value is"} (mismatch the topology key) but that have existing pods that match the labelSelector get low score
// the node(machine2) that have the label {"region": "China"} (match the topology key) but that have existing pods that mismatch the labelSelector get low score
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "Affinity: pod that matches topology key & pods in nodes will get high score comparing to others" +
"which doesn't match either pods in nodes or in topology key",
},
// the node1(machine1) that have the label {"region": "China"} (match the topology key) and that have existing pods that match the labelSelector get high score
// the node2(machine2) that have the label {"region": "China"}, match the topology key and have the same label value with node1, get the same high score with node1
// the node3(machine3) that have the label {"region": "India"}, match the topology key but have a different label value, don't have existing pods that match the labelSelector,
// get a low score.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegion}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChinaAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "All the nodes that have the same topology key & label value with one of them has an existing pod that match the affinity rules, have the same score",
},
// there are 2 regions, say regionChina(machine1,machine3,machine4) and regionIndia(machine2,machine5), both regions have nodes that match the preference.
// But there are more nodes(actually more existing pods) in regionChina that match the preference than regionIndia.
// Then, nodes in regionChina get higher score than nodes in regionIndia, and all the nodes in regionChina should get a same score(high score),
// while all the nodes in regionIndia should get another same score(low score).
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS2InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine4"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine5"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine5", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 50}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: framework.MaxNodeScore}, {Name: "machine5", Score: 50}},
name: "Affinity: nodes in one region has more matching pods comparing to other reqion, so the region which has more macthes will get high score",
},
// Test with the different operators and values for pod affinity scheduling preference, including some match failures.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: affinity3}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 20}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity: different Label operators and values for pod affinity scheduling preference, including some match failures ",
},
// Test the symmetry cases for affinity, the difference between affinity and symmetry is not the pod wants to run together with some existing pods,
// but the existing pods have the inter pod affinity preference while the pod to schedule satisfy the preference.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: stayWithS1InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: stayWithS2InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity symmetry: considered only the preferredDuringSchedulingIgnoredDuringExecution in pod affinity symmetry",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardAffinity}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardAffinity}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity symmetry: considered RequiredDuringSchedulingIgnoredDuringExecution in pod affinity symmetry",
},
// The pod to schedule prefer to stay away from some existing pods at node level using the pod anti affinity.
// the nodes that have the label {"node": "bar"} (match the topology key) and that have existing pods that match the labelSelector get low score
// the nodes that don't have the label {"node": "whatever the value is"} (mismatch the topology key) but that have existing pods that match the labelSelector get high score
// the nodes that have the label {"node": "bar"} (match the topology key) but that have existing pods that mismatch the labelSelector get high score
// there are 2 nodes, say node1 and node2, both nodes have pods that match the labelSelector and have topology-key in node.Labels.
// But there are more pods on node1 that match the preference than node2. Then, node1 get a lower score than node2.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: pod that doesnot match existing pods in node will get high score ",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: pod that does not matches topology key & matches the pods in nodes will get higher score comparing to others ",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: one node has more matching pods comparing to other node, so the node which has more unmacthes will get high score",
},
// Test the symmetry cases for anti affinity
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: awayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz2}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity symmetry: the existing pods in node which has anti affinity match will get high score",
},
// Test both affinity and anti-affinity
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}},
name: "Affinity and Anti Affinity: considered only preferredDuringSchedulingIgnoredDuringExecution in both pod affinity & anti affinity",
},
// Combined cases considering both affinity and anti-affinity, the pod to schedule and existing pods have the same labels (they are in the same RC/service),
// the pod prefer to run together with its brother pods in the same region, but wants to stay away from them at node level,
// so that all the pods of a RC/service can stay in a same region but trying to separate with each other
// machine-1,machine-3,machine-4 are in ChinaRegion others machin-2,machine-5 are in IndiaRegion
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine4"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine5"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChinaAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine5", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 40}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: framework.MaxNodeScore}, {Name: "machine5", Score: 40}},
name: "Affinity and Anti Affinity: considering both affinity and anti-affinity, the pod to schedule and existing pods have the same labels",
},
// Consider Affinity, Anti Affinity and symmetry together.
// for Affinity, the weights are: 8, 0, 0, 0
// for Anti Affinity, the weights are: 0, -5, 0, 0
// for Affinity symmetry, the weights are: 0, 0, 8, 0
// for Anti Affinity symmetry, the weights are: 0, 0, 0, -5
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3", Affinity: stayWithS1InRegionAwayFromS2InAz}},
{Spec: v1.PodSpec{NodeName: "machine4", Affinity: awayFromS1InAz}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelAzAz2}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: 0}},
name: "Affinity and Anti Affinity and symmetry: considered only preferredDuringSchedulingIgnoredDuringExecution in both pod affinity & anti affinity & symmetry",
},
// Cover https://github.com/kubernetes/kubernetes/issues/82796 which panics upon:
// 1. Some nodes in a topology don't have pods with affinity, but other nodes in the same topology have.
// 2. The incoming pod doesn't have affinity.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: stayWithS1InRegionAwayFromS2InAz}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Avoid panic when partial nodes in a topology don't have pods with affinity",
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
state := framework.NewCycleState()
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, test.nodes))
p := &InterPodAffinity{
sharedLister: snapshot,
podAffinityChecker: predicates.NewPodAffinityChecker(snapshot),
hardPodAffinityWeight: 1,
}
status := p.PostFilter(context.Background(), state, test.pod, test.nodes, nil)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
var gotList framework.NodeScoreList
for _, n := range test.nodes {
nodeName := n.ObjectMeta.Name
score, status := p.Score(context.Background(), state, test.pod, nodeName)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
gotList = append(gotList, framework.NodeScore{Name: nodeName, Score: score})
}
status = p.ScoreExtensions().NormalizeScore(context.Background(), state, test.pod, gotList)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
if !reflect.DeepEqual(test.expectedList, gotList) {
t.Errorf("expected:\n\t%+v,\ngot:\n\t%+v", test.expectedList, gotList)
}
})
}
}
func TestPreferredAffinityWithHardPodAffinitySymmetricWeight(t *testing.T) {
podLabelServiceS1 := map[string]string{
"service": "S1",
}
labelRgChina := map[string]string{
"region": "China",
}
labelRgIndia := map[string]string{
"region": "India",
}
labelAzAz1 := map[string]string{
"az": "az1",
}
hardPodAffinity := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "service",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
}
tests := []struct {
pod *v1.Pod
pods []*v1.Pod
nodes []*v1.Node
hardPodAffinityWeight int32
expectedList framework.NodeScoreList
name string
}{
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelServiceS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardPodAffinity}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardPodAffinity}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
hardPodAffinityWeight: v1.DefaultHardPodAffinitySymmetricWeight,
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Hard Pod Affinity symmetry: hard pod affinity symmetry weights 1 by default, then nodes that match the hard pod affinity symmetry rules, get a high score",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelServiceS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardPodAffinity}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardPodAffinity}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
hardPodAffinityWeight: 0,
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "Hard Pod Affinity symmetry: hard pod affinity symmetry is closed(weights 0), then nodes that match the hard pod affinity symmetry rules, get same score with those not match",
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
state := framework.NewCycleState()
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, test.nodes))
fh, _ := framework.NewFramework(nil, nil, nil, framework.WithSnapshotSharedLister(snapshot))
args := &runtime.Unknown{Raw: []byte(fmt.Sprintf(`{"hardPodAffinityWeight":%d}`, test.hardPodAffinityWeight))}
p, _ := New(args, fh)
status := p.(framework.PostFilterPlugin).PostFilter(context.Background(), state, test.pod, test.nodes, nil)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
var gotList framework.NodeScoreList
for _, n := range test.nodes {
nodeName := n.ObjectMeta.Name
score, status := p.(framework.ScorePlugin).Score(context.Background(), state, test.pod, nodeName)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
gotList = append(gotList, framework.NodeScore{Name: nodeName, Score: score})
}
status = p.(framework.ScorePlugin).ScoreExtensions().NormalizeScore(context.Background(), state, test.pod, gotList)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
if !reflect.DeepEqual(test.expectedList, gotList) {
t.Errorf("expected:\n\t%+v,\ngot:\n\t%+v", test.expectedList, gotList)
}
})
}
}
func TestPreFilterStateAddRemovePod(t *testing.T) {
var label1 = map[string]string{
"region": "r1",
@ -2381,7 +1751,7 @@ func TestPreFilterStateAddRemovePod(t *testing.T) {
},
},
{
name: "metadata anti-affinity terms are updated correctly after adding and removing a pod",
name: "preFilterState anti-affinity terms are updated correctly after adding and removing a pod",
pendingPod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "pending", Labels: selector1},
},
@ -2414,7 +1784,7 @@ func TestPreFilterStateAddRemovePod(t *testing.T) {
},
},
{
name: "metadata anti-affinity terms are updated correctly after adding and removing a pod",
name: "preFilterState anti-affinity terms are updated correctly after adding and removing a pod",
pendingPod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "pending", Labels: selector1},
},
@ -2448,7 +1818,7 @@ func TestPreFilterStateAddRemovePod(t *testing.T) {
},
},
{
name: "metadata matching pod affinity and anti-affinity are updated correctly after adding and removing a pod",
name: "preFilterState matching pod affinity and anti-affinity are updated correctly after adding and removing a pod",
pendingPod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "pending", Labels: selector1},
},
@ -2487,12 +1857,11 @@ func TestPreFilterStateAddRemovePod(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
// getMeta creates predicate meta data given the list of pods.
getState := func(pods []*v1.Pod) (*InterPodAffinity, *framework.CycleState, *predicates.PodAffinityMetadata, *nodeinfosnapshot.Snapshot) {
getState := func(pods []*v1.Pod) (*InterPodAffinity, *framework.CycleState, *preFilterState, *nodeinfosnapshot.Snapshot) {
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(pods, test.nodes))
p := &InterPodAffinity{
sharedLister: snapshot,
podAffinityChecker: predicates.NewPodAffinityChecker(snapshot),
sharedLister: snapshot,
}
cycleState := framework.NewCycleState()
preFilterStatus := p.PreFilter(context.Background(), cycleState, test.pendingPod)
@ -2500,37 +1869,279 @@ func TestPreFilterStateAddRemovePod(t *testing.T) {
t.Errorf("prefilter failed with status: %v", preFilterStatus)
}
meta, err := getPodAffinityMetadata(cycleState)
state, err := getPreFilterState(cycleState)
if err != nil {
t.Errorf("failed to get metadata from cycleState: %v", err)
t.Errorf("failed to get preFilterState from cycleState: %v", err)
}
return p, cycleState, meta, snapshot
return p, cycleState, state, snapshot
}
// allPodsState is the state produced when all pods, including test.addedPod are given to prefilter.
_, _, allPodsMeta, _ := getState(append(test.existingPods, test.addedPod))
_, _, allPodsState, _ := getState(append(test.existingPods, test.addedPod))
// state is produced for test.existingPods (without test.addedPod).
ipa, state, meta, snapshot := getState(test.existingPods)
ipa, cycleState, state, snapshot := getState(test.existingPods)
// clone the state so that we can compare it later when performing Remove.
originalMeta := meta.Clone()
originalState := state.Clone()
// Add test.addedPod to state1 and verify it is equal to allPodsState.
if err := ipa.AddPod(context.Background(), state, test.pendingPod, test.addedPod, snapshot.NodeInfoMap[test.addedPod.Spec.NodeName]); err != nil {
if err := ipa.AddPod(context.Background(), cycleState, test.pendingPod, test.addedPod, snapshot.NodeInfoMap[test.addedPod.Spec.NodeName]); err != nil {
t.Errorf("error adding pod to meta: %v", err)
}
if !reflect.DeepEqual(allPodsMeta, meta) {
t.Errorf("State is not equal, got: %v, want: %v", meta, allPodsMeta)
if !reflect.DeepEqual(allPodsState, state) {
t.Errorf("State is not equal, got: %v, want: %v", state, allPodsState)
}
// Remove the added pod pod and make sure it is equal to the original state.
if err := ipa.RemovePod(context.Background(), state, test.pendingPod, test.addedPod, snapshot.NodeInfoMap[test.addedPod.Spec.NodeName]); err != nil {
if err := ipa.RemovePod(context.Background(), cycleState, test.pendingPod, test.addedPod, snapshot.NodeInfoMap[test.addedPod.Spec.NodeName]); err != nil {
t.Errorf("error removing pod from meta: %v", err)
}
if !reflect.DeepEqual(originalMeta, meta) {
t.Errorf("State is not equal, got: %v, want: %v", meta, originalMeta)
if !reflect.DeepEqual(originalState, state) {
t.Errorf("State is not equal, got: %v, want: %v", state, originalState)
}
})
}
}
func TestPreFilterStateClone(t *testing.T) {
source := &preFilterState{
topologyToMatchedExistingAntiAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "machine1"}: 1,
{key: "name", value: "machine2"}: 1,
},
topologyToMatchedAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "nodeA"}: 1,
{key: "name", value: "nodeC"}: 2,
},
topologyToMatchedAntiAffinityTerms: topologyToMatchedTermCount{
{key: "name", value: "nodeN"}: 3,
{key: "name", value: "nodeM"}: 1,
},
}
clone := source.Clone()
if clone == source {
t.Errorf("Clone returned the exact same object!")
}
if !reflect.DeepEqual(clone, source) {
t.Errorf("Copy is not equal to source!")
}
}
// TestGetTPMapMatchingIncomingAffinityAntiAffinity tests against method getTPMapMatchingIncomingAffinityAntiAffinity
// on Anti Affinity cases
func TestGetTPMapMatchingIncomingAffinityAntiAffinity(t *testing.T) {
newPodAffinityTerms := func(keys ...string) []v1.PodAffinityTerm {
var terms []v1.PodAffinityTerm
for _, key := range keys {
terms = append(terms, v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: key,
Operator: metav1.LabelSelectorOpExists,
},
},
},
TopologyKey: "hostname",
})
}
return terms
}
newPod := func(labels ...string) *v1.Pod {
labelMap := make(map[string]string)
for _, l := range labels {
labelMap[l] = ""
}
return &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "normal", Labels: labelMap},
Spec: v1.PodSpec{NodeName: "nodeA"},
}
}
normalPodA := newPod("aaa")
normalPodB := newPod("bbb")
normalPodAB := newPod("aaa", "bbb")
nodeA := &v1.Node{ObjectMeta: metav1.ObjectMeta{Name: "nodeA", Labels: map[string]string{"hostname": "nodeA"}}}
tests := []struct {
name string
existingPods []*v1.Pod
nodes []*v1.Node
pod *v1.Pod
wantAffinityPodsMap topologyToMatchedTermCount
wantAntiAffinityPodsMap topologyToMatchedTermCount
wantErr bool
}{
{
name: "nil test",
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-normal"},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "incoming pod without affinity/anti-affinity causes a no-op",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-normal"},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "no pod has label that violates incoming pod's affinity and anti-affinity",
existingPods: []*v1.Pod{normalPodB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "aaa-anti"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: make(topologyToMatchedTermCount),
},
{
name: "existing pod matches incoming pod's affinity and anti-affinity - single term case",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "existing pod matches incoming pod's affinity and anti-affinity - multiple terms case",
existingPods: []*v1.Pod{normalPodAB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa"),
},
},
},
},
wantAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 2, // 2 one for each term.
},
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "existing pod not match incoming pod's affinity but matches anti-affinity",
existingPods: []*v1.Pod{normalPodA},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "incoming pod's anti-affinity has more than one term - existing pod violates partial term - case 1",
existingPods: []*v1.Pod{normalPodAB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "anaffi-antiaffiti"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "ccc"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "ccc"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
{
name: "incoming pod's anti-affinity has more than one term - existing pod violates partial term - case 2",
existingPods: []*v1.Pod{normalPodB},
nodes: []*v1.Node{nodeA},
pod: &v1.Pod{
ObjectMeta: metav1.ObjectMeta{Name: "affi-antiaffi"},
Spec: v1.PodSpec{
Affinity: &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
PodAntiAffinity: &v1.PodAntiAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: newPodAffinityTerms("aaa", "bbb"),
},
},
},
},
wantAffinityPodsMap: make(topologyToMatchedTermCount),
wantAntiAffinityPodsMap: topologyToMatchedTermCount{
{key: "hostname", value: "nodeA"}: 1,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
s := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(tt.existingPods, tt.nodes))
l, _ := s.NodeInfos().List()
gotAffinityPodsMap, gotAntiAffinityPodsMap, err := getTPMapMatchingIncomingAffinityAntiAffinity(tt.pod, l)
if (err != nil) != tt.wantErr {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() error = %v, wantErr %v", err, tt.wantErr)
return
}
if !reflect.DeepEqual(gotAffinityPodsMap, tt.wantAffinityPodsMap) {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() gotAffinityPodsMap = %#v, want %#v", gotAffinityPodsMap, tt.wantAffinityPodsMap)
}
if !reflect.DeepEqual(gotAntiAffinityPodsMap, tt.wantAntiAffinityPodsMap) {
t.Errorf("getTPMapMatchingIncomingAffinityAntiAffinity() gotAntiAffinityPodsMap = %#v, want %#v", gotAntiAffinityPodsMap, tt.wantAntiAffinityPodsMap)
}
})
}

View File

@ -0,0 +1,68 @@
/*
Copyright 2019 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.
*/
package interpodaffinity
import (
"fmt"
"sync"
"k8s.io/apimachinery/pkg/runtime"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
schedulerlisters "k8s.io/kubernetes/pkg/scheduler/listers"
)
// Name is the name of the plugin used in the plugin registry and configurations.
const Name = "InterPodAffinity"
// Args holds the args that are used to configure the plugin.
type Args struct {
HardPodAffinityWeight int32 `json:"hardPodAffinityWeight,omitempty"`
}
var _ framework.PreFilterPlugin = &InterPodAffinity{}
var _ framework.FilterPlugin = &InterPodAffinity{}
var _ framework.PostFilterPlugin = &InterPodAffinity{}
var _ framework.ScorePlugin = &InterPodAffinity{}
// InterPodAffinity is a plugin that checks inter pod affinity
type InterPodAffinity struct {
sharedLister schedulerlisters.SharedLister
hardPodAffinityWeight int32
sync.Mutex
}
// Name returns name of the plugin. It is used in logs, etc.
func (pl *InterPodAffinity) Name() string {
return Name
}
// New initializes a new plugin and returns it.
func New(plArgs *runtime.Unknown, h framework.FrameworkHandle) (framework.Plugin, error) {
if h.SnapshotSharedLister() == nil {
return nil, fmt.Errorf("SnapshotSharedlister is nil")
}
args := &Args{}
if err := framework.DecodeInto(plArgs, args); err != nil {
return nil, err
}
return &InterPodAffinity{
sharedLister: h.SnapshotSharedLister(),
hardPodAffinityWeight: args.HardPodAffinityWeight,
}, nil
}

View File

@ -19,153 +19,19 @@ package interpodaffinity
import (
"context"
"fmt"
"sync"
"k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/labels"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/client-go/util/workqueue"
"k8s.io/klog"
"k8s.io/kubernetes/pkg/scheduler/algorithm/predicates"
priorityutil "k8s.io/kubernetes/pkg/scheduler/algorithm/priorities/util"
"k8s.io/kubernetes/pkg/scheduler/framework/plugins/migration"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
schedulerlisters "k8s.io/kubernetes/pkg/scheduler/listers"
"k8s.io/kubernetes/pkg/scheduler/nodeinfo"
schedutil "k8s.io/kubernetes/pkg/scheduler/util"
)
// InterPodAffinity is a plugin that checks inter pod affinity
type InterPodAffinity struct {
sharedLister schedulerlisters.SharedLister
podAffinityChecker *predicates.PodAffinityChecker
hardPodAffinityWeight int32
sync.Mutex
}
// Args holds the args that are used to configure the plugin.
type Args struct {
HardPodAffinityWeight int32 `json:"hardPodAffinityWeight,omitempty"`
}
var _ framework.PreFilterPlugin = &InterPodAffinity{}
var _ framework.FilterPlugin = &InterPodAffinity{}
var _ framework.PostFilterPlugin = &InterPodAffinity{}
var _ framework.ScorePlugin = &InterPodAffinity{}
const (
// Name is the name of the plugin used in the plugin registry and configurations.
Name = "InterPodAffinity"
// preFilterStateKey is the key in CycleState to InterPodAffinity pre-computed data for Filtering.
// Using the name of the plugin will likely help us avoid collisions with other plugins.
preFilterStateKey = "PreFilter" + Name
// postFilterStateKey is the key in CycleState to InterPodAffinity pre-computed data for Scoring.
postFilterStateKey = "PostFilter" + Name
)
// preFilterState computed at PreFilter and used at Filter.
type preFilterState struct {
meta *predicates.PodAffinityMetadata
}
// Clone the prefilter state.
func (s *preFilterState) Clone() framework.StateData {
copy := &preFilterState{
meta: s.meta.Clone(),
}
return copy
}
// Name returns name of the plugin. It is used in logs, etc.
func (pl *InterPodAffinity) Name() string {
return Name
}
// PreFilter invoked at the prefilter extension point.
func (pl *InterPodAffinity) PreFilter(ctx context.Context, cycleState *framework.CycleState, pod *v1.Pod) *framework.Status {
var meta *predicates.PodAffinityMetadata
var allNodes []*nodeinfo.NodeInfo
var havePodsWithAffinityNodes []*nodeinfo.NodeInfo
var err error
if allNodes, err = pl.sharedLister.NodeInfos().List(); err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("failed to list NodeInfos: %v", err))
}
if havePodsWithAffinityNodes, err = pl.sharedLister.NodeInfos().HavePodsWithAffinityList(); err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("failed to list NodeInfos with pods with affinity: %v", err))
}
if meta, err = predicates.GetPodAffinityMetadata(pod, allNodes, havePodsWithAffinityNodes); err != nil {
return framework.NewStatus(framework.Error, fmt.Sprintf("Error calculating podAffinityMetadata: %v", err))
}
s := &preFilterState{
meta: meta,
}
cycleState.Write(preFilterStateKey, s)
return nil
}
// PreFilterExtensions returns prefilter extensions, pod add and remove.
func (pl *InterPodAffinity) PreFilterExtensions() framework.PreFilterExtensions {
return pl
}
// AddPod from pre-computed data in cycleState.
func (pl *InterPodAffinity) AddPod(ctx context.Context, cycleState *framework.CycleState, podToSchedule *v1.Pod, podToAdd *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
meta, err := getPodAffinityMetadata(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
meta.UpdateWithPod(podToAdd, podToSchedule, nodeInfo.Node(), 1)
return nil
}
// RemovePod from pre-computed data in cycleState.
func (pl *InterPodAffinity) RemovePod(ctx context.Context, cycleState *framework.CycleState, podToSchedule *v1.Pod, podToRemove *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
meta, err := getPodAffinityMetadata(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
meta.UpdateWithPod(podToRemove, podToSchedule, nodeInfo.Node(), -1)
return nil
}
func getPodAffinityMetadata(cycleState *framework.CycleState) (*predicates.PodAffinityMetadata, error) {
c, err := cycleState.Read(preFilterStateKey)
if err != nil {
// The metadata wasn't pre-computed in prefilter. We ignore the error for now since
// Filter is able to handle that by computing it again.
klog.V(5).Infof("Error reading %q from cycleState: %v", preFilterStateKey, err)
return nil, nil
}
s, ok := c.(*preFilterState)
if !ok {
return nil, fmt.Errorf("%+v convert to interpodaffinity.state error", c)
}
return s.meta, nil
}
// Filter invoked at the filter extension point.
func (pl *InterPodAffinity) Filter(ctx context.Context, cycleState *framework.CycleState, pod *v1.Pod, nodeInfo *nodeinfo.NodeInfo) *framework.Status {
meta, err := getPodAffinityMetadata(cycleState)
if err != nil {
return framework.NewStatus(framework.Error, err.Error())
}
_, reasons, err := pl.podAffinityChecker.InterPodAffinityMatches(pod, meta, nodeInfo)
return migration.PredicateResultToFrameworkStatus(reasons, err)
}
// A "processed" representation of v1.WeightedAffinityTerm.
type weightedAffinityTerm struct {
namespaces sets.String
selector labels.Selector
weight int32
topologyKey string
}
// postFilterStateKey is the key in CycleState to InterPodAffinity pre-computed data for Scoring.
const postFilterStateKey = "PostFilter" + Name
// postFilterState computed at PostFilter and used at Score.
type postFilterState struct {
@ -180,29 +46,35 @@ func (s *postFilterState) Clone() framework.StateData {
return s
}
// A "processed" representation of v1.WeightedAffinityTerm.
type weightedAffinityTerm struct {
affinityTerm
weight int32
}
func newWeightedAffinityTerm(pod *v1.Pod, term *v1.PodAffinityTerm, weight int32) (*weightedAffinityTerm, error) {
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(pod, term)
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)
if err != nil {
return nil, err
}
return &weightedAffinityTerm{namespaces: namespaces, selector: selector, topologyKey: term.TopologyKey, weight: weight}, nil
return &weightedAffinityTerm{affinityTerm: affinityTerm{namespaces: namespaces, selector: selector, topologyKey: term.TopologyKey}, weight: weight}, nil
}
func getProcessedTerms(pod *v1.Pod, terms []v1.WeightedPodAffinityTerm) ([]*weightedAffinityTerm, error) {
if terms == nil {
func getWeightedAffinityTerms(pod *v1.Pod, v1Terms []v1.WeightedPodAffinityTerm) ([]*weightedAffinityTerm, error) {
if v1Terms == nil {
return nil, nil
}
var processedTerms []*weightedAffinityTerm
for i := range terms {
p, err := newWeightedAffinityTerm(pod, &terms[i].PodAffinityTerm, terms[i].Weight)
var terms []*weightedAffinityTerm
for i := range v1Terms {
p, err := newWeightedAffinityTerm(pod, &v1Terms[i].PodAffinityTerm, v1Terms[i].Weight)
if err != nil {
return nil, err
}
processedTerms = append(processedTerms, p)
terms = append(terms, p)
}
return processedTerms, nil
return terms, nil
}
func (pl *InterPodAffinity) processTerm(
@ -274,7 +146,7 @@ func (pl *InterPodAffinity) processExistingPod(state *postFilterState, existingP
// For every soft pod affinity term of <existingPod>, if <pod> matches the term,
// increment <p.counts> for every node in the cluster with the same <term.TopologyKey>
// value as that of <existingPod>'s node by the term's weight.
terms, err := getProcessedTerms(existingPod, existingPodAffinity.PodAffinity.PreferredDuringSchedulingIgnoredDuringExecution)
terms, err := getWeightedAffinityTerms(existingPod, existingPodAffinity.PodAffinity.PreferredDuringSchedulingIgnoredDuringExecution)
if err != nil {
klog.Error(err)
return nil
@ -286,7 +158,7 @@ func (pl *InterPodAffinity) processExistingPod(state *postFilterState, existingP
// For every soft pod anti-affinity term of <existingPod>, if <pod> matches the term,
// decrement <pm.counts> for every node in the cluster with the same <term.TopologyKey>
// value as that of <existingPod>'s node by the term's weight.
terms, err := getProcessedTerms(existingPod, existingPodAffinity.PodAntiAffinity.PreferredDuringSchedulingIgnoredDuringExecution)
terms, err := getWeightedAffinityTerms(existingPod, existingPodAffinity.PodAntiAffinity.PreferredDuringSchedulingIgnoredDuringExecution)
if err != nil {
return err
}
@ -332,13 +204,13 @@ func (pl *InterPodAffinity) PostFilter(
var affinityTerms []*weightedAffinityTerm
var antiAffinityTerms []*weightedAffinityTerm
if hasAffinityConstraints {
if affinityTerms, err = getProcessedTerms(pod, affinity.PodAffinity.PreferredDuringSchedulingIgnoredDuringExecution); err != nil {
if affinityTerms, err = getWeightedAffinityTerms(pod, affinity.PodAffinity.PreferredDuringSchedulingIgnoredDuringExecution); err != nil {
klog.Error(err)
return nil
}
}
if hasAntiAffinityConstraints {
if antiAffinityTerms, err = getProcessedTerms(pod, affinity.PodAntiAffinity.PreferredDuringSchedulingIgnoredDuringExecution); err != nil {
if antiAffinityTerms, err = getWeightedAffinityTerms(pod, affinity.PodAntiAffinity.PreferredDuringSchedulingIgnoredDuringExecution); err != nil {
klog.Error(err)
return nil
}
@ -458,21 +330,3 @@ func (pl *InterPodAffinity) NormalizeScore(ctx context.Context, cycleState *fram
func (pl *InterPodAffinity) ScoreExtensions() framework.ScoreExtensions {
return pl
}
// New initializes a new plugin and returns it.
func New(plArgs *runtime.Unknown, h framework.FrameworkHandle) (framework.Plugin, error) {
if h.SnapshotSharedLister() == nil {
return nil, fmt.Errorf("SnapshotSharedlister is nil")
}
args := &Args{}
if err := framework.DecodeInto(plArgs, args); err != nil {
return nil, err
}
return &InterPodAffinity{
sharedLister: h.SnapshotSharedLister(),
podAffinityChecker: predicates.NewPodAffinityChecker(h.SnapshotSharedLister()),
hardPodAffinityWeight: args.HardPodAffinityWeight,
}, nil
}

View File

@ -0,0 +1,655 @@
/*
Copyright 2019 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.
*/
package interpodaffinity
import (
"context"
"fmt"
"reflect"
"testing"
"k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
nodeinfosnapshot "k8s.io/kubernetes/pkg/scheduler/nodeinfo/snapshot"
)
func TestPreferredAffinity(t *testing.T) {
labelRgChina := map[string]string{
"region": "China",
}
labelRgIndia := map[string]string{
"region": "India",
}
labelAzAz1 := map[string]string{
"az": "az1",
}
labelAzAz2 := map[string]string{
"az": "az2",
}
labelRgChinaAzAz1 := map[string]string{
"region": "China",
"az": "az1",
}
podLabelSecurityS1 := map[string]string{
"security": "S1",
}
podLabelSecurityS2 := map[string]string{
"security": "S2",
}
// considered only preferredDuringSchedulingIgnoredDuringExecution in pod affinity
stayWithS1InRegion := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
},
}
stayWithS2InRegion := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 6,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "region",
},
},
},
},
}
affinity3 := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 8,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpNotIn,
Values: []string{"S1"},
}, {
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "region",
},
}, {
Weight: 2,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpExists,
}, {
Key: "wrongkey",
Operator: metav1.LabelSelectorOpDoesNotExist,
},
},
},
TopologyKey: "region",
},
},
},
},
}
hardAffinity := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1", "value2"},
},
},
},
TopologyKey: "region",
}, {
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpExists,
}, {
Key: "wrongkey",
Operator: metav1.LabelSelectorOpDoesNotExist,
},
},
},
TopologyKey: "region",
},
},
},
}
awayFromS1InAz := &v1.Affinity{
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
// to stay away from security S2 in any az.
awayFromS2InAz := &v1.Affinity{
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
// to stay with security S1 in same region, stay away from security S2 in any az.
stayWithS1InRegionAwayFromS2InAz := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 8,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
},
PodAntiAffinity: &v1.PodAntiAffinity{
PreferredDuringSchedulingIgnoredDuringExecution: []v1.WeightedPodAffinityTerm{
{
Weight: 5,
PodAffinityTerm: v1.PodAffinityTerm{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "security",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S2"},
},
},
},
TopologyKey: "az",
},
},
},
},
}
tests := []struct {
pod *v1.Pod
pods []*v1.Pod
nodes []*v1.Node
expectedList framework.NodeScoreList
name string
}{
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "all machines are same priority as Affinity is nil",
},
// the node(machine1) that have the label {"region": "China"} (match the topology key) and that have existing pods that match the labelSelector get high score
// the node(machine3) that don't have the label {"region": "whatever the value is"} (mismatch the topology key) but that have existing pods that match the labelSelector get low score
// the node(machine2) that have the label {"region": "China"} (match the topology key) but that have existing pods that mismatch the labelSelector get low score
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "Affinity: pod that matches topology key & pods in nodes will get high score comparing to others" +
"which doesn't match either pods in nodes or in topology key",
},
// the node1(machine1) that have the label {"region": "China"} (match the topology key) and that have existing pods that match the labelSelector get high score
// the node2(machine2) that have the label {"region": "China"}, match the topology key and have the same label value with node1, get the same high score with node1
// the node3(machine3) that have the label {"region": "India"}, match the topology key but have a different label value, don't have existing pods that match the labelSelector,
// get a low score.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegion}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChinaAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "All the nodes that have the same topology key & label value with one of them has an existing pod that match the affinity rules, have the same score",
},
// there are 2 regions, say regionChina(machine1,machine3,machine4) and regionIndia(machine2,machine5), both regions have nodes that match the preference.
// But there are more nodes(actually more existing pods) in regionChina that match the preference than regionIndia.
// Then, nodes in regionChina get higher score than nodes in regionIndia, and all the nodes in regionChina should get a same score(high score),
// while all the nodes in regionIndia should get another same score(low score).
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS2InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine4"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine5"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine5", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 50}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: framework.MaxNodeScore}, {Name: "machine5", Score: 50}},
name: "Affinity: nodes in one region has more matching pods comparing to other reqion, so the region which has more macthes will get high score",
},
// Test with the different operators and values for pod affinity scheduling preference, including some match failures.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: affinity3}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 20}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity: different Label operators and values for pod affinity scheduling preference, including some match failures ",
},
// Test the symmetry cases for affinity, the difference between affinity and symmetry is not the pod wants to run together with some existing pods,
// but the existing pods have the inter pod affinity preference while the pod to schedule satisfy the preference.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: stayWithS1InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: stayWithS2InRegion}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity symmetry: considered only the preferredDuringSchedulingIgnoredDuringExecution in pod affinity symmetry",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardAffinity}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardAffinity}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Affinity symmetry: considered RequiredDuringSchedulingIgnoredDuringExecution in pod affinity symmetry",
},
// The pod to schedule prefer to stay away from some existing pods at node level using the pod anti affinity.
// the nodes that have the label {"node": "bar"} (match the topology key) and that have existing pods that match the labelSelector get low score
// the nodes that don't have the label {"node": "whatever the value is"} (mismatch the topology key) but that have existing pods that match the labelSelector get high score
// the nodes that have the label {"node": "bar"} (match the topology key) but that have existing pods that mismatch the labelSelector get high score
// there are 2 nodes, say node1 and node2, both nodes have pods that match the labelSelector and have topology-key in node.Labels.
// But there are more pods on node1 that match the preference than node2. Then, node1 get a lower score than node2.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: pod that doesnot match existing pods in node will get high score ",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: pod that does not matches topology key & matches the pods in nodes will get higher score comparing to others ",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity: one node has more matching pods comparing to other node, so the node which has more unmacthes will get high score",
},
// Test the symmetry cases for anti affinity
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: awayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: awayFromS1InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz2}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Anti Affinity symmetry: the existing pods in node which has anti affinity match will get high score",
},
// Test both affinity and anti-affinity
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz1}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}},
name: "Affinity and Anti Affinity: considered only preferredDuringSchedulingIgnoredDuringExecution in both pod affinity & anti affinity",
},
// Combined cases considering both affinity and anti-affinity, the pod to schedule and existing pods have the same labels (they are in the same RC/service),
// the pod prefer to run together with its brother pods in the same region, but wants to stay away from them at node level,
// so that all the pods of a RC/service can stay in a same region but trying to separate with each other
// machine-1,machine-3,machine-4 are in ChinaRegion others machin-2,machine-5 are in IndiaRegion
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine3"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine4"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine5"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChinaAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine5", Labels: labelRgIndia}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 40}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: framework.MaxNodeScore}, {Name: "machine5", Score: 40}},
name: "Affinity and Anti Affinity: considering both affinity and anti-affinity, the pod to schedule and existing pods have the same labels",
},
// Consider Affinity, Anti Affinity and symmetry together.
// for Affinity, the weights are: 8, 0, 0, 0
// for Anti Affinity, the weights are: 0, -5, 0, 0
// for Affinity symmetry, the weights are: 0, 0, 8, 0
// for Anti Affinity symmetry, the weights are: 0, 0, 0, -5
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: "", Affinity: stayWithS1InRegionAwayFromS2InAz}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS2}},
{Spec: v1.PodSpec{NodeName: "machine3", Affinity: stayWithS1InRegionAwayFromS2InAz}},
{Spec: v1.PodSpec{NodeName: "machine4", Affinity: awayFromS1InAz}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelAzAz1}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine4", Labels: labelAzAz2}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: framework.MaxNodeScore}, {Name: "machine4", Score: 0}},
name: "Affinity and Anti Affinity and symmetry: considered only preferredDuringSchedulingIgnoredDuringExecution in both pod affinity & anti affinity & symmetry",
},
// Cover https://github.com/kubernetes/kubernetes/issues/82796 which panics upon:
// 1. Some nodes in a topology don't have pods with affinity, but other nodes in the same topology have.
// 2. The incoming pod doesn't have affinity.
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1"}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelSecurityS1}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: stayWithS1InRegionAwayFromS2InAz}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgChina}},
},
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}},
name: "Avoid panic when partial nodes in a topology don't have pods with affinity",
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
state := framework.NewCycleState()
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, test.nodes))
p := &InterPodAffinity{
sharedLister: snapshot,
hardPodAffinityWeight: 1,
}
status := p.PostFilter(context.Background(), state, test.pod, test.nodes, nil)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
var gotList framework.NodeScoreList
for _, n := range test.nodes {
nodeName := n.ObjectMeta.Name
score, status := p.Score(context.Background(), state, test.pod, nodeName)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
gotList = append(gotList, framework.NodeScore{Name: nodeName, Score: score})
}
status = p.ScoreExtensions().NormalizeScore(context.Background(), state, test.pod, gotList)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
if !reflect.DeepEqual(test.expectedList, gotList) {
t.Errorf("expected:\n\t%+v,\ngot:\n\t%+v", test.expectedList, gotList)
}
})
}
}
func TestPreferredAffinityWithHardPodAffinitySymmetricWeight(t *testing.T) {
podLabelServiceS1 := map[string]string{
"service": "S1",
}
labelRgChina := map[string]string{
"region": "China",
}
labelRgIndia := map[string]string{
"region": "India",
}
labelAzAz1 := map[string]string{
"az": "az1",
}
hardPodAffinity := &v1.Affinity{
PodAffinity: &v1.PodAffinity{
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
{
LabelSelector: &metav1.LabelSelector{
MatchExpressions: []metav1.LabelSelectorRequirement{
{
Key: "service",
Operator: metav1.LabelSelectorOpIn,
Values: []string{"S1"},
},
},
},
TopologyKey: "region",
},
},
},
}
tests := []struct {
pod *v1.Pod
pods []*v1.Pod
nodes []*v1.Node
hardPodAffinityWeight int32
expectedList framework.NodeScoreList
name string
}{
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelServiceS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardPodAffinity}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardPodAffinity}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
hardPodAffinityWeight: v1.DefaultHardPodAffinitySymmetricWeight,
expectedList: []framework.NodeScore{{Name: "machine1", Score: framework.MaxNodeScore}, {Name: "machine2", Score: framework.MaxNodeScore}, {Name: "machine3", Score: 0}},
name: "Hard Pod Affinity symmetry: hard pod affinity symmetry weights 1 by default, then nodes that match the hard pod affinity symmetry rules, get a high score",
},
{
pod: &v1.Pod{Spec: v1.PodSpec{NodeName: ""}, ObjectMeta: metav1.ObjectMeta{Labels: podLabelServiceS1}},
pods: []*v1.Pod{
{Spec: v1.PodSpec{NodeName: "machine1", Affinity: hardPodAffinity}},
{Spec: v1.PodSpec{NodeName: "machine2", Affinity: hardPodAffinity}},
},
nodes: []*v1.Node{
{ObjectMeta: metav1.ObjectMeta{Name: "machine1", Labels: labelRgChina}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine2", Labels: labelRgIndia}},
{ObjectMeta: metav1.ObjectMeta{Name: "machine3", Labels: labelAzAz1}},
},
hardPodAffinityWeight: 0,
expectedList: []framework.NodeScore{{Name: "machine1", Score: 0}, {Name: "machine2", Score: 0}, {Name: "machine3", Score: 0}},
name: "Hard Pod Affinity symmetry: hard pod affinity symmetry is closed(weights 0), then nodes that match the hard pod affinity symmetry rules, get same score with those not match",
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
state := framework.NewCycleState()
snapshot := nodeinfosnapshot.NewSnapshot(nodeinfosnapshot.CreateNodeInfoMap(test.pods, test.nodes))
fh, _ := framework.NewFramework(nil, nil, nil, framework.WithSnapshotSharedLister(snapshot))
args := &runtime.Unknown{Raw: []byte(fmt.Sprintf(`{"hardPodAffinityWeight":%d}`, test.hardPodAffinityWeight))}
p, _ := New(args, fh)
status := p.(framework.PostFilterPlugin).PostFilter(context.Background(), state, test.pod, test.nodes, nil)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
var gotList framework.NodeScoreList
for _, n := range test.nodes {
nodeName := n.ObjectMeta.Name
score, status := p.(framework.ScorePlugin).Score(context.Background(), state, test.pod, nodeName)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
gotList = append(gotList, framework.NodeScore{Name: nodeName, Score: score})
}
status = p.(framework.ScorePlugin).ScoreExtensions().NormalizeScore(context.Background(), state, test.pod, gotList)
if !status.IsSuccess() {
t.Errorf("unexpected error: %v", status)
}
if !reflect.DeepEqual(test.expectedList, gotList) {
t.Errorf("expected:\n\t%+v,\ngot:\n\t%+v", test.expectedList, gotList)
}
})
}
}

View File

@ -11,7 +11,6 @@ go_library(
visibility = ["//pkg/scheduler:__subpackages__"],
deps = [
"//pkg/api/v1/pod:go_default_library",
"//pkg/scheduler/algorithm/predicates:go_default_library",
"//pkg/scheduler/algorithm/priorities/util:go_default_library",
"//pkg/scheduler/framework/v1alpha1:go_default_library",
"//pkg/scheduler/internal/heap:go_default_library",

View File

@ -37,7 +37,6 @@ import (
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/tools/cache"
"k8s.io/kubernetes/pkg/api/v1/pod"
"k8s.io/kubernetes/pkg/scheduler/algorithm/predicates"
priorityutil "k8s.io/kubernetes/pkg/scheduler/algorithm/priorities/util"
framework "k8s.io/kubernetes/pkg/scheduler/framework/v1alpha1"
"k8s.io/kubernetes/pkg/scheduler/internal/heap"
@ -587,7 +586,7 @@ func (p *PriorityQueue) getUnschedulablePodsWithMatchingAffinityTerm(pod *v1.Pod
up := pInfo.Pod
affinity := up.Spec.Affinity
if affinity != nil && affinity.PodAffinity != nil {
terms := predicates.GetPodAffinityTerms(affinity.PodAffinity)
terms := util.GetPodAffinityTerms(affinity.PodAffinity)
for _, term := range terms {
namespaces := priorityutil.GetNamespacesFromPodAffinityTerm(up, &term)
selector, err := metav1.LabelSelectorAsSelector(term.LabelSelector)

View File

@ -103,3 +103,31 @@ func MoreImportantPod(pod1, pod2 *v1.Pod) bool {
}
return GetPodStartTime(pod1).Before(GetPodStartTime(pod2))
}
// GetPodAffinityTerms gets pod affinity terms by a pod affinity object.
func GetPodAffinityTerms(podAffinity *v1.PodAffinity) (terms []v1.PodAffinityTerm) {
if podAffinity != nil {
if len(podAffinity.RequiredDuringSchedulingIgnoredDuringExecution) != 0 {
terms = podAffinity.RequiredDuringSchedulingIgnoredDuringExecution
}
// TODO: Uncomment this block when implement RequiredDuringSchedulingRequiredDuringExecution.
//if len(podAffinity.RequiredDuringSchedulingRequiredDuringExecution) != 0 {
// terms = append(terms, podAffinity.RequiredDuringSchedulingRequiredDuringExecution...)
//}
}
return terms
}
// GetPodAntiAffinityTerms gets pod affinity terms by a pod anti-affinity.
func GetPodAntiAffinityTerms(podAntiAffinity *v1.PodAntiAffinity) (terms []v1.PodAffinityTerm) {
if podAntiAffinity != nil {
if len(podAntiAffinity.RequiredDuringSchedulingIgnoredDuringExecution) != 0 {
terms = podAntiAffinity.RequiredDuringSchedulingIgnoredDuringExecution
}
// TODO: Uncomment this block when implement RequiredDuringSchedulingRequiredDuringExecution.
//if len(podAntiAffinity.RequiredDuringSchedulingRequiredDuringExecution) != 0 {
// terms = append(terms, podAntiAffinity.RequiredDuringSchedulingRequiredDuringExecution...)
//}
}
return terms
}