Skip to content

Commit

Permalink
[Feature][RayJob] Remove the deprecated RuntimeEnv from CRD. Use Runt…
Browse files Browse the repository at this point in the history
…imeEnvYAML instead. (#1792)
  • Loading branch information
rueian authored Dec 31, 2023
1 parent c814963 commit b2dbb15
Show file tree
Hide file tree
Showing 11 changed files with 19 additions and 75 deletions.
3 changes: 0 additions & 3 deletions apiserver/pkg/util/job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,6 @@ func TestBuildRayJob(t *testing.T) {
assert.Equal(t, "test", job.ObjectMeta.Labels[RayClusterUserLabelKey])
assert.Equal(t, 1, len(job.ObjectMeta.Annotations))
assert.Greater(t, len(job.Spec.RuntimeEnvYAML), 1)
assert.Equal(t, len(job.Spec.RuntimeEnv), 0)
assert.Equal(t, 1, len(job.Spec.Metadata))
assert.Nil(t, job.Spec.ClusterSelector)
assert.NotNil(t, job.Spec.RayClusterSpec)
Expand All @@ -96,7 +95,6 @@ func TestBuildRayJob(t *testing.T) {
assert.Equal(t, 4, len(job.ObjectMeta.Labels))
assert.Equal(t, "test", job.ObjectMeta.Labels[RayClusterUserLabelKey])
assert.Greater(t, len(job.Spec.RuntimeEnvYAML), 1)
assert.Equal(t, len(job.Spec.RuntimeEnv), 0)
assert.NotNil(t, job.Spec.ClusterSelector)
assert.Nil(t, job.Spec.RayClusterSpec)
assert.Nil(t, job.Spec.SubmitterPodTemplate)
Expand All @@ -109,7 +107,6 @@ func TestBuildRayJob(t *testing.T) {
assert.Equal(t, 4, len(job.ObjectMeta.Labels))
assert.Equal(t, "test", job.ObjectMeta.Labels[RayClusterUserLabelKey])
assert.Greater(t, len(job.Spec.RuntimeEnvYAML), 1)
assert.Equal(t, len(job.Spec.RuntimeEnv), 0)
assert.NotNil(t, job.Spec.ClusterSelector)
assert.Nil(t, job.Spec.RayClusterSpec)
assert.Equal(t, float32(2), job.Spec.EntrypointNumCpus)
Expand Down
2 changes: 0 additions & 2 deletions docs/reference/api.md
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,6 @@ _Appears in:_
| --- | --- |
| `entrypoint` _string_ | INSERT ADDITIONAL SPEC FIELDS - desired state of cluster Important: Run "make" to regenerate code after modifying this file |
| `metadata` _object (keys:string, values:string)_ | Refer to Kubernetes API documentation for fields of `metadata`. |
| `runtimeEnv` _string_ | RuntimeEnv is base64 encoded. This field is deprecated, please use RuntimeEnvYAML instead. |
| `runtimeEnvYAML` _string_ | RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. |
| `jobId` _string_ | If jobId is not set, a new jobId will be auto-generated. |
| `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. |
Expand Down Expand Up @@ -344,7 +343,6 @@ _Appears in:_
| --- | --- |
| `entrypoint` _string_ | INSERT ADDITIONAL SPEC FIELDS - desired state of cluster Important: Run "make" to regenerate code after modifying this file |
| `metadata` _object (keys:string, values:string)_ | Refer to Kubernetes API documentation for fields of `metadata`. |
| `runtimeEnv` _string_ | RuntimeEnv is base64 encoded. This field is deprecated, please use RuntimeEnvYAML instead. |
| `runtimeEnvYAML` _string_ | RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. |
| `jobId` _string_ | If jobId is not set, a new jobId will be auto-generated. |
| `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. |
Expand Down
4 changes: 0 additions & 4 deletions helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

3 changes: 0 additions & 3 deletions ray-operator/apis/ray/v1/rayjob_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,9 +48,6 @@ type RayJobSpec struct {
Entrypoint string `json:"entrypoint"`
// Metadata is data to store along with this job.
Metadata map[string]string `json:"metadata,omitempty"`
// RuntimeEnv is base64 encoded.
// This field is deprecated, please use RuntimeEnvYAML instead.
RuntimeEnv string `json:"runtimeEnv,omitempty"`
// RuntimeEnvYAML represents the runtime environment configuration
// provided as a multi-line YAML string.
RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"`
Expand Down
3 changes: 0 additions & 3 deletions ray-operator/apis/ray/v1alpha1/rayjob_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,9 +48,6 @@ type RayJobSpec struct {
Entrypoint string `json:"entrypoint"`
// Metadata is data to store along with this job.
Metadata map[string]string `json:"metadata,omitempty"`
// RuntimeEnv is base64 encoded.
// This field is deprecated, please use RuntimeEnvYAML instead.
RuntimeEnv string `json:"runtimeEnv,omitempty"`
// RuntimeEnvYAML represents the runtime environment configuration
// provided as a multi-line YAML string.
RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"`
Expand Down
4 changes: 0 additions & 4 deletions ray-operator/config/crd/bases/ray.io_rayjobs.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

20 changes: 0 additions & 20 deletions ray-operator/controllers/ray/common/job.go
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
package common

import (
"encoding/base64"
"encoding/json"
"fmt"
"strings"
Expand All @@ -15,29 +14,10 @@ import (
"sigs.k8s.io/yaml"
)

// GetDecodedRuntimeEnv decodes the runtime environment for the Ray job from a base64-encoded string.
func GetDecodedRuntimeEnv(runtimeEnv string) (string, error) {
decodedBytes, err := base64.StdEncoding.DecodeString(runtimeEnv)
if err != nil {
return "", fmt.Errorf("failed to decode runtimeEnv: %v: %v", runtimeEnv, err)
}
return string(decodedBytes), nil
}

// GetRuntimeEnvJson returns the JSON string of the runtime environment for the Ray job.
func getRuntimeEnvJson(rayJobInstance *rayv1.RayJob) (string, error) {
runtimeEnv := rayJobInstance.Spec.RuntimeEnv
runtimeEnvYAML := rayJobInstance.Spec.RuntimeEnvYAML

// Check if both runtimeEnv and RuntimeEnvYAML are specified.
if len(runtimeEnv) > 0 && len(runtimeEnvYAML) > 0 {
return "", fmt.Errorf("Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.")
}

if len(runtimeEnv) > 0 {
return GetDecodedRuntimeEnv(runtimeEnv)
}

if len(runtimeEnvYAML) > 0 {
// Convert YAML to JSON
jsonData, err := yaml.YAMLToJSON([]byte(runtimeEnvYAML))
Expand Down
20 changes: 1 addition & 19 deletions ray-operator/controllers/ray/common/job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ import (

var testRayJob = &rayv1.RayJob{
Spec: rayv1.RayJobSpec{
RuntimeEnv: "eyJ0ZXN0IjoidGVzdCJ9", // {"test":"test"} in base64
RuntimeEnvYAML: "test: test",
Metadata: map[string]string{
"testKey": "testValue",
},
Expand All @@ -30,12 +30,6 @@ var testRayJob = &rayv1.RayJob{
},
}

func TestGetDecodedRuntimeEnv(t *testing.T) {
decoded, err := GetDecodedRuntimeEnv(testRayJob.Spec.RuntimeEnv)
assert.NoError(t, err)
assert.Equal(t, `{"test":"test"}`, decoded)
}

func TestGetRuntimeEnvJsonFromBase64(t *testing.T) {
expected := `{"test":"test"}`
jsonOutput, err := getRuntimeEnvJson(testRayJob)
Expand Down Expand Up @@ -67,18 +61,6 @@ pip: ["python-multipart==0.0.6"]
assert.Equal(t, expectedMap, actualMap)
}

func TestGetRuntimeEnvJsonErrorWithBothFields(t *testing.T) {
rayJobWithBoth := &rayv1.RayJob{
Spec: rayv1.RayJobSpec{
RuntimeEnv: "eyJ0ZXN0IjoidGVzdCJ9",
RuntimeEnvYAML: `pip: ["python-multipart==0.0.6"]`,
},
}
_, err := getRuntimeEnvJson(rayJobWithBoth)
assert.Error(t, err)
assert.Contains(t, err.Error(), "Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.")
}

func TestGetBaseRayJobCommand(t *testing.T) {
expected := []string{"ray", "job", "submit", "--address", "http://127.0.0.1:8265"}
command := GetBaseRayJobCommand(testRayJob.Status.DashboardURL)
Expand Down
5 changes: 0 additions & 5 deletions ray-operator/controllers/ray/rayjob_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,11 +324,6 @@ func (r *RayJobReconciler) getSubmitterTemplate(rayJobInstance *rayv1.RayJob, ra

// If the command in the submitter pod template isn't set, use the default command.
if len(submitterTemplate.Spec.Containers[utils.RayContainerIndex].Command) == 0 {
// Check for deprecated 'runtimeEnv' field usage and log a warning.
if len(rayJobInstance.Spec.RuntimeEnv) > 0 {
r.Log.Info("Warning: The 'runtimeEnv' field is deprecated. Please use 'runtimeEnvYAML' instead.")
}

k8sJobCommand, err := common.GetK8sJobCommand(rayJobInstance)
if err != nil {
return corev1.PodTemplateSpec{}, err
Expand Down
13 changes: 5 additions & 8 deletions ray-operator/controllers/ray/utils/dashboard_httpclient.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,12 +3,13 @@ package utils
import (
"bytes"
"context"
"encoding/base64"
"fmt"
"io"
"net/http"
"time"

"k8s.io/apimachinery/pkg/util/yaml"

"github.com/go-logr/logr"
fmtErrors "github.com/pkg/errors"
corev1 "k8s.io/api/core/v1"
Expand Down Expand Up @@ -414,17 +415,13 @@ func ConvertRayJobToReq(rayJob *rayv1.RayJob) (*RayJobRequest, error) {
Metadata: rayJob.Spec.Metadata,
JobId: rayJob.Status.JobId,
}
if len(rayJob.Spec.RuntimeEnv) == 0 {
if len(rayJob.Spec.RuntimeEnvYAML) == 0 {
return req, nil
}
decodeBytes, err := base64.StdEncoding.DecodeString(rayJob.Spec.RuntimeEnv)
if err != nil {
return nil, fmt.Errorf("Failed to decode runtimeEnv: %v: %v", rayJob.Spec.RuntimeEnv, err)
}
var runtimeEnv map[string]interface{}
err = json.Unmarshal(decodeBytes, &runtimeEnv)
err := yaml.Unmarshal([]byte(rayJob.Spec.RuntimeEnvYAML), &runtimeEnv)
if err != nil {
return nil, fmt.Errorf("failed to unmarshal runtimeEnv: %v: %v", decodeBytes, err)
return nil, fmt.Errorf("failed to unmarshal runtimeEnv: %v: %v", rayJob.Spec.RuntimeEnvYAML, err)
}
req.RuntimeEnv = runtimeEnv
return req, nil
Expand Down
17 changes: 13 additions & 4 deletions ray-operator/controllers/ray/utils/dashboard_httpclient_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@ package utils

import (
"context"
"encoding/base64"
"encoding/json"
"net/http"

Expand All @@ -16,7 +15,18 @@ import (
)

const (
runtimeEnvStr = "{\n \"working_dir\": \"./\",\n \"pip\": [\n \"requests==2.26.0\",\n \"pendulum==2.1.2\"\n ],\n \"conda\": {\n \"dependencies\": [\n \"pytorch\",\n \"torchvision\",\n \"pip\",\n {\n \"pip\": [\n \"pendulum\"\n ]\n }\n ]\n },\n \"eager_install\": false\n}"
runtimeEnvStr = `working_dir: "./"
pip:
- requests==2.26.0
- pendulum==2.1.2
conda:
dependencies:
- pytorch
- torchvision
- pip
- pip:
- pendulum
eager_install: false`
)

var _ = Describe("RayFrameworkGenerator", func() {
Expand All @@ -29,7 +39,6 @@ var _ = Describe("RayFrameworkGenerator", func() {

BeforeEach(func() {
expectJobId = "raysubmit_test001"
encodedRuntimeEnv := base64.StdEncoding.EncodeToString([]byte(runtimeEnvStr))
rayJob = &rayv1.RayJob{
ObjectMeta: metav1.ObjectMeta{
Name: "rayjob-sample",
Expand All @@ -40,7 +49,7 @@ var _ = Describe("RayFrameworkGenerator", func() {
Metadata: map[string]string{
"owner": "test1",
},
RuntimeEnv: encodedRuntimeEnv,
RuntimeEnvYAML: runtimeEnvStr,
},
}
rayDashboardClient = &RayDashboardClient{}
Expand Down

0 comments on commit b2dbb15

Please sign in to comment.