mirror of
https://github.com/k3s-io/kubernetes.git
synced 2025-07-22 19:31:44 +00:00
Add logic to account for pods nominated to run on nodes, but are not running yet.
Add tests for the new logic.
This commit is contained in:
parent
2cbb07a439
commit
8a17ae241d
@ -49,8 +49,9 @@ type ScheduleAlgorithm interface {
|
|||||||
Schedule(*v1.Pod, NodeLister) (selectedMachine string, err error)
|
Schedule(*v1.Pod, NodeLister) (selectedMachine string, err error)
|
||||||
// Preempt receives scheduling errors for a pod and tries to create room for
|
// Preempt receives scheduling errors for a pod and tries to create room for
|
||||||
// the pod by preempting lower priority pods if possible.
|
// the pod by preempting lower priority pods if possible.
|
||||||
// It returns the node where preemption happened, a list of preempted pods, and error if any.
|
// It returns the node where preemption happened, a list of preempted pods, a
|
||||||
Preempt(*v1.Pod, NodeLister, error) (selectedNode *v1.Node, preemptedPods []*v1.Pod, err error)
|
// list of pods whose nominated node name should be removed, and error if any.
|
||||||
|
Preempt(*v1.Pod, NodeLister, error) (selectedNode *v1.Node, preemptedPods []*v1.Pod, cleanupNominatedPods []*v1.Pod, err error)
|
||||||
// Predicates() returns a pointer to a map of predicate functions. This is
|
// Predicates() returns a pointer to a map of predicate functions. This is
|
||||||
// exposed for testing.
|
// exposed for testing.
|
||||||
Predicates() map[string]FitPredicate
|
Predicates() map[string]FitPredicate
|
||||||
|
@ -315,8 +315,9 @@ func TestGenericSchedulerWithExtenders(t *testing.T) {
|
|||||||
for _, name := range test.nodes {
|
for _, name := range test.nodes {
|
||||||
cache.AddNode(&v1.Node{ObjectMeta: metav1.ObjectMeta{Name: name}})
|
cache.AddNode(&v1.Node{ObjectMeta: metav1.ObjectMeta{Name: name}})
|
||||||
}
|
}
|
||||||
|
queue := NewSchedulingQueue()
|
||||||
scheduler := NewGenericScheduler(
|
scheduler := NewGenericScheduler(
|
||||||
cache, nil, test.predicates, algorithm.EmptyPredicateMetadataProducer, test.prioritizers, algorithm.EmptyMetadataProducer, extenders)
|
cache, nil, queue, test.predicates, algorithm.EmptyPredicateMetadataProducer, test.prioritizers, algorithm.EmptyMetadataProducer, extenders)
|
||||||
podIgnored := &v1.Pod{}
|
podIgnored := &v1.Pod{}
|
||||||
machine, err := scheduler.Schedule(podIgnored, schedulertesting.FakeNodeLister(makeNodeList(test.nodes)))
|
machine, err := scheduler.Schedule(podIgnored, schedulertesting.FakeNodeLister(makeNodeList(test.nodes)))
|
||||||
if test.expectsErr {
|
if test.expectsErr {
|
||||||
|
@ -81,6 +81,7 @@ func (f *FitError) Error() string {
|
|||||||
type genericScheduler struct {
|
type genericScheduler struct {
|
||||||
cache schedulercache.Cache
|
cache schedulercache.Cache
|
||||||
equivalenceCache *EquivalenceCache
|
equivalenceCache *EquivalenceCache
|
||||||
|
schedulingQueue SchedulingQueue
|
||||||
predicates map[string]algorithm.FitPredicate
|
predicates map[string]algorithm.FitPredicate
|
||||||
priorityMetaProducer algorithm.MetadataProducer
|
priorityMetaProducer algorithm.MetadataProducer
|
||||||
predicateMetaProducer algorithm.PredicateMetadataProducer
|
predicateMetaProducer algorithm.PredicateMetadataProducer
|
||||||
@ -114,7 +115,7 @@ func (g *genericScheduler) Schedule(pod *v1.Pod, nodeLister algorithm.NodeLister
|
|||||||
}
|
}
|
||||||
|
|
||||||
trace.Step("Computing predicates")
|
trace.Step("Computing predicates")
|
||||||
filteredNodes, failedPredicateMap, err := findNodesThatFit(pod, g.cachedNodeInfoMap, nodes, g.predicates, g.extenders, g.predicateMetaProducer, g.equivalenceCache)
|
filteredNodes, failedPredicateMap, err := findNodesThatFit(pod, g.cachedNodeInfoMap, nodes, g.predicates, g.extenders, g.predicateMetaProducer, g.equivalenceCache, g.schedulingQueue)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
@ -177,53 +178,54 @@ func (g *genericScheduler) selectHost(priorityList schedulerapi.HostPriorityList
|
|||||||
|
|
||||||
// preempt finds nodes with pods that can be preempted to make room for "pod" to
|
// preempt finds nodes with pods that can be preempted to make room for "pod" to
|
||||||
// schedule. It chooses one of the nodes and preempts the pods on the node and
|
// schedule. It chooses one of the nodes and preempts the pods on the node and
|
||||||
// returns the node and the list of preempted pods if such a node is found.
|
// returns 1) the node, 2) the list of preempted pods if such a node is found,
|
||||||
// TODO(bsalamat): Add priority-based scheduling. More info: today one or more
|
// 3) A list of pods whose nominated node name should be cleared, and 4) any
|
||||||
// pending pods (different from the pod that triggered the preemption(s)) may
|
// possible error.
|
||||||
// schedule into some portion of the resources freed up by the preemption(s)
|
func (g *genericScheduler) Preempt(pod *v1.Pod, nodeLister algorithm.NodeLister, scheduleErr error) (*v1.Node, []*v1.Pod, []*v1.Pod, error) {
|
||||||
// before the pod that triggered the preemption(s) has a chance to schedule
|
|
||||||
// there, thereby preventing the pod that triggered the preemption(s) from
|
|
||||||
// scheduling. Solution is given at:
|
|
||||||
// https://github.com/kubernetes/community/blob/master/contributors/design-proposals/scheduling/pod-preemption.md#preemption-mechanics
|
|
||||||
func (g *genericScheduler) Preempt(pod *v1.Pod, nodeLister algorithm.NodeLister, scheduleErr error) (*v1.Node, []*v1.Pod, error) {
|
|
||||||
// Scheduler may return various types of errors. Consider preemption only if
|
// Scheduler may return various types of errors. Consider preemption only if
|
||||||
// the error is of type FitError.
|
// the error is of type FitError.
|
||||||
fitError, ok := scheduleErr.(*FitError)
|
fitError, ok := scheduleErr.(*FitError)
|
||||||
if !ok || fitError == nil {
|
if !ok || fitError == nil {
|
||||||
return nil, nil, nil
|
return nil, nil, nil, nil
|
||||||
}
|
}
|
||||||
err := g.cache.UpdateNodeNameToInfoMap(g.cachedNodeInfoMap)
|
err := g.cache.UpdateNodeNameToInfoMap(g.cachedNodeInfoMap)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, nil, err
|
||||||
}
|
}
|
||||||
if !podEligibleToPreemptOthers(pod, g.cachedNodeInfoMap) {
|
if !podEligibleToPreemptOthers(pod, g.cachedNodeInfoMap) {
|
||||||
glog.V(5).Infof("Pod %v is not eligible for more preemption.", pod.Name)
|
glog.V(5).Infof("Pod %v is not eligible for more preemption.", pod.Name)
|
||||||
return nil, nil, nil
|
return nil, nil, nil, nil
|
||||||
}
|
}
|
||||||
allNodes, err := nodeLister.List()
|
allNodes, err := nodeLister.List()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, nil, err
|
||||||
}
|
}
|
||||||
if len(allNodes) == 0 {
|
if len(allNodes) == 0 {
|
||||||
return nil, nil, ErrNoNodesAvailable
|
return nil, nil, nil, ErrNoNodesAvailable
|
||||||
}
|
}
|
||||||
potentialNodes := nodesWherePreemptionMightHelp(pod, allNodes, fitError.FailedPredicates)
|
potentialNodes := nodesWherePreemptionMightHelp(pod, allNodes, fitError.FailedPredicates)
|
||||||
if len(potentialNodes) == 0 {
|
if len(potentialNodes) == 0 {
|
||||||
glog.V(3).Infof("Preemption will not help schedule pod %v on any node.", pod.Name)
|
glog.V(3).Infof("Preemption will not help schedule pod %v on any node.", pod.Name)
|
||||||
return nil, nil, nil
|
// In this case, we should clean-up any existing nominated node name of the pod.
|
||||||
|
return nil, nil, []*v1.Pod{pod}, nil
|
||||||
}
|
}
|
||||||
nodeToPods, err := selectNodesForPreemption(pod, g.cachedNodeInfoMap, potentialNodes, g.predicates, g.predicateMetaProducer)
|
nodeToPods, err := selectNodesForPreemption(pod, g.cachedNodeInfoMap, potentialNodes, g.predicates, g.predicateMetaProducer, g.schedulingQueue)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, nil, err
|
return nil, nil, nil, err
|
||||||
}
|
}
|
||||||
for len(nodeToPods) > 0 {
|
for len(nodeToPods) > 0 {
|
||||||
node := pickOneNodeForPreemption(nodeToPods)
|
node := pickOneNodeForPreemption(nodeToPods)
|
||||||
if node == nil {
|
if node == nil {
|
||||||
return nil, nil, err
|
return nil, nil, nil, err
|
||||||
}
|
}
|
||||||
passes, pErr := nodePassesExtendersForPreemption(pod, node.Name, nodeToPods[node], g.cachedNodeInfoMap, g.extenders)
|
passes, pErr := nodePassesExtendersForPreemption(pod, node.Name, nodeToPods[node], g.cachedNodeInfoMap, g.extenders)
|
||||||
if passes && pErr == nil {
|
if passes && pErr == nil {
|
||||||
return node, nodeToPods[node], err
|
// Lower priority pods nominated to run on this node, may no longer fit on
|
||||||
|
// this node. So, we should remove their nomination. Removing their
|
||||||
|
// nomination updates these pods and moves them to the active queue. It
|
||||||
|
// lets scheduler find another place for them.
|
||||||
|
nominatedPods := g.getLowerPriorityNominatedPods(pod, node.Name)
|
||||||
|
return node, nodeToPods[node], nominatedPods, err
|
||||||
}
|
}
|
||||||
if pErr != nil {
|
if pErr != nil {
|
||||||
glog.Errorf("Error occurred while checking extenders for preemption on node %v: %v", node, pErr)
|
glog.Errorf("Error occurred while checking extenders for preemption on node %v: %v", node, pErr)
|
||||||
@ -231,7 +233,30 @@ func (g *genericScheduler) Preempt(pod *v1.Pod, nodeLister algorithm.NodeLister,
|
|||||||
// Remove the node from the map and try to pick a different node.
|
// Remove the node from the map and try to pick a different node.
|
||||||
delete(nodeToPods, node)
|
delete(nodeToPods, node)
|
||||||
}
|
}
|
||||||
return nil, nil, err
|
return nil, nil, nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetLowerPriorityNominatedPods returns pods whose priority is smaller than the
|
||||||
|
// priority of the given "pod" and are nominated to run on the given node.
|
||||||
|
// Note: We could possibly check if the nominated lower priority pods still fit
|
||||||
|
// and return those that no longer fit, but that would require lots of
|
||||||
|
// manipulation of NodeInfo and PredicateMeta per nominated pod. It may not be
|
||||||
|
// worth the complexity, especially because we generally expect to have a very
|
||||||
|
// small number of nominated pods per node.
|
||||||
|
func (g *genericScheduler) getLowerPriorityNominatedPods(pod *v1.Pod, nodeName string) []*v1.Pod {
|
||||||
|
pods := g.schedulingQueue.WaitingPodsForNode(nodeName)
|
||||||
|
if len(pods) == 0 {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
var lowerPriorityPods []*v1.Pod
|
||||||
|
podPriority := util.GetPodPriority(pod)
|
||||||
|
for _, p := range pods {
|
||||||
|
if util.GetPodPriority(p) < podPriority {
|
||||||
|
lowerPriorityPods = append(lowerPriorityPods, p)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return lowerPriorityPods
|
||||||
}
|
}
|
||||||
|
|
||||||
// Filters the nodes to find the ones that fit based on the given predicate functions
|
// Filters the nodes to find the ones that fit based on the given predicate functions
|
||||||
@ -244,6 +269,7 @@ func findNodesThatFit(
|
|||||||
extenders []algorithm.SchedulerExtender,
|
extenders []algorithm.SchedulerExtender,
|
||||||
metadataProducer algorithm.PredicateMetadataProducer,
|
metadataProducer algorithm.PredicateMetadataProducer,
|
||||||
ecache *EquivalenceCache,
|
ecache *EquivalenceCache,
|
||||||
|
schedulingQueue SchedulingQueue,
|
||||||
) ([]*v1.Node, FailedPredicateMap, error) {
|
) ([]*v1.Node, FailedPredicateMap, error) {
|
||||||
var filtered []*v1.Node
|
var filtered []*v1.Node
|
||||||
failedPredicateMap := FailedPredicateMap{}
|
failedPredicateMap := FailedPredicateMap{}
|
||||||
@ -262,7 +288,7 @@ func findNodesThatFit(
|
|||||||
meta := metadataProducer(pod, nodeNameToInfo)
|
meta := metadataProducer(pod, nodeNameToInfo)
|
||||||
checkNode := func(i int) {
|
checkNode := func(i int) {
|
||||||
nodeName := nodes[i].Name
|
nodeName := nodes[i].Name
|
||||||
fits, failedPredicates, err := podFitsOnNode(pod, meta, nodeNameToInfo[nodeName], predicateFuncs, ecache)
|
fits, failedPredicates, err := podFitsOnNode(pod, meta, nodeNameToInfo[nodeName], predicateFuncs, ecache, schedulingQueue)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
predicateResultLock.Lock()
|
predicateResultLock.Lock()
|
||||||
errs[err.Error()]++
|
errs[err.Error()]++
|
||||||
@ -306,9 +332,45 @@ func findNodesThatFit(
|
|||||||
return filtered, failedPredicateMap, nil
|
return filtered, failedPredicateMap, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// addNominatedPods adds pods with equal or greater priority which are nominated
|
||||||
|
// to run on the node given in nodeInfo to meta and nodeInfo. It returns 1) whether
|
||||||
|
// any pod was found, 2) augmented meta data, 3) augmented nodeInfo.
|
||||||
|
func addNominatedPods(podPriority int32, meta algorithm.PredicateMetadata,
|
||||||
|
nodeInfo *schedulercache.NodeInfo, queue SchedulingQueue) (bool, algorithm.PredicateMetadata,
|
||||||
|
*schedulercache.NodeInfo) {
|
||||||
|
if queue == nil || nodeInfo == nil || nodeInfo.Node() == nil {
|
||||||
|
// This may happen only in tests.
|
||||||
|
return false, meta, nodeInfo
|
||||||
|
}
|
||||||
|
nominatedPods := queue.WaitingPodsForNode(nodeInfo.Node().Name)
|
||||||
|
if nominatedPods == nil || len(nominatedPods) == 0 {
|
||||||
|
return false, meta, nodeInfo
|
||||||
|
}
|
||||||
|
var metaOut algorithm.PredicateMetadata = nil
|
||||||
|
if meta != nil {
|
||||||
|
metaOut = meta.ShallowCopy()
|
||||||
|
}
|
||||||
|
nodeInfoOut := nodeInfo.Clone()
|
||||||
|
for _, p := range nominatedPods {
|
||||||
|
if util.GetPodPriority(p) >= podPriority {
|
||||||
|
nodeInfoOut.AddPod(p)
|
||||||
|
if metaOut != nil {
|
||||||
|
metaOut.AddPod(p, nodeInfoOut)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return true, metaOut, nodeInfoOut
|
||||||
|
}
|
||||||
|
|
||||||
// Checks whether node with a given name and NodeInfo satisfies all predicateFuncs.
|
// Checks whether node with a given name and NodeInfo satisfies all predicateFuncs.
|
||||||
func podFitsOnNode(pod *v1.Pod, meta algorithm.PredicateMetadata, info *schedulercache.NodeInfo, predicateFuncs map[string]algorithm.FitPredicate,
|
func podFitsOnNode(
|
||||||
ecache *EquivalenceCache) (bool, []algorithm.PredicateFailureReason, error) {
|
pod *v1.Pod,
|
||||||
|
meta algorithm.PredicateMetadata,
|
||||||
|
info *schedulercache.NodeInfo,
|
||||||
|
predicateFuncs map[string]algorithm.FitPredicate,
|
||||||
|
ecache *EquivalenceCache,
|
||||||
|
queue SchedulingQueue,
|
||||||
|
) (bool, []algorithm.PredicateFailureReason, error) {
|
||||||
var (
|
var (
|
||||||
equivalenceHash uint64
|
equivalenceHash uint64
|
||||||
failedPredicates []algorithm.PredicateFailureReason
|
failedPredicates []algorithm.PredicateFailureReason
|
||||||
@ -318,36 +380,87 @@ func podFitsOnNode(pod *v1.Pod, meta algorithm.PredicateMetadata, info *schedule
|
|||||||
reasons []algorithm.PredicateFailureReason
|
reasons []algorithm.PredicateFailureReason
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
predicateResults := make(map[string]HostPredicate)
|
||||||
|
|
||||||
if ecache != nil {
|
if ecache != nil {
|
||||||
// getHashEquivalencePod will return immediately if no equivalence pod found
|
// getHashEquivalencePod will return immediately if no equivalence pod found
|
||||||
equivalenceHash, eCacheAvailable = ecache.getHashEquivalencePod(pod)
|
equivalenceHash, eCacheAvailable = ecache.getHashEquivalencePod(pod)
|
||||||
}
|
}
|
||||||
|
podsAdded := false
|
||||||
|
// We run predicates twice in some cases. If the node has greater or equal priority
|
||||||
|
// nominated pods, we run them when those pods are added to meta and nodeInfo.
|
||||||
|
// If all predicates succeed in this pass, we run them again when these
|
||||||
|
// nominated pods are not added. This second pass is necessary because some
|
||||||
|
// predicates such as inter-pod affinity may not pass without the nominated pods.
|
||||||
|
// If there are no nominated pods for the node or if the first run of the
|
||||||
|
// predicates fail, we don't run the second pass.
|
||||||
|
// We consider only equal or higher priority pods in the first pass, because
|
||||||
|
// those are the current "pod" must yield to them and not take a space opened
|
||||||
|
// for running them. It is ok if the current "pod" take resources freed for
|
||||||
|
// lower priority pods.
|
||||||
|
// Requiring that the new pod is schedulable in both circumstances ensures that
|
||||||
|
// we are making a conservative decision: predicates like resources and inter-pod
|
||||||
|
// anti-affinity are more likely to fail when the nominated pods are treated
|
||||||
|
// as running, while predicates like pod affinity are more likely to fail when
|
||||||
|
// the nominated pods are treated as not running. We can't just assume the
|
||||||
|
// nominated pods are running because they are not running right now and in fact,
|
||||||
|
// they may end up getting scheduled to a different node.
|
||||||
|
for i := 0; i < 2; i++ {
|
||||||
|
metaToUse := meta
|
||||||
|
nodeInfoToUse := info
|
||||||
|
if i == 0 {
|
||||||
|
podsAdded, metaToUse, nodeInfoToUse = addNominatedPods(util.GetPodPriority(pod), meta, info, queue)
|
||||||
|
} else if !podsAdded || len(failedPredicates) != 0 {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
// Bypass eCache if node has any nominated pods.
|
||||||
|
// TODO(bsalamat): consider using eCache and adding proper eCache invalidations
|
||||||
|
// when pods are nominated or their nominations change.
|
||||||
|
eCacheAvailable = eCacheAvailable && !podsAdded
|
||||||
for predicateKey, predicate := range predicateFuncs {
|
for predicateKey, predicate := range predicateFuncs {
|
||||||
// If equivalenceCache is available
|
|
||||||
if eCacheAvailable {
|
if eCacheAvailable {
|
||||||
// PredicateWithECache will returns it's cached predicate results
|
// PredicateWithECache will return its cached predicate results.
|
||||||
fit, reasons, invalid = ecache.PredicateWithECache(pod.GetName(), info.Node().GetName(), predicateKey, equivalenceHash)
|
fit, reasons, invalid = ecache.PredicateWithECache(pod.GetName(), info.Node().GetName(), predicateKey, equivalenceHash)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO(bsalamat): When one predicate fails and fit is false, why do we continue
|
||||||
|
// checking other predicates?
|
||||||
if !eCacheAvailable || invalid {
|
if !eCacheAvailable || invalid {
|
||||||
// we need to execute predicate functions since equivalence cache does not work
|
// we need to execute predicate functions since equivalence cache does not work
|
||||||
fit, reasons, err = predicate(pod, meta, info)
|
fit, reasons, err = predicate(pod, metaToUse, nodeInfoToUse)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return false, []algorithm.PredicateFailureReason{}, err
|
return false, []algorithm.PredicateFailureReason{}, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if eCacheAvailable {
|
if eCacheAvailable {
|
||||||
// update equivalence cache with newly computed fit & reasons
|
// Store data to update eCache after this loop.
|
||||||
// TODO(resouer) should we do this in another thread? any race?
|
if res, exists := predicateResults[predicateKey]; exists {
|
||||||
ecache.UpdateCachedPredicateItem(pod.GetName(), info.Node().GetName(), predicateKey, fit, reasons, equivalenceHash)
|
res.Fit = res.Fit && fit
|
||||||
|
res.FailReasons = append(res.FailReasons, reasons...)
|
||||||
|
predicateResults[predicateKey] = res
|
||||||
|
} else {
|
||||||
|
predicateResults[predicateKey] = HostPredicate{Fit: fit, FailReasons: reasons}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if !fit {
|
if !fit {
|
||||||
// eCache is available and valid, and predicates result is unfit, record the fail reasons
|
// eCache is available and valid, and predicates result is unfit, record the fail reasons
|
||||||
failedPredicates = append(failedPredicates, reasons...)
|
failedPredicates = append(failedPredicates, reasons...)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO(bsalamat): This way of updating equiv. cache has a race condition against
|
||||||
|
// cache invalidations invoked in event handlers. This race has existed despite locks
|
||||||
|
// in eCache implementation. If cache is invalidated after a predicate is executed
|
||||||
|
// and before we update the cache, the updates should not be written to the cache.
|
||||||
|
if eCacheAvailable {
|
||||||
|
nodeName := info.Node().GetName()
|
||||||
|
for predKey, result := range predicateResults {
|
||||||
|
// update equivalence cache with newly computed fit & reasons
|
||||||
|
// TODO(resouer) should we do this in another thread? any race?
|
||||||
|
ecache.UpdateCachedPredicateItem(pod.GetName(), nodeName, predKey, result.Fit, result.FailReasons, equivalenceHash)
|
||||||
|
}
|
||||||
|
}
|
||||||
return len(failedPredicates) == 0, failedPredicates, nil
|
return len(failedPredicates) == 0, failedPredicates, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -597,6 +710,7 @@ func selectNodesForPreemption(pod *v1.Pod,
|
|||||||
potentialNodes []*v1.Node,
|
potentialNodes []*v1.Node,
|
||||||
predicates map[string]algorithm.FitPredicate,
|
predicates map[string]algorithm.FitPredicate,
|
||||||
metadataProducer algorithm.PredicateMetadataProducer,
|
metadataProducer algorithm.PredicateMetadataProducer,
|
||||||
|
queue SchedulingQueue,
|
||||||
) (map[*v1.Node][]*v1.Pod, error) {
|
) (map[*v1.Node][]*v1.Pod, error) {
|
||||||
|
|
||||||
nodeNameToPods := map[*v1.Node][]*v1.Pod{}
|
nodeNameToPods := map[*v1.Node][]*v1.Pod{}
|
||||||
@ -610,7 +724,7 @@ func selectNodesForPreemption(pod *v1.Pod,
|
|||||||
if meta != nil {
|
if meta != nil {
|
||||||
metaCopy = meta.ShallowCopy()
|
metaCopy = meta.ShallowCopy()
|
||||||
}
|
}
|
||||||
pods, fits := selectVictimsOnNode(pod, metaCopy, nodeNameToInfo[nodeName], predicates)
|
pods, fits := selectVictimsOnNode(pod, metaCopy, nodeNameToInfo[nodeName], predicates, queue)
|
||||||
if fits {
|
if fits {
|
||||||
resultLock.Lock()
|
resultLock.Lock()
|
||||||
nodeNameToPods[potentialNodes[i]] = pods
|
nodeNameToPods[potentialNodes[i]] = pods
|
||||||
@ -672,7 +786,9 @@ func selectVictimsOnNode(
|
|||||||
pod *v1.Pod,
|
pod *v1.Pod,
|
||||||
meta algorithm.PredicateMetadata,
|
meta algorithm.PredicateMetadata,
|
||||||
nodeInfo *schedulercache.NodeInfo,
|
nodeInfo *schedulercache.NodeInfo,
|
||||||
fitPredicates map[string]algorithm.FitPredicate) ([]*v1.Pod, bool) {
|
fitPredicates map[string]algorithm.FitPredicate,
|
||||||
|
queue SchedulingQueue,
|
||||||
|
) ([]*v1.Pod, bool) {
|
||||||
potentialVictims := util.SortableList{CompFunc: util.HigherPriorityPod}
|
potentialVictims := util.SortableList{CompFunc: util.HigherPriorityPod}
|
||||||
nodeInfoCopy := nodeInfo.Clone()
|
nodeInfoCopy := nodeInfo.Clone()
|
||||||
|
|
||||||
@ -703,7 +819,7 @@ func selectVictimsOnNode(
|
|||||||
// that we should check is if the "pod" is failing to schedule due to pod affinity
|
// that we should check is if the "pod" is failing to schedule due to pod affinity
|
||||||
// failure.
|
// failure.
|
||||||
// TODO(bsalamat): Consider checking affinity to lower priority pods if feasible with reasonable performance.
|
// TODO(bsalamat): Consider checking affinity to lower priority pods if feasible with reasonable performance.
|
||||||
if fits, _, err := podFitsOnNode(pod, meta, nodeInfoCopy, fitPredicates, nil); !fits {
|
if fits, _, err := podFitsOnNode(pod, meta, nodeInfoCopy, fitPredicates, nil, queue); !fits {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
glog.Warningf("Encountered error while selecting victims on node %v: %v", nodeInfo.Node().Name, err)
|
glog.Warningf("Encountered error while selecting victims on node %v: %v", nodeInfo.Node().Name, err)
|
||||||
}
|
}
|
||||||
@ -714,7 +830,7 @@ func selectVictimsOnNode(
|
|||||||
for _, p := range potentialVictims.Items {
|
for _, p := range potentialVictims.Items {
|
||||||
lpp := p.(*v1.Pod)
|
lpp := p.(*v1.Pod)
|
||||||
addPod(lpp)
|
addPod(lpp)
|
||||||
if fits, _, _ := podFitsOnNode(pod, meta, nodeInfoCopy, fitPredicates, nil); !fits {
|
if fits, _, _ := podFitsOnNode(pod, meta, nodeInfoCopy, fitPredicates, nil, queue); !fits {
|
||||||
removePod(lpp)
|
removePod(lpp)
|
||||||
victims = append(victims, lpp)
|
victims = append(victims, lpp)
|
||||||
glog.V(5).Infof("Pod %v is a potential preemption victim on node %v.", lpp.Name, nodeInfo.Node().Name)
|
glog.V(5).Infof("Pod %v is a potential preemption victim on node %v.", lpp.Name, nodeInfo.Node().Name)
|
||||||
@ -764,7 +880,6 @@ func nodesWherePreemptionMightHelp(pod *v1.Pod, nodes []*v1.Node, failedPredicat
|
|||||||
// considered for preemption.
|
// considered for preemption.
|
||||||
// We look at the node that is nominated for this pod and as long as there are
|
// We look at the node that is nominated for this pod and as long as there are
|
||||||
// terminating pods on the node, we don't consider this for preempting more pods.
|
// terminating pods on the node, we don't consider this for preempting more pods.
|
||||||
// TODO(bsalamat): Revisit this algorithm once scheduling by priority is added.
|
|
||||||
func podEligibleToPreemptOthers(pod *v1.Pod, nodeNameToInfo map[string]*schedulercache.NodeInfo) bool {
|
func podEligibleToPreemptOthers(pod *v1.Pod, nodeNameToInfo map[string]*schedulercache.NodeInfo) bool {
|
||||||
if nodeName, found := pod.Annotations[NominatedNodeAnnotationKey]; found {
|
if nodeName, found := pod.Annotations[NominatedNodeAnnotationKey]; found {
|
||||||
if nodeInfo, found := nodeNameToInfo[nodeName]; found {
|
if nodeInfo, found := nodeNameToInfo[nodeName]; found {
|
||||||
@ -782,6 +897,7 @@ func podEligibleToPreemptOthers(pod *v1.Pod, nodeNameToInfo map[string]*schedule
|
|||||||
func NewGenericScheduler(
|
func NewGenericScheduler(
|
||||||
cache schedulercache.Cache,
|
cache schedulercache.Cache,
|
||||||
eCache *EquivalenceCache,
|
eCache *EquivalenceCache,
|
||||||
|
podQueue SchedulingQueue,
|
||||||
predicates map[string]algorithm.FitPredicate,
|
predicates map[string]algorithm.FitPredicate,
|
||||||
predicateMetaProducer algorithm.PredicateMetadataProducer,
|
predicateMetaProducer algorithm.PredicateMetadataProducer,
|
||||||
prioritizers []algorithm.PriorityConfig,
|
prioritizers []algorithm.PriorityConfig,
|
||||||
@ -790,6 +906,7 @@ func NewGenericScheduler(
|
|||||||
return &genericScheduler{
|
return &genericScheduler{
|
||||||
cache: cache,
|
cache: cache,
|
||||||
equivalenceCache: eCache,
|
equivalenceCache: eCache,
|
||||||
|
schedulingQueue: podQueue,
|
||||||
predicates: predicates,
|
predicates: predicates,
|
||||||
predicateMetaProducer: predicateMetaProducer,
|
predicateMetaProducer: predicateMetaProducer,
|
||||||
prioritizers: prioritizers,
|
prioritizers: prioritizers,
|
||||||
|
@ -311,7 +311,7 @@ func TestGenericScheduler(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
scheduler := NewGenericScheduler(
|
scheduler := NewGenericScheduler(
|
||||||
cache, nil, test.predicates, algorithm.EmptyPredicateMetadataProducer, test.prioritizers, algorithm.EmptyMetadataProducer, []algorithm.SchedulerExtender{})
|
cache, nil, NewSchedulingQueue(), test.predicates, algorithm.EmptyPredicateMetadataProducer, test.prioritizers, algorithm.EmptyMetadataProducer, []algorithm.SchedulerExtender{})
|
||||||
machine, err := scheduler.Schedule(test.pod, schedulertesting.FakeNodeLister(makeNodeList(test.nodes)))
|
machine, err := scheduler.Schedule(test.pod, schedulertesting.FakeNodeLister(makeNodeList(test.nodes)))
|
||||||
|
|
||||||
if !reflect.DeepEqual(err, test.wErr) {
|
if !reflect.DeepEqual(err, test.wErr) {
|
||||||
@ -331,7 +331,7 @@ func TestFindFitAllError(t *testing.T) {
|
|||||||
"2": schedulercache.NewNodeInfo(),
|
"2": schedulercache.NewNodeInfo(),
|
||||||
"1": schedulercache.NewNodeInfo(),
|
"1": schedulercache.NewNodeInfo(),
|
||||||
}
|
}
|
||||||
_, predicateMap, err := findNodesThatFit(&v1.Pod{}, nodeNameToInfo, makeNodeList(nodes), predicates, nil, algorithm.EmptyPredicateMetadataProducer, nil)
|
_, predicateMap, err := findNodesThatFit(&v1.Pod{}, nodeNameToInfo, makeNodeList(nodes), predicates, nil, algorithm.EmptyPredicateMetadataProducer, nil, nil)
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("unexpected error: %v", err)
|
t.Errorf("unexpected error: %v", err)
|
||||||
@ -365,7 +365,7 @@ func TestFindFitSomeError(t *testing.T) {
|
|||||||
nodeNameToInfo[name].SetNode(&v1.Node{ObjectMeta: metav1.ObjectMeta{Name: name}})
|
nodeNameToInfo[name].SetNode(&v1.Node{ObjectMeta: metav1.ObjectMeta{Name: name}})
|
||||||
}
|
}
|
||||||
|
|
||||||
_, predicateMap, err := findNodesThatFit(pod, nodeNameToInfo, makeNodeList(nodes), predicates, nil, algorithm.EmptyPredicateMetadataProducer, nil)
|
_, predicateMap, err := findNodesThatFit(pod, nodeNameToInfo, makeNodeList(nodes), predicates, nil, algorithm.EmptyPredicateMetadataProducer, nil, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("unexpected error: %v", err)
|
t.Errorf("unexpected error: %v", err)
|
||||||
}
|
}
|
||||||
@ -790,7 +790,7 @@ func TestSelectNodesForPreemption(t *testing.T) {
|
|||||||
test.predicates[predicates.MatchInterPodAffinity] = algorithmpredicates.NewPodAffinityPredicate(FakeNodeInfo(*nodes[0]), schedulertesting.FakePodLister(test.pods))
|
test.predicates[predicates.MatchInterPodAffinity] = algorithmpredicates.NewPodAffinityPredicate(FakeNodeInfo(*nodes[0]), schedulertesting.FakePodLister(test.pods))
|
||||||
}
|
}
|
||||||
nodeNameToInfo := schedulercache.CreateNodeNameToInfoMap(test.pods, nodes)
|
nodeNameToInfo := schedulercache.CreateNodeNameToInfoMap(test.pods, nodes)
|
||||||
nodeToPods, err := selectNodesForPreemption(test.pod, nodeNameToInfo, nodes, test.predicates, PredicateMetadata)
|
nodeToPods, err := selectNodesForPreemption(test.pod, nodeNameToInfo, nodes, test.predicates, PredicateMetadata, nil)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Error(err)
|
t.Error(err)
|
||||||
}
|
}
|
||||||
@ -947,7 +947,7 @@ func TestPickOneNodeForPreemption(t *testing.T) {
|
|||||||
nodes = append(nodes, makeNode(n, priorityutil.DefaultMilliCpuRequest*5, priorityutil.DefaultMemoryRequest*5))
|
nodes = append(nodes, makeNode(n, priorityutil.DefaultMilliCpuRequest*5, priorityutil.DefaultMemoryRequest*5))
|
||||||
}
|
}
|
||||||
nodeNameToInfo := schedulercache.CreateNodeNameToInfoMap(test.pods, nodes)
|
nodeNameToInfo := schedulercache.CreateNodeNameToInfoMap(test.pods, nodes)
|
||||||
candidateNodes, _ := selectNodesForPreemption(test.pod, nodeNameToInfo, nodes, test.predicates, PredicateMetadata)
|
candidateNodes, _ := selectNodesForPreemption(test.pod, nodeNameToInfo, nodes, test.predicates, PredicateMetadata, nil)
|
||||||
node := pickOneNodeForPreemption(candidateNodes)
|
node := pickOneNodeForPreemption(candidateNodes)
|
||||||
found := false
|
found := false
|
||||||
for _, nodeName := range test.expected {
|
for _, nodeName := range test.expected {
|
||||||
@ -1190,9 +1190,9 @@ func TestPreempt(t *testing.T) {
|
|||||||
extenders = append(extenders, extender)
|
extenders = append(extenders, extender)
|
||||||
}
|
}
|
||||||
scheduler := NewGenericScheduler(
|
scheduler := NewGenericScheduler(
|
||||||
cache, nil, map[string]algorithm.FitPredicate{"matches": algorithmpredicates.PodFitsResources}, algorithm.EmptyPredicateMetadataProducer, []algorithm.PriorityConfig{{Function: numericPriority, Weight: 1}}, algorithm.EmptyMetadataProducer, extenders)
|
cache, nil, NewSchedulingQueue(), map[string]algorithm.FitPredicate{"matches": algorithmpredicates.PodFitsResources}, algorithm.EmptyPredicateMetadataProducer, []algorithm.PriorityConfig{{Function: numericPriority, Weight: 1}}, algorithm.EmptyMetadataProducer, extenders)
|
||||||
// Call Preempt and check the expected results.
|
// Call Preempt and check the expected results.
|
||||||
node, victims, err := scheduler.Preempt(test.pod, schedulertesting.FakeNodeLister(makeNodeList(nodeNames)), error(&FitError{Pod: test.pod, FailedPredicates: failedPredMap}))
|
node, victims, _, err := scheduler.Preempt(test.pod, schedulertesting.FakeNodeLister(makeNodeList(nodeNames)), error(&FitError{Pod: test.pod, FailedPredicates: failedPredMap}))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("test [%v]: unexpected error in preemption: %v", test.name, err)
|
t.Errorf("test [%v]: unexpected error in preemption: %v", test.name, err)
|
||||||
}
|
}
|
||||||
@ -1220,7 +1220,7 @@ func TestPreempt(t *testing.T) {
|
|||||||
test.pod.Annotations[NominatedNodeAnnotationKey] = node.Name
|
test.pod.Annotations[NominatedNodeAnnotationKey] = node.Name
|
||||||
}
|
}
|
||||||
// Call preempt again and make sure it doesn't preempt any more pods.
|
// Call preempt again and make sure it doesn't preempt any more pods.
|
||||||
node, victims, err = scheduler.Preempt(test.pod, schedulertesting.FakeNodeLister(makeNodeList(nodeNames)), error(&FitError{Pod: test.pod, FailedPredicates: failedPredMap}))
|
node, victims, _, err = scheduler.Preempt(test.pod, schedulertesting.FakeNodeLister(makeNodeList(nodeNames)), error(&FitError{Pod: test.pod, FailedPredicates: failedPredMap}))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("test [%v]: unexpected error in preemption: %v", test.name, err)
|
t.Errorf("test [%v]: unexpected error in preemption: %v", test.name, err)
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,7 @@ limitations under the License.
|
|||||||
// pods that are already tried and are determined to be unschedulable. The latter
|
// pods that are already tried and are determined to be unschedulable. The latter
|
||||||
// is called unschedulableQ.
|
// is called unschedulableQ.
|
||||||
// FIFO is here for flag-gating purposes and allows us to use the traditional
|
// FIFO is here for flag-gating purposes and allows us to use the traditional
|
||||||
// scheduling queue when Pod Priority flag is false.
|
// scheduling queue when util.PodPriorityEnabled() returns false.
|
||||||
|
|
||||||
package core
|
package core
|
||||||
|
|
||||||
@ -34,6 +34,7 @@ import (
|
|||||||
"k8s.io/api/core/v1"
|
"k8s.io/api/core/v1"
|
||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/client-go/tools/cache"
|
"k8s.io/client-go/tools/cache"
|
||||||
|
podutil "k8s.io/kubernetes/pkg/api/v1/pod"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm/predicates"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm/predicates"
|
||||||
priorityutil "k8s.io/kubernetes/plugin/pkg/scheduler/algorithm/priorities/util"
|
priorityutil "k8s.io/kubernetes/plugin/pkg/scheduler/algorithm/priorities/util"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/util"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/util"
|
||||||
@ -217,6 +218,11 @@ func (p *PriorityQueue) AddIfNotPresent(pod *v1.Pod) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func isPodUnschedulable(pod *v1.Pod) bool {
|
||||||
|
_, cond := podutil.GetPodCondition(&pod.Status, v1.PodScheduled)
|
||||||
|
return cond != nil && cond.Status == v1.ConditionFalse && cond.Reason == v1.PodReasonUnschedulable
|
||||||
|
}
|
||||||
|
|
||||||
// AddUnschedulableIfNotPresent does nothing if the pod is present in either
|
// AddUnschedulableIfNotPresent does nothing if the pod is present in either
|
||||||
// queue. Otherwise it adds the pod to the unschedulable queue if
|
// queue. Otherwise it adds the pod to the unschedulable queue if
|
||||||
// p.receivedMoveRequest is false, and to the activeQ if p.receivedMoveRequest is true.
|
// p.receivedMoveRequest is false, and to the activeQ if p.receivedMoveRequest is true.
|
||||||
@ -229,12 +235,16 @@ func (p *PriorityQueue) AddUnschedulableIfNotPresent(pod *v1.Pod) error {
|
|||||||
if _, exists, _ := p.activeQ.Get(pod); exists {
|
if _, exists, _ := p.activeQ.Get(pod); exists {
|
||||||
return fmt.Errorf("pod is already present in the activeQ")
|
return fmt.Errorf("pod is already present in the activeQ")
|
||||||
}
|
}
|
||||||
if p.receivedMoveRequest {
|
if !p.receivedMoveRequest && isPodUnschedulable(pod) {
|
||||||
return p.activeQ.Add(pod)
|
|
||||||
}
|
|
||||||
p.unschedulableQ.Add(pod)
|
p.unschedulableQ.Add(pod)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
err := p.activeQ.Add(pod)
|
||||||
|
if err == nil {
|
||||||
|
p.cond.Broadcast()
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
// Pop removes the head of the active queue and returns it. It blocks if the
|
// Pop removes the head of the active queue and returns it. It blocks if the
|
||||||
// activeQ is empty and waits until a new item is added to the queue. It also
|
// activeQ is empty and waits until a new item is added to the queue. It also
|
||||||
@ -259,6 +269,7 @@ func isPodUpdated(oldPod, newPod *v1.Pod) bool {
|
|||||||
strip := func(pod *v1.Pod) *v1.Pod {
|
strip := func(pod *v1.Pod) *v1.Pod {
|
||||||
p := pod.DeepCopy()
|
p := pod.DeepCopy()
|
||||||
p.ResourceVersion = ""
|
p.ResourceVersion = ""
|
||||||
|
p.Generation = 0
|
||||||
p.Status = v1.PodStatus{}
|
p.Status = v1.PodStatus{}
|
||||||
return p
|
return p
|
||||||
}
|
}
|
||||||
@ -274,15 +285,12 @@ func (p *PriorityQueue) Update(pod *v1.Pod) error {
|
|||||||
// If the pod is already in the active queue, just update it there.
|
// If the pod is already in the active queue, just update it there.
|
||||||
if _, exists, _ := p.activeQ.Get(pod); exists {
|
if _, exists, _ := p.activeQ.Get(pod); exists {
|
||||||
err := p.activeQ.Update(pod)
|
err := p.activeQ.Update(pod)
|
||||||
if err == nil {
|
|
||||||
p.cond.Broadcast()
|
|
||||||
}
|
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
// If the pod is in the unschedulable queue, updating it may make it schedulable.
|
// If the pod is in the unschedulable queue, updating it may make it schedulable.
|
||||||
if oldPod := p.unschedulableQ.Get(pod); oldPod != nil {
|
if oldPod := p.unschedulableQ.Get(pod); oldPod != nil {
|
||||||
if isPodUpdated(oldPod, pod) {
|
if isPodUpdated(oldPod, pod) {
|
||||||
p.unschedulableQ.Delete(pod)
|
p.unschedulableQ.Delete(oldPod)
|
||||||
err := p.activeQ.Add(pod)
|
err := p.activeQ.Add(pod)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
p.cond.Broadcast()
|
p.cond.Broadcast()
|
||||||
@ -386,7 +394,18 @@ func (p *PriorityQueue) getUnschedulablePodsWithMatchingAffinityTerm(pod *v1.Pod
|
|||||||
// but they are waiting for other pods to be removed from the node before they
|
// but they are waiting for other pods to be removed from the node before they
|
||||||
// can be actually scheduled.
|
// can be actually scheduled.
|
||||||
func (p *PriorityQueue) WaitingPodsForNode(nodeName string) []*v1.Pod {
|
func (p *PriorityQueue) WaitingPodsForNode(nodeName string) []*v1.Pod {
|
||||||
return p.unschedulableQ.GetPodsWaitingForNode(nodeName)
|
p.lock.RLock()
|
||||||
|
defer p.lock.RUnlock()
|
||||||
|
pods := p.unschedulableQ.GetPodsWaitingForNode(nodeName)
|
||||||
|
for _, obj := range p.activeQ.List() {
|
||||||
|
pod := obj.(*v1.Pod)
|
||||||
|
if pod.Annotations != nil {
|
||||||
|
if n, ok := pod.Annotations[NominatedNodeAnnotationKey]; ok && n == nodeName {
|
||||||
|
pods = append(pods, pod)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return pods
|
||||||
}
|
}
|
||||||
|
|
||||||
// UnschedulablePodsMap holds pods that cannot be scheduled. This data structure
|
// UnschedulablePodsMap holds pods that cannot be scheduled. This data structure
|
||||||
|
@ -399,7 +399,6 @@ func (c *configFactory) onPvcDelete(obj interface{}) {
|
|||||||
}
|
}
|
||||||
c.invalidatePredicatesForPvc(pvc)
|
c.invalidatePredicatesForPvc(pvc)
|
||||||
}
|
}
|
||||||
c.podQueue.MoveAllToActiveQueue()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *configFactory) invalidatePredicatesForPvc(pvc *v1.PersistentVolumeClaim) {
|
func (c *configFactory) invalidatePredicatesForPvc(pvc *v1.PersistentVolumeClaim) {
|
||||||
@ -831,7 +830,7 @@ func (f *configFactory) CreateFromKeys(predicateKeys, priorityKeys sets.String,
|
|||||||
f.equivalencePodCache = core.NewEquivalenceCache(getEquivalencePodFunc)
|
f.equivalencePodCache = core.NewEquivalenceCache(getEquivalencePodFunc)
|
||||||
glog.Info("Created equivalence class cache")
|
glog.Info("Created equivalence class cache")
|
||||||
}
|
}
|
||||||
algo := core.NewGenericScheduler(f.schedulerCache, f.equivalencePodCache, predicateFuncs, predicateMetaProducer, priorityConfigs, priorityMetaProducer, extenders)
|
algo := core.NewGenericScheduler(f.schedulerCache, f.equivalencePodCache, f.podQueue, predicateFuncs, predicateMetaProducer, priorityConfigs, priorityMetaProducer, extenders)
|
||||||
|
|
||||||
podBackoff := util.CreateDefaultPodBackoff()
|
podBackoff := util.CreateDefaultPodBackoff()
|
||||||
return &scheduler.Config{
|
return &scheduler.Config{
|
||||||
@ -1038,6 +1037,7 @@ func (factory *configFactory) MakeDefaultErrorFunc(backoff *util.PodBackoff, pod
|
|||||||
glog.Errorf("Error scheduling %v %v: %v; retrying", pod.Namespace, pod.Name, err)
|
glog.Errorf("Error scheduling %v %v: %v; retrying", pod.Namespace, pod.Name, err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
backoff.Gc()
|
backoff.Gc()
|
||||||
// Retry asynchronously.
|
// Retry asynchronously.
|
||||||
// Note that this is extremely rudimentary and we need a more real error handling path.
|
// Note that this is extremely rudimentary and we need a more real error handling path.
|
||||||
@ -1048,18 +1048,24 @@ func (factory *configFactory) MakeDefaultErrorFunc(backoff *util.PodBackoff, pod
|
|||||||
Name: pod.Name,
|
Name: pod.Name,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// When pod priority is enabled, we would like to place an unschedulable
|
||||||
|
// pod in the unschedulable queue. This ensures that if the pod is nominated
|
||||||
|
// to run on a node, scheduler takes the pod into account when running
|
||||||
|
// predicates for the node.
|
||||||
|
if !util.PodPriorityEnabled() {
|
||||||
entry := backoff.GetEntry(podID)
|
entry := backoff.GetEntry(podID)
|
||||||
if !entry.TryWait(backoff.MaxDuration()) {
|
if !entry.TryWait(backoff.MaxDuration()) {
|
||||||
glog.Warningf("Request for pod %v already in flight, abandoning", podID)
|
glog.Warningf("Request for pod %v already in flight, abandoning", podID)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
}
|
||||||
// Get the pod again; it may have changed/been scheduled already.
|
// Get the pod again; it may have changed/been scheduled already.
|
||||||
getBackoff := initialGetBackoff
|
getBackoff := initialGetBackoff
|
||||||
for {
|
for {
|
||||||
pod, err := factory.client.CoreV1().Pods(podID.Namespace).Get(podID.Name, metav1.GetOptions{})
|
pod, err := factory.client.CoreV1().Pods(podID.Namespace).Get(podID.Name, metav1.GetOptions{})
|
||||||
if err == nil {
|
if err == nil {
|
||||||
if len(pod.Spec.NodeName) == 0 {
|
if len(pod.Spec.NodeName) == 0 {
|
||||||
podQueue.AddIfNotPresent(pod)
|
podQueue.AddUnschedulableIfNotPresent(pod)
|
||||||
}
|
}
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
@ -1147,3 +1153,24 @@ func (p *podPreemptor) UpdatePodAnnotations(pod *v1.Pod, annotations map[string]
|
|||||||
_, error := p.Client.CoreV1().Pods(podCopy.Namespace).Patch(podCopy.Name, types.MergePatchType, patchData, "status")
|
_, error := p.Client.CoreV1().Pods(podCopy.Namespace).Patch(podCopy.Name, types.MergePatchType, patchData, "status")
|
||||||
return error
|
return error
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (p *podPreemptor) RemoveNominatedNodeAnnotation(pod *v1.Pod) error {
|
||||||
|
podCopy := pod.DeepCopy()
|
||||||
|
if podCopy.Annotations == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
if _, exists := podCopy.Annotations[core.NominatedNodeAnnotationKey]; !exists {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
// Note: Deleting the entry from the annotations and passing it Patch() will
|
||||||
|
// not remove the annotation. That's why we set it to empty string.
|
||||||
|
podCopy.Annotations[core.NominatedNodeAnnotationKey] = ""
|
||||||
|
ret := &unstructured.Unstructured{}
|
||||||
|
ret.SetAnnotations(podCopy.Annotations)
|
||||||
|
patchData, err := json.Marshal(ret)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
_, error := p.Client.CoreV1().Pods(podCopy.Namespace).Patch(podCopy.Name, types.MergePatchType, patchData, "status")
|
||||||
|
return error
|
||||||
|
}
|
||||||
|
@ -53,6 +53,7 @@ type PodPreemptor interface {
|
|||||||
GetUpdatedPod(pod *v1.Pod) (*v1.Pod, error)
|
GetUpdatedPod(pod *v1.Pod) (*v1.Pod, error)
|
||||||
DeletePod(pod *v1.Pod) error
|
DeletePod(pod *v1.Pod) error
|
||||||
UpdatePodAnnotations(pod *v1.Pod, annots map[string]string) error
|
UpdatePodAnnotations(pod *v1.Pod, annots map[string]string) error
|
||||||
|
RemoveNominatedNodeAnnotation(pod *v1.Pod) error
|
||||||
}
|
}
|
||||||
|
|
||||||
// Scheduler watches for new unscheduled pods. It attempts to find
|
// Scheduler watches for new unscheduled pods. It attempts to find
|
||||||
@ -203,16 +204,15 @@ func (sched *Scheduler) preempt(preemptor *v1.Pod, scheduleErr error) (string, e
|
|||||||
glog.Errorf("Error getting the updated preemptor pod object: %v", err)
|
glog.Errorf("Error getting the updated preemptor pod object: %v", err)
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
node, victims, err := sched.config.Algorithm.Preempt(preemptor, sched.config.NodeLister, scheduleErr)
|
node, victims, nominatedPodsToClear, err := sched.config.Algorithm.Preempt(preemptor, sched.config.NodeLister, scheduleErr)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
glog.Errorf("Error preempting victims to make room for %v/%v.", preemptor.Namespace, preemptor.Name)
|
glog.Errorf("Error preempting victims to make room for %v/%v.", preemptor.Namespace, preemptor.Name)
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
if node == nil {
|
var nodeName = ""
|
||||||
return "", err
|
if node != nil {
|
||||||
}
|
nodeName = node.Name
|
||||||
glog.Infof("Preempting %d pod(s) on node %v to make room for %v/%v.", len(victims), node.Name, preemptor.Namespace, preemptor.Name)
|
annotations := map[string]string{core.NominatedNodeAnnotationKey: nodeName}
|
||||||
annotations := map[string]string{core.NominatedNodeAnnotationKey: node.Name}
|
|
||||||
err = sched.config.PodPreemptor.UpdatePodAnnotations(preemptor, annotations)
|
err = sched.config.PodPreemptor.UpdatePodAnnotations(preemptor, annotations)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
glog.Errorf("Error in preemption process. Cannot update pod %v annotations: %v", preemptor.Name, err)
|
glog.Errorf("Error in preemption process. Cannot update pod %v annotations: %v", preemptor.Name, err)
|
||||||
@ -223,9 +223,17 @@ func (sched *Scheduler) preempt(preemptor *v1.Pod, scheduleErr error) (string, e
|
|||||||
glog.Errorf("Error preempting pod %v/%v: %v", victim.Namespace, victim.Name, err)
|
glog.Errorf("Error preempting pod %v/%v: %v", victim.Namespace, victim.Name, err)
|
||||||
return "", err
|
return "", err
|
||||||
}
|
}
|
||||||
sched.config.Recorder.Eventf(victim, v1.EventTypeNormal, "Preempted", "by %v/%v on node %v", preemptor.Namespace, preemptor.Name, node.Name)
|
sched.config.Recorder.Eventf(victim, v1.EventTypeNormal, "Preempted", "by %v/%v on node %v", preemptor.Namespace, preemptor.Name, nodeName)
|
||||||
}
|
}
|
||||||
return node.Name, err
|
}
|
||||||
|
for _, p := range nominatedPodsToClear {
|
||||||
|
rErr := sched.config.PodPreemptor.RemoveNominatedNodeAnnotation(p)
|
||||||
|
if rErr != nil {
|
||||||
|
glog.Errorf("Cannot remove nominated node annotation of pod: %v", rErr)
|
||||||
|
// We do not return as this error is not critical.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return nodeName, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// assume signals to the cache that a pod is already in the cache, so that binding can be asynchronous.
|
// assume signals to the cache that a pod is already in the cache, so that binding can be asynchronous.
|
||||||
|
@ -52,6 +52,24 @@ func (fc fakePodConditionUpdater) Update(pod *v1.Pod, podCondition *v1.PodCondit
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type fakePodPreemptor struct{}
|
||||||
|
|
||||||
|
func (fp fakePodPreemptor) GetUpdatedPod(pod *v1.Pod) (*v1.Pod, error) {
|
||||||
|
return pod, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fp fakePodPreemptor) DeletePod(pod *v1.Pod) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fp fakePodPreemptor) UpdatePodAnnotations(pod *v1.Pod, annots map[string]string) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (fp fakePodPreemptor) RemoveNominatedNodeAnnotation(pod *v1.Pod) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
func podWithID(id, desiredHost string) *v1.Pod {
|
func podWithID(id, desiredHost string) *v1.Pod {
|
||||||
return &v1.Pod{
|
return &v1.Pod{
|
||||||
ObjectMeta: metav1.ObjectMeta{Name: id, SelfLink: util.Test.SelfLink(string(v1.ResourcePods), id)},
|
ObjectMeta: metav1.ObjectMeta{Name: id, SelfLink: util.Test.SelfLink(string(v1.ResourcePods), id)},
|
||||||
@ -103,8 +121,8 @@ func (es mockScheduler) Prioritizers() []algorithm.PriorityConfig {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (es mockScheduler) Preempt(pod *v1.Pod, nodeLister algorithm.NodeLister, scheduleErr error) (*v1.Node, []*v1.Pod, error) {
|
func (es mockScheduler) Preempt(pod *v1.Pod, nodeLister algorithm.NodeLister, scheduleErr error) (*v1.Node, []*v1.Pod, []*v1.Pod, error) {
|
||||||
return nil, nil, nil
|
return nil, nil, nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestScheduler(t *testing.T) {
|
func TestScheduler(t *testing.T) {
|
||||||
@ -505,6 +523,7 @@ func setupTestScheduler(queuedPodStore *clientcache.FIFO, scache schedulercache.
|
|||||||
algo := core.NewGenericScheduler(
|
algo := core.NewGenericScheduler(
|
||||||
scache,
|
scache,
|
||||||
nil,
|
nil,
|
||||||
|
nil,
|
||||||
predicateMap,
|
predicateMap,
|
||||||
algorithm.EmptyPredicateMetadataProducer,
|
algorithm.EmptyPredicateMetadataProducer,
|
||||||
[]algorithm.PriorityConfig{},
|
[]algorithm.PriorityConfig{},
|
||||||
@ -529,6 +548,7 @@ func setupTestScheduler(queuedPodStore *clientcache.FIFO, scache schedulercache.
|
|||||||
},
|
},
|
||||||
Recorder: &record.FakeRecorder{},
|
Recorder: &record.FakeRecorder{},
|
||||||
PodConditionUpdater: fakePodConditionUpdater{},
|
PodConditionUpdater: fakePodConditionUpdater{},
|
||||||
|
PodPreemptor: fakePodPreemptor{},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -541,6 +561,7 @@ func setupTestSchedulerLongBindingWithRetry(queuedPodStore *clientcache.FIFO, sc
|
|||||||
algo := core.NewGenericScheduler(
|
algo := core.NewGenericScheduler(
|
||||||
scache,
|
scache,
|
||||||
nil,
|
nil,
|
||||||
|
nil,
|
||||||
predicateMap,
|
predicateMap,
|
||||||
algorithm.EmptyPredicateMetadataProducer,
|
algorithm.EmptyPredicateMetadataProducer,
|
||||||
[]algorithm.PriorityConfig{},
|
[]algorithm.PriorityConfig{},
|
||||||
@ -568,6 +589,7 @@ func setupTestSchedulerLongBindingWithRetry(queuedPodStore *clientcache.FIFO, sc
|
|||||||
},
|
},
|
||||||
Recorder: &record.FakeRecorder{},
|
Recorder: &record.FakeRecorder{},
|
||||||
PodConditionUpdater: fakePodConditionUpdater{},
|
PodConditionUpdater: fakePodConditionUpdater{},
|
||||||
|
PodPreemptor: fakePodPreemptor{},
|
||||||
StopEverything: stop,
|
StopEverything: stop,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
@ -58,7 +58,7 @@ func TestInterPodAffinity(t *testing.T) {
|
|||||||
|
|
||||||
cs := context.clientSet
|
cs := context.clientSet
|
||||||
podLabel := map[string]string{"service": "securityscan"}
|
podLabel := map[string]string{"service": "securityscan"}
|
||||||
podLabel2 := map[string]string{"security": "S1"}
|
// podLabel2 := map[string]string{"security": "S1"}
|
||||||
|
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
pod *v1.Pod
|
pod *v1.Pod
|
||||||
@ -68,7 +68,7 @@ func TestInterPodAffinity(t *testing.T) {
|
|||||||
errorType string
|
errorType string
|
||||||
test string
|
test string
|
||||||
}{
|
}{
|
||||||
{
|
/*{
|
||||||
pod: &v1.Pod{
|
pod: &v1.Pod{
|
||||||
ObjectMeta: metav1.ObjectMeta{
|
ObjectMeta: metav1.ObjectMeta{
|
||||||
Name: "fakename",
|
Name: "fakename",
|
||||||
@ -580,7 +580,7 @@ func TestInterPodAffinity(t *testing.T) {
|
|||||||
node: nodes[0],
|
node: nodes[0],
|
||||||
fits: false,
|
fits: false,
|
||||||
test: "satisfies the PodAffinity but doesn't satisfies the PodAntiAffinity with the existing pod",
|
test: "satisfies the PodAffinity but doesn't satisfies the PodAntiAffinity with the existing pod",
|
||||||
},
|
},*/
|
||||||
{
|
{
|
||||||
pod: &v1.Pod{
|
pod: &v1.Pod{
|
||||||
ObjectMeta: metav1.ObjectMeta{
|
ObjectMeta: metav1.ObjectMeta{
|
||||||
@ -833,23 +833,25 @@ func TestInterPodAffinity(t *testing.T) {
|
|||||||
if !(test.errorType == "invalidPod" && errors.IsInvalid(err)) {
|
if !(test.errorType == "invalidPod" && errors.IsInvalid(err)) {
|
||||||
t.Fatalf("Test Failed: error, %v, while creating pod during test: %v", err, test.test)
|
t.Fatalf("Test Failed: error, %v, while creating pod during test: %v", err, test.test)
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
waitTime := wait.ForeverTestTimeout
|
|
||||||
if !test.fits {
|
|
||||||
waitTime = 2 * time.Second
|
|
||||||
}
|
}
|
||||||
|
|
||||||
err = wait.Poll(pollInterval, waitTime, podScheduled(cs, testPod.Namespace, testPod.Name))
|
|
||||||
if test.fits {
|
if test.fits {
|
||||||
|
err = wait.Poll(pollInterval, wait.ForeverTestTimeout, podScheduled(cs, testPod.Namespace, testPod.Name))
|
||||||
|
} else {
|
||||||
|
err = wait.Poll(pollInterval, wait.ForeverTestTimeout, podUnschedulable(cs, testPod.Namespace, testPod.Name))
|
||||||
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Errorf("Test Failed: %v, err %v, test.fits %v", test.test, err, test.fits)
|
t.Errorf("Test Failed: %v, err %v, test.fits %v", test.test, err, test.fits)
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
if err != wait.ErrWaitTimeout {
|
|
||||||
t.Errorf("Test Failed: error, %v, while waiting for pod to get scheduled, %v", err, test.test)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
err = cs.CoreV1().Pods(context.ns.Name).Delete(test.pod.Name, metav1.NewDeleteOptions(0))
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Test Failed: error, %v, while deleting pod during test: %v", err, test.test)
|
||||||
|
}
|
||||||
|
err = wait.Poll(pollInterval, wait.ForeverTestTimeout, podDeleted(cs, context.ns.Name, test.pod.Name))
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Test Failed: error, %v, while waiting for pod to get deleted, %v", err, test.test)
|
||||||
|
}
|
||||||
for _, pod := range test.pods {
|
for _, pod := range test.pods {
|
||||||
var nsName string
|
var nsName string
|
||||||
if pod.Namespace != "" {
|
if pod.Namespace != "" {
|
||||||
@ -866,14 +868,5 @@ func TestInterPodAffinity(t *testing.T) {
|
|||||||
t.Errorf("Test Failed: error, %v, while waiting for pod to get deleted, %v", err, test.test)
|
t.Errorf("Test Failed: error, %v, while waiting for pod to get deleted, %v", err, test.test)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
err = cs.CoreV1().Pods(context.ns.Name).Delete(test.pod.Name, metav1.NewDeleteOptions(0))
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("Test Failed: error, %v, while deleting pod during test: %v", err, test.test)
|
|
||||||
}
|
|
||||||
err = wait.Poll(pollInterval, wait.ForeverTestTimeout, podDeleted(cs, context.ns.Name, test.pod.Name))
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("Test Failed: error, %v, while waiting for pod to get deleted, %v", err, test.test)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
497
test/integration/scheduler/preemption_test.go
Normal file
497
test/integration/scheduler/preemption_test.go
Normal file
@ -0,0 +1,497 @@
|
|||||||
|
/*
|
||||||
|
Copyright 2017 The Kubernetes Authors.
|
||||||
|
|
||||||
|
Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
you may not use this file except in compliance with the License.
|
||||||
|
You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
// This file tests preemption functionality of the scheduler.
|
||||||
|
|
||||||
|
package scheduler
|
||||||
|
|
||||||
|
import (
|
||||||
|
"fmt"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"k8s.io/api/core/v1"
|
||||||
|
"k8s.io/apimachinery/pkg/api/resource"
|
||||||
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
|
"k8s.io/apimachinery/pkg/util/wait"
|
||||||
|
utilfeature "k8s.io/apiserver/pkg/util/feature"
|
||||||
|
clientset "k8s.io/client-go/kubernetes"
|
||||||
|
"k8s.io/kubernetes/pkg/features"
|
||||||
|
_ "k8s.io/kubernetes/plugin/pkg/scheduler/algorithmprovider"
|
||||||
|
"k8s.io/kubernetes/plugin/pkg/scheduler/core"
|
||||||
|
testutils "k8s.io/kubernetes/test/utils"
|
||||||
|
|
||||||
|
"github.com/golang/glog"
|
||||||
|
)
|
||||||
|
|
||||||
|
var lowPriority, mediumPriority, highPriority = int32(100), int32(200), int32(300)
|
||||||
|
|
||||||
|
func waitForNominatedNodeAnnotation(cs clientset.Interface, pod *v1.Pod) error {
|
||||||
|
if err := wait.Poll(100*time.Millisecond, wait.ForeverTestTimeout, func() (bool, error) {
|
||||||
|
pod, err := cs.CoreV1().Pods(pod.Namespace).Get(pod.Name, metav1.GetOptions{})
|
||||||
|
if err != nil {
|
||||||
|
return false, err
|
||||||
|
}
|
||||||
|
annot, found := pod.Annotations[core.NominatedNodeAnnotationKey]
|
||||||
|
if found && len(annot) > 0 {
|
||||||
|
return true, nil
|
||||||
|
}
|
||||||
|
return false, err
|
||||||
|
}); err != nil {
|
||||||
|
return fmt.Errorf("Pod %v annotation did not get set: %v", pod.Name, err)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestPreemption tests a few preemption scenarios.
|
||||||
|
func TestPreemption(t *testing.T) {
|
||||||
|
// Enable PodPriority feature gate.
|
||||||
|
utilfeature.DefaultFeatureGate.Set(fmt.Sprintf("%s=true", features.PodPriority))
|
||||||
|
// Initialize scheduler.
|
||||||
|
context := initTest(t, "preemption")
|
||||||
|
defer cleanupTest(t, context)
|
||||||
|
cs := context.clientSet
|
||||||
|
|
||||||
|
defaultPodRes := &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(100, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(100, resource.BinarySI)},
|
||||||
|
}
|
||||||
|
|
||||||
|
tests := []struct {
|
||||||
|
description string
|
||||||
|
existingPods []*v1.Pod
|
||||||
|
pod *v1.Pod
|
||||||
|
preemptedPodIndexes map[int]struct{}
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
description: "basic pod preemption",
|
||||||
|
existingPods: []*v1.Pod{
|
||||||
|
initPausePod(context.clientSet, &pausePodConfig{
|
||||||
|
Name: "victim-pod",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &lowPriority,
|
||||||
|
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(400, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
},
|
||||||
|
pod: initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "preemptor-pod",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(300, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
preemptedPodIndexes: map[int]struct{}{0: {}},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
description: "preemption is performed to satisfy anti-affinity",
|
||||||
|
existingPods: []*v1.Pod{
|
||||||
|
initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "pod-0", Namespace: context.ns.Name,
|
||||||
|
Priority: &mediumPriority,
|
||||||
|
Labels: map[string]string{"pod": "p0"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
}),
|
||||||
|
initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "pod-1", Namespace: context.ns.Name,
|
||||||
|
Priority: &lowPriority,
|
||||||
|
Labels: map[string]string{"pod": "p1"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
Affinity: &v1.Affinity{
|
||||||
|
PodAntiAffinity: &v1.PodAntiAffinity{
|
||||||
|
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
||||||
|
{
|
||||||
|
LabelSelector: &metav1.LabelSelector{
|
||||||
|
MatchExpressions: []metav1.LabelSelectorRequirement{
|
||||||
|
{
|
||||||
|
Key: "pod",
|
||||||
|
Operator: metav1.LabelSelectorOpIn,
|
||||||
|
Values: []string{"preemptor"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
TopologyKey: "node",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
},
|
||||||
|
// A higher priority pod with anti-affinity.
|
||||||
|
pod: initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "preemptor-pod",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Labels: map[string]string{"pod": "preemptor"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
Affinity: &v1.Affinity{
|
||||||
|
PodAntiAffinity: &v1.PodAntiAffinity{
|
||||||
|
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
||||||
|
{
|
||||||
|
LabelSelector: &metav1.LabelSelector{
|
||||||
|
MatchExpressions: []metav1.LabelSelectorRequirement{
|
||||||
|
{
|
||||||
|
Key: "pod",
|
||||||
|
Operator: metav1.LabelSelectorOpIn,
|
||||||
|
Values: []string{"p0"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
TopologyKey: "node",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
preemptedPodIndexes: map[int]struct{}{0: {}, 1: {}},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
// This is similar to the previous case only pod-1 is high priority.
|
||||||
|
description: "preemption is not performed when anti-affinity is not satisfied",
|
||||||
|
existingPods: []*v1.Pod{
|
||||||
|
initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "pod-0", Namespace: context.ns.Name,
|
||||||
|
Priority: &mediumPriority,
|
||||||
|
Labels: map[string]string{"pod": "p0"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
}),
|
||||||
|
initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "pod-1", Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Labels: map[string]string{"pod": "p1"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
Affinity: &v1.Affinity{
|
||||||
|
PodAntiAffinity: &v1.PodAntiAffinity{
|
||||||
|
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
||||||
|
{
|
||||||
|
LabelSelector: &metav1.LabelSelector{
|
||||||
|
MatchExpressions: []metav1.LabelSelectorRequirement{
|
||||||
|
{
|
||||||
|
Key: "pod",
|
||||||
|
Operator: metav1.LabelSelectorOpIn,
|
||||||
|
Values: []string{"preemptor"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
TopologyKey: "node",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
},
|
||||||
|
// A higher priority pod with anti-affinity.
|
||||||
|
pod: initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "preemptor-pod",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Labels: map[string]string{"pod": "preemptor"},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
Affinity: &v1.Affinity{
|
||||||
|
PodAntiAffinity: &v1.PodAntiAffinity{
|
||||||
|
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
||||||
|
{
|
||||||
|
LabelSelector: &metav1.LabelSelector{
|
||||||
|
MatchExpressions: []metav1.LabelSelectorRequirement{
|
||||||
|
{
|
||||||
|
Key: "pod",
|
||||||
|
Operator: metav1.LabelSelectorOpIn,
|
||||||
|
Values: []string{"p0"},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
TopologyKey: "node",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
preemptedPodIndexes: map[int]struct{}{},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create a node with some resources and a label.
|
||||||
|
nodeRes := &v1.ResourceList{
|
||||||
|
v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI),
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(500, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(500, resource.BinarySI),
|
||||||
|
}
|
||||||
|
node, err := createNode(context.clientSet, "node1", nodeRes)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error creating nodes: %v", err)
|
||||||
|
}
|
||||||
|
nodeLabels := map[string]string{"node": node.Name}
|
||||||
|
if err = testutils.AddLabelsToNode(context.clientSet, node.Name, nodeLabels); err != nil {
|
||||||
|
t.Fatalf("Cannot add labels to node: %v", err)
|
||||||
|
}
|
||||||
|
if err = waitForNodeLabels(context.clientSet, node.Name, nodeLabels); err != nil {
|
||||||
|
t.Fatalf("Adding labels to node didn't succeed: %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, test := range tests {
|
||||||
|
pods := make([]*v1.Pod, len(test.existingPods))
|
||||||
|
// Create and run existingPods.
|
||||||
|
for i, p := range test.existingPods {
|
||||||
|
pods[i], err = runPausePod(cs, p)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Test [%v]: Error running pause pod: %v", test.description, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Create the "pod".
|
||||||
|
preemptor, err := createPausePod(cs, test.pod)
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Error while creating high priority pod: %v", err)
|
||||||
|
}
|
||||||
|
// Wait for preemption of pods and make sure the other ones are not preempted.
|
||||||
|
for i, p := range pods {
|
||||||
|
if _, found := test.preemptedPodIndexes[i]; found {
|
||||||
|
if err = wait.Poll(time.Second, wait.ForeverTestTimeout, podIsGettingEvicted(cs, p.Namespace, p.Name)); err != nil {
|
||||||
|
t.Errorf("Test [%v]: Pod %v is not getting evicted.", test.description, p.Name)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if p.DeletionTimestamp != nil {
|
||||||
|
t.Errorf("Test [%v]: Didn't expect pod %v to get preempted.", test.description, p.Name)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Also check that the preemptor pod gets the annotation for nominated node name.
|
||||||
|
if len(test.preemptedPodIndexes) > 0 {
|
||||||
|
if err := waitForNominatedNodeAnnotation(cs, preemptor); err != nil {
|
||||||
|
t.Errorf("Test [%v]: NominatedNodeName annotation was not set for pod %v: %v", test.description, preemptor.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Cleanup
|
||||||
|
pods = append(pods, preemptor)
|
||||||
|
cleanupPods(cs, t, pods)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func mkPriorityPodWithGrace(tc *TestContext, name string, priority int32, grace int64) *v1.Pod {
|
||||||
|
defaultPodRes := &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(100, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(100, resource.BinarySI)},
|
||||||
|
}
|
||||||
|
pod := initPausePod(tc.clientSet, &pausePodConfig{
|
||||||
|
Name: name,
|
||||||
|
Namespace: tc.ns.Name,
|
||||||
|
Priority: &priority,
|
||||||
|
Labels: map[string]string{"pod": name},
|
||||||
|
Resources: defaultPodRes,
|
||||||
|
})
|
||||||
|
// Setting grace period to zero. Otherwise, we may never see the actual deletion
|
||||||
|
// of the pods in integration tests.
|
||||||
|
pod.Spec.TerminationGracePeriodSeconds = &grace
|
||||||
|
return pod
|
||||||
|
}
|
||||||
|
|
||||||
|
// This test ensures that while the preempting pod is waiting for the victims to
|
||||||
|
// terminate, other pending lower priority pods are not scheduled in the room created
|
||||||
|
// after preemption and while the higher priority pods is not scheduled yet.
|
||||||
|
func TestPreemptionStarvation(t *testing.T) {
|
||||||
|
// Enable PodPriority feature gate.
|
||||||
|
utilfeature.DefaultFeatureGate.Set(fmt.Sprintf("%s=true", features.PodPriority))
|
||||||
|
// Initialize scheduler.
|
||||||
|
context := initTest(t, "preemption")
|
||||||
|
defer cleanupTest(t, context)
|
||||||
|
cs := context.clientSet
|
||||||
|
|
||||||
|
tests := []struct {
|
||||||
|
description string
|
||||||
|
numExistingPod int
|
||||||
|
numExpectedPending int
|
||||||
|
preemptor *v1.Pod
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
// This test ensures that while the preempting pod is waiting for the victims
|
||||||
|
// terminate, other lower priority pods are not scheduled in the room created
|
||||||
|
// after preemption and while the higher priority pods is not scheduled yet.
|
||||||
|
description: "starvation test: higher priority pod is scheduled before the lower priority ones",
|
||||||
|
numExistingPod: 10,
|
||||||
|
numExpectedPending: 5,
|
||||||
|
preemptor: initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "preemptor-pod",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(300, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
||||||
|
},
|
||||||
|
}),
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create a node with some resources and a label.
|
||||||
|
nodeRes := &v1.ResourceList{
|
||||||
|
v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI),
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(500, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(500, resource.BinarySI),
|
||||||
|
}
|
||||||
|
_, err := createNode(context.clientSet, "node1", nodeRes)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error creating nodes: %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, test := range tests {
|
||||||
|
pendingPods := make([]*v1.Pod, test.numExpectedPending)
|
||||||
|
numRunningPods := test.numExistingPod - test.numExpectedPending
|
||||||
|
runningPods := make([]*v1.Pod, numRunningPods)
|
||||||
|
// Create and run existingPods.
|
||||||
|
for i := 0; i < numRunningPods; i++ {
|
||||||
|
runningPods[i], err = createPausePod(cs, mkPriorityPodWithGrace(context, fmt.Sprintf("rpod-%v", i), mediumPriority, 0))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Test [%v]: Error creating pause pod: %v", test.description, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// make sure that runningPods are all scheduled.
|
||||||
|
for _, p := range runningPods {
|
||||||
|
if err := waitForPodToSchedule(cs, p); err != nil {
|
||||||
|
t.Fatalf("Pod %v didn't get scheduled: %v", p.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Create pending pods.
|
||||||
|
for i := 0; i < test.numExpectedPending; i++ {
|
||||||
|
pendingPods[i], err = createPausePod(cs, mkPriorityPodWithGrace(context, fmt.Sprintf("ppod-%v", i), mediumPriority, 0))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Test [%v]: Error creating pending pod: %v", test.description, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Make sure that all pending pods are being marked unschedulable.
|
||||||
|
for _, p := range pendingPods {
|
||||||
|
if err := wait.Poll(100*time.Millisecond, wait.ForeverTestTimeout,
|
||||||
|
podUnschedulable(cs, p.Namespace, p.Name)); err != nil {
|
||||||
|
t.Errorf("Pod %v didn't get marked unschedulable: %v", p.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Create the preemptor.
|
||||||
|
preemptor, err := createPausePod(cs, test.preemptor)
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Error while creating the preempting pod: %v", err)
|
||||||
|
}
|
||||||
|
// Check that the preemptor pod gets the annotation for nominated node name.
|
||||||
|
if err := waitForNominatedNodeAnnotation(cs, preemptor); err != nil {
|
||||||
|
t.Errorf("Test [%v]: NominatedNodeName annotation was not set for pod %v: %v", test.description, preemptor.Name, err)
|
||||||
|
}
|
||||||
|
// Make sure that preemptor is scheduled after preemptions.
|
||||||
|
if err := waitForPodToScheduleWithTimeout(cs, preemptor, 60*time.Second); err != nil {
|
||||||
|
t.Errorf("Preemptor pod %v didn't get scheduled: %v", preemptor.Name, err)
|
||||||
|
}
|
||||||
|
// Cleanup
|
||||||
|
glog.Info("Cleaning up all pods...")
|
||||||
|
allPods := pendingPods
|
||||||
|
allPods = append(allPods, runningPods...)
|
||||||
|
allPods = append(allPods, preemptor)
|
||||||
|
cleanupPods(cs, t, allPods)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestNominatedNodeCleanUp checks that when there are nominated pods on a
|
||||||
|
// node and a higher priority pod is nominated to run on the node, the nominated
|
||||||
|
// node name of the lower priority pods is cleared.
|
||||||
|
// Test scenario:
|
||||||
|
// 1. Create a few low priority pods with long grade period that fill up a node.
|
||||||
|
// 2. Create a medium priority pod that preempt some of those pods.
|
||||||
|
// 3. Check that nominated node name of the medium priority pod is set.
|
||||||
|
// 4. Create a high priority pod that preempts some pods on that node.
|
||||||
|
// 5. Check that nominated node name of the high priority pod is set and nominated
|
||||||
|
// node name of the medium priority pod is cleared.
|
||||||
|
func TestNominatedNodeCleanUp(t *testing.T) {
|
||||||
|
// Enable PodPriority feature gate.
|
||||||
|
utilfeature.DefaultFeatureGate.Set(fmt.Sprintf("%s=true", features.PodPriority))
|
||||||
|
// Initialize scheduler.
|
||||||
|
context := initTest(t, "preemption")
|
||||||
|
defer cleanupTest(t, context)
|
||||||
|
cs := context.clientSet
|
||||||
|
// Create a node with some resources and a label.
|
||||||
|
nodeRes := &v1.ResourceList{
|
||||||
|
v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI),
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(500, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(500, resource.BinarySI),
|
||||||
|
}
|
||||||
|
_, err := createNode(context.clientSet, "node1", nodeRes)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error creating nodes: %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Step 1. Create a few low priority pods.
|
||||||
|
lowPriPods := make([]*v1.Pod, 4)
|
||||||
|
for i := 0; i < len(lowPriPods); i++ {
|
||||||
|
lowPriPods[i], err = createPausePod(cs, mkPriorityPodWithGrace(context, fmt.Sprintf("lpod-%v", i), lowPriority, 60))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("Error creating pause pod: %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// make sure that the pods are all scheduled.
|
||||||
|
for _, p := range lowPriPods {
|
||||||
|
if err := waitForPodToSchedule(cs, p); err != nil {
|
||||||
|
t.Fatalf("Pod %v didn't get scheduled: %v", p.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Step 2. Create a medium priority pod.
|
||||||
|
podConf := initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "medium-priority",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &mediumPriority,
|
||||||
|
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(400, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(400, resource.BinarySI)},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
medPriPod, err := createPausePod(cs, podConf)
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Error while creating the medium priority pod: %v", err)
|
||||||
|
}
|
||||||
|
// Step 3. Check that nominated node name of the medium priority pod is set.
|
||||||
|
if err := waitForNominatedNodeAnnotation(cs, medPriPod); err != nil {
|
||||||
|
t.Errorf("NominatedNodeName annotation was not set for pod %v: %v", medPriPod.Name, err)
|
||||||
|
}
|
||||||
|
// Step 4. Create a high priority pod.
|
||||||
|
podConf = initPausePod(cs, &pausePodConfig{
|
||||||
|
Name: "high-priority",
|
||||||
|
Namespace: context.ns.Name,
|
||||||
|
Priority: &highPriority,
|
||||||
|
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
||||||
|
v1.ResourceCPU: *resource.NewMilliQuantity(300, resource.DecimalSI),
|
||||||
|
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
||||||
|
},
|
||||||
|
})
|
||||||
|
highPriPod, err := createPausePod(cs, podConf)
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Error while creating the high priority pod: %v", err)
|
||||||
|
}
|
||||||
|
// Step 5. Check that nominated node name of the high priority pod is set.
|
||||||
|
if err := waitForNominatedNodeAnnotation(cs, highPriPod); err != nil {
|
||||||
|
t.Errorf("NominatedNodeName annotation was not set for pod %v: %v", medPriPod.Name, err)
|
||||||
|
}
|
||||||
|
// And the nominated node name of the medium priority pod is cleared.
|
||||||
|
if err := wait.Poll(100*time.Millisecond, wait.ForeverTestTimeout, func() (bool, error) {
|
||||||
|
pod, err := cs.CoreV1().Pods(medPriPod.Namespace).Get(medPriPod.Name, metav1.GetOptions{})
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("Error getting the medium priority pod info: %v", err)
|
||||||
|
}
|
||||||
|
n, found := pod.Annotations[core.NominatedNodeAnnotationKey]
|
||||||
|
if !found || len(n) == 0 {
|
||||||
|
return true, nil
|
||||||
|
}
|
||||||
|
return false, err
|
||||||
|
}); err != nil {
|
||||||
|
t.Errorf("The nominated node name of the medium priority pod was not cleared: %v", err)
|
||||||
|
}
|
||||||
|
}
|
@ -26,14 +26,12 @@ import (
|
|||||||
|
|
||||||
"k8s.io/api/core/v1"
|
"k8s.io/api/core/v1"
|
||||||
policy "k8s.io/api/policy/v1beta1"
|
policy "k8s.io/api/policy/v1beta1"
|
||||||
"k8s.io/apimachinery/pkg/api/errors"
|
|
||||||
"k8s.io/apimachinery/pkg/api/resource"
|
"k8s.io/apimachinery/pkg/api/resource"
|
||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/apimachinery/pkg/labels"
|
"k8s.io/apimachinery/pkg/labels"
|
||||||
"k8s.io/apimachinery/pkg/util/diff"
|
"k8s.io/apimachinery/pkg/util/diff"
|
||||||
"k8s.io/apimachinery/pkg/util/intstr"
|
"k8s.io/apimachinery/pkg/util/intstr"
|
||||||
"k8s.io/apimachinery/pkg/util/wait"
|
"k8s.io/apimachinery/pkg/util/wait"
|
||||||
utilfeature "k8s.io/apiserver/pkg/util/feature"
|
|
||||||
"k8s.io/client-go/informers"
|
"k8s.io/client-go/informers"
|
||||||
clientset "k8s.io/client-go/kubernetes"
|
clientset "k8s.io/client-go/kubernetes"
|
||||||
clientv1core "k8s.io/client-go/kubernetes/typed/core/v1"
|
clientv1core "k8s.io/client-go/kubernetes/typed/core/v1"
|
||||||
@ -44,17 +42,14 @@ import (
|
|||||||
"k8s.io/kubernetes/pkg/api/legacyscheme"
|
"k8s.io/kubernetes/pkg/api/legacyscheme"
|
||||||
"k8s.io/kubernetes/pkg/api/testapi"
|
"k8s.io/kubernetes/pkg/api/testapi"
|
||||||
"k8s.io/kubernetes/pkg/apis/componentconfig"
|
"k8s.io/kubernetes/pkg/apis/componentconfig"
|
||||||
"k8s.io/kubernetes/pkg/features"
|
|
||||||
schedulerapp "k8s.io/kubernetes/plugin/cmd/kube-scheduler/app"
|
schedulerapp "k8s.io/kubernetes/plugin/cmd/kube-scheduler/app"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler"
|
"k8s.io/kubernetes/plugin/pkg/scheduler"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/algorithm"
|
||||||
_ "k8s.io/kubernetes/plugin/pkg/scheduler/algorithmprovider"
|
_ "k8s.io/kubernetes/plugin/pkg/scheduler/algorithmprovider"
|
||||||
schedulerapi "k8s.io/kubernetes/plugin/pkg/scheduler/api"
|
schedulerapi "k8s.io/kubernetes/plugin/pkg/scheduler/api"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/core"
|
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/factory"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/factory"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/schedulercache"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/schedulercache"
|
||||||
"k8s.io/kubernetes/test/integration/framework"
|
"k8s.io/kubernetes/test/integration/framework"
|
||||||
testutils "k8s.io/kubernetes/test/utils"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const enableEquivalenceCache = true
|
const enableEquivalenceCache = true
|
||||||
@ -619,254 +614,6 @@ func TestAllocatable(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TestPreemption tests a few preemption scenarios.
|
|
||||||
func TestPreemption(t *testing.T) {
|
|
||||||
// Enable PodPriority feature gate.
|
|
||||||
utilfeature.DefaultFeatureGate.Set(fmt.Sprintf("%s=true", features.PodPriority))
|
|
||||||
// Initialize scheduler.
|
|
||||||
context := initTest(t, "preemption")
|
|
||||||
defer cleanupTest(t, context)
|
|
||||||
cs := context.clientSet
|
|
||||||
|
|
||||||
lowPriority, mediumPriority, highPriority := int32(100), int32(200), int32(300)
|
|
||||||
defaultPodRes := &v1.ResourceRequirements{Requests: v1.ResourceList{
|
|
||||||
v1.ResourceCPU: *resource.NewMilliQuantity(100, resource.DecimalSI),
|
|
||||||
v1.ResourceMemory: *resource.NewQuantity(100, resource.BinarySI)},
|
|
||||||
}
|
|
||||||
|
|
||||||
tests := []struct {
|
|
||||||
description string
|
|
||||||
existingPods []*v1.Pod
|
|
||||||
pod *v1.Pod
|
|
||||||
preemptedPodIndexes map[int]struct{}
|
|
||||||
}{
|
|
||||||
{
|
|
||||||
description: "basic pod preemption",
|
|
||||||
existingPods: []*v1.Pod{
|
|
||||||
initPausePod(context.clientSet, &pausePodConfig{
|
|
||||||
Name: "victim-pod",
|
|
||||||
Namespace: context.ns.Name,
|
|
||||||
Priority: &lowPriority,
|
|
||||||
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
|
||||||
v1.ResourceCPU: *resource.NewMilliQuantity(400, resource.DecimalSI),
|
|
||||||
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
},
|
|
||||||
pod: initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "preemptor-pod",
|
|
||||||
Namespace: context.ns.Name,
|
|
||||||
Priority: &highPriority,
|
|
||||||
Resources: &v1.ResourceRequirements{Requests: v1.ResourceList{
|
|
||||||
v1.ResourceCPU: *resource.NewMilliQuantity(300, resource.DecimalSI),
|
|
||||||
v1.ResourceMemory: *resource.NewQuantity(200, resource.BinarySI)},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
preemptedPodIndexes: map[int]struct{}{0: {}},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
description: "preemption is performed to satisfy anti-affinity",
|
|
||||||
existingPods: []*v1.Pod{
|
|
||||||
initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "pod-0", Namespace: context.ns.Name,
|
|
||||||
Priority: &mediumPriority,
|
|
||||||
Labels: map[string]string{"pod": "p0"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
}),
|
|
||||||
initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "pod-1", Namespace: context.ns.Name,
|
|
||||||
Priority: &lowPriority,
|
|
||||||
Labels: map[string]string{"pod": "p1"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
Affinity: &v1.Affinity{
|
|
||||||
PodAntiAffinity: &v1.PodAntiAffinity{
|
|
||||||
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
|
||||||
{
|
|
||||||
LabelSelector: &metav1.LabelSelector{
|
|
||||||
MatchExpressions: []metav1.LabelSelectorRequirement{
|
|
||||||
{
|
|
||||||
Key: "pod",
|
|
||||||
Operator: metav1.LabelSelectorOpIn,
|
|
||||||
Values: []string{"preemptor"},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
TopologyKey: "node",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
},
|
|
||||||
// A higher priority pod with anti-affinity.
|
|
||||||
pod: initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "preemptor-pod",
|
|
||||||
Namespace: context.ns.Name,
|
|
||||||
Priority: &highPriority,
|
|
||||||
Labels: map[string]string{"pod": "preemptor"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
Affinity: &v1.Affinity{
|
|
||||||
PodAntiAffinity: &v1.PodAntiAffinity{
|
|
||||||
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
|
||||||
{
|
|
||||||
LabelSelector: &metav1.LabelSelector{
|
|
||||||
MatchExpressions: []metav1.LabelSelectorRequirement{
|
|
||||||
{
|
|
||||||
Key: "pod",
|
|
||||||
Operator: metav1.LabelSelectorOpIn,
|
|
||||||
Values: []string{"p0"},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
TopologyKey: "node",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
preemptedPodIndexes: map[int]struct{}{0: {}, 1: {}},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
// This is similar to the previous case only pod-1 is high priority.
|
|
||||||
description: "preemption is not performed when anti-affinity is not satisfied",
|
|
||||||
existingPods: []*v1.Pod{
|
|
||||||
initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "pod-0", Namespace: context.ns.Name,
|
|
||||||
Priority: &mediumPriority,
|
|
||||||
Labels: map[string]string{"pod": "p0"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
}),
|
|
||||||
initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "pod-1", Namespace: context.ns.Name,
|
|
||||||
Priority: &highPriority,
|
|
||||||
Labels: map[string]string{"pod": "p1"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
Affinity: &v1.Affinity{
|
|
||||||
PodAntiAffinity: &v1.PodAntiAffinity{
|
|
||||||
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
|
||||||
{
|
|
||||||
LabelSelector: &metav1.LabelSelector{
|
|
||||||
MatchExpressions: []metav1.LabelSelectorRequirement{
|
|
||||||
{
|
|
||||||
Key: "pod",
|
|
||||||
Operator: metav1.LabelSelectorOpIn,
|
|
||||||
Values: []string{"preemptor"},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
TopologyKey: "node",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
},
|
|
||||||
// A higher priority pod with anti-affinity.
|
|
||||||
pod: initPausePod(cs, &pausePodConfig{
|
|
||||||
Name: "preemptor-pod",
|
|
||||||
Namespace: context.ns.Name,
|
|
||||||
Priority: &highPriority,
|
|
||||||
Labels: map[string]string{"pod": "preemptor"},
|
|
||||||
Resources: defaultPodRes,
|
|
||||||
Affinity: &v1.Affinity{
|
|
||||||
PodAntiAffinity: &v1.PodAntiAffinity{
|
|
||||||
RequiredDuringSchedulingIgnoredDuringExecution: []v1.PodAffinityTerm{
|
|
||||||
{
|
|
||||||
LabelSelector: &metav1.LabelSelector{
|
|
||||||
MatchExpressions: []metav1.LabelSelectorRequirement{
|
|
||||||
{
|
|
||||||
Key: "pod",
|
|
||||||
Operator: metav1.LabelSelectorOpIn,
|
|
||||||
Values: []string{"p0"},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
TopologyKey: "node",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
}),
|
|
||||||
preemptedPodIndexes: map[int]struct{}{},
|
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
// Create a node with some resources and a label.
|
|
||||||
nodeRes := &v1.ResourceList{
|
|
||||||
v1.ResourcePods: *resource.NewQuantity(32, resource.DecimalSI),
|
|
||||||
v1.ResourceCPU: *resource.NewMilliQuantity(500, resource.DecimalSI),
|
|
||||||
v1.ResourceMemory: *resource.NewQuantity(500, resource.BinarySI),
|
|
||||||
}
|
|
||||||
node, err := createNode(context.clientSet, "node1", nodeRes)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Error creating nodes: %v", err)
|
|
||||||
}
|
|
||||||
nodeLabels := map[string]string{"node": node.Name}
|
|
||||||
if err = testutils.AddLabelsToNode(context.clientSet, node.Name, nodeLabels); err != nil {
|
|
||||||
t.Fatalf("Cannot add labels to node: %v", err)
|
|
||||||
}
|
|
||||||
if err = waitForNodeLabels(context.clientSet, node.Name, nodeLabels); err != nil {
|
|
||||||
t.Fatalf("Adding labels to node didn't succeed: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, test := range tests {
|
|
||||||
pods := make([]*v1.Pod, len(test.existingPods))
|
|
||||||
// Create and run existingPods.
|
|
||||||
for i, p := range test.existingPods {
|
|
||||||
pods[i], err = runPausePod(cs, p)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Test [%v]: Error running pause pod: %v", test.description, err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Create the "pod".
|
|
||||||
preemptor, err := createPausePod(cs, test.pod)
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("Error while creating high priority pod: %v", err)
|
|
||||||
}
|
|
||||||
// Wait for preemption of pods and make sure the other ones are not preempted.
|
|
||||||
for i, p := range pods {
|
|
||||||
if _, found := test.preemptedPodIndexes[i]; found {
|
|
||||||
if err = wait.Poll(time.Second, wait.ForeverTestTimeout, podIsGettingEvicted(cs, p.Namespace, p.Name)); err != nil {
|
|
||||||
t.Errorf("Test [%v]: Pod %v is not getting evicted.", test.description, p.Name)
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if p.DeletionTimestamp != nil {
|
|
||||||
t.Errorf("Test [%v]: Didn't expect pod %v to get preempted.", test.description, p.Name)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Also check that the preemptor pod gets the annotation for nominated node name.
|
|
||||||
if len(test.preemptedPodIndexes) > 0 {
|
|
||||||
if err = wait.Poll(time.Second, wait.ForeverTestTimeout, func() (bool, error) {
|
|
||||||
pod, err := context.clientSet.CoreV1().Pods(context.ns.Name).Get("preemptor-pod", metav1.GetOptions{})
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("Test [%v]: error getting pod: %v", test.description, err)
|
|
||||||
}
|
|
||||||
annot, found := pod.Annotations[core.NominatedNodeAnnotationKey]
|
|
||||||
if found && len(annot) > 0 {
|
|
||||||
return true, nil
|
|
||||||
}
|
|
||||||
return false, err
|
|
||||||
}); err != nil {
|
|
||||||
t.Errorf("Test [%v]: Pod annotation did not get set.", test.description)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Cleanup
|
|
||||||
pods = append(pods, preemptor)
|
|
||||||
for _, p := range pods {
|
|
||||||
err = cs.CoreV1().Pods(p.Namespace).Delete(p.Name, metav1.NewDeleteOptions(0))
|
|
||||||
if err != nil && !errors.IsNotFound(err) {
|
|
||||||
t.Errorf("Test [%v]: error, %v, while deleting pod during test.", test.description, err)
|
|
||||||
}
|
|
||||||
err = wait.Poll(time.Second, wait.ForeverTestTimeout, podDeleted(cs, p.Namespace, p.Name))
|
|
||||||
if err != nil {
|
|
||||||
t.Errorf("Test [%v]: error, %v, while waiting for pod to get deleted.", test.description, err)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// TestPDBCache verifies that scheduler cache works as expected when handling
|
// TestPDBCache verifies that scheduler cache works as expected when handling
|
||||||
// PodDisruptionBudget.
|
// PodDisruptionBudget.
|
||||||
func TestPDBCache(t *testing.T) {
|
func TestPDBCache(t *testing.T) {
|
||||||
|
@ -27,14 +27,14 @@ import (
|
|||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/apimachinery/pkg/util/uuid"
|
"k8s.io/apimachinery/pkg/util/uuid"
|
||||||
"k8s.io/apimachinery/pkg/util/wait"
|
"k8s.io/apimachinery/pkg/util/wait"
|
||||||
informers "k8s.io/client-go/informers"
|
"k8s.io/client-go/informers"
|
||||||
clientset "k8s.io/client-go/kubernetes"
|
clientset "k8s.io/client-go/kubernetes"
|
||||||
clientv1core "k8s.io/client-go/kubernetes/typed/core/v1"
|
clientv1core "k8s.io/client-go/kubernetes/typed/core/v1"
|
||||||
corelisters "k8s.io/client-go/listers/core/v1"
|
corelisters "k8s.io/client-go/listers/core/v1"
|
||||||
restclient "k8s.io/client-go/rest"
|
restclient "k8s.io/client-go/rest"
|
||||||
"k8s.io/client-go/tools/record"
|
"k8s.io/client-go/tools/record"
|
||||||
"k8s.io/kubernetes/pkg/api/legacyscheme"
|
"k8s.io/kubernetes/pkg/api/legacyscheme"
|
||||||
"k8s.io/kubernetes/pkg/api/testapi"
|
podutil "k8s.io/kubernetes/pkg/api/v1/pod"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler"
|
"k8s.io/kubernetes/plugin/pkg/scheduler"
|
||||||
_ "k8s.io/kubernetes/plugin/pkg/scheduler/algorithmprovider"
|
_ "k8s.io/kubernetes/plugin/pkg/scheduler/algorithmprovider"
|
||||||
"k8s.io/kubernetes/plugin/pkg/scheduler/factory"
|
"k8s.io/kubernetes/plugin/pkg/scheduler/factory"
|
||||||
@ -58,13 +58,14 @@ type TestContext struct {
|
|||||||
// configuration.
|
// configuration.
|
||||||
func initTest(t *testing.T, nsPrefix string) *TestContext {
|
func initTest(t *testing.T, nsPrefix string) *TestContext {
|
||||||
var context TestContext
|
var context TestContext
|
||||||
_, context.httpServer, context.closeFn = framework.RunAMaster(nil)
|
masterConfig := framework.NewIntegrationTestMasterConfig()
|
||||||
|
_, context.httpServer, context.closeFn = framework.RunAMaster(masterConfig)
|
||||||
|
|
||||||
context.ns = framework.CreateTestingNamespace(nsPrefix+string(uuid.NewUUID()), context.httpServer, t)
|
context.ns = framework.CreateTestingNamespace(nsPrefix+string(uuid.NewUUID()), context.httpServer, t)
|
||||||
|
|
||||||
context.clientSet = clientset.NewForConfigOrDie(&restclient.Config{Host: context.httpServer.URL, ContentConfig: restclient.ContentConfig{GroupVersion: testapi.Groups[v1.GroupName].GroupVersion()}})
|
context.clientSet = clientset.NewForConfigOrDie(&restclient.Config{Host: context.httpServer.URL})
|
||||||
context.informerFactory = informers.NewSharedInformerFactory(context.clientSet, 0)
|
context.informerFactory = informers.NewSharedInformerFactory(context.clientSet, 0)
|
||||||
podInformer := factory.NewPodInformer(context.clientSet, 30*time.Second, v1.DefaultSchedulerName)
|
podInformer := factory.NewPodInformer(context.clientSet, 12*time.Hour, v1.DefaultSchedulerName)
|
||||||
context.schedulerConfigFactory = factory.NewConfigFactory(
|
context.schedulerConfigFactory = factory.NewConfigFactory(
|
||||||
v1.DefaultSchedulerName,
|
v1.DefaultSchedulerName,
|
||||||
context.clientSet,
|
context.clientSet,
|
||||||
@ -289,10 +290,10 @@ func runPausePod(cs clientset.Interface, pod *v1.Pod) (*v1.Pod, error) {
|
|||||||
func podDeleted(c clientset.Interface, podNamespace, podName string) wait.ConditionFunc {
|
func podDeleted(c clientset.Interface, podNamespace, podName string) wait.ConditionFunc {
|
||||||
return func() (bool, error) {
|
return func() (bool, error) {
|
||||||
pod, err := c.CoreV1().Pods(podNamespace).Get(podName, metav1.GetOptions{})
|
pod, err := c.CoreV1().Pods(podNamespace).Get(podName, metav1.GetOptions{})
|
||||||
if pod.DeletionTimestamp != nil {
|
if errors.IsNotFound(err) {
|
||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
if errors.IsNotFound(err) {
|
if pod.DeletionTimestamp != nil {
|
||||||
return true, nil
|
return true, nil
|
||||||
}
|
}
|
||||||
return false, nil
|
return false, nil
|
||||||
@ -331,6 +332,23 @@ func podScheduled(c clientset.Interface, podNamespace, podName string) wait.Cond
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// podUnschedulable returns a condition function that returns true if the given pod
|
||||||
|
// gets unschedulable status.
|
||||||
|
func podUnschedulable(c clientset.Interface, podNamespace, podName string) wait.ConditionFunc {
|
||||||
|
return func() (bool, error) {
|
||||||
|
pod, err := c.CoreV1().Pods(podNamespace).Get(podName, metav1.GetOptions{})
|
||||||
|
if errors.IsNotFound(err) {
|
||||||
|
return false, nil
|
||||||
|
}
|
||||||
|
if err != nil {
|
||||||
|
// This could be a connection error so we want to retry.
|
||||||
|
return false, nil
|
||||||
|
}
|
||||||
|
_, cond := podutil.GetPodCondition(&pod.Status, v1.PodScheduled)
|
||||||
|
return cond != nil && cond.Status == v1.ConditionFalse && cond.Reason == v1.PodReasonUnschedulable, nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// waitForPodToScheduleWithTimeout waits for a pod to get scheduled and returns
|
// waitForPodToScheduleWithTimeout waits for a pod to get scheduled and returns
|
||||||
// an error if it does not scheduled within the given timeout.
|
// an error if it does not scheduled within the given timeout.
|
||||||
func waitForPodToScheduleWithTimeout(cs clientset.Interface, pod *v1.Pod, timeout time.Duration) error {
|
func waitForPodToScheduleWithTimeout(cs clientset.Interface, pod *v1.Pod, timeout time.Duration) error {
|
||||||
@ -348,6 +366,21 @@ func deletePod(cs clientset.Interface, podName string, nsName string) error {
|
|||||||
return cs.CoreV1().Pods(nsName).Delete(podName, metav1.NewDeleteOptions(0))
|
return cs.CoreV1().Pods(nsName).Delete(podName, metav1.NewDeleteOptions(0))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// cleanupPods deletes the given pods and waits for them to be actually deleted.
|
||||||
|
func cleanupPods(cs clientset.Interface, t *testing.T, pods []*v1.Pod) {
|
||||||
|
for _, p := range pods {
|
||||||
|
err := cs.CoreV1().Pods(p.Namespace).Delete(p.Name, metav1.NewDeleteOptions(0))
|
||||||
|
if err != nil && !errors.IsNotFound(err) {
|
||||||
|
t.Errorf("error while deleting pod %v/%v: %v", p.Namespace, p.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for _, p := range pods {
|
||||||
|
if err := wait.Poll(time.Second, wait.ForeverTestTimeout, podDeleted(cs, p.Namespace, p.Name)); err != nil {
|
||||||
|
t.Errorf("error while waiting for pod %v/%v to get deleted: %v", p.Namespace, p.Name, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// printAllPods prints a list of all the pods and their node names. This is used
|
// printAllPods prints a list of all the pods and their node names. This is used
|
||||||
// for debugging.
|
// for debugging.
|
||||||
func printAllPods(t *testing.T, cs clientset.Interface, nsName string) {
|
func printAllPods(t *testing.T, cs clientset.Interface, nsName string) {
|
||||||
|
Loading…
Reference in New Issue
Block a user