841 lines
25 KiB
Go
841 lines
25 KiB
Go
/*
|
|
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.
|
|
*/
|
|
|
|
package cache
|
|
|
|
import (
|
|
"fmt"
|
|
"k8s.io/api/core/v1"
|
|
"k8s.io/api/scheduling/v1beta1"
|
|
apierrors "k8s.io/apimachinery/pkg/api/errors"
|
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
|
"k8s.io/apimachinery/pkg/runtime"
|
|
utilruntime "k8s.io/apimachinery/pkg/util/runtime"
|
|
"k8s.io/apimachinery/pkg/util/wait"
|
|
"k8s.io/client-go/informers"
|
|
infov1 "k8s.io/client-go/informers/core/v1"
|
|
schedv1 "k8s.io/client-go/informers/scheduling/v1beta1"
|
|
storagev1 "k8s.io/client-go/informers/storage/v1"
|
|
"k8s.io/client-go/kubernetes"
|
|
corev1 "k8s.io/client-go/kubernetes/typed/core/v1"
|
|
"k8s.io/client-go/rest"
|
|
"k8s.io/client-go/tools/cache"
|
|
"k8s.io/client-go/tools/record"
|
|
"k8s.io/client-go/util/workqueue"
|
|
"k8s.io/klog"
|
|
podutil "k8s.io/kubernetes/pkg/api/v1/pod"
|
|
"k8s.io/kubernetes/pkg/scheduler/volumebinder"
|
|
"sync"
|
|
"time"
|
|
vcv1beta1 "volcano.sh/volcano/pkg/apis/scheduling/v1beta1"
|
|
|
|
"volcano.sh/volcano/cmd/scheduler/app/options"
|
|
"volcano.sh/volcano/pkg/apis/scheduling"
|
|
schedulingscheme "volcano.sh/volcano/pkg/apis/scheduling/scheme"
|
|
vcclient "volcano.sh/volcano/pkg/client/clientset/versioned"
|
|
"volcano.sh/volcano/pkg/client/clientset/versioned/scheme"
|
|
vcinformer "volcano.sh/volcano/pkg/client/informers/externalversions"
|
|
vcinformerv1 "volcano.sh/volcano/pkg/client/informers/externalversions/scheduling/v1beta1"
|
|
schedulingapi "volcano.sh/volcano/pkg/scheduler/api"
|
|
)
|
|
|
|
func init() {
|
|
schemeBuilder := runtime.SchemeBuilder{
|
|
v1.AddToScheme,
|
|
}
|
|
|
|
utilruntime.Must(schemeBuilder.AddToScheme(scheme.Scheme))
|
|
}
|
|
|
|
// New returns a Cache implementation.
|
|
func New(config *rest.Config, schedulerName string, defaultQueue string) Cache {
|
|
return newSchedulerCache(config, schedulerName, defaultQueue)
|
|
}
|
|
|
|
//SchedulerCache cache for the kube batch
|
|
type SchedulerCache struct {
|
|
sync.Mutex
|
|
|
|
kubeclient *kubernetes.Clientset
|
|
vcclient *vcclient.Clientset
|
|
|
|
defaultQueue string
|
|
// schedulerName is the name for kube batch scheduler
|
|
schedulerName string
|
|
|
|
podInformer infov1.PodInformer
|
|
nodeInformer infov1.NodeInformer
|
|
nsInformer infov1.NamespaceInformer
|
|
podGroupInformerV1beta1 vcinformerv1.PodGroupInformer
|
|
queueInformerV1beta1 vcinformerv1.QueueInformer
|
|
pvInformer infov1.PersistentVolumeInformer
|
|
pvcInformer infov1.PersistentVolumeClaimInformer
|
|
scInformer storagev1.StorageClassInformer
|
|
pcInformer schedv1.PriorityClassInformer
|
|
quotaInformer infov1.ResourceQuotaInformer
|
|
|
|
Binder Binder
|
|
Evictor Evictor
|
|
StatusUpdater StatusUpdater
|
|
VolumeBinder VolumeBinder
|
|
|
|
Recorder record.EventRecorder
|
|
|
|
Jobs map[schedulingapi.JobID]*schedulingapi.JobInfo
|
|
Nodes map[string]*schedulingapi.NodeInfo
|
|
Queues map[schedulingapi.QueueID]*schedulingapi.QueueInfo
|
|
PriorityClasses map[string]*v1beta1.PriorityClass
|
|
defaultPriorityClass *v1beta1.PriorityClass
|
|
defaultPriority int32
|
|
|
|
NamespaceCollection map[string]*schedulingapi.NamespaceCollection
|
|
|
|
errTasks workqueue.RateLimitingInterface
|
|
deletedJobs workqueue.RateLimitingInterface
|
|
}
|
|
|
|
type defaultBinder struct {
|
|
kubeclient *kubernetes.Clientset
|
|
}
|
|
|
|
//Bind will send bind request to api server
|
|
func (db *defaultBinder) Bind(p *v1.Pod, hostname string) error {
|
|
if err := db.kubeclient.CoreV1().Pods(p.Namespace).Bind(&v1.Binding{
|
|
ObjectMeta: metav1.ObjectMeta{Namespace: p.Namespace, Name: p.Name, UID: p.UID, Annotations: p.Annotations},
|
|
Target: v1.ObjectReference{
|
|
Kind: "Node",
|
|
Name: hostname,
|
|
},
|
|
}); err != nil {
|
|
klog.Errorf("Failed to bind pod <%v/%v>: %#v", p.Namespace, p.Name, err)
|
|
return err
|
|
}
|
|
return nil
|
|
}
|
|
|
|
type defaultEvictor struct {
|
|
kubeclient *kubernetes.Clientset
|
|
}
|
|
|
|
//Evict will send delete pod request to api server
|
|
func (de *defaultEvictor) Evict(p *v1.Pod) error {
|
|
klog.V(3).Infof("Evicting pod %v/%v", p.Namespace, p.Name)
|
|
|
|
if err := de.kubeclient.CoreV1().Pods(p.Namespace).Delete(p.Name, nil); err != nil {
|
|
klog.Errorf("Failed to evict pod <%v/%v>: %#v", p.Namespace, p.Name, err)
|
|
return err
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// defaultStatusUpdater is the default implementation of the StatusUpdater interface
|
|
type defaultStatusUpdater struct {
|
|
kubeclient *kubernetes.Clientset
|
|
vcclient *vcclient.Clientset
|
|
}
|
|
|
|
// following the same logic as podutil.UpdatePodCondition
|
|
func podConditionHaveUpdate(status *v1.PodStatus, condition *v1.PodCondition) bool {
|
|
lastTransitionTime := metav1.Now()
|
|
// Try to find this pod condition.
|
|
_, oldCondition := podutil.GetPodCondition(status, condition.Type)
|
|
|
|
if oldCondition == nil {
|
|
// We are adding new pod condition.
|
|
return true
|
|
}
|
|
// We are updating an existing condition, so we need to check if it has changed.
|
|
if condition.Status == oldCondition.Status {
|
|
lastTransitionTime = oldCondition.LastTransitionTime
|
|
}
|
|
|
|
isEqual := condition.Status == oldCondition.Status &&
|
|
condition.Reason == oldCondition.Reason &&
|
|
condition.Message == oldCondition.Message &&
|
|
condition.LastProbeTime.Equal(&oldCondition.LastProbeTime) &&
|
|
lastTransitionTime.Equal(&oldCondition.LastTransitionTime)
|
|
|
|
// Return true if one of the fields have changed.
|
|
return !isEqual
|
|
}
|
|
|
|
// UpdatePodCondition will Update pod with podCondition
|
|
func (su *defaultStatusUpdater) UpdatePodCondition(pod *v1.Pod, condition *v1.PodCondition) (*v1.Pod, error) {
|
|
klog.V(3).Infof("Updating pod condition for %s/%s to (%s==%s)", pod.Namespace, pod.Name, condition.Type, condition.Status)
|
|
if podutil.UpdatePodCondition(&pod.Status, condition) {
|
|
return su.kubeclient.CoreV1().Pods(pod.Namespace).UpdateStatus(pod)
|
|
}
|
|
return pod, nil
|
|
}
|
|
|
|
// UpdatePodGroup will Update pod with podCondition
|
|
func (su *defaultStatusUpdater) UpdatePodGroup(pg *schedulingapi.PodGroup) (*schedulingapi.PodGroup, error) {
|
|
podgroup := &vcv1beta1.PodGroup{}
|
|
if err := schedulingscheme.Scheme.Convert(&pg.PodGroup, podgroup, nil); err != nil {
|
|
klog.Errorf("Error while converting PodGroup to v1alpha1.PodGroup with error: %v", err)
|
|
return nil, err
|
|
}
|
|
|
|
updated, err := su.vcclient.SchedulingV1beta1().PodGroups(podgroup.Namespace).Update(podgroup)
|
|
if err != nil {
|
|
klog.Errorf("Error while updating PodGroup with error: %v", err)
|
|
return nil, err
|
|
}
|
|
|
|
podGroupInfo := &schedulingapi.PodGroup{Version: schedulingapi.PodGroupVersionV1Beta1}
|
|
if err := schedulingscheme.Scheme.Convert(updated, &podGroupInfo.PodGroup, nil); err != nil {
|
|
klog.Errorf("Error while converting v1alpha.PodGroup to api.PodGroup with error: %v", err)
|
|
return nil, err
|
|
}
|
|
|
|
return podGroupInfo, nil
|
|
}
|
|
|
|
type defaultVolumeBinder struct {
|
|
volumeBinder *volumebinder.VolumeBinder
|
|
}
|
|
|
|
// AllocateVolumes allocates volume on the host to the task
|
|
func (dvb *defaultVolumeBinder) AllocateVolumes(task *schedulingapi.TaskInfo, hostname string) error {
|
|
allBound, err := dvb.volumeBinder.Binder.AssumePodVolumes(task.Pod, hostname)
|
|
task.VolumeReady = allBound
|
|
|
|
return err
|
|
}
|
|
|
|
// BindVolumes binds volumes to the task
|
|
func (dvb *defaultVolumeBinder) BindVolumes(task *schedulingapi.TaskInfo) error {
|
|
// If task's volumes are ready, did not bind them again.
|
|
if task.VolumeReady {
|
|
return nil
|
|
}
|
|
|
|
return dvb.volumeBinder.Binder.BindPodVolumes(task.Pod)
|
|
}
|
|
|
|
func newSchedulerCache(config *rest.Config, schedulerName string, defaultQueue string) *SchedulerCache {
|
|
kubeClient, err := kubernetes.NewForConfig(config)
|
|
if err != nil {
|
|
panic(fmt.Sprintf("failed init kubeClient, with err: %v", err))
|
|
}
|
|
vcClient, err := vcclient.NewForConfig(config)
|
|
if err != nil {
|
|
panic(fmt.Sprintf("failed init vcClient, with err: %v", err))
|
|
}
|
|
eventClient, err := kubernetes.NewForConfig(config)
|
|
if err != nil {
|
|
panic(fmt.Sprintf("failed init eventClient, with err: %v", err))
|
|
}
|
|
|
|
// create default queue
|
|
defaultQue := vcv1beta1.Queue{
|
|
ObjectMeta: metav1.ObjectMeta{
|
|
Name: defaultQueue,
|
|
},
|
|
Spec: vcv1beta1.QueueSpec{
|
|
Weight: 1,
|
|
},
|
|
}
|
|
if _, err := vcClient.SchedulingV1beta1().Queues().Create(&defaultQue); err != nil && !apierrors.IsAlreadyExists(err) {
|
|
panic(fmt.Sprintf("failed init default queue, with err: %v", err))
|
|
}
|
|
|
|
sc := &SchedulerCache{
|
|
Jobs: make(map[schedulingapi.JobID]*schedulingapi.JobInfo),
|
|
Nodes: make(map[string]*schedulingapi.NodeInfo),
|
|
Queues: make(map[schedulingapi.QueueID]*schedulingapi.QueueInfo),
|
|
PriorityClasses: make(map[string]*v1beta1.PriorityClass),
|
|
errTasks: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()),
|
|
deletedJobs: workqueue.NewRateLimitingQueue(workqueue.DefaultControllerRateLimiter()),
|
|
kubeclient: kubeClient,
|
|
vcclient: vcClient,
|
|
defaultQueue: defaultQueue,
|
|
schedulerName: schedulerName,
|
|
|
|
NamespaceCollection: make(map[string]*schedulingapi.NamespaceCollection),
|
|
}
|
|
|
|
// Prepare event clients.
|
|
broadcaster := record.NewBroadcaster()
|
|
broadcaster.StartRecordingToSink(&corev1.EventSinkImpl{Interface: eventClient.CoreV1().Events("")})
|
|
sc.Recorder = broadcaster.NewRecorder(scheme.Scheme, v1.EventSource{Component: schedulerName})
|
|
|
|
sc.Binder = &defaultBinder{
|
|
kubeclient: sc.kubeclient,
|
|
}
|
|
|
|
sc.Evictor = &defaultEvictor{
|
|
kubeclient: sc.kubeclient,
|
|
}
|
|
|
|
sc.StatusUpdater = &defaultStatusUpdater{
|
|
kubeclient: sc.kubeclient,
|
|
vcclient: sc.vcclient,
|
|
}
|
|
|
|
informerFactory := informers.NewSharedInformerFactory(sc.kubeclient, 0)
|
|
|
|
sc.pvcInformer = informerFactory.Core().V1().PersistentVolumeClaims()
|
|
sc.pvInformer = informerFactory.Core().V1().PersistentVolumes()
|
|
sc.scInformer = informerFactory.Storage().V1().StorageClasses()
|
|
sc.VolumeBinder = &defaultVolumeBinder{
|
|
volumeBinder: volumebinder.NewVolumeBinder(
|
|
sc.kubeclient,
|
|
sc.nodeInformer,
|
|
sc.pvcInformer,
|
|
sc.pvInformer,
|
|
sc.scInformer,
|
|
30*time.Second,
|
|
),
|
|
}
|
|
|
|
// create informer for node information
|
|
sc.nodeInformer = informerFactory.Core().V1().Nodes()
|
|
sc.nodeInformer.Informer().AddEventHandlerWithResyncPeriod(
|
|
cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddNode,
|
|
UpdateFunc: sc.UpdateNode,
|
|
DeleteFunc: sc.DeleteNode,
|
|
},
|
|
0,
|
|
)
|
|
|
|
// create informer for pod information
|
|
sc.podInformer = informerFactory.Core().V1().Pods()
|
|
sc.podInformer.Informer().AddEventHandler(
|
|
cache.FilteringResourceEventHandler{
|
|
FilterFunc: func(obj interface{}) bool {
|
|
switch obj.(type) {
|
|
case *v1.Pod:
|
|
pod := obj.(*v1.Pod)
|
|
if !responsibleForPod(pod, schedulerName) {
|
|
if len(pod.Spec.NodeName) == 0 {
|
|
return false
|
|
}
|
|
}
|
|
return true
|
|
default:
|
|
return false
|
|
}
|
|
},
|
|
Handler: cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddPod,
|
|
UpdateFunc: sc.UpdatePod,
|
|
DeleteFunc: sc.DeletePod,
|
|
},
|
|
})
|
|
|
|
sc.pcInformer = informerFactory.Scheduling().V1beta1().PriorityClasses()
|
|
sc.pcInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddPriorityClass,
|
|
UpdateFunc: sc.UpdatePriorityClass,
|
|
DeleteFunc: sc.DeletePriorityClass,
|
|
})
|
|
|
|
sc.quotaInformer = informerFactory.Core().V1().ResourceQuotas()
|
|
sc.quotaInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddResourceQuota,
|
|
UpdateFunc: sc.UpdateResourceQuota,
|
|
DeleteFunc: sc.DeleteResourceQuota,
|
|
})
|
|
|
|
vcinformers := vcinformer.NewSharedInformerFactory(sc.vcclient, 0)
|
|
|
|
// create informer for PodGroup(v1beta1) information
|
|
sc.podGroupInformerV1beta1 = vcinformers.Scheduling().V1beta1().PodGroups()
|
|
sc.podGroupInformerV1beta1.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddPodGroupV1beta1,
|
|
UpdateFunc: sc.UpdatePodGroupV1beta1,
|
|
DeleteFunc: sc.DeletePodGroupV1beta1,
|
|
})
|
|
|
|
// create informer(v1beta1) for Queue information
|
|
sc.queueInformerV1beta1 = vcinformers.Scheduling().V1beta1().Queues()
|
|
sc.queueInformerV1beta1.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
|
|
AddFunc: sc.AddQueueV1beta1,
|
|
UpdateFunc: sc.UpdateQueueV1beta1,
|
|
DeleteFunc: sc.DeleteQueueV1beta1,
|
|
})
|
|
|
|
return sc
|
|
}
|
|
|
|
// Run starts the schedulerCache
|
|
func (sc *SchedulerCache) Run(stopCh <-chan struct{}) {
|
|
go sc.podInformer.Informer().Run(stopCh)
|
|
go sc.nodeInformer.Informer().Run(stopCh)
|
|
go sc.podGroupInformerV1beta1.Informer().Run(stopCh)
|
|
go sc.pvInformer.Informer().Run(stopCh)
|
|
go sc.pvcInformer.Informer().Run(stopCh)
|
|
go sc.scInformer.Informer().Run(stopCh)
|
|
go sc.queueInformerV1beta1.Informer().Run(stopCh)
|
|
go sc.quotaInformer.Informer().Run(stopCh)
|
|
|
|
if options.ServerOpts.EnablePriorityClass {
|
|
go sc.pcInformer.Informer().Run(stopCh)
|
|
}
|
|
|
|
// Re-sync error tasks.
|
|
go wait.Until(sc.processResyncTask, 0, stopCh)
|
|
|
|
// Cleanup jobs.
|
|
go wait.Until(sc.processCleanupJob, 0, stopCh)
|
|
}
|
|
|
|
// WaitForCacheSync sync the cache with the api server
|
|
func (sc *SchedulerCache) WaitForCacheSync(stopCh <-chan struct{}) bool {
|
|
|
|
return cache.WaitForCacheSync(stopCh,
|
|
func() []cache.InformerSynced {
|
|
informerSynced := []cache.InformerSynced{
|
|
sc.podInformer.Informer().HasSynced,
|
|
sc.podGroupInformerV1beta1.Informer().HasSynced,
|
|
sc.nodeInformer.Informer().HasSynced,
|
|
sc.pvInformer.Informer().HasSynced,
|
|
sc.pvcInformer.Informer().HasSynced,
|
|
sc.scInformer.Informer().HasSynced,
|
|
sc.queueInformerV1beta1.Informer().HasSynced,
|
|
sc.quotaInformer.Informer().HasSynced,
|
|
}
|
|
if options.ServerOpts.EnablePriorityClass {
|
|
informerSynced = append(informerSynced, sc.pcInformer.Informer().HasSynced)
|
|
}
|
|
return informerSynced
|
|
}()...,
|
|
)
|
|
}
|
|
|
|
// findJobAndTask returns job and the task info
|
|
func (sc *SchedulerCache) findJobAndTask(taskInfo *schedulingapi.TaskInfo) (*schedulingapi.JobInfo, *schedulingapi.TaskInfo, error) {
|
|
job, found := sc.Jobs[taskInfo.Job]
|
|
if !found {
|
|
return nil, nil, fmt.Errorf("failed to find Job %v for Task %v",
|
|
taskInfo.Job, taskInfo.UID)
|
|
}
|
|
|
|
task, found := job.Tasks[taskInfo.UID]
|
|
if !found {
|
|
return nil, nil, fmt.Errorf("failed to find task in status %v by id %v",
|
|
taskInfo.Status, taskInfo.UID)
|
|
}
|
|
|
|
return job, task, nil
|
|
}
|
|
|
|
// Evict will evict the pod.
|
|
//
|
|
// If error occurs both task and job are guaranteed to be in the original state.
|
|
func (sc *SchedulerCache) Evict(taskInfo *schedulingapi.TaskInfo, reason string) error {
|
|
sc.Mutex.Lock()
|
|
defer sc.Mutex.Unlock()
|
|
|
|
job, task, err := sc.findJobAndTask(taskInfo)
|
|
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
node, found := sc.Nodes[task.NodeName]
|
|
if !found {
|
|
return fmt.Errorf("failed to bind Task %v to host %v, host does not exist",
|
|
task.UID, task.NodeName)
|
|
}
|
|
|
|
originalStatus := task.Status
|
|
if err := job.UpdateTaskStatus(task, schedulingapi.Releasing); err != nil {
|
|
return err
|
|
}
|
|
|
|
// Add new task to node.
|
|
if err := node.UpdateTask(task); err != nil {
|
|
// After failing to update task to a node we need to revert task status from Releasing,
|
|
// otherwise task might be stuck in the Releasing state indefinitely.
|
|
if err := job.UpdateTaskStatus(task, originalStatus); err != nil {
|
|
klog.Errorf("Task <%s/%s> will be resynchronized after failing to revert status "+
|
|
"from %s to %s after failing to update Task on Node <%s>: %v",
|
|
task.Namespace, task.Name, task.Status, originalStatus, node.Name, err)
|
|
sc.resyncTask(task)
|
|
}
|
|
return err
|
|
}
|
|
|
|
p := task.Pod
|
|
|
|
go func() {
|
|
err := sc.Evictor.Evict(p)
|
|
if err != nil {
|
|
sc.resyncTask(task)
|
|
}
|
|
}()
|
|
|
|
podgroup := &vcv1beta1.PodGroup{}
|
|
if err := schedulingscheme.Scheme.Convert(&job.PodGroup.PodGroup, podgroup, nil); err != nil {
|
|
klog.Errorf("Error while converting PodGroup to v1alpha1.PodGroup with error: %v", err)
|
|
return err
|
|
}
|
|
sc.Recorder.Eventf(podgroup, v1.EventTypeNormal, "Evict", reason)
|
|
return nil
|
|
}
|
|
|
|
// Bind binds task to the target host.
|
|
func (sc *SchedulerCache) Bind(taskInfo *schedulingapi.TaskInfo, hostname string) error {
|
|
sc.Mutex.Lock()
|
|
defer sc.Mutex.Unlock()
|
|
|
|
job, task, err := sc.findJobAndTask(taskInfo)
|
|
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
node, found := sc.Nodes[hostname]
|
|
if !found {
|
|
return fmt.Errorf("failed to bind Task %v to host %v, host does not exist",
|
|
task.UID, hostname)
|
|
}
|
|
|
|
originalStatus := task.Status
|
|
if err := job.UpdateTaskStatus(task, schedulingapi.Binding); err != nil {
|
|
return err
|
|
}
|
|
|
|
// Add task to the node.
|
|
if err := node.AddTask(task); err != nil {
|
|
// After failing to update task to a node we need to revert task status from Releasing,
|
|
// otherwise task might be stuck in the Releasing state indefinitely.
|
|
if err := job.UpdateTaskStatus(task, originalStatus); err != nil {
|
|
klog.Errorf("Task <%s/%s> will be resynchronized after failing to revert status "+
|
|
"from %s to %s after failing to update Task on Node <%s>: %v",
|
|
task.Namespace, task.Name, task.Status, originalStatus, node.Name, err)
|
|
sc.resyncTask(task)
|
|
}
|
|
return err
|
|
}
|
|
|
|
p := task.Pod
|
|
|
|
var pgCopy schedulingapi.PodGroup
|
|
if job.PodGroup != nil {
|
|
pgCopy = schedulingapi.PodGroup{
|
|
Version: job.PodGroup.Version,
|
|
PodGroup: *job.PodGroup.PodGroup.DeepCopy(),
|
|
}
|
|
}
|
|
|
|
go func() {
|
|
if err := sc.Binder.Bind(p, hostname); err != nil {
|
|
sc.resyncTask(task)
|
|
} else {
|
|
sc.Recorder.Eventf(p, v1.EventTypeNormal, "Scheduled", "Successfully assigned %v/%v to %v", p.Namespace, p.Name, hostname)
|
|
|
|
if job.PodGroup != nil {
|
|
msg := fmt.Sprintf("%v/%v tasks in gang unschedulable: %v, %v minAvailable, %v Pending",
|
|
len(job.TaskStatusIndex[schedulingapi.Pending]),
|
|
len(job.Tasks),
|
|
scheduling.PodGroupReady,
|
|
job.MinAvailable,
|
|
len(job.TaskStatusIndex[schedulingapi.Pending]))
|
|
sc.recordPodGroupEvent(&pgCopy, v1.EventTypeNormal, string(scheduling.PodGroupScheduled), msg)
|
|
}
|
|
}
|
|
}()
|
|
|
|
return nil
|
|
}
|
|
|
|
// AllocateVolumes allocates volume on the host to the task
|
|
func (sc *SchedulerCache) AllocateVolumes(task *schedulingapi.TaskInfo, hostname string) error {
|
|
return sc.VolumeBinder.AllocateVolumes(task, hostname)
|
|
}
|
|
|
|
// BindVolumes binds volumes to the task
|
|
func (sc *SchedulerCache) BindVolumes(task *schedulingapi.TaskInfo) error {
|
|
return sc.VolumeBinder.BindVolumes(task)
|
|
}
|
|
|
|
// taskUnschedulable updates pod status of pending task
|
|
func (sc *SchedulerCache) taskUnschedulable(task *schedulingapi.TaskInfo, message string) error {
|
|
pod := task.Pod
|
|
|
|
condition := &v1.PodCondition{
|
|
Type: v1.PodScheduled,
|
|
Status: v1.ConditionFalse,
|
|
Reason: v1.PodReasonUnschedulable,
|
|
Message: message,
|
|
}
|
|
|
|
if podConditionHaveUpdate(&pod.Status, condition) {
|
|
pod = pod.DeepCopy()
|
|
|
|
// The reason field in 'Events' should be "FailedScheduling", there is not constants defined for this in
|
|
// k8s core, so using the same string here.
|
|
// The reason field in PodCondition should be "Unschedulable"
|
|
sc.Recorder.Eventf(pod, v1.EventTypeWarning, "FailedScheduling", message)
|
|
if _, err := sc.StatusUpdater.UpdatePodCondition(pod, condition); err != nil {
|
|
return err
|
|
}
|
|
} else {
|
|
klog.V(4).Infof("task unscheduleable %s/%s, message: %s, skip by no condition update", pod.Namespace, pod.Name, message)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (sc *SchedulerCache) deleteJob(job *schedulingapi.JobInfo) {
|
|
klog.V(3).Infof("Try to delete Job <%v:%v/%v>", job.UID, job.Namespace, job.Name)
|
|
|
|
sc.deletedJobs.AddRateLimited(job)
|
|
}
|
|
|
|
func (sc *SchedulerCache) processCleanupJob() {
|
|
obj, shutdown := sc.deletedJobs.Get()
|
|
if shutdown {
|
|
return
|
|
}
|
|
|
|
defer sc.deletedJobs.Done(obj)
|
|
|
|
job, found := obj.(*schedulingapi.JobInfo)
|
|
if !found {
|
|
klog.Errorf("Failed to convert <%v> to *JobInfo", obj)
|
|
return
|
|
}
|
|
|
|
sc.Mutex.Lock()
|
|
defer sc.Mutex.Unlock()
|
|
|
|
if schedulingapi.JobTerminated(job) {
|
|
delete(sc.Jobs, job.UID)
|
|
klog.V(3).Infof("Job <%v:%v/%v> was deleted.", job.UID, job.Namespace, job.Name)
|
|
} else {
|
|
// Retry
|
|
sc.deleteJob(job)
|
|
}
|
|
}
|
|
|
|
func (sc *SchedulerCache) resyncTask(task *schedulingapi.TaskInfo) {
|
|
sc.errTasks.AddRateLimited(task)
|
|
}
|
|
|
|
func (sc *SchedulerCache) processResyncTask() {
|
|
obj, shutdown := sc.errTasks.Get()
|
|
if shutdown {
|
|
return
|
|
}
|
|
|
|
defer sc.errTasks.Done(obj)
|
|
|
|
task, ok := obj.(*schedulingapi.TaskInfo)
|
|
if !ok {
|
|
klog.Errorf("failed to convert %v to *v1.Pod", obj)
|
|
return
|
|
}
|
|
|
|
if err := sc.syncTask(task); err != nil {
|
|
klog.Errorf("Failed to sync pod <%v/%v>, retry it.", task.Namespace, task.Name)
|
|
sc.resyncTask(task)
|
|
}
|
|
}
|
|
|
|
// Snapshot returns the complete snapshot of the cluster from cache
|
|
func (sc *SchedulerCache) Snapshot() *schedulingapi.ClusterInfo {
|
|
sc.Mutex.Lock()
|
|
defer sc.Mutex.Unlock()
|
|
|
|
snapshot := &schedulingapi.ClusterInfo{
|
|
Nodes: make(map[string]*schedulingapi.NodeInfo),
|
|
Jobs: make(map[schedulingapi.JobID]*schedulingapi.JobInfo),
|
|
Queues: make(map[schedulingapi.QueueID]*schedulingapi.QueueInfo),
|
|
NamespaceInfo: make(map[schedulingapi.NamespaceName]*schedulingapi.NamespaceInfo),
|
|
}
|
|
|
|
for _, value := range sc.Nodes {
|
|
if !value.Ready() {
|
|
continue
|
|
}
|
|
|
|
snapshot.Nodes[value.Name] = value.Clone()
|
|
}
|
|
|
|
for _, value := range sc.Queues {
|
|
snapshot.Queues[value.UID] = value.Clone()
|
|
}
|
|
|
|
var cloneJobLock sync.Mutex
|
|
var wg sync.WaitGroup
|
|
|
|
cloneJob := func(value *schedulingapi.JobInfo) {
|
|
defer wg.Done()
|
|
if value.PodGroup != nil {
|
|
value.Priority = sc.defaultPriority
|
|
|
|
priName := value.PodGroup.Spec.PriorityClassName
|
|
if priorityClass, found := sc.PriorityClasses[priName]; found {
|
|
value.Priority = priorityClass.Value
|
|
}
|
|
|
|
klog.V(4).Infof("The priority of job <%s/%s> is <%s/%d>",
|
|
value.Namespace, value.Name, priName, value.Priority)
|
|
}
|
|
|
|
clonedJob := value.Clone()
|
|
|
|
cloneJobLock.Lock()
|
|
snapshot.Jobs[value.UID] = clonedJob
|
|
cloneJobLock.Unlock()
|
|
}
|
|
|
|
for _, value := range sc.NamespaceCollection {
|
|
info := value.Snapshot()
|
|
snapshot.NamespaceInfo[info.Name] = info
|
|
klog.V(4).Infof("Namespace %s has weight %v",
|
|
value.Name, info.GetWeight())
|
|
}
|
|
|
|
for _, value := range sc.Jobs {
|
|
// If no scheduling spec, does not handle it.
|
|
if value.PodGroup == nil {
|
|
klog.V(4).Infof("The scheduling spec of Job <%v:%s/%s> is nil, ignore it.",
|
|
value.UID, value.Namespace, value.Name)
|
|
|
|
continue
|
|
}
|
|
|
|
if _, found := snapshot.Queues[value.Queue]; !found {
|
|
klog.V(3).Infof("The Queue <%v> of Job <%v/%v> does not exist, ignore it.",
|
|
value.Queue, value.Namespace, value.Name)
|
|
continue
|
|
}
|
|
|
|
wg.Add(1)
|
|
go cloneJob(value)
|
|
}
|
|
wg.Wait()
|
|
|
|
klog.V(3).Infof("There are <%d> Jobs, <%d> Queues and <%d> Nodes in total for scheduling.",
|
|
len(snapshot.Jobs), len(snapshot.Queues), len(snapshot.Nodes))
|
|
|
|
return snapshot
|
|
}
|
|
|
|
// String returns information about the cache in a string format
|
|
func (sc *SchedulerCache) String() string {
|
|
sc.Mutex.Lock()
|
|
defer sc.Mutex.Unlock()
|
|
|
|
str := "Cache:\n"
|
|
|
|
if len(sc.Nodes) != 0 {
|
|
str = str + "Nodes:\n"
|
|
for _, n := range sc.Nodes {
|
|
str = str + fmt.Sprintf("\t %s: idle(%v) used(%v) allocatable(%v) pods(%d)\n",
|
|
n.Name, n.Idle, n.Used, n.Allocatable, len(n.Tasks))
|
|
|
|
i := 0
|
|
for _, p := range n.Tasks {
|
|
str = str + fmt.Sprintf("\t\t %d: %v\n", i, p)
|
|
i++
|
|
}
|
|
}
|
|
}
|
|
|
|
if len(sc.Jobs) != 0 {
|
|
str = str + "Jobs:\n"
|
|
for _, job := range sc.Jobs {
|
|
str = str + fmt.Sprintf("\t %s\n", job)
|
|
}
|
|
}
|
|
|
|
if len(sc.NamespaceCollection) != 0 {
|
|
str = str + "Namespaces:\n"
|
|
for _, ns := range sc.NamespaceCollection {
|
|
info := ns.Snapshot()
|
|
str = str + fmt.Sprintf("\t Namespace(%s) Weight(%v)\n",
|
|
info.Name, info.Weight)
|
|
}
|
|
}
|
|
|
|
return str
|
|
}
|
|
|
|
// RecordJobStatusEvent records related events according to job status.
|
|
func (sc *SchedulerCache) RecordJobStatusEvent(job *schedulingapi.JobInfo) {
|
|
baseErrorMessage := job.JobFitErrors
|
|
if baseErrorMessage == "" {
|
|
baseErrorMessage = schedulingapi.AllNodeUnavailableMsg
|
|
}
|
|
|
|
pgUnschedulable := job.PodGroup != nil &&
|
|
(job.PodGroup.Status.Phase == scheduling.PodGroupUnknown ||
|
|
job.PodGroup.Status.Phase == scheduling.PodGroupPending ||
|
|
job.PodGroup.Status.Phase == scheduling.PodGroupInqueue)
|
|
|
|
// If pending or unschedulable, record unschedulable event.
|
|
if pgUnschedulable {
|
|
msg := fmt.Sprintf("%v/%v tasks in gang unschedulable: %v",
|
|
len(job.TaskStatusIndex[schedulingapi.Pending]),
|
|
len(job.Tasks),
|
|
job.FitError())
|
|
sc.recordPodGroupEvent(job.PodGroup, v1.EventTypeWarning, string(scheduling.PodGroupUnschedulableType), msg)
|
|
}
|
|
|
|
// Update podCondition for tasks Allocated and Pending before job discarded
|
|
for _, status := range []schedulingapi.TaskStatus{schedulingapi.Allocated, schedulingapi.Pending, schedulingapi.Pipelined} {
|
|
for _, taskInfo := range job.TaskStatusIndex[status] {
|
|
msg := baseErrorMessage
|
|
fitError := job.NodesFitErrors[taskInfo.UID]
|
|
if fitError != nil {
|
|
msg = fitError.Error()
|
|
}
|
|
if err := sc.taskUnschedulable(taskInfo, msg); err != nil {
|
|
klog.Errorf("Failed to update unschedulable task status <%s/%s>: %v",
|
|
taskInfo.Namespace, taskInfo.Name, err)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// UpdateJobStatus update the status of job and its tasks.
|
|
func (sc *SchedulerCache) UpdateJobStatus(job *schedulingapi.JobInfo, updatePG bool) (*schedulingapi.JobInfo, error) {
|
|
if updatePG {
|
|
pg, err := sc.StatusUpdater.UpdatePodGroup(job.PodGroup)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
job.PodGroup = pg
|
|
}
|
|
|
|
sc.RecordJobStatusEvent(job)
|
|
|
|
return job, nil
|
|
}
|
|
|
|
func (sc *SchedulerCache) recordPodGroupEvent(podGroup *schedulingapi.PodGroup, eventType, reason, msg string) {
|
|
if podGroup == nil {
|
|
return
|
|
}
|
|
|
|
pg := &vcv1beta1.PodGroup{}
|
|
if err := schedulingscheme.Scheme.Convert(&podGroup.PodGroup, pg, nil); err != nil {
|
|
klog.Errorf("Error while converting PodGroup to v1alpha1.PodGroup with error: %v", err)
|
|
return
|
|
}
|
|
|
|
sc.Recorder.Eventf(pg, eventType, reason, msg)
|
|
|
|
return
|
|
}
|