Skip to content

Commit

Permalink
Provide Kubeflow PyTorchJob support for MultiKueue (#2735)
Browse files Browse the repository at this point in the history
* Provide Kubeflow PyTorchJob support for MultiKueue

* update after code review
  • Loading branch information
mszadkow authored Aug 2, 2024
1 parent facc679 commit bdc0749
Show file tree
Hide file tree
Showing 13 changed files with 638 additions and 40 deletions.
1 change: 1 addition & 0 deletions charts/kueue/templates/rbac/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,7 @@ rules:
- pytorchjobs/status
verbs:
- get
- patch
- update
- apiGroups:
- kubeflow.org
Expand Down
1 change: 1 addition & 0 deletions config/components/rbac/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -277,6 +277,7 @@ rules:
- pytorchjobs/status
verbs:
- get
- patch
- update
- apiGroups:
- kubeflow.org
Expand Down
3 changes: 2 additions & 1 deletion pkg/controller/jobframework/validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,8 @@ var (
batchv1.SchemeGroupVersion.WithKind("Job").String(),
jobset.SchemeGroupVersion.WithKind("JobSet").String(),
kftraining.SchemeGroupVersion.WithKind(kftraining.TFJobKind).String(),
kftraining.SchemeGroupVersion.WithKind(kftraining.PaddleJobKind).String())
kftraining.SchemeGroupVersion.WithKind(kftraining.PaddleJobKind).String(),
kftraining.SchemeGroupVersion.WithKind(kftraining.PyTorchJobKind).String())
)

// ValidateJobOnCreate encapsulates all GenericJob validations that must be performed on a Create operation
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
Copyright 2024 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 pytorchjob

import (
"context"
"errors"
"fmt"

kftraining "github.com/kubeflow/training-operator/pkg/apis/kubeflow.org/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/apimachinery/pkg/types"
"k8s.io/klog/v2"
"sigs.k8s.io/controller-runtime/pkg/client"

kueuealpha "sigs.k8s.io/kueue/apis/kueue/v1alpha1"
"sigs.k8s.io/kueue/pkg/controller/constants"
"sigs.k8s.io/kueue/pkg/controller/jobframework"
"sigs.k8s.io/kueue/pkg/util/api"
clientutil "sigs.k8s.io/kueue/pkg/util/client"
)

type multikueueAdapter struct{}

var _ jobframework.MultiKueueAdapter = (*multikueueAdapter)(nil)

func (b *multikueueAdapter) SyncJob(ctx context.Context, localClient client.Client, remoteClient client.Client, key types.NamespacedName, workloadName, origin string) error {
localJob := kftraining.PyTorchJob{}
err := localClient.Get(ctx, key, &localJob)
if err != nil {
return err
}

remoteJob := &kftraining.PyTorchJob{}
err = remoteClient.Get(ctx, key, remoteJob)
if client.IgnoreNotFound(err) != nil {
return err
}

// if the remote exists, just copy the status
if err == nil {
return clientutil.PatchStatus(ctx, localClient, &localJob, func() (bool, error) {
localJob.Status = remoteJob.Status
return true, nil
})
}

remoteJob = &kftraining.PyTorchJob{
ObjectMeta: api.CloneObjectMetaForCreation(&localJob.ObjectMeta),
Spec: *localJob.Spec.DeepCopy(),
}

// add the prebuilt workload
if remoteJob.Labels == nil {
remoteJob.Labels = make(map[string]string, 2)
}
remoteJob.Labels[constants.PrebuiltWorkloadLabel] = workloadName
remoteJob.Labels[kueuealpha.MultiKueueOriginLabel] = origin

return remoteClient.Create(ctx, remoteJob)
}

func (b *multikueueAdapter) DeleteRemoteObject(ctx context.Context, remoteClient client.Client, key types.NamespacedName) error {
job := kftraining.PyTorchJob{}
err := remoteClient.Get(ctx, key, &job)
if err != nil {
return client.IgnoreNotFound(err)
}
return client.IgnoreNotFound(remoteClient.Delete(ctx, &job))
}

func (b *multikueueAdapter) KeepAdmissionCheckPending() bool {
return false
}

func (b *multikueueAdapter) IsJobManagedByKueue(context.Context, client.Client, types.NamespacedName) (bool, string, error) {
return true, "", nil
}

func (b *multikueueAdapter) GVK() schema.GroupVersionKind {
return gvk
}

var _ jobframework.MultiKueueWatcher = (*multikueueAdapter)(nil)

func (*multikueueAdapter) GetEmptyList() client.ObjectList {
return &kftraining.PyTorchJobList{}
}

func (*multikueueAdapter) WorkloadKeyFor(o runtime.Object) (types.NamespacedName, error) {
pyTorchJob, isPyTorchJob := o.(*kftraining.PyTorchJob)
if !isPyTorchJob {
return types.NamespacedName{}, errors.New("not a PyTorchJob")
}

prebuiltWl, hasPrebuiltWorkload := pyTorchJob.Labels[constants.PrebuiltWorkloadLabel]
if !hasPrebuiltWorkload {
return types.NamespacedName{}, fmt.Errorf("no prebuilt workload found for PyTorchJob: %s", klog.KObj(pyTorchJob))
}

return types.NamespacedName{Name: prebuiltWl, Namespace: pyTorchJob.Namespace}, nil
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,160 @@
/*
Copyright 2024 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 pytorchjob

import (
"context"
"testing"

"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
kftraining "github.com/kubeflow/training-operator/pkg/apis/kubeflow.org/v1"
corev1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"

"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/client/interceptor"

kueuealpha "sigs.k8s.io/kueue/apis/kueue/v1alpha1"
"sigs.k8s.io/kueue/pkg/controller/constants"
"sigs.k8s.io/kueue/pkg/util/slices"
utiltesting "sigs.k8s.io/kueue/pkg/util/testing"
kfutiltesting "sigs.k8s.io/kueue/pkg/util/testingjobs/pytorchjob"
)

const (
TestNamespace = "ns"
)

func TestMultikueueAdapter(t *testing.T) {
objCheckOpts := []cmp.Option{
cmpopts.IgnoreFields(metav1.ObjectMeta{}, "ResourceVersion"),
cmpopts.EquateEmpty(),
}

pyTorchJobBuilder := kfutiltesting.MakePyTorchJob("pytorchjob1", TestNamespace).Queue("queue").Suspend(false)

cases := map[string]struct {
managersPyTorchJobs []kftraining.PyTorchJob
workerPyTorchJobs []kftraining.PyTorchJob

operation func(ctx context.Context, adapter *multikueueAdapter, managerClient, workerClient client.Client) error

wantError error
wantManagersPyTorchJobs []kftraining.PyTorchJob
wantWorkerPyTorchJobs []kftraining.PyTorchJob
}{
"sync creates missing remote pytorchjob": {
managersPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().Obj(),
},
operation: func(ctx context.Context, adapter *multikueueAdapter, managerClient, workerClient client.Client) error {
return adapter.SyncJob(ctx, managerClient, workerClient, types.NamespacedName{Name: "pytorchjob1", Namespace: TestNamespace}, "wl1", "origin1")
},

wantManagersPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().Obj(),
},
wantWorkerPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().
Label(constants.PrebuiltWorkloadLabel, "wl1").
Label(kueuealpha.MultiKueueOriginLabel, "origin1").
Obj(),
},
},
"sync status from remote pytorchjob": {
managersPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().Obj(),
},
workerPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().
Label(constants.PrebuiltWorkloadLabel, "wl1").
Label(kueuealpha.MultiKueueOriginLabel, "origin1").
StatusConditions(kftraining.JobCondition{Type: kftraining.JobSucceeded, Status: corev1.ConditionTrue}).
Obj(),
},
operation: func(ctx context.Context, adapter *multikueueAdapter, managerClient, workerClient client.Client) error {
return adapter.SyncJob(ctx, managerClient, workerClient, types.NamespacedName{Name: "pytorchjob1", Namespace: TestNamespace}, "wl1", "origin1")
},

wantManagersPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().
StatusConditions(kftraining.JobCondition{Type: kftraining.JobSucceeded, Status: corev1.ConditionTrue}).
Obj(),
},
wantWorkerPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().
Label(constants.PrebuiltWorkloadLabel, "wl1").
Label(kueuealpha.MultiKueueOriginLabel, "origin1").
StatusConditions(kftraining.JobCondition{Type: kftraining.JobSucceeded, Status: corev1.ConditionTrue}).
Obj(),
},
},
"remote pytorchjob is deleted": {
workerPyTorchJobs: []kftraining.PyTorchJob{
*pyTorchJobBuilder.Clone().
Label(constants.PrebuiltWorkloadLabel, "wl1").
Label(kueuealpha.MultiKueueOriginLabel, "origin1").
Obj(),
},
operation: func(ctx context.Context, adapter *multikueueAdapter, managerClient, workerClient client.Client) error {
return adapter.DeleteRemoteObject(ctx, workerClient, types.NamespacedName{Name: "pytorchjob1", Namespace: TestNamespace})
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
managerBuilder := utiltesting.NewClientBuilder(kftraining.AddToScheme).WithInterceptorFuncs(interceptor.Funcs{SubResourcePatch: utiltesting.TreatSSAAsStrategicMerge})
managerBuilder = managerBuilder.WithLists(&kftraining.PyTorchJobList{Items: tc.managersPyTorchJobs})
managerBuilder = managerBuilder.WithStatusSubresource(slices.Map(tc.managersPyTorchJobs, func(w *kftraining.PyTorchJob) client.Object { return w })...)
managerClient := managerBuilder.Build()

workerBuilder := utiltesting.NewClientBuilder(kftraining.AddToScheme).WithInterceptorFuncs(interceptor.Funcs{SubResourcePatch: utiltesting.TreatSSAAsStrategicMerge})
workerBuilder = workerBuilder.WithLists(&kftraining.PyTorchJobList{Items: tc.workerPyTorchJobs})
workerClient := workerBuilder.Build()

ctx, _ := utiltesting.ContextWithLog(t)

adapter := &multikueueAdapter{}

gotErr := tc.operation(ctx, adapter, managerClient, workerClient)

if diff := cmp.Diff(tc.wantError, gotErr, cmpopts.EquateErrors()); diff != "" {
t.Errorf("unexpected error (-want/+got):\n%s", diff)
}

gotManagersPyTorchJob := &kftraining.PyTorchJobList{}
if err := managerClient.List(ctx, gotManagersPyTorchJob); err != nil {
t.Errorf("unexpected list manager's pytorchjobs error %s", err)
} else {
if diff := cmp.Diff(tc.wantManagersPyTorchJobs, gotManagersPyTorchJob.Items, objCheckOpts...); diff != "" {
t.Errorf("unexpected manager's pytorchjobs (-want/+got):\n%s", diff)
}
}

gotWorkerPyTorchJobs := &kftraining.PyTorchJobList{}
if err := workerClient.List(ctx, gotWorkerPyTorchJobs); err != nil {
t.Errorf("unexpected list worker's pytorchjobs error %s", err)
} else {
if diff := cmp.Diff(tc.wantWorkerPyTorchJobs, gotWorkerPyTorchJobs.Items, objCheckOpts...); diff != "" {
t.Errorf("unexpected worker's pytorchjobs (-want/+got):\n%s", diff)
}
}
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,13 +45,14 @@ func init() {
JobType: &kftraining.PyTorchJob{},
AddToScheme: kftraining.AddToScheme,
IsManagingObjectsOwner: isPyTorchJob,
MultiKueueAdapter: &multikueueAdapter{},
}))
}

// +kubebuilder:rbac:groups=scheduling.k8s.io,resources=priorityclasses,verbs=list;get;watch
// +kubebuilder:rbac:groups="",resources=events,verbs=create;watch;update;patch
// +kubebuilder:rbac:groups=kubeflow.org,resources=pytorchjobs,verbs=get;list;watch;update;patch
// +kubebuilder:rbac:groups=kubeflow.org,resources=pytorchjobs/status,verbs=get;update
// +kubebuilder:rbac:groups=kubeflow.org,resources=pytorchjobs/status,verbs=get;update;patch
// +kubebuilder:rbac:groups=kubeflow.org,resources=pytorchjobs/finalizers,verbs=get;update
// +kubebuilder:rbac:groups=kueue.x-k8s.io,resources=workloads,verbs=get;list;watch;create;update;patch;delete
// +kubebuilder:rbac:groups=kueue.x-k8s.io,resources=workloads/status,verbs=get;update;patch
Expand Down
Loading

0 comments on commit bdc0749

Please sign in to comment.