Skip to content

Commit

Permalink
[Refactor] Use global constants for Ray versions and Ray image versio…
Browse files Browse the repository at this point in the history
…ns in go tests (#2641)
  • Loading branch information
simotw authored Dec 13, 2024
1 parent fa21edd commit 3a62419
Show file tree
Hide file tree
Showing 12 changed files with 63 additions and 35 deletions.
2 changes: 1 addition & 1 deletion apiserver/test/e2e/cluster_server_autoscaler_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func TestCreateClusterAutoscaler(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
EnableInTreeAutoscaling: true,
Expand Down
20 changes: 10 additions & 10 deletions apiserver/test/e2e/cluster_server_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "3cpo",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -80,7 +80,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -140,7 +140,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -248,7 +248,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "bullwinkle",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: nil,
},
Expand All @@ -265,7 +265,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "bullwinkle",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{},
},
Expand All @@ -282,7 +282,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -310,7 +310,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand All @@ -335,7 +335,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -365,7 +365,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down Expand Up @@ -399,7 +399,7 @@ func TestCreateClusterEndpoint(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down
6 changes: 6 additions & 0 deletions apiserver/test/e2e/defaults.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
package e2e

const (
RayVersion = "2.9.0"
RayImage = "rayproject/ray:2.9.0"
)
12 changes: 6 additions & 6 deletions apiserver/test/e2e/job_server_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func TestCreateJobWithDisposableClusters(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "natacha",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/counter_sample.py",
RuntimeEnv: "pip:\n - requests==2.26.0\n - pendulum==2.1.2\nenv_vars:\n counter_name: test_counter\n",
ShutdownAfterJobFinishes: true,
Expand All @@ -105,7 +105,7 @@ func TestCreateJobWithDisposableClusters(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "natacha",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/fail_fast.py",
ShutdownAfterJobFinishes: true,
ClusterSpec: clusterSpec,
Expand Down Expand Up @@ -196,7 +196,7 @@ func TestCreateJobWithDisposableClusters(t *testing.T) {
Namespace: tCtx.GetNamespaceName(),
Name: tCtx.GetNextName(),
User: "bullwinkle",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/counter_sample.py",
RuntimeEnv: "pip:\n - requests==2.26.0\n - pendulum==2.1.2\nenv_vars:\n counter_name: test_counter\n",
ShutdownAfterJobFinishes: true,
Expand Down Expand Up @@ -444,7 +444,7 @@ func TestCreateJobWithClusterSelector(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "r2d2",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/counter_sample.py",
Metadata: map[string]string{},
RuntimeEnv: "pip:\n - requests==2.26.0\n - pendulum==2.1.2\nenv_vars:\n counter_name: test_counter\n",
Expand All @@ -466,7 +466,7 @@ func TestCreateJobWithClusterSelector(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "r2d2",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/fail_fast.py",
RuntimeEnv: "pip:\n - requests==2.26.0\n - pendulum==2.1.2\nenv_vars:\n counter_name: test_counter\n",
ShutdownAfterJobFinishes: true,
Expand Down Expand Up @@ -532,7 +532,7 @@ func createTestJob(t *testing.T, tCtx *End2EndTestingContext) *api.CreateRayJobR
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "natacha",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Entrypoint: "python /home/ray/samples/counter_sample.py",
RuntimeEnv: "pip:\n - requests==2.26.0\n - pendulum==2.1.2\nenv_vars:\n counter_name: test_counter\n",
ShutdownAfterJobFinishes: true,
Expand Down
2 changes: 1 addition & 1 deletion apiserver/test/e2e/job_submission_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func TestCreateJobSubmission(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "boris",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
Environment: api.Cluster_DEV,
ClusterSpec: &api.ClusterSpec{
HeadGroupSpec: &api.HeadGroupSpec{
Expand Down
4 changes: 2 additions & 2 deletions apiserver/test/e2e/service_server_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func TestCreateServiceV2(t *testing.T) {
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "user1",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
ServeConfig_V2: "applications:\n - name: fruit_app\n import_path: fruit.deployment_graph\n route_prefix: /fruit\n runtime_env:\n working_dir: \"https://github.com/ray-project/test_dag/archive/78b4a5da38796123d9f9ffff59bab2792a043e95.zip\"\n deployments:\n - name: MangoStand\n num_replicas: 2\n max_replicas_per_node: 1\n user_config:\n price: 3\n ray_actor_options:\n num_cpus: 0.1\n - name: OrangeStand\n num_replicas: 1\n user_config:\n price: 2\n ray_actor_options:\n num_cpus: 0.1\n - name: PearStand\n num_replicas: 1\n user_config:\n price: 1\n ray_actor_options:\n num_cpus: 0.1\n - name: FruitMarket\n num_replicas: 1\n ray_actor_options:\n num_cpus: 0.1\n - name: math_app\n import_path: conditional_dag.serve_dag\n route_prefix: /calc\n runtime_env:\n working_dir: \"https://github.com/ray-project/test_dag/archive/78b4a5da38796123d9f9ffff59bab2792a043e95.zip\"\n deployments:\n - name: Adder\n num_replicas: 1\n user_config:\n increment: 3\n ray_actor_options:\n num_cpus: 0.1\n - name: Multiplier\n num_replicas: 1\n user_config:\n factor: 5\n ray_actor_options:\n num_cpus: 0.1\n - name: Router\n num_replicas: 1\n",
ServiceUnhealthySecondThreshold: 10,
DeploymentUnhealthySecondThreshold: 20,
Expand Down Expand Up @@ -356,7 +356,7 @@ func createTestServiceV2(t *testing.T, tCtx *End2EndTestingContext) *api.CreateR
Name: tCtx.GetNextName(),
Namespace: tCtx.GetNamespaceName(),
User: "user1",
Version: "2.9.0",
Version: tCtx.GetRayVersion(),
ServeConfig_V2: "applications:\n - name: fruit_app\n import_path: fruit.deployment_graph\n route_prefix: /fruit\n runtime_env:\n working_dir: \"https://github.com/ray-project/test_dag/archive/78b4a5da38796123d9f9ffff59bab2792a043e95.zip\"\n deployments:\n - name: MangoStand\n num_replicas: 2\n max_replicas_per_node: 1\n user_config:\n price: 3\n ray_actor_options:\n num_cpus: 0.1\n - name: OrangeStand\n num_replicas: 1\n user_config:\n price: 2\n ray_actor_options:\n num_cpus: 0.1\n - name: PearStand\n num_replicas: 1\n user_config:\n price: 1\n ray_actor_options:\n num_cpus: 0.1\n - name: FruitMarket\n num_replicas: 1\n ray_actor_options:\n num_cpus: 0.1\n - name: math_app\n import_path: conditional_dag.serve_dag\n route_prefix: /calc\n runtime_env:\n working_dir: \"https://github.com/ray-project/test_dag/archive/78b4a5da38796123d9f9ffff59bab2792a043e95.zip\"\n deployments:\n - name: Adder\n num_replicas: 1\n user_config:\n increment: 3\n ray_actor_options:\n num_cpus: 0.1\n - name: Multiplier\n num_replicas: 1\n user_config:\n factor: 5\n ray_actor_options:\n num_cpus: 0.1\n - name: Router\n num_replicas: 1\n",
ServiceUnhealthySecondThreshold: 10,
DeploymentUnhealthySecondThreshold: 20,
Expand Down
19 changes: 18 additions & 1 deletion apiserver/test/e2e/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ type End2EndTestingContext struct {
k8client *kubernetes.Clientset
apiServerBaseURL string
rayImage string
rayVersion string
namespaceName string
computeTemplateName string
clusterName string
Expand All @@ -61,6 +62,7 @@ func NewEnd2EndTestingContext(t *testing.T) (*End2EndTestingContext, error) {
// ordering is important as there dependencies between field values
return newEnd2EndTestingContext(t,
withRayImage(),
withRayVersion(),
withBaseURL(),
withHttpClient(),
withContext(),
Expand Down Expand Up @@ -117,7 +119,7 @@ func withRayImage() contextOption {
return func(_ *testing.T, testingContext *End2EndTestingContext) error {
rayImage := os.Getenv("E2E_API_SERVER_RAY_IMAGE")
if strings.TrimSpace(rayImage) == "" {
rayImage = "rayproject/ray:2.9.0-py310"
rayImage = RayImage + "-py310"
}
// detect if we are running on arm64 machine, most likely apple silicon
// the os name is not checked as it also possible that it might be linux
Expand All @@ -130,6 +132,17 @@ func withRayImage() contextOption {
}
}

func withRayVersion() contextOption {
return func(_ *testing.T, testingContext *End2EndTestingContext) error {
rayVersion := os.Getenv("E2E_API_SERVER_RAY_VERSION")
if strings.TrimSpace(rayVersion) == "" {
rayVersion = RayVersion
}
testingContext.rayVersion = rayVersion
return nil
}
}

func withK8sClient() contextOption {
return func(t *testing.T, testingContext *End2EndTestingContext) error {
cfg, err := config.GetConfig()
Expand Down Expand Up @@ -210,6 +223,10 @@ func (e2etc *End2EndTestingContext) GetRayImage() string {
return e2etc.rayImage
}

func (e2etc *End2EndTestingContext) GetRayVersion() string {
return e2etc.rayVersion
}

func (e2etc *End2EndTestingContext) GetRayApiServerClient() *kuberayHTTP.KuberayAPIServerClient {
return e2etc.kuberayAPIServerClient
}
Expand Down
5 changes: 3 additions & 2 deletions ray-operator/controllers/ray/raycluster_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"k8s.io/apimachinery/pkg/api/meta"

"github.com/ray-project/kuberay/ray-operator/pkg/features"
"github.com/ray-project/kuberay/ray-operator/test/support"

"github.com/ray-project/kuberay/ray-operator/controllers/ray/common"
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
Expand Down Expand Up @@ -59,7 +60,7 @@ func rayClusterTemplate(name string, namespace string) *rayv1.RayCluster {
Containers: []corev1.Container{
{
Name: "ray-head",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
},
},
},
Expand All @@ -77,7 +78,7 @@ func rayClusterTemplate(name string, namespace string) *rayv1.RayCluster {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
},
},
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
"github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/scheme"
"github.com/ray-project/kuberay/ray-operator/pkg/features"
"github.com/ray-project/kuberay/ray-operator/test/support"

. "github.com/onsi/ginkgo/v2"
"github.com/stretchr/testify/assert"
Expand Down Expand Up @@ -231,7 +232,7 @@ func setupTest(t *testing.T) {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Command: []string{"echo"},
Args: []string{"Hello Ray"},
},
Expand Down Expand Up @@ -261,7 +262,7 @@ func setupTest(t *testing.T) {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Command: []string{"echo"},
Args: []string{"Hello Ray"},
},
Expand Down Expand Up @@ -316,7 +317,7 @@ func setupTest(t *testing.T) {
Containers: []corev1.Container{
{
Name: "ray-head",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Command: []string{"python"},
Args: []string{"/opt/code.py"},
Env: []corev1.EnvVar{
Expand Down Expand Up @@ -350,7 +351,7 @@ func setupTest(t *testing.T) {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Command: []string{"echo"},
Args: []string{"Hello Ray"},
Env: []corev1.EnvVar{
Expand Down
7 changes: 4 additions & 3 deletions ray-operator/controllers/ray/rayjob_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
"github.com/ray-project/kuberay/ray-operator/controllers/ray/common"
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
"github.com/ray-project/kuberay/ray-operator/test/support"

batchv1 "k8s.io/api/batch/v1"
corev1 "k8s.io/api/core/v1"
Expand All @@ -50,15 +51,15 @@ func rayJobTemplate(name string, namespace string) *rayv1.RayJob {
SubmissionMode: rayv1.K8sJobMode,
ShutdownAfterJobFinishes: true,
RayClusterSpec: &rayv1.RayClusterSpec{
RayVersion: "2.9.0",
RayVersion: support.GetRayVersion(),
HeadGroupSpec: rayv1.HeadGroupSpec{
RayStartParams: map[string]string{},
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: []corev1.Container{
{
Name: "ray-head",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Resources: corev1.ResourceRequirements{
Limits: corev1.ResourceList{
corev1.ResourceCPU: resource.MustParse("1"),
Expand Down Expand Up @@ -100,7 +101,7 @@ func rayJobTemplate(name string, namespace string) *rayv1.RayJob {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
},
},
},
Expand Down
7 changes: 4 additions & 3 deletions ray-operator/controllers/ray/rayservice_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"time"

"github.com/ray-project/kuberay/ray-operator/controllers/ray/common"
"github.com/ray-project/kuberay/ray-operator/test/support"

"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"

Expand Down Expand Up @@ -81,7 +82,7 @@ var _ = Context("Inside the default namespace", func() {
Spec: rayv1.RayServiceSpec{
ServeConfigV2: testServeConfigV2,
RayClusterSpec: rayv1.RayClusterSpec{
RayVersion: "2.9.0",
RayVersion: support.GetRayVersion(),
HeadGroupSpec: rayv1.HeadGroupSpec{
RayStartParams: map[string]string{
"port": "6379",
Expand All @@ -104,7 +105,7 @@ var _ = Context("Inside the default namespace", func() {
Containers: []corev1.Container{
{
Name: "ray-head",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Env: []corev1.EnvVar{
{
Name: "MY_POD_IP",
Expand Down Expand Up @@ -178,7 +179,7 @@ var _ = Context("Inside the default namespace", func() {
Containers: []corev1.Container{
{
Name: "ray-worker",
Image: "rayproject/ray:2.9.0",
Image: support.GetRayImage(),
Command: []string{"echo"},
Args: []string{"Hello Ray"},
Env: []corev1.EnvVar{
Expand Down
Loading

0 comments on commit 3a62419

Please sign in to comment.