Added VolumeScheduling.

Signed-off-by: Da K. Ma <klaus1982.cn@gmail.com>
This commit is contained in:
Da K. Ma 2019-01-17 10:22:19 +08:00
parent 1f16f6804a
commit 3f7c4561fb
7 changed files with 165 additions and 81 deletions

View File

@ -24,7 +24,7 @@ import (
"testing"
"time"
v1 "k8s.io/api/core/v1"
"k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/api/resource"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
@ -95,14 +95,6 @@ func buildPod(ns, n, nn string, p v1.PodPhase, req v1.ResourceList, groupName st
}
}
func buildOwnerReference(owner string) metav1.OwnerReference {
controller := true
return metav1.OwnerReference{
Controller: &controller,
UID: types.UID(owner),
}
}
type fakeBinder struct {
sync.Mutex
binds map[string]string
@ -129,6 +121,16 @@ func (ftsu *fakeTaskStatusUpdater) Update(pod *v1.Pod, podCondition *v1.PodCondi
return nil
}
type fakeVolumeBinder struct {
}
func (fvb *fakeVolumeBinder) AllocateVolumes(task *api.TaskInfo, hostname string) error {
return nil
}
func (fvb *fakeVolumeBinder) BindVolumes(task *api.TaskInfo) error {
return nil
}
func TestAllocate(t *testing.T) {
framework.RegisterPluginBuilder("drf", drf.New)
framework.RegisterPluginBuilder("proportion", proportion.New)
@ -237,11 +239,12 @@ func TestAllocate(t *testing.T) {
c: make(chan string),
}
schedulerCache := &cache.SchedulerCache{
Nodes: make(map[string]*api.NodeInfo),
Jobs: make(map[api.JobID]*api.JobInfo),
Queues: make(map[api.QueueID]*api.QueueInfo),
Binder: binder,
TsUpdater: &fakeTaskStatusUpdater{},
Nodes: make(map[string]*api.NodeInfo),
Jobs: make(map[api.JobID]*api.JobInfo),
Queues: make(map[api.QueueID]*api.QueueInfo),
Binder: binder,
TaskStatusUpdater: &fakeTaskStatusUpdater{},
VolumeBinder: &fakeVolumeBinder{},
}
for _, node := range test.nodes {
schedulerCache.AddNode(node)

View File

@ -42,9 +42,10 @@ type TaskInfo struct {
Resreq *Resource
NodeName string
Status TaskStatus
Priority int32
NodeName string
Status TaskStatus
Priority int32
VolumeReady bool
Pod *v1.Pod
}
@ -89,15 +90,16 @@ func NewTaskInfo(pod *v1.Pod) *TaskInfo {
func (ti *TaskInfo) Clone() *TaskInfo {
return &TaskInfo{
UID: ti.UID,
Job: ti.Job,
Name: ti.Name,
Namespace: ti.Namespace,
NodeName: ti.NodeName,
Status: ti.Status,
Priority: ti.Priority,
Pod: ti.Pod,
Resreq: ti.Resreq.Clone(),
UID: ti.UID,
Job: ti.Job,
Name: ti.Name,
Namespace: ti.Namespace,
NodeName: ti.NodeName,
Status: ti.Status,
Priority: ti.Priority,
Pod: ti.Pod,
Resreq: ti.Resreq.Clone(),
VolumeReady: ti.VolumeReady,
}
}

View File

@ -24,25 +24,27 @@ import (
"github.com/golang/glog"
v1 "k8s.io/api/core/v1"
"k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/client-go/informers"
infov1 "k8s.io/client-go/informers/core/v1"
policyv1 "k8s.io/client-go/informers/policy/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"
podutil "k8s.io/kubernetes/pkg/api/v1/pod"
"k8s.io/kubernetes/pkg/scheduler/volumebinder"
arbcorev1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1"
"github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned"
kbv1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1"
kbver "github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned"
"github.com/kubernetes-sigs/kube-batch/pkg/client/clientset/versioned/scheme"
arbinfo "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions"
arbcoreinfo "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions/scheduling/v1alpha1"
kbinfo "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions"
kbinfov1 "github.com/kubernetes-sigs/kube-batch/pkg/client/informers/externalversions/scheduling/v1alpha1"
"github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api"
arbapi "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api"
kbapi "github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api"
)
// New returns a Cache implementation.
@ -54,24 +56,28 @@ type SchedulerCache struct {
sync.Mutex
kubeclient *kubernetes.Clientset
arbclient *versioned.Clientset
kbclient *kbver.Clientset
podInformer infov1.PodInformer
nodeInformer infov1.NodeInformer
pdbInformer policyv1.PodDisruptionBudgetInformer
nsInformer infov1.NamespaceInformer
podGroupInformer arbcoreinfo.PodGroupInformer
queueInformer arbcoreinfo.QueueInformer
podGroupInformer kbinfov1.PodGroupInformer
queueInformer kbinfov1.QueueInformer
pvInformer infov1.PersistentVolumeInformer
pvcInformer infov1.PersistentVolumeClaimInformer
scInformer storagev1.StorageClassInformer
Binder Binder
Evictor Evictor
TsUpdater TaskStatusUpdater
Binder Binder
Evictor Evictor
TaskStatusUpdater TaskStatusUpdater
VolumeBinder VolumeBinder
recorder record.EventRecorder
Jobs map[arbapi.JobID]*arbapi.JobInfo
Nodes map[string]*arbapi.NodeInfo
Queues map[arbapi.QueueID]*arbapi.QueueInfo
Jobs map[kbapi.JobID]*kbapi.JobInfo
Nodes map[string]*kbapi.NodeInfo
Queues map[kbapi.QueueID]*kbapi.QueueInfo
errTasks *cache.FIFO
deletedJobs *cache.FIFO
@ -131,12 +137,34 @@ func (tsUpdater *defaultTaskStatusUpdater) Update(pod *v1.Pod, condition *v1.Pod
return nil
}
type defaultVolumeBinder struct {
volumeBinder *volumebinder.VolumeBinder
}
// AllocateVolume allocates volume on the host to the task
func (dvb *defaultVolumeBinder) AllocateVolumes(task *api.TaskInfo, hostname string) error {
allBound, err := dvb.volumeBinder.Binder.AssumePodVolumes(task.Pod, hostname)
task.VolumeReady = allBound
return err
}
// BindVolume binds volumes to the task
func (dvb *defaultVolumeBinder) BindVolumes(task *api.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 taskKey(obj interface{}) (string, error) {
if obj == nil {
return "", fmt.Errorf("the object is nil")
}
task, ok := obj.(*arbapi.TaskInfo)
task, ok := obj.(*kbapi.TaskInfo)
if !ok {
return "", fmt.Errorf("failed to convert %v to TaskInfo", obj)
@ -150,7 +178,7 @@ func jobKey(obj interface{}) (string, error) {
return "", fmt.Errorf("the object is nil")
}
job, ok := obj.(*arbapi.JobInfo)
job, ok := obj.(*kbapi.JobInfo)
if !ok {
return "", fmt.Errorf("failed to convert %v to TaskInfo", obj)
@ -161,13 +189,13 @@ func jobKey(obj interface{}) (string, error) {
func newSchedulerCache(config *rest.Config, schedulerName string, nsAsQueue bool) *SchedulerCache {
sc := &SchedulerCache{
Jobs: make(map[arbapi.JobID]*arbapi.JobInfo),
Nodes: make(map[string]*arbapi.NodeInfo),
Queues: make(map[arbapi.QueueID]*arbapi.QueueInfo),
Jobs: make(map[kbapi.JobID]*kbapi.JobInfo),
Nodes: make(map[string]*kbapi.NodeInfo),
Queues: make(map[kbapi.QueueID]*kbapi.QueueInfo),
errTasks: cache.NewFIFO(taskKey),
deletedJobs: cache.NewFIFO(jobKey),
kubeclient: kubernetes.NewForConfigOrDie(config),
arbclient: versioned.NewForConfigOrDie(config),
kbclient: kbver.NewForConfigOrDie(config),
namespaceAsQueue: nsAsQueue,
}
@ -184,10 +212,23 @@ func newSchedulerCache(config *rest.Config, schedulerName string, nsAsQueue bool
kubeclient: sc.kubeclient,
}
sc.TsUpdater = &defaultTaskStatusUpdater{kubeclient: sc.kubeclient}
sc.TaskStatusUpdater = &defaultTaskStatusUpdater{kubeclient: sc.kubeclient}
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.pvcInformer,
sc.pvInformer,
sc.scInformer,
30*time.Second,
),
}
// create informer for node information
sc.nodeInformer = informerFactory.Core().V1().Nodes()
sc.nodeInformer.Informer().AddEventHandlerWithResyncPeriod(
@ -229,9 +270,9 @@ func newSchedulerCache(config *rest.Config, schedulerName string, nsAsQueue bool
DeleteFunc: sc.DeletePDB,
})
arbinformer := arbinfo.NewSharedInformerFactory(sc.arbclient, 0)
kbinformer := kbinfo.NewSharedInformerFactory(sc.kbclient, 0)
// create informer for PodGroup information
sc.podGroupInformer = arbinformer.Scheduling().V1alpha1().PodGroups()
sc.podGroupInformer = kbinformer.Scheduling().V1alpha1().PodGroups()
sc.podGroupInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
AddFunc: sc.AddPodGroup,
UpdateFunc: sc.UpdatePodGroup,
@ -248,7 +289,7 @@ func newSchedulerCache(config *rest.Config, schedulerName string, nsAsQueue bool
})
} else {
// create informer for Queue information
sc.queueInformer = arbinformer.Scheduling().V1alpha1().Queues()
sc.queueInformer = kbinformer.Scheduling().V1alpha1().Queues()
sc.queueInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
AddFunc: sc.AddQueue,
UpdateFunc: sc.UpdateQueue,
@ -264,6 +305,9 @@ func (sc *SchedulerCache) Run(stopCh <-chan struct{}) {
go sc.podInformer.Informer().Run(stopCh)
go sc.nodeInformer.Informer().Run(stopCh)
go sc.podGroupInformer.Informer().Run(stopCh)
go sc.pvInformer.Informer().Run(stopCh)
go sc.pvcInformer.Informer().Run(stopCh)
go sc.scInformer.Informer().Run(stopCh)
if sc.namespaceAsQueue {
go sc.nsInformer.Informer().Run(stopCh)
@ -291,11 +335,14 @@ func (sc *SchedulerCache) WaitForCacheSync(stopCh <-chan struct{}) bool {
sc.podInformer.Informer().HasSynced,
sc.podGroupInformer.Informer().HasSynced,
sc.nodeInformer.Informer().HasSynced,
sc.pvInformer.Informer().HasSynced,
sc.pvcInformer.Informer().HasSynced,
sc.scInformer.Informer().HasSynced,
queueSync,
)
}
func (sc *SchedulerCache) findJobAndTask(taskInfo *arbapi.TaskInfo) (*arbapi.JobInfo, *arbapi.TaskInfo, error) {
func (sc *SchedulerCache) findJobAndTask(taskInfo *kbapi.TaskInfo) (*kbapi.JobInfo, *kbapi.TaskInfo, error) {
job, found := sc.Jobs[taskInfo.Job]
if !found {
return nil, nil, fmt.Errorf("failed to find Job %v for Task %v",
@ -311,7 +358,7 @@ func (sc *SchedulerCache) findJobAndTask(taskInfo *arbapi.TaskInfo) (*arbapi.Job
return job, task, nil
}
func (sc *SchedulerCache) Evict(taskInfo *arbapi.TaskInfo, reason string) error {
func (sc *SchedulerCache) Evict(taskInfo *kbapi.TaskInfo, reason string) error {
sc.Mutex.Lock()
defer sc.Mutex.Unlock()
@ -327,7 +374,7 @@ func (sc *SchedulerCache) Evict(taskInfo *arbapi.TaskInfo, reason string) error
task.UID, task.NodeName)
}
err = job.UpdateTaskStatus(task, arbapi.Releasing)
err = job.UpdateTaskStatus(task, kbapi.Releasing)
if err != nil {
return err
}
@ -344,13 +391,13 @@ func (sc *SchedulerCache) Evict(taskInfo *arbapi.TaskInfo, reason string) error
}
}()
sc.recorder.Eventf(job.PodGroup, v1.EventTypeNormal, string(arbcorev1.EvictEvent), reason)
sc.recorder.Eventf(job.PodGroup, v1.EventTypeNormal, string(kbv1.EvictEvent), reason)
return nil
}
// Bind binds task to the target host.
func (sc *SchedulerCache) Bind(taskInfo *arbapi.TaskInfo, hostname string) error {
func (sc *SchedulerCache) Bind(taskInfo *kbapi.TaskInfo, hostname string) error {
sc.Mutex.Lock()
defer sc.Mutex.Unlock()
@ -366,7 +413,7 @@ func (sc *SchedulerCache) Bind(taskInfo *arbapi.TaskInfo, hostname string) error
task.UID, hostname)
}
err = job.UpdateTaskStatus(task, arbapi.Binding)
err = job.UpdateTaskStatus(task, kbapi.Binding)
if err != nil {
return err
}
@ -388,15 +435,25 @@ func (sc *SchedulerCache) Bind(taskInfo *arbapi.TaskInfo, hostname string) error
return nil
}
// AllocateVolume allocates volume on the host to the task
func (sc *SchedulerCache) AllocateVolumes(task *api.TaskInfo, hostname string) error {
return sc.VolumeBinder.AllocateVolumes(task, hostname)
}
// BindVolume binds volumes to the task
func (sc *SchedulerCache) BindVolumes(task *api.TaskInfo) error {
return sc.VolumeBinder.BindVolumes(task)
}
// TaskUnschedulable updates pod status of pending task
func (sc *SchedulerCache) TaskUnschedulable(task *api.TaskInfo, event arbcorev1.Event, reason string) error {
func (sc *SchedulerCache) TaskUnschedulable(task *api.TaskInfo, event kbv1.Event, reason string) error {
sc.Mutex.Lock()
defer sc.Mutex.Unlock()
pod := task.Pod.DeepCopy()
sc.recorder.Eventf(pod, v1.EventTypeWarning, string(event), reason)
sc.TsUpdater.Update(pod, &v1.PodCondition{
sc.TaskStatusUpdater.Update(pod, &v1.PodCondition{
Type: v1.PodScheduled,
Status: v1.ConditionFalse,
Reason: v1.PodReasonUnschedulable,
@ -406,7 +463,7 @@ func (sc *SchedulerCache) TaskUnschedulable(task *api.TaskInfo, event arbcorev1.
return nil
}
func (sc *SchedulerCache) deleteJob(job *arbapi.JobInfo) {
func (sc *SchedulerCache) deleteJob(job *kbapi.JobInfo) {
glog.V(3).Infof("Try to delete Job <%v:%v/%v>", job.UID, job.Namespace, job.Name)
time.AfterFunc(5*time.Second, func() {
@ -416,7 +473,7 @@ func (sc *SchedulerCache) deleteJob(job *arbapi.JobInfo) {
func (sc *SchedulerCache) processCleanupJob() error {
_, err := sc.deletedJobs.Pop(func(obj interface{}) error {
job, ok := obj.(*arbapi.JobInfo)
job, ok := obj.(*kbapi.JobInfo)
if !ok {
return fmt.Errorf("failed to convert %v to *v1.Pod", obj)
}
@ -425,7 +482,7 @@ func (sc *SchedulerCache) processCleanupJob() error {
sc.Mutex.Lock()
defer sc.Mutex.Unlock()
if arbapi.JobTerminated(job) {
if kbapi.JobTerminated(job) {
delete(sc.Jobs, job.UID)
glog.V(3).Infof("Job <%v:%v/%v> was deleted.", job.UID, job.Namespace, job.Name)
} else {
@ -449,7 +506,7 @@ func (sc *SchedulerCache) cleanupJobs() {
}
}
func (sc *SchedulerCache) resyncTask(task *arbapi.TaskInfo) {
func (sc *SchedulerCache) resyncTask(task *kbapi.TaskInfo) {
sc.errTasks.AddIfNotPresent(task)
}
@ -464,7 +521,7 @@ func (sc *SchedulerCache) resync() {
func (sc *SchedulerCache) processResyncTask() error {
_, err := sc.errTasks.Pop(func(obj interface{}) error {
task, ok := obj.(*arbapi.TaskInfo)
task, ok := obj.(*kbapi.TaskInfo)
if !ok {
return fmt.Errorf("failed to convert %v to *v1.Pod", obj)
}
@ -479,22 +536,22 @@ func (sc *SchedulerCache) processResyncTask() error {
return err
}
func (sc *SchedulerCache) Snapshot() *arbapi.ClusterInfo {
func (sc *SchedulerCache) Snapshot() *kbapi.ClusterInfo {
sc.Mutex.Lock()
defer sc.Mutex.Unlock()
snapshot := &arbapi.ClusterInfo{
Nodes: make([]*arbapi.NodeInfo, 0, len(sc.Nodes)),
Jobs: make([]*arbapi.JobInfo, 0, len(sc.Jobs)),
Queues: make([]*arbapi.QueueInfo, 0, len(sc.Queues)),
Others: make([]*arbapi.TaskInfo, 0, 10),
snapshot := &kbapi.ClusterInfo{
Nodes: make([]*kbapi.NodeInfo, 0, len(sc.Nodes)),
Jobs: make([]*kbapi.JobInfo, 0, len(sc.Jobs)),
Queues: make([]*kbapi.QueueInfo, 0, len(sc.Queues)),
Others: make([]*kbapi.TaskInfo, 0, 10),
}
for _, value := range sc.Nodes {
snapshot.Nodes = append(snapshot.Nodes, value.Clone())
}
queues := map[arbapi.QueueID]struct{}{}
queues := map[kbapi.QueueID]struct{}{}
for _, value := range sc.Queues {
snapshot.Queues = append(snapshot.Queues, value.Clone())
queues[value.UID] = struct{}{}
@ -507,7 +564,7 @@ func (sc *SchedulerCache) Snapshot() *arbapi.ClusterInfo {
value.UID, value.Namespace, value.Name)
// Also tracing the running task assigned by other scheduler.
for _, task := range value.TaskStatusIndex[arbapi.Running] {
for _, task := range value.TaskStatusIndex[kbapi.Running] {
snapshot.Others = append(snapshot.Others, task.Clone())
}
@ -581,7 +638,7 @@ func (sc *SchedulerCache) String() string {
}
// Backoff record event for job
func (sc *SchedulerCache) Backoff(job *arbapi.JobInfo, event arbcorev1.Event, reason string) error {
func (sc *SchedulerCache) Backoff(job *kbapi.JobInfo, event kbv1.Event, reason string) error {
if job.PodGroup != nil {
sc.recorder.Eventf(job.PodGroup, v1.EventTypeWarning, string(event), reason)
} else if job.PDB != nil {

View File

@ -251,10 +251,10 @@ func TestTaskUnschedulable(t *testing.T) {
tsUpdater.On("Update", taskInfo.Pod, mock.Anything).Return(nil)
cache := &SchedulerCache{
Nodes: make(map[string]*api.NodeInfo),
Jobs: make(map[api.JobID]*api.JobInfo),
TsUpdater: tsUpdater,
recorder: record.NewFakeRecorder(100),
Nodes: make(map[string]*api.NodeInfo),
Jobs: make(map[api.JobID]*api.JobInfo),
TaskStatusUpdater: tsUpdater,
recorder: record.NewFakeRecorder(100),
}
result := cache.TaskUnschedulable(taskInfo, arbcorev1.FailedSchedulingEvent, jobInfo.FitError())

View File

@ -19,7 +19,7 @@ package cache
import (
arbcorev1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1"
"github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api"
v1 "k8s.io/api/core/v1"
"k8s.io/api/core/v1"
)
// Cache collects pods/nodes/queues information
@ -49,6 +49,17 @@ type Cache interface {
// TaskUnschedulable updates pod status of pending task
TaskUnschedulable(task *api.TaskInfo, event arbcorev1.Event, reason string) error
// AllocateVolumes allocates volume on the host to the task
AllocateVolumes(task *api.TaskInfo, hostname string) error
// BindVolumes binds volumes to the task
BindVolumes(task *api.TaskInfo) error
}
type VolumeBinder interface {
AllocateVolumes(task *api.TaskInfo, hostname string) error
BindVolumes(task *api.TaskInfo) error
}
type Binder interface {

View File

@ -160,6 +160,10 @@ func (ssn *Session) Pipeline(task *api.TaskInfo, hostname string) error {
}
func (ssn *Session) Allocate(task *api.TaskInfo, hostname string) error {
if err := ssn.cache.AllocateVolumes(task, hostname); err != nil {
return err
}
// Only update status in session
job, found := ssn.JobIndex[task.Job]
if found {
@ -208,6 +212,10 @@ func (ssn *Session) Allocate(task *api.TaskInfo, hostname string) error {
}
func (ssn *Session) dispatch(task *api.TaskInfo) error {
if err := ssn.cache.BindVolumes(task); err != nil {
return err
}
if err := ssn.cache.Bind(task, task.NodeName); err != nil {
return err
}

View File

@ -21,7 +21,7 @@ import (
"github.com/golang/glog"
arbcorev1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1"
kbv1 "github.com/kubernetes-sigs/kube-batch/pkg/apis/scheduling/v1alpha1"
"github.com/kubernetes-sigs/kube-batch/pkg/scheduler/api"
"github.com/kubernetes-sigs/kube-batch/pkg/scheduler/framework"
)
@ -77,7 +77,7 @@ func jobReady(obj interface{}) bool {
func (gp *gangPlugin) OnSessionOpen(ssn *framework.Session) {
for _, job := range ssn.Jobs {
if validTaskNum(job) < job.MinAvailable {
ssn.Backoff(job, arbcorev1.UnschedulableEvent, "not enough valid tasks for gang-scheduling")
ssn.Backoff(job, kbv1.UnschedulableEvent, "not enough valid tasks for gang-scheduling")
}
}
@ -151,7 +151,10 @@ func (gp *gangPlugin) OnSessionClose(ssn *framework.Session) {
if len(job.TaskStatusIndex[api.Pending]) != 0 {
glog.V(3).Infof("Gang: <%v/%v> allocated: %v, pending: %v", job.Namespace, job.Name, len(job.TaskStatusIndex[api.Allocated]), len(job.TaskStatusIndex[api.Pending]))
msg := fmt.Sprintf("%v/%v tasks in gang unschedulable: %v", len(job.TaskStatusIndex[api.Pending]), len(job.Tasks), job.FitError())
ssn.Backoff(job, arbcorev1.UnschedulableEvent, msg)
if err := ssn.Backoff(job, kbv1.UnschedulableEvent, msg); err != nil {
glog.Errorf("Failed to backoff job <%s/%s>: %v",
job.Namespace, job.Name, err)
}
}
}
}