-
Notifications
You must be signed in to change notification settings - Fork 436
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[RayCluster][Feature] add GcsFaultToleranceOptions to the RayCluster CRD #2712
Conversation
Signed-off-by: Rueian <[email protected]>
…tions Signed-off-by: Rueian <[email protected]>
Signed-off-by: Rueian <[email protected]>
Signed-off-by: Rueian <[email protected]>
Signed-off-by: Rueian <[email protected]>
…edis-password Signed-off-by: Rueian <[email protected]>
@@ -96,6 +96,9 @@ func initTemplateAnnotations(instance rayv1.RayCluster, podTemplate *corev1.PodT | |||
podTemplate.Annotations[utils.RayExternalStorageNSAnnotationKey] = v | |||
} | |||
} | |||
if options := instance.Spec.GcsFaultToleranceOptions; options != nil && options.ExternalStorageNamespace != "" { | |||
podTemplate.Annotations[utils.RayExternalStorageNSAnnotationKey] = options.ExternalStorageNamespace |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Override the ray.io/external-storage-namespace
annotation if the new ExternalStorageNamespace
is specified. We rely on this annotation to set the RAY_external_storage_namespace
env in the later setContainerEnvVars
function.
@@ -322,7 +325,7 @@ func initLivenessAndReadinessProbe(rayContainer *corev1.Container, rayNodeType r | |||
} | |||
|
|||
// BuildPod a pod config | |||
func BuildPod(ctx context.Context, podTemplateSpec corev1.PodTemplateSpec, rayNodeType rayv1.RayNodeType, rayStartParams map[string]string, headPort string, enableRayAutoscaler *bool, creatorCRDType utils.CRDType, fqdnRayIP string) (aPod corev1.Pod) { | |||
func BuildPod(ctx context.Context, podTemplateSpec corev1.PodTemplateSpec, rayNodeType rayv1.RayNodeType, gcsOptions *rayv1.GcsFaultToleranceOptions, rayStartParams map[string]string, headPort string, enableRayAutoscaler *bool, creatorCRDType utils.CRDType, fqdnRayIP string) (aPod corev1.Pod) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just add a new gcsOptions
parameter and pass it to the later setContainerEnvVars
function.
if rayNodeType == rayv1.HeadNode && gcsOptions != nil { | ||
if gcsOptions.RedisUsername != nil { | ||
rayStartParams["redis-username"] = "$REDIS_USERNAME" | ||
} | ||
if gcsOptions.RedisPassword != nil { | ||
rayStartParams["redis-password"] = "$REDIS_PASSWORD" | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If the new gcsOptions is specified with redis username and password, we add --redis-username and --redis-password flags to the ray start command.
The $REDIS_USERNAME and $REDIS_PASSWORD will be replaced with env vars by the shell.
if !utils.EnvVarExists(utils.REDIS_USERNAME, container.Env) { | ||
// setting the REDIS_USERNAME env var from the params | ||
if value, ok := rayStartParams["redis-username"]; ok { | ||
container.Env = append(container.Env, corev1.EnvVar{Name: utils.REDIS_USERNAME, Value: value}) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Although the support for Redis usernames in the Ray core has been merged but hasn't been released yet, I think we can add related code in KubeRay 1.3.
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), use_ssl=parsed.scheme=='rediss', storage_namespace=os.getenv('RAY_external_storage_namespace')) else None\"" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We add the username=os.getenv('REDIS_USERNAME', parsed.username)
positional argument only when the REDIS_USERNAME
env is present because the cleanup_redis_storage
function in old Ray doesn't support the argument.
Signed-off-by: Rueian <[email protected]>
Would you mind separating this into several smaller PRs? Perhaps open a PR to add the fields in the CRD as the first step. |
@kevin85421, no problem! Here is the first step which only contains the addition to the CRD. #2715 |
This will be split into even smaller PRs. |
Hi @rueian, I had previously inquired whether turning on GcsFaultTolerance was effective for Ray Cluster, i.e. whether the data corresponding to the Ray Dashboard could be recovered after the head node reboot, and the answer I got was no. I would like to ask if this feature is designed to support this? |
Hi @Moonquakes, The answer is still no. Ray only stores its cluster metadata in the Global Control Service (GCS); other data, such as metrics and logs, are not stored there. You will need to prepare additional external log storage (e.g., NFS) and metrics storage (e.g., Prometheus) for them if you want them to be recovered from failures. |
@rueian Thank you for your answer! However, GCS should also store task-related information. I don’t need to obtain logs and indicators. I just want to restore the original ray job and task information. |
Why are these changes needed?
Currently, we need users to set the
ray.io/ft-enabled
annotation, their redis address in envs and their redis password in both envs andRayStartParams
in their RayCluster CR to enable GCS FT. Too many places to set up.This PR adds
GcsFaultToleranceOptions
to the RayCluster CRD, unifying all the GCS FT settings in one field to provide a better user experience and all required annotations, envs, and ray start parameters will be set according to this newGcsFaultToleranceOptions
.With the new
GcsFaultToleranceOptions
, a GCS FT enabled RayCluster CR will look like this:Related issue number
Resolves #2695
Checks