From acc3e5b5ded2c616856f5ae116d1ca27ba5a451c Mon Sep 17 00:00:00 2001 From: win5923 Date: Thu, 16 Jan 2025 22:35:41 +0800 Subject: [PATCH 1/3] [RayCluster][Feature] add redis username to head pod from GcsFaultToleranceOptions Signed-off-by: win5923 --- ray-operator/controllers/ray/common/pod.go | 21 +- .../controllers/ray/common/pod_test.go | 35 ++++ .../controllers/ray/raycluster_controller.go | 8 +- .../controllers/ray/utils/constant.go | 1 + ray-operator/ray-cluster.external-redis.yaml | 181 ++++++++++++++++++ 5 files changed, 243 insertions(+), 3 deletions(-) create mode 100644 ray-operator/ray-cluster.external-redis.yaml diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 3ab5cc6fad8..3b8071ea2a2 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -104,7 +104,7 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra container.Env = append(container.Env, gcsTimeout) } - // Configure the Redis address and password for GCS FT. + // Configure the Redis address, username and password for GCS FT. if rayNodeType == rayv1.HeadNode { // Configure the external storage namespace for GCS FT. storageNS := string(instance.UID) @@ -125,6 +125,17 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra Name: utils.RAY_REDIS_ADDRESS, Value: options.RedisAddress, }) + if options.RedisUsername != nil { + // If `GcsFaultToleranceOptions.RedisUsername` is set, it will be put into the + // `REDIS_USERNAME` environment variable later. Here, we use `$REDIS_USERNAME` in + // rayStartParams to refer to the environment variable. + instance.Spec.HeadGroupSpec.RayStartParams["redis-username"] = "$REDIS_USERNAME" + container.Env = append(container.Env, corev1.EnvVar{ + Name: utils.REDIS_USERNAME, + Value: options.RedisUsername.Value, + ValueFrom: options.RedisUsername.ValueFrom, + }) + } if options.RedisPassword != nil { // If `GcsFaultToleranceOptions.RedisPassword` is set, it will be put into the // `REDIS_PASSWORD` environment variable later. Here, we use `$REDIS_PASSWORD` in @@ -137,6 +148,14 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra }) } } else { + if !utils.EnvVarExists(utils.REDIS_USERNAME, container.Env) { + // setting the REDIS_USERNAME env var from the params + redisUsernameEnv := corev1.EnvVar{Name: utils.REDIS_USERNAME} + if value, ok := instance.Spec.HeadGroupSpec.RayStartParams["redis-username"]; ok { + redisUsernameEnv.Value = value + } + container.Env = append(container.Env, redisUsernameEnv) + } // If users directly set the `redis-password` in `rayStartParams` instead of referring // to a K8s secret, we need to set the `REDIS_PASSWORD` env var so that the Redis cleanup // job can connect to Redis using the password. This is not recommended. diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index e1e08655bcb..1d14275a2f6 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -312,7 +312,9 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { tests := []struct { name string storageNS string + redisUsernameEnv string redisPasswordEnv string + redisUsernameRayStartParams string redisPasswordRayStartParams string isHeadPod bool gcsFTEnabled bool @@ -334,12 +336,26 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { redisPasswordEnv: "test-password", isHeadPod: true, }, + { + name: "GCS FT enabled with redis username and password env", + gcsFTEnabled: true, + redisUsernameEnv: "test-username", + redisPasswordEnv: "test-password", + isHeadPod: true, + }, { name: "GCS FT enabled with redis password ray start params", gcsFTEnabled: true, redisPasswordRayStartParams: "test-password", isHeadPod: true, }, + { + name: "GCS FT enabled with redis username and password ray start params", + gcsFTEnabled: true, + redisUsernameRayStartParams: "test-username", + redisPasswordRayStartParams: "test-password", + isHeadPod: true, + }, { // The most common case. name: "GCS FT enabled with redis password env and ray start params referring to env", @@ -348,6 +364,16 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { redisPasswordRayStartParams: "$REDIS_PASSWORD", isHeadPod: false, }, + { + // The most common case. + name: "GCS FT enabled with redis username and password env and ray start params referring to env", + gcsFTEnabled: true, + redisUsernameEnv: "test-username", + redisUsernameRayStartParams: "$REDIS_USERNAME", + redisPasswordEnv: "test-password", + redisPasswordRayStartParams: "$REDIS_PASSWORD", + isHeadPod: false, + }, { name: "GCS FT enabled / worker Pod", gcsFTEnabled: true, @@ -410,12 +436,21 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { if test.storageNS != "" { cluster.Annotations[utils.RayExternalStorageNSAnnotationKey] = test.storageNS } + if test.redisUsernameEnv != "" { + cluster.Spec.HeadGroupSpec.Template.Spec.Containers[utils.RayContainerIndex].Env = append(cluster.Spec.HeadGroupSpec.Template.Spec.Containers[utils.RayContainerIndex].Env, corev1.EnvVar{ + Name: utils.REDIS_USERNAME, + Value: test.redisUsernameEnv, + }) + } if test.redisPasswordEnv != "" { cluster.Spec.HeadGroupSpec.Template.Spec.Containers[utils.RayContainerIndex].Env = append(cluster.Spec.HeadGroupSpec.Template.Spec.Containers[utils.RayContainerIndex].Env, corev1.EnvVar{ Name: utils.REDIS_PASSWORD, Value: test.redisPasswordEnv, }) } + if test.redisUsernameRayStartParams != "" { + cluster.Spec.HeadGroupSpec.RayStartParams["redis-username"] = test.redisUsernameRayStartParams + } if test.redisPasswordRayStartParams != "" { cluster.Spec.HeadGroupSpec.RayStartParams["redis-password"] = test.redisPasswordRayStartParams } diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 7e3fa6edca7..d69e1ee4ad2 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -1234,8 +1234,12 @@ func (r *RayClusterReconciler) buildRedisCleanupJob(ctx context.Context, instanc "import sys; " + "redis_address = os.getenv('RAY_REDIS_ADDRESS', '').split(',')[0]; " + "redis_address = redis_address if '://' in redis_address else 'redis://' + redis_address; " + - "parsed = urlparse(redis_address); " + - "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, password=os.getenv('REDIS_PASSWORD', parsed.password or ''), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"", + "parsed = urlparse(redis_address); ", + } + if utils.EnvVarExists(utils.REDIS_USERNAME, pod.Spec.Containers[utils.RayContainerIndex].Env) { + pod.Spec.Containers[utils.RayContainerIndex].Args[0] += "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, username=os.getenv('REDIS_USERNAME', parsed.username), password=os.getenv('REDIS_PASSWORD', parsed.password), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" + } else { + pod.Spec.Containers[utils.RayContainerIndex].Args[0] += "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, password=os.getenv('REDIS_PASSWORD', parsed.password or ''), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" } // Disable liveness and readiness probes because the Job will not launch processes like Raylet and GCS. diff --git a/ray-operator/controllers/ray/utils/constant.go b/ray-operator/controllers/ray/utils/constant.go index c5b6111e971..f108981d17b 100644 --- a/ray-operator/controllers/ray/utils/constant.go +++ b/ray-operator/controllers/ray/utils/constant.go @@ -97,6 +97,7 @@ const ( RAY_ADDRESS = "RAY_ADDRESS" RAY_REDIS_ADDRESS = "RAY_REDIS_ADDRESS" REDIS_PASSWORD = "REDIS_PASSWORD" + REDIS_USERNAME = "REDIS_USERNAME" RAY_DASHBOARD_ENABLE_K8S_DISK_USAGE = "RAY_DASHBOARD_ENABLE_K8S_DISK_USAGE" RAY_EXTERNAL_STORAGE_NS = "RAY_external_storage_namespace" RAY_GCS_RPC_SERVER_RECONNECT_TIMEOUT_S = "RAY_gcs_rpc_server_reconnect_timeout_s" diff --git a/ray-operator/ray-cluster.external-redis.yaml b/ray-operator/ray-cluster.external-redis.yaml new file mode 100644 index 00000000000..1f2ae1455c1 --- /dev/null +++ b/ray-operator/ray-cluster.external-redis.yaml @@ -0,0 +1,181 @@ +kind: ConfigMap +apiVersion: v1 +metadata: + name: redis-config + labels: + app: redis +data: + redis.conf: |- + dir /data + port 6379 + bind 0.0.0.0 + appendonly yes + protected-mode no + requirepass 5241590000000000 + pidfile /data/redis-6379.pid + + user username on >5241590000000000 ~* +@all +--- +apiVersion: v1 +kind: Service +metadata: + name: redis + labels: + app: redis +spec: + type: ClusterIP + ports: + - name: redis + port: 6379 + selector: + app: redis +--- +apiVersion: apps/v1 +kind: Deployment +metadata: + name: redis + labels: + app: redis +spec: + replicas: 1 + selector: + matchLabels: + app: redis + template: + metadata: + labels: + app: redis + spec: + containers: + - name: redis + image: redis:7.4.2 + command: + - "sh" + - "-c" + - "redis-server /usr/local/etc/redis/redis.conf" + ports: + - containerPort: 6379 + volumeMounts: + - name: config + mountPath: /usr/local/etc/redis/redis.conf + subPath: redis.conf + volumes: + - name: config + configMap: + name: redis-config +--- +# Redis password +apiVersion: v1 +kind: Secret +metadata: + name: redis-password-secret +type: Opaque +data: + # echo -n "username" | base64 + # echo -n "5241590000000000" | base64 + username: dXNlcm5hbWU= + password: NTI0MTU5MDAwMDAwMDAwMA== +--- +apiVersion: ray.io/v1 +kind: RayCluster +metadata: + name: raycluster-external-redis +spec: + rayVersion: 'nightly' + gcsFaultToleranceOptions: + redisAddress: redis:6379 + redisUsername: + valueFrom: + secretKeyRef: + name: redis-password-secret + key: username + redisPassword: + valueFrom: + secretKeyRef: + name: redis-password-secret + key: password + headGroupSpec: + rayStartParams: + num-cpus: "0" + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:nightly + resources: + limits: + cpu: "1" + requests: + cpu: "1" + ports: + - containerPort: 6379 + name: redis + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + - mountPath: /home/ray/samples + name: ray-example-configmap + volumes: + - name: ray-logs + emptyDir: {} + - name: ray-example-configmap + configMap: + name: ray-example + defaultMode: 0777 + items: + - key: detached_actor.py + path: detached_actor.py + - key: increment_counter.py + path: increment_counter.py + workerGroupSpecs: + - replicas: 1 + minReplicas: 1 + maxReplicas: 10 + groupName: small-group + rayStartParams: {} + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray:nightly + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + resources: + limits: + cpu: "1" + requests: + cpu: "1" + volumes: + - name: ray-logs + emptyDir: {} +--- +apiVersion: v1 +kind: ConfigMap +metadata: + name: ray-example +data: + detached_actor.py: | + import ray + + @ray.remote(num_cpus=1) + class Counter: + def __init__(self): + self.value = 0 + + def increment(self): + self.value += 1 + return self.value + + ray.init(namespace="default_namespace") + Counter.options(name="counter_actor", lifetime="detached").remote() + increment_counter.py: | + import ray + + ray.init(namespace="default_namespace") + counter = ray.get_actor("counter_actor") + print(ray.get(counter.increment.remote())) From 4abe1b171a255e825c9eee287746099780e71715 Mon Sep 17 00:00:00 2001 From: win5923 Date: Fri, 17 Jan 2025 22:30:57 +0800 Subject: [PATCH 2/3] Resloved conflicts Signed-off-by: win5923 --- ray-operator/controllers/ray/common/pod.go | 3 + .../controllers/ray/common/pod_test.go | 8 +- ray-operator/ray-cluster.external-redis.yaml | 181 ------------------ 3 files changed, 10 insertions(+), 182 deletions(-) delete mode 100644 ray-operator/ray-cluster.external-redis.yaml diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 3b8071ea2a2..3d4d4d1d0cf 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -148,6 +148,9 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra }) } } else { + // If users directly set the `redis-username` in `rayStartParams` instead of referring + // to a K8s secret, we need to set the `REDIS_USERNAME` env var so that the Redis cleanup + // job can connect to Redis using the username. if !utils.EnvVarExists(utils.REDIS_USERNAME, container.Env) { // setting the REDIS_USERNAME env var from the params redisUsernameEnv := corev1.EnvVar{Name: utils.REDIS_USERNAME} diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index 1d14275a2f6..d8a22148640 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -365,7 +365,6 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { isHeadPod: false, }, { - // The most common case. name: "GCS FT enabled with redis username and password env and ray start params referring to env", gcsFTEnabled: true, redisUsernameEnv: "test-username", @@ -389,6 +388,9 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { // Validate the test input + if test.redisUsernameEnv != "" && test.redisUsernameRayStartParams != "" { + assert.True(t, test.redisUsernameRayStartParams == "$REDIS_USERNAME") + } if test.redisPasswordEnv != "" && test.redisPasswordRayStartParams != "" { assert.True(t, test.redisPasswordRayStartParams == "$REDIS_PASSWORD") } @@ -475,6 +477,10 @@ func TestConfigureGCSFaultToleranceWithAnnotations(t *testing.T) { assert.Equal(t, podTemplate.Annotations[utils.RayExternalStorageNSAnnotationKey], test.storageNS) assert.True(t, utils.EnvVarExists(utils.RAY_EXTERNAL_STORAGE_NS, container.Env)) } + if test.redisUsernameEnv != "" { + env := getEnvVar(container, utils.REDIS_USERNAME) + assert.Equal(t, env.Value, test.redisUsernameEnv) + } if test.redisPasswordEnv != "" { env := getEnvVar(container, utils.REDIS_PASSWORD) assert.Equal(t, env.Value, test.redisPasswordEnv) diff --git a/ray-operator/ray-cluster.external-redis.yaml b/ray-operator/ray-cluster.external-redis.yaml deleted file mode 100644 index 1f2ae1455c1..00000000000 --- a/ray-operator/ray-cluster.external-redis.yaml +++ /dev/null @@ -1,181 +0,0 @@ -kind: ConfigMap -apiVersion: v1 -metadata: - name: redis-config - labels: - app: redis -data: - redis.conf: |- - dir /data - port 6379 - bind 0.0.0.0 - appendonly yes - protected-mode no - requirepass 5241590000000000 - pidfile /data/redis-6379.pid - - user username on >5241590000000000 ~* +@all ---- -apiVersion: v1 -kind: Service -metadata: - name: redis - labels: - app: redis -spec: - type: ClusterIP - ports: - - name: redis - port: 6379 - selector: - app: redis ---- -apiVersion: apps/v1 -kind: Deployment -metadata: - name: redis - labels: - app: redis -spec: - replicas: 1 - selector: - matchLabels: - app: redis - template: - metadata: - labels: - app: redis - spec: - containers: - - name: redis - image: redis:7.4.2 - command: - - "sh" - - "-c" - - "redis-server /usr/local/etc/redis/redis.conf" - ports: - - containerPort: 6379 - volumeMounts: - - name: config - mountPath: /usr/local/etc/redis/redis.conf - subPath: redis.conf - volumes: - - name: config - configMap: - name: redis-config ---- -# Redis password -apiVersion: v1 -kind: Secret -metadata: - name: redis-password-secret -type: Opaque -data: - # echo -n "username" | base64 - # echo -n "5241590000000000" | base64 - username: dXNlcm5hbWU= - password: NTI0MTU5MDAwMDAwMDAwMA== ---- -apiVersion: ray.io/v1 -kind: RayCluster -metadata: - name: raycluster-external-redis -spec: - rayVersion: 'nightly' - gcsFaultToleranceOptions: - redisAddress: redis:6379 - redisUsername: - valueFrom: - secretKeyRef: - name: redis-password-secret - key: username - redisPassword: - valueFrom: - secretKeyRef: - name: redis-password-secret - key: password - headGroupSpec: - rayStartParams: - num-cpus: "0" - template: - spec: - containers: - - name: ray-head - image: rayproject/ray:nightly - resources: - limits: - cpu: "1" - requests: - cpu: "1" - ports: - - containerPort: 6379 - name: redis - - containerPort: 8265 - name: dashboard - - containerPort: 10001 - name: client - volumeMounts: - - mountPath: /tmp/ray - name: ray-logs - - mountPath: /home/ray/samples - name: ray-example-configmap - volumes: - - name: ray-logs - emptyDir: {} - - name: ray-example-configmap - configMap: - name: ray-example - defaultMode: 0777 - items: - - key: detached_actor.py - path: detached_actor.py - - key: increment_counter.py - path: increment_counter.py - workerGroupSpecs: - - replicas: 1 - minReplicas: 1 - maxReplicas: 10 - groupName: small-group - rayStartParams: {} - template: - spec: - containers: - - name: ray-worker - image: rayproject/ray:nightly - volumeMounts: - - mountPath: /tmp/ray - name: ray-logs - resources: - limits: - cpu: "1" - requests: - cpu: "1" - volumes: - - name: ray-logs - emptyDir: {} ---- -apiVersion: v1 -kind: ConfigMap -metadata: - name: ray-example -data: - detached_actor.py: | - import ray - - @ray.remote(num_cpus=1) - class Counter: - def __init__(self): - self.value = 0 - - def increment(self): - self.value += 1 - return self.value - - ray.init(namespace="default_namespace") - Counter.options(name="counter_actor", lifetime="detached").remote() - increment_counter.py: | - import ray - - ray.init(namespace="default_namespace") - counter = ray.get_actor("counter_actor") - print(ray.get(counter.increment.remote())) From 152b126a05743c56bbd279c643228a91fc5a1633 Mon Sep 17 00:00:00 2001 From: win5923 Date: Mon, 20 Jan 2025 21:13:06 +0800 Subject: [PATCH 3/3] Resloved comments Signed-off-by: win5923 --- ray-operator/controllers/ray/common/pod.go | 12 +----- .../controllers/ray/common/pod_test.go | 42 ++++++++++++++++++- .../controllers/ray/raycluster_controller.go | 2 +- 3 files changed, 43 insertions(+), 13 deletions(-) diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 3d4d4d1d0cf..31c35d31a5a 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -126,6 +126,7 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra Value: options.RedisAddress, }) if options.RedisUsername != nil { + // Note that `redis-username` will be supported starting from Ray 2.41. // If `GcsFaultToleranceOptions.RedisUsername` is set, it will be put into the // `REDIS_USERNAME` environment variable later. Here, we use `$REDIS_USERNAME` in // rayStartParams to refer to the environment variable. @@ -148,17 +149,6 @@ func configureGCSFaultTolerance(podTemplate *corev1.PodTemplateSpec, instance ra }) } } else { - // If users directly set the `redis-username` in `rayStartParams` instead of referring - // to a K8s secret, we need to set the `REDIS_USERNAME` env var so that the Redis cleanup - // job can connect to Redis using the username. - if !utils.EnvVarExists(utils.REDIS_USERNAME, container.Env) { - // setting the REDIS_USERNAME env var from the params - redisUsernameEnv := corev1.EnvVar{Name: utils.REDIS_USERNAME} - if value, ok := instance.Spec.HeadGroupSpec.RayStartParams["redis-username"]; ok { - redisUsernameEnv.Value = value - } - container.Env = append(container.Env, redisUsernameEnv) - } // If users directly set the `redis-password` in `rayStartParams` instead of referring // to a K8s secret, we need to set the `REDIS_PASSWORD` env var so that the Redis cleanup // job can connect to Redis using the password. This is not recommended. diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index d8a22148640..38ff5003099 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -518,6 +518,19 @@ func TestConfigureGCSFaultToleranceWithGcsFTOptions(t *testing.T) { }, isHeadPod: true, }, + { + name: "GCS FT enabled with redis username and password", + gcsFTOptions: &rayv1.GcsFaultToleranceOptions{ + RedisAddress: "redis:6379", + RedisUsername: &rayv1.RedisCredential{ + Value: "test-username", + }, + RedisPassword: &rayv1.RedisCredential{ + Value: "test-password", + }, + }, + isHeadPod: true, + }, { name: "GCS FT enabled with redis password in secret", gcsFTOptions: &rayv1.GcsFaultToleranceOptions{ @@ -533,7 +546,28 @@ func TestConfigureGCSFaultToleranceWithGcsFTOptions(t *testing.T) { isHeadPod: true, }, { - name: "GCS FT enabled with redis password in secret", + name: "GCS FT enabled with redis username and password in secret", + gcsFTOptions: &rayv1.GcsFaultToleranceOptions{ + RedisAddress: "redis:6379", + RedisUsername: &rayv1.RedisCredential{ + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "spec.redisUsername", + }, + }, + }, + RedisPassword: &rayv1.RedisCredential{ + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "spec.redisPassword", + }, + }, + }, + }, + isHeadPod: true, + }, + { + name: "GCS FT enabled with redis external storage namespace", gcsFTOptions: &rayv1.GcsFaultToleranceOptions{ RedisAddress: "redis:6379", ExternalStorageNamespace: "test-ns", @@ -593,6 +627,12 @@ func TestConfigureGCSFaultToleranceWithGcsFTOptions(t *testing.T) { env := getEnvVar(container, utils.RAY_REDIS_ADDRESS) assert.Equal(t, env.Value, "redis:6379") + if test.gcsFTOptions.RedisUsername != nil { + env := getEnvVar(container, utils.REDIS_USERNAME) + assert.Equal(t, env.Value, test.gcsFTOptions.RedisUsername.Value) + assert.Equal(t, env.ValueFrom, test.gcsFTOptions.RedisUsername.ValueFrom) + } + if test.gcsFTOptions.RedisPassword != nil { env := getEnvVar(container, utils.REDIS_PASSWORD) assert.Equal(t, env.Value, test.gcsFTOptions.RedisPassword.Value) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index d69e1ee4ad2..8cf80089b97 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -1237,7 +1237,7 @@ func (r *RayClusterReconciler) buildRedisCleanupJob(ctx context.Context, instanc "parsed = urlparse(redis_address); ", } if utils.EnvVarExists(utils.REDIS_USERNAME, pod.Spec.Containers[utils.RayContainerIndex].Env) { - pod.Spec.Containers[utils.RayContainerIndex].Args[0] += "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, username=os.getenv('REDIS_USERNAME', parsed.username), password=os.getenv('REDIS_PASSWORD', parsed.password), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" + pod.Spec.Containers[utils.RayContainerIndex].Args[0] += "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, username=os.getenv('REDIS_USERNAME', parsed.username), password=os.getenv('REDIS_PASSWORD', parsed.password or ''), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" } else { pod.Spec.Containers[utils.RayContainerIndex].Args[0] += "sys.exit(1) if not cleanup_redis_storage(host=parsed.hostname, port=parsed.port, password=os.getenv('REDIS_PASSWORD', parsed.password or ''), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" }