-
Notifications
You must be signed in to change notification settings - Fork 402
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Support gang scheduling with Apache YuniKorn #2396
Changes from all commits
b561e0a
d72d73a
708b86f
1ceada8
2e962f9
4ee2405
f9b68db
08cc7b3
6a06417
6a3d044
aae020e
10119ed
3ea3b64
31ade1c
d932703
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -12,14 +12,17 @@ import ( | |
|
||
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" | ||
schedulerinterface "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler/interface" | ||
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" | ||
) | ||
|
||
const ( | ||
SchedulerName string = "yunikorn" | ||
YuniKornPodApplicationIDLabelName string = "applicationId" | ||
YuniKornPodQueueLabelName string = "queue" | ||
RayClusterApplicationIDLabelName string = "yunikorn.apache.org/application-id" | ||
RayClusterQueueLabelName string = "yunikorn.apache.org/queue-name" | ||
SchedulerName string = "yunikorn" | ||
YuniKornPodApplicationIDLabelName string = "applicationId" | ||
YuniKornPodQueueLabelName string = "queue" | ||
RayClusterApplicationIDLabelName string = "yunikorn.apache.org/app-id" | ||
RayClusterQueueLabelName string = "yunikorn.apache.org/queue" | ||
YuniKornTaskGroupNameAnnotationName string = "yunikorn.apache.org/task-group-name" | ||
YuniKornTaskGroupsAnnotationName string = "yunikorn.apache.org/task-groups" | ||
) | ||
|
||
type YuniKornScheduler struct { | ||
|
@@ -42,19 +45,66 @@ func (y *YuniKornScheduler) DoBatchSchedulingOnSubmission(_ context.Context, _ * | |
return nil | ||
} | ||
|
||
// populatePodLabels is a helper function that copies RayCluster's label to the given pod based on the label key | ||
// TODO: remove the legacy labels, i.e "applicationId" and "queue", directly populate labels | ||
// RayClusterApplicationIDLabelName to RayClusterQueueLabelName to pod labels. | ||
// Currently we use this function to translate labels "yunikorn.apache.org/app-id" and "yunikorn.apache.org/queue" | ||
// to legacy labels "applicationId" and "queue", this is for the better compatibilities to support older yunikorn | ||
// versions. | ||
func (y *YuniKornScheduler) populatePodLabels(app *rayv1.RayCluster, pod *corev1.Pod, sourceKey string, targetKey string) { | ||
// check labels | ||
if value, exist := app.Labels[sourceKey]; exist { | ||
y.log.Info("Updating pod label based on RayCluster annotations", | ||
y.log.Info("Updating pod label based on RayCluster labels", | ||
"sourceKey", sourceKey, "targetKey", targetKey, "value", value) | ||
pod.Labels[targetKey] = value | ||
} | ||
} | ||
|
||
func (y *YuniKornScheduler) AddMetadataToPod(app *rayv1.RayCluster, _ string, pod *corev1.Pod) { | ||
// AddMetadataToPod adds essential labels and annotations to the Ray pods | ||
// the yunikorn scheduler needs these labels and annotations in order to do the scheduling properly | ||
func (y *YuniKornScheduler) AddMetadataToPod(app *rayv1.RayCluster, groupName string, pod *corev1.Pod) { | ||
// the applicationID and queue name must be provided in the labels | ||
y.populatePodLabels(app, pod, RayClusterApplicationIDLabelName, YuniKornPodApplicationIDLabelName) | ||
y.populatePodLabels(app, pod, RayClusterQueueLabelName, YuniKornPodQueueLabelName) | ||
pod.Spec.SchedulerName = y.Name() | ||
|
||
// when gang scheduling is enabled, extra annotations need to be added to all pods | ||
if y.isGangSchedulingEnabled(app) { | ||
// populate the taskGroups info to each pod | ||
y.populateTaskGroupsAnnotationToPod(app, pod) | ||
|
||
// set the task group name based on the head or worker group name | ||
// the group name for the head and each of the worker group should be different | ||
pod.Annotations[YuniKornTaskGroupNameAnnotationName] = groupName | ||
kevin85421 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
} | ||
|
||
func (y *YuniKornScheduler) isGangSchedulingEnabled(app *rayv1.RayCluster) bool { | ||
_, exist := app.Labels[utils.RayClusterGangSchedulingEnabled] | ||
return exist | ||
} | ||
|
||
func (y *YuniKornScheduler) populateTaskGroupsAnnotationToPod(app *rayv1.RayCluster, pod *corev1.Pod) { | ||
taskGroups := newTaskGroupsFromApp(app) | ||
taskGroupsAnnotationValue, err := taskGroups.marshal() | ||
if err != nil { | ||
y.log.Error(err, "failed to add gang scheduling related annotations to pod, "+ | ||
"gang scheduling will not be enabled for this workload", | ||
"rayCluster", app.Name, "name", pod.Name, "namespace", pod.Namespace) | ||
return | ||
} | ||
|
||
y.log.Info("add task groups info to pod's annotation", | ||
kevin85421 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
"key", YuniKornTaskGroupsAnnotationName, | ||
"value", taskGroupsAnnotationValue, | ||
"numOfTaskGroups", taskGroups.size()) | ||
if pod.Annotations == nil { | ||
pod.Annotations = make(map[string]string) | ||
} | ||
pod.Annotations[YuniKornTaskGroupsAnnotationName] = taskGroupsAnnotationValue | ||
|
||
y.log.Info("Gang Scheduling enabled for RayCluster", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not familiar with Yunikorn. Can we infer that gang scheduling is enabled if the annotation if y.isGangSchedulingEnabled(app) {
...
} instead? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I may not fully understand this. Basically "yunikorn.apache.org/task-groups" only needs to be set to Ray pods by the operator, it is internal metadata. We only use one flag "ray.io/gang-scheduling-enabled" in Ray to tell if we want to enable Gang scheduling or not for this Ray cluster. Only when enabled, the operator populates the task groups info to the pods, then yunikorn will honor and apply gang scheduling. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I found it odd that we print the log saying "Gang Scheduling enabled for RayCluster" in That's why I’m asking if we should move the log to: Copy code
if y.isGangSchedulingEnabled(app) {
...
} instead? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see what you mean. Basically There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it. Thanks! |
||
"RayCluster", app.Name, "Namespace", app.Namespace) | ||
} | ||
|
||
func (yf *YuniKornSchedulerFactory) New(_ *rest.Config) (schedulerinterface.BatchScheduler, error) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't know why the label names need to be translated from
RayClusterApplicationIDLabelName
toYuniKornPodApplicationIDLabelName
and fromRayClusterQueueLabelName
toYuniKornPodQueueLabelName
.Does Yunikorn read the labels on resources other than Pods? If not, why not set
YuniKornPodApplicationIDLabelName
toyunikorn.apache.org/app-id
andYuniKornPodQueueLabelName
toyunikorn.apache.org/queue
and set them directly onRayCluster
, and then apply the same labels to all the Pods that controlled byRayCluster
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the caveat is, yunikorn depends on labels to identify apps, but they are currently using very simple format:
YuniKornPodApplicationIDLabelName
: applicationIdYuniKornPodQueueLabelName
: queuebut in Ray, I think it makes sense to add the domain name to the label, to indicate these lables are set for the scheduler.
the code logic is very simple, just translate these labels and set them on pods, without introducing any inconsistency in Ray cluster spec. Hope that makes sense.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But I viewed the doc here
https://yunikorn.apache.org/docs/next/user_guide/labels_and_annotations_in_yunikorn
there are two labels
yunikorn.apache.org/app-id
andyunikorn.apache.org/queue
. AndapplicationId
andqueue
labels are legacy now.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, good catch. Seems like the community is moving to a domain scoped label approach recently. Let me take a look, and do some tests, will update the PR once confims these labels works. Thanks!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hi @MortalHappiness I just found these labels are only supported in v1.6 which was released just a few weeks ago. For better compatibility with some older version of yunikorn deployment, I think it's better to continue to support the legacy label names. However, I have modified
RayClusterApplicationIDLabelName
andRayClusterQueueLabelName
in the package to be the same as yunikorn upstream. We can remove this translation layer and directly populate the labels to pods after a few releases. Does this sound good?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SGTM. Thanks!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about add a TODO comment here to let us know we need to remove this in the future release?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, updated.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@yangwwei Would you mind briefly adding comments for the context you mentioned in https://github.com/ray-project/kuberay/pull/2396/files#r1777710831?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure