Skip to content

Commit 40b97cf

Browse files
committed
Fix: MinReplicas / MaxReplicas should verify if autoscaling is not
enabled 1. fix golangci-lint test (unused ctx)
1 parent 717af47 commit 40b97cf

File tree

7 files changed

+19
-19
lines changed

7 files changed

+19
-19
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ func createPodGroup(ctx context.Context, app *rayv1.RayCluster) *v1alpha1.PodGro
6262
},
6363
},
6464
Spec: v1alpha1.PodGroupSpec{
65-
MinMember: utils.CalculateDesiredReplicas(ctx, app) + 1, // +1 for the head pod
65+
MinMember: utils.CalculateDesiredReplicas(app) + 1, // +1 for the head pod
6666
MinResources: utils.CalculateDesiredResources(app),
6767
},
6868
}

ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ func (v *VolcanoBatchScheduler) DoBatchSchedulingOnSubmission(ctx context.Contex
5050
var minMember int32
5151
var totalResource corev1.ResourceList
5252
if !utils.IsAutoscalingEnabled(&app.Spec) {
53-
minMember = utils.CalculateDesiredReplicas(ctx, app) + 1
53+
minMember = utils.CalculateDesiredReplicas(app) + 1
5454
totalResource = utils.CalculateDesiredResources(app)
5555
} else {
5656
minMember = utils.CalculateMinReplicas(app) + 1

ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler_test.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,6 @@
11
package volcano
22

33
import (
4-
"context"
54
"testing"
65

76
"github.com/stretchr/testify/assert"
@@ -83,7 +82,7 @@ func TestCreatePodGroup(t *testing.T) {
8382

8483
cluster := createTestRayCluster(1)
8584

86-
minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1
85+
minMember := utils.CalculateDesiredReplicas(&cluster) + 1
8786
totalResource := utils.CalculateDesiredResources(&cluster)
8887
pg := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource)
8988

@@ -107,7 +106,7 @@ func TestCreatePodGroup_NumOfHosts2(t *testing.T) {
107106

108107
cluster := createTestRayCluster(2)
109108

110-
minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1
109+
minMember := utils.CalculateDesiredReplicas(&cluster) + 1
111110
totalResource := utils.CalculateDesiredResources(&cluster)
112111
pg := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource)
113112

ray-operator/controllers/ray/raycluster_controller.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1169,7 +1169,7 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra
11691169

11701170
newInstance.Status.ReadyWorkerReplicas = utils.CalculateReadyReplicas(runtimePods)
11711171
newInstance.Status.AvailableWorkerReplicas = utils.CalculateAvailableReplicas(runtimePods)
1172-
newInstance.Status.DesiredWorkerReplicas = utils.CalculateDesiredReplicas(ctx, newInstance)
1172+
newInstance.Status.DesiredWorkerReplicas = utils.CalculateDesiredReplicas(newInstance)
11731173
newInstance.Status.MinWorkerReplicas = utils.CalculateMinReplicas(newInstance)
11741174
newInstance.Status.MaxWorkerReplicas = utils.CalculateMaxReplicas(newInstance)
11751175

ray-operator/controllers/ray/utils/util.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -352,7 +352,7 @@ func GetWorkerGroupDesiredReplicas(workerGroupSpec rayv1.WorkerGroupSpec) int32
352352
}
353353

354354
// CalculateDesiredReplicas calculate desired worker replicas at the cluster level
355-
func CalculateDesiredReplicas(ctx context.Context, cluster *rayv1.RayCluster) int32 {
355+
func CalculateDesiredReplicas(cluster *rayv1.RayCluster) int32 {
356356
count := int32(0)
357357
for _, nodeGroup := range cluster.Spec.WorkerGroupSpecs {
358358
count += GetWorkerGroupDesiredReplicas(nodeGroup)

ray-operator/controllers/ray/utils/util_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -781,7 +781,7 @@ func TestCalculateDesiredReplicas(t *testing.T) {
781781
},
782782
},
783783
}
784-
assert.Equal(t, CalculateDesiredReplicas(context.Background(), &cluster), tc.answer)
784+
assert.Equal(t, CalculateDesiredReplicas(&cluster), tc.answer)
785785
})
786786
}
787787
}

ray-operator/controllers/ray/utils/validation.go

Lines changed: 12 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -39,23 +39,27 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s
3939
return fmt.Errorf("headGroupSpec should have at least one container")
4040
}
4141

42+
// Check if autoscaling is enabled once to avoid repeated calls
43+
isAutoscalingEnabled := IsAutoscalingEnabled(spec)
44+
4245
for _, workerGroup := range spec.WorkerGroupSpecs {
4346
if len(workerGroup.Template.Spec.Containers) == 0 {
4447
return fmt.Errorf("workerGroupSpec should have at least one container")
4548
}
46-
if workerGroup.MinReplicas == nil || workerGroup.MaxReplicas == nil {
47-
return fmt.Errorf("worker group %s must set both minReplicas and maxReplicas", workerGroup.GroupName)
49+
// When autoscaling is enabled, MinReplicas and MaxReplicas are optional
50+
// as users can manually update them and the autoscaler will handle the adjustment.
51+
if !isAutoscalingEnabled && (workerGroup.MinReplicas == nil || workerGroup.MaxReplicas == nil) {
52+
return fmt.Errorf("worker group %s must set both minReplicas and maxReplicas when autoscaling is disabled", workerGroup.GroupName)
4853
}
49-
if *workerGroup.MinReplicas < 0 {
54+
if workerGroup.MinReplicas != nil && *workerGroup.MinReplicas < 0 {
5055
return fmt.Errorf("worker group %s has negative minReplicas %d", workerGroup.GroupName, *workerGroup.MinReplicas)
5156
}
52-
if *workerGroup.MaxReplicas < 0 {
57+
if workerGroup.MaxReplicas != nil && *workerGroup.MaxReplicas < 0 {
5358
return fmt.Errorf("worker group %s has negative maxReplicas %d", workerGroup.GroupName, *workerGroup.MaxReplicas)
5459
}
55-
if *workerGroup.MinReplicas > *workerGroup.MaxReplicas {
56-
return fmt.Errorf("worker group %s has minReplicas %d greater than maxReplicas %d", workerGroup.GroupName, *workerGroup.MinReplicas, *workerGroup.MaxReplicas)
57-
}
58-
if workerGroup.Replicas != nil {
60+
// When autoscaling is enabled, the Ray Autoscaler will manage replicas and
61+
// eventually adjust them to fall within minReplicas/maxReplicas bounds.
62+
if workerGroup.Replicas != nil && !isAutoscalingEnabled && workerGroup.MinReplicas != nil && workerGroup.MaxReplicas != nil {
5963
if *workerGroup.Replicas < *workerGroup.MinReplicas {
6064
return fmt.Errorf("worker group %s has replicas %d smaller than minReplicas %d", workerGroup.GroupName, *workerGroup.Replicas, *workerGroup.MinReplicas)
6165
}
@@ -113,9 +117,6 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s
113117
}
114118
}
115119

116-
// Check if autoscaling is enabled once to avoid repeated calls
117-
isAutoscalingEnabled := IsAutoscalingEnabled(spec)
118-
119120
// Validate that RAY_enable_autoscaler_v2 environment variable is not set to "1" or "true" when autoscaler is disabled
120121
if !isAutoscalingEnabled {
121122
if envVar, exists := EnvVarByName(RAY_ENABLE_AUTOSCALER_V2, spec.HeadGroupSpec.Template.Spec.Containers[RayContainerIndex].Env); exists {

0 commit comments

Comments
 (0)