Added VolumeScheduling.
Signed-off-by: Da K. Ma <klaus1982.cn@gmail.com>
This commit is contained in:
parent
1f16f6804a
commit
3f7c4561fb
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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())
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue