Skip to content

Commit b6bcf10

Browse files
authored
[Feature] [scheduler-plugins] Support second scheduler mode (#3852)
Signed-off-by: Cheyu Wu <[email protected]>
1 parent 67c6b38 commit b6bcf10

File tree

4 files changed

+72
-5
lines changed

4 files changed

+72
-5
lines changed

helm-chart/kuberay-operator/values.yaml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ batchScheduler:
7575
# Deprecated. This option will be removed in the future.
7676
# Note, for backwards compatibility. When it sets to true, it enables volcano scheduler integration.
7777
enabled: false
78-
# Set the customized scheduler name, supported values are "volcano" or "yunikorn", do not set
78+
# Set the customized scheduler name, supported values are "volcano", "yunikorn" or "scheduler-plugins", do not set
7979
# "batchScheduler.enabled=true" at the same time as it will override this option.
8080
name: ""
8181

ray-operator/apis/config/v1alpha1/config_utils_test.go

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ import (
66
"github.com/go-logr/logr"
77
"github.com/go-logr/logr/testr"
88

9+
schedulerPlugins "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler/scheduler-plugins"
910
"github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler/volcano"
1011
"github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler/yunikorn"
1112
)
@@ -60,6 +61,16 @@ func TestValidateBatchSchedulerConfig(t *testing.T) {
6061
},
6162
wantErr: false,
6263
},
64+
{
65+
name: "valid option, batch-scheduler=scheduler-plugins",
66+
args: args{
67+
logger: testr.New(t),
68+
config: Configuration{
69+
BatchScheduler: schedulerPlugins.GetPluginName(),
70+
},
71+
},
72+
wantErr: false,
73+
},
6374
{
6475
name: "invalid option, invalid scheduler name",
6576
args: args{

ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins.go

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,13 @@ import (
2121
)
2222

2323
const (
24-
schedulerName string = "scheduler-plugins"
24+
// This is the batchScheduler name used in the Ray Operator.
25+
// We use this name because it is easier to understand and remember.
26+
// It is also consistent with the name used in the Helm chart values.yaml.
27+
schedulerName string = "scheduler-plugins"
28+
// The default scheduler plugins name is "scheduler-plugins-scheduler".
29+
// https://github.com/kubernetes-sigs/scheduler-plugins/blob/b3127ba4cc420430ca5322740103220043697eec/manifests/install/charts/as-a-second-scheduler/values.yaml#L6C9-L6C36
30+
schedulerInstanceName string = "scheduler-plugins-scheduler"
2531
kubeSchedulerPodGroupLabelKey string = "scheduling.x-k8s.io/pod-group"
2632
)
2733

@@ -36,7 +42,7 @@ func GetPluginName() string {
3642
}
3743

3844
func (k *KubeScheduler) Name() string {
39-
return GetPluginName()
45+
return schedulerInstanceName
4046
}
4147

4248
func createPodGroup(ctx context.Context, app *rayv1.RayCluster) *v1alpha1.PodGroup {
@@ -90,8 +96,7 @@ func (k *KubeScheduler) AddMetadataToPod(_ context.Context, app *rayv1.RayCluste
9096
if k.isGangSchedulingEnabled(app) {
9197
pod.Labels[kubeSchedulerPodGroupLabelKey] = app.Name
9298
}
93-
// TODO(kevin85421): Currently, we only support "single scheduler" mode. If we want to support
94-
// "second scheduler" mode, we need to add `schedulerName` to the pod spec.
99+
pod.Spec.SchedulerName = k.Name()
95100
}
96101

97102
func (k *KubeScheduler) isGangSchedulingEnabled(app *rayv1.RayCluster) bool {

ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins_test.go

Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -116,3 +116,54 @@ func TestCreatePodGroupWithMultipleHosts(t *testing.T) {
116116
// 1 head and 4 workers
117117
a.Equal(int32(5), podGroup.Spec.MinMember)
118118
}
119+
120+
func TestAddMetadataToPod(t *testing.T) {
121+
tests := []struct {
122+
name string
123+
enableGang bool
124+
podHasLabels bool
125+
}{
126+
{"GangEnabled_WithLabels", true, true},
127+
{"GangDisabled_WithLabels", false, true},
128+
{"GangDisabled_WithoutLabels", false, false},
129+
}
130+
131+
for _, tt := range tests {
132+
t.Run(tt.name, func(t *testing.T) {
133+
a := assert.New(t)
134+
cluster := createTestRayCluster(1)
135+
cluster.Labels = make(map[string]string)
136+
137+
if tt.enableGang {
138+
cluster.Labels["ray.io/gang-scheduling-enabled"] = "true"
139+
}
140+
141+
var pod *corev1.Pod
142+
if tt.podHasLabels {
143+
pod = &corev1.Pod{
144+
ObjectMeta: metav1.ObjectMeta{
145+
Labels: map[string]string{},
146+
},
147+
}
148+
} else {
149+
pod = &corev1.Pod{}
150+
}
151+
152+
scheduler := &KubeScheduler{}
153+
scheduler.AddMetadataToPod(context.TODO(), &cluster, "worker", pod)
154+
155+
if tt.enableGang {
156+
a.Equal(cluster.Name, pod.Labels[kubeSchedulerPodGroupLabelKey])
157+
} else {
158+
_, exists := pod.Labels[kubeSchedulerPodGroupLabelKey]
159+
a.False(exists)
160+
}
161+
162+
a.Equal(scheduler.Name(), pod.Spec.SchedulerName)
163+
// The default scheduler plugins name is "scheduler-plugins-scheduler"
164+
// The batchScheduler name is "scheduler-plugins"
165+
// This is to ensure batchScheduler and default scheduler plugins name are not the same.
166+
a.NotEqual(scheduler.Name(), GetPluginName())
167+
})
168+
}
169+
}

0 commit comments

Comments
 (0)