From 5f62532c4e9d704df3c6048e51e9a0113ebabe15 Mon Sep 17 00:00:00 2001 From: Bobbins228 Date: Thu, 31 Oct 2024 14:37:43 +0000 Subject: [PATCH 1/2] refactor: creation of ray cluster/appwrapper --- .../ray/cluster/build_ray_cluster.py | 553 ++++++++++++++++++ src/codeflare_sdk/ray/cluster/cluster.py | 337 ++++++----- src/codeflare_sdk/ray/cluster/config.py | 8 - .../ray/cluster/generate_yaml.py | 308 ---------- .../ray/templates/base-template.yaml | 192 ------ 5 files changed, 746 insertions(+), 652 deletions(-) create mode 100644 src/codeflare_sdk/ray/cluster/build_ray_cluster.py delete mode 100755 src/codeflare_sdk/ray/cluster/generate_yaml.py delete mode 100644 src/codeflare_sdk/ray/templates/base-template.yaml diff --git a/src/codeflare_sdk/ray/cluster/build_ray_cluster.py b/src/codeflare_sdk/ray/cluster/build_ray_cluster.py new file mode 100644 index 000000000..e590d4834 --- /dev/null +++ b/src/codeflare_sdk/ray/cluster/build_ray_cluster.py @@ -0,0 +1,553 @@ +# Copyright 2024 IBM, Red Hat +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" + This sub-module exists primarily to be used internally by the Cluster object + (in the cluster sub-module) for RayCluster/AppWrapper generation. +""" +from typing import Union, Tuple, Dict +from ...common import _kube_api_error_handling +from ...common.kubernetes_cluster import get_api_client, config_check +from kubernetes.client.exceptions import ApiException +import codeflare_sdk +import os + +from kubernetes import client +from kubernetes.client import ( + V1ObjectMeta, + V1KeyToPath, + V1ConfigMapVolumeSource, + V1Volume, + V1VolumeMount, + V1ResourceRequirements, + V1Container, + V1ContainerPort, + V1Lifecycle, + V1ExecAction, + V1LifecycleHandler, + V1EnvVar, + V1PodTemplateSpec, + V1PodSpec, + V1LocalObjectReference, +) + +import yaml +import uuid +import sys +import warnings +import json + +FORBIDDEN_CUSTOM_RESOURCE_TYPES = ["GPU", "CPU", "memory"] +VOLUME_MOUNTS = [ + V1VolumeMount( + mount_path="/etc/pki/tls/certs/odh-trusted-ca-bundle.crt", + name="odh-trusted-ca-cert", + sub_path="odh-trusted-ca-bundle.crt", + ), + V1VolumeMount( + mount_path="/etc/ssl/certs/odh-trusted-ca-bundle.crt", + name="odh-trusted-ca-cert", + sub_path="odh-trusted-ca-bundle.crt", + ), + V1VolumeMount( + mount_path="/etc/pki/tls/certs/odh-ca-bundle.crt", + name="odh-ca-cert", + sub_path="odh-ca-bundle.crt", + ), + V1VolumeMount( + mount_path="/etc/ssl/certs/odh-ca-bundle.crt", + name="odh-ca-cert", + sub_path="odh-ca-bundle.crt", + ), +] + +VOLUMES = [ + V1Volume( + name="odh-trusted-ca-cert", + config_map=V1ConfigMapVolumeSource( + name="odh-trusted-ca-bundle", + items=[V1KeyToPath(key="ca-bundle.crt", path="odh-trusted-ca-bundle.crt")], + optional=True, + ), + ), + V1Volume( + name="odh-ca-cert", + config_map=V1ConfigMapVolumeSource( + name="odh-trusted-ca-bundle", + items=[V1KeyToPath(key="odh-ca-bundle.crt", path="odh-ca-bundle.crt")], + optional=True, + ), + ), +] + +SUPPORTED_PYTHON_VERSIONS = { + "3.9": "quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06", + "3.11": "quay.io/modh/ray@sha256:db667df1bc437a7b0965e8031e905d3ab04b86390d764d120e05ea5a5c18d1b4", +} + + +# RayCluster/AppWrapper builder function +def build_ray_cluster(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """build_ray_cluster is used for creating a Ray Cluster/AppWrapper dict + + The resource is a dict template which uses Kubernetes Objects for creating metadata, resource requests, + specs and containers. The result is sanitised and returned either as a dict or written as a yaml file. + """ + ray_version = "2.35.0" + + # GPU related variables + head_gpu_count, worker_gpu_count = head_worker_gpu_count_from_cluster(cluster) + head_resources, worker_resources = head_worker_extended_resources_from_cluster( + cluster + ) + head_resources = json.dumps(head_resources).replace('"', '\\"') + head_resources = f'"{head_resources}"' + worker_resources = json.dumps(worker_resources).replace('"', '\\"') + worker_resources = f'"{worker_resources}"' + + # Create the Ray Cluster using the V1RayCluster Object + resource = { + "apiVersion": "ray.io/v1", + "kind": "RayCluster", + "metadata": get_metadata(cluster), + "spec": { + "rayVersion": ray_version, + "enableInTreeAutoscaling": False, + "autoscalerOptions": { + "upscalingMode": "Default", + "idleTimeoutSeconds": 60, + "resources": get_resources("500m", "500m", "512Mi", "512Mi"), + }, + "headGroupSpec": { + "serviceType": "ClusterIP", + "enableIngress": False, + "rayStartParams": { + "dashboard-host": "0.0.0.0", + "block": "true", + "num-gpus": str(head_gpu_count), + "resources": head_resources, + }, + "template": { + "spec": get_pod_spec(cluster, [get_head_container_spec(cluster)]) + }, + }, + "workerGroupSpecs": [ + { + "replicas": cluster.config.num_workers, + "minReplicas": cluster.config.num_workers, + "maxReplicas": cluster.config.num_workers, + "groupName": f"small-group-{cluster.config.name}", + "rayStartParams": { + "block": "true", + "num-gpus": str(worker_gpu_count), + "resources": worker_resources, + }, + "template": V1PodTemplateSpec( + spec=get_pod_spec(cluster, [get_worker_container_spec(cluster)]) + ), + } + ], + }, + } + + config_check() + k8s_client = get_api_client() or client.ApiClient() + + if cluster.config.appwrapper: + # Wrap the Ray Cluster in an AppWrapper + appwrapper_name, _ = gen_names(cluster.config.name) + resource = wrap_cluster(cluster, appwrapper_name, resource) + + resource = k8s_client.sanitize_for_serialization(resource) + + # write_to_file functionality + if cluster.config.write_to_file: + return write_to_file(cluster, resource) # Writes the file and returns its name + else: + print(f"Yaml resources loaded for {cluster.config.name}") + return resource # Returns the Resource as a dict + + +# Metadata related functions +def get_metadata(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """ + The get_metadata() function builds and returns a V1ObjectMeta Object using cluster configurtation parameters + """ + object_meta = V1ObjectMeta( + name=cluster.config.name, + namespace=cluster.config.namespace, + labels=get_labels(cluster), + ) + + # Get the NB annotation if it exists - could be useful in future for a "annotations" parameter. + annotations = get_nb_annotations() + if annotations != {}: + object_meta.annotations = annotations # As annotations are not a guarantee they are appended to the metadata after creation. + return object_meta + + +def get_labels(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """ + The get_labels() function generates a dict "labels" which includes the base label, local queue label and user defined labels + """ + labels = { + "controller-tools.k8s.io": "1.0", + } + if cluster.config.labels != {}: + labels.update(cluster.config.labels) + + if cluster.config.appwrapper is False: + add_queue_label(cluster, labels) + + return labels + + +def get_nb_annotations(): + """ + The get_nb_annotations() function generates the annotation for NB Prefix if the SDK is running in a notebook + """ + annotations = {} + + # Notebook annotation + nb_prefix = os.environ.get("NB_PREFIX") + if nb_prefix: + annotations.update({"app.kubernetes.io/managed-by": nb_prefix}) + + return annotations + + +# Head/Worker container related functions +def update_image(image) -> str: + """ + The update_image() function automatically sets the image config parameter to a preset image based on Python version if not specified. + If no Ray image exists for the given Python version a warning is produced. + """ + if not image: + python_version = f"{sys.version_info.major}.{sys.version_info.minor}" + if python_version in SUPPORTED_PYTHON_VERSIONS: + image = SUPPORTED_PYTHON_VERSIONS[python_version] + else: + warnings.warn( + f"No default Ray image defined for {python_version}. Please provide your own image or use one of the following python versions: {', '.join(SUPPORTED_PYTHON_VERSIONS.keys())}." + ) + return image + + +def get_pod_spec(cluster: "codeflare_sdk.ray.cluster.Cluster", containers): + """ + The get_pod_spec() function generates a V1PodSpec for the head/worker containers + """ + pod_spec = V1PodSpec( + containers=containers, + volumes=VOLUMES, + ) + if cluster.config.image_pull_secrets != []: + pod_spec.image_pull_secrets = generate_image_pull_secrets(cluster) + + return pod_spec + + +def generate_image_pull_secrets(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """ + The generate_image_pull_secrets() methods generates a list of V1LocalObjectReference including each of the specified image pull secrets + """ + pull_secrets = [] + for pull_secret in cluster.config.image_pull_secrets: + pull_secrets.append(V1LocalObjectReference(name=pull_secret)) + + return pull_secrets + + +def get_head_container_spec( + cluster: "codeflare_sdk.ray.cluster.Cluster", +): + """ + The get_head_container_spec() function builds and returns a V1Container object including user defined resource requests/limits + """ + head_container = V1Container( + name="ray-head", + image=update_image(cluster.config.image), + image_pull_policy="Always", + ports=[ + V1ContainerPort(name="gcs", container_port=6379), + V1ContainerPort(name="dashboard", container_port=8265), + V1ContainerPort(name="client", container_port=10001), + ], + lifecycle=V1Lifecycle( + pre_stop=V1LifecycleHandler( + _exec=V1ExecAction(["/bin/sh", "-c", "ray stop"]) + ) + ), + resources=get_resources( + cluster.config.head_cpu_requests, + cluster.config.head_cpu_limits, + cluster.config.head_memory_requests, + cluster.config.head_memory_limits, + cluster.config.head_extended_resource_requests, + ), + volume_mounts=VOLUME_MOUNTS, + ) + if cluster.config.envs != {}: + head_container.env = generate_env_vars(cluster) + + return head_container + + +def generate_env_vars(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """ + The generate_env_vars() builds and returns a V1EnvVar object which is populated by user specified environment variables + """ + envs = [] + for key, value in cluster.config.envs.items(): + env_var = V1EnvVar(name=key, value=value) + envs.append(env_var) + + return envs + + +def get_worker_container_spec( + cluster: "codeflare_sdk.ray.cluster.Cluster", +): + """ + The get_worker_container_spec() function builds and returns a V1Container object including user defined resource requests/limits + """ + worker_container = V1Container( + name="machine-learning", + image=update_image(cluster.config.image), + image_pull_policy="Always", + lifecycle=V1Lifecycle( + pre_stop=V1LifecycleHandler( + _exec=V1ExecAction(["/bin/sh", "-c", "ray stop"]) + ) + ), + resources=get_resources( + cluster.config.worker_cpu_requests, + cluster.config.worker_cpu_limits, + cluster.config.worker_memory_requests, + cluster.config.worker_memory_limits, + cluster.config.worker_extended_resource_requests, + ), + volume_mounts=VOLUME_MOUNTS, + ) + + if cluster.config.envs != {}: + worker_container.env = generate_env_vars(cluster) + + return worker_container + + +def get_resources( + cpu_requests: Union[int, str], + cpu_limits: Union[int, str], + memory_requests: Union[int, str], + memory_limits: Union[int, str], + custom_extended_resource_requests: Dict[str, int] = None, +): + """ + The get_resources() function generates a V1ResourceRequirements object for cpu/memory request/limits and GPU resources + """ + resource_requirements = V1ResourceRequirements( + requests={"cpu": cpu_requests, "memory": memory_requests}, + limits={"cpu": cpu_limits, "memory": memory_limits}, + ) + + # Append the resource/limit requests with custom extended resources + if custom_extended_resource_requests is not None: + for k in custom_extended_resource_requests.keys(): + resource_requirements.limits[k] = custom_extended_resource_requests[k] + resource_requirements.requests[k] = custom_extended_resource_requests[k] + + return resource_requirements + + +# GPU related functions +def head_worker_gpu_count_from_cluster( + cluster: "codeflare_sdk.ray.cluster.Cluster", +) -> Tuple[int, int]: + """ + The head_worker_gpu_count_from_cluster() function returns the total number of requested GPUs for the head and worker separately + """ + head_gpus = 0 + worker_gpus = 0 + for k in cluster.config.head_extended_resource_requests.keys(): + resource_type = cluster.config.extended_resource_mapping[k] + if resource_type == "GPU": + head_gpus += int(cluster.config.head_extended_resource_requests[k]) + for k in cluster.config.worker_extended_resource_requests.keys(): + resource_type = cluster.config.extended_resource_mapping[k] + if resource_type == "GPU": + worker_gpus += int(cluster.config.worker_extended_resource_requests[k]) + + return head_gpus, worker_gpus + + +def head_worker_extended_resources_from_cluster( + cluster: "codeflare_sdk.ray.cluster.Cluster", +) -> Tuple[dict, dict]: + """ + The head_worker_extended_resources_from_cluster() function returns 2 dicts for head/worker respectively populated by the GPU type requested by the user + """ + head_worker_extended_resources = {}, {} + for k in cluster.config.head_extended_resource_requests.keys(): + resource_type = cluster.config.extended_resource_mapping[k] + if resource_type in FORBIDDEN_CUSTOM_RESOURCE_TYPES: + continue + head_worker_extended_resources[0][ + resource_type + ] = cluster.config.head_extended_resource_requests[ + k + ] + head_worker_extended_resources[ + 0 + ].get( + resource_type, 0 + ) + + for k in cluster.config.worker_extended_resource_requests.keys(): + resource_type = cluster.config.extended_resource_mapping[k] + if resource_type in FORBIDDEN_CUSTOM_RESOURCE_TYPES: + continue + head_worker_extended_resources[1][ + resource_type + ] = cluster.config.worker_extended_resource_requests[ + k + ] + head_worker_extended_resources[ + 1 + ].get( + resource_type, 0 + ) + return head_worker_extended_resources + + +# Local Queue related functions +def add_queue_label(cluster: "codeflare_sdk.ray.cluster.Cluster", labels: dict): + """ + The add_queue_label() function updates the given base labels with the local queue label if Kueue exists on the Cluster + """ + lq_name = cluster.config.local_queue or get_default_local_queue(cluster, labels) + if lq_name == None: + return + elif not local_queue_exists(cluster): + raise ValueError( + "local_queue provided does not exist or is not in this namespace. Please provide the correct local_queue name in Cluster Configuration" + ) + labels.update({"kueue.x-k8s.io/queue-name": lq_name}) + + +def local_queue_exists(cluster: "codeflare_sdk.ray.cluster.Cluster"): + """ + The local_queue_exists() checks if the user inputted local_queue exists in the given namespace and returns a bool + """ + # get all local queues in the namespace + try: + config_check() + api_instance = client.CustomObjectsApi(get_api_client()) + local_queues = api_instance.list_namespaced_custom_object( + group="kueue.x-k8s.io", + version="v1beta1", + namespace=cluster.config.namespace, + plural="localqueues", + ) + except Exception as e: # pragma: no cover + return _kube_api_error_handling(e) + # check if local queue with the name provided in cluster config exists + for lq in local_queues["items"]: + if lq["metadata"]["name"] == cluster.config.local_queue: + return True + return False + + +def get_default_local_queue(cluster: "codeflare_sdk.ray.cluster.Cluster", labels: dict): + """ + The get_default_local_queue() function attempts to find a local queue with the default label == true, if that is the case the labels variable is updated with that local queue + """ + try: + # Try to get the default local queue if it exists and append the label list + config_check() + api_instance = client.CustomObjectsApi(get_api_client()) + local_queues = api_instance.list_namespaced_custom_object( + group="kueue.x-k8s.io", + version="v1beta1", + namespace=cluster.config.namespace, + plural="localqueues", + ) + except ApiException as e: # pragma: no cover + if e.status == 404 or e.status == 403: + return + else: + return _kube_api_error_handling(e) + + for lq in local_queues["items"]: + if ( + "annotations" in lq["metadata"] + and "kueue.x-k8s.io/default-queue" in lq["metadata"]["annotations"] + and lq["metadata"]["annotations"]["kueue.x-k8s.io/default-queue"].lower() + == "true" + ): + labels.update({"kueue.x-k8s.io/queue-name": lq["metadata"]["name"]}) + + +# AppWrapper related functions +def wrap_cluster( + cluster: "codeflare_sdk.ray.cluster.Cluster", + appwrapper_name: str, + ray_cluster_yaml: dict, +): + """ + Wraps the pre-built Ray Cluster dict in an AppWrapper + """ + wrapping = { + "apiVersion": "workload.codeflare.dev/v1beta2", + "kind": "AppWrapper", + "metadata": {"name": appwrapper_name, "namespace": cluster.config.namespace}, + "spec": {"components": [{"template": ray_cluster_yaml}]}, + } + # Add local queue label if it is necessary + labels = {} + add_queue_label(cluster, labels) + if labels != {}: + wrapping["metadata"]["labels"] = labels + + return wrapping + + +# Etc. +def write_to_file(cluster: "codeflare_sdk.ray.cluster.Cluster", resource: dict): + """ + The write_to_file function writes the built Ray Cluster/AppWrapper dict as a yaml file in the .codeflare folder + """ + directory_path = os.path.expanduser("~/.codeflare/resources/") + output_file_name = os.path.join(directory_path, cluster.config.name + ".yaml") + + directory_path = os.path.dirname(output_file_name) + if not os.path.exists(directory_path): + os.makedirs(directory_path) + + with open(output_file_name, "w") as outfile: + yaml.dump(resource, outfile, default_flow_style=False) + + print(f"Written to: {output_file_name}") + return output_file_name + + +def gen_names(name): + """ + Generates a unique name for the appwrapper and Ray Cluster + """ + if not name: + gen_id = str(uuid.uuid4()) + appwrapper_name = "appwrapper-" + gen_id + cluster_name = "cluster-" + gen_id + return appwrapper_name, cluster_name + else: + return name, name diff --git a/src/codeflare_sdk/ray/cluster/cluster.py b/src/codeflare_sdk/ray/cluster/cluster.py index da87639c5..fe29eaa97 100644 --- a/src/codeflare_sdk/ray/cluster/cluster.py +++ b/src/codeflare_sdk/ray/cluster/cluster.py @@ -28,12 +28,9 @@ get_api_client, ) from . import pretty_print -from .generate_yaml import ( - generate_appwrapper, - head_worker_gpu_count_from_cluster, -) +from .build_ray_cluster import build_ray_cluster, head_worker_gpu_count_from_cluster +from .build_ray_cluster import write_to_file as write_cluster_to_file from ...common import _kube_api_error_handling -from .generate_yaml import is_openshift_cluster from .config import ClusterConfiguration from .status import ( @@ -56,6 +53,7 @@ from kubernetes import config from kubernetes.client.rest import ApiException +import warnings class Cluster: @@ -74,9 +72,15 @@ def __init__(self, config: ClusterConfiguration): request. """ self.config = config - self.app_wrapper_yaml = self.create_app_wrapper() + if self.config is None: + warnings.warn( + "Please provide a ClusterConfiguration to initialise the Cluster object" + ) + return + else: + self.resource_yaml = self.create_resource() + self._job_submission_client = None - self.app_wrapper_name = self.config.name if is_notebook(): cluster_up_down_buttons(self) @@ -91,7 +95,7 @@ def _client_headers(self): @property def _client_verify_tls(self): - if not is_openshift_cluster or not self.config.verify_tls: + if not _is_openshift_cluster or not self.config.verify_tls: return False return True @@ -100,7 +104,7 @@ def job_client(self): k8client = get_api_client() if self._job_submission_client: return self._job_submission_client - if is_openshift_cluster(): + if _is_openshift_cluster(): self._job_submission_client = JobSubmissionClient( self.cluster_dashboard_uri(), headers=self._client_headers, @@ -112,7 +116,7 @@ def job_client(self): ) return self._job_submission_client - def create_app_wrapper(self): + def create_resource(self): """ Called upon cluster object creation, creates an AppWrapper yaml based on the specifications of the ClusterConfiguration. @@ -127,7 +131,7 @@ def create_app_wrapper(self): f"Namespace {self.config.namespace} is of type {type(self.config.namespace)}. Check your Kubernetes Authentication." ) - return generate_appwrapper(self) + return build_ray_cluster(self) # creates a new cluster with the provided or default spec def up(self): @@ -146,7 +150,7 @@ def up(self): api_instance = client.CustomObjectsApi(get_api_client()) if self.config.appwrapper: if self.config.write_to_file: - with open(self.app_wrapper_yaml) as f: + with open(self.resource_yaml) as f: aw = yaml.load(f, Loader=yaml.FullLoader) api_instance.create_namespaced_custom_object( group="workload.codeflare.dev", @@ -156,13 +160,12 @@ def up(self): body=aw, ) else: - aw = yaml.safe_load(self.app_wrapper_yaml) api_instance.create_namespaced_custom_object( group="workload.codeflare.dev", version="v1beta2", namespace=namespace, plural="appwrappers", - body=aw, + body=self.resource_yaml, ) print(f"AppWrapper: '{self.config.name}' has successfully been created") else: @@ -198,6 +201,7 @@ def down(self): associated with the cluster. """ namespace = self.config.namespace + resource_name = self.config.name self._throw_for_no_raycluster() try: config_check() @@ -208,11 +212,11 @@ def down(self): version="v1beta2", namespace=namespace, plural="appwrappers", - name=self.app_wrapper_name, + name=resource_name, ) - print(f"AppWrapper: '{self.config.name}' has successfully been deleted") + print(f"AppWrapper: '{resource_name}' has successfully been deleted") else: - self._component_resources_down(namespace, api_instance) + _delete_resources(resource_name, namespace, api_instance) print( f"Ray Cluster: '{self.config.name}' has successfully been deleted" ) @@ -358,7 +362,7 @@ def cluster_dashboard_uri(self) -> str: Returns a string containing the cluster's dashboard URI. """ config_check() - if is_openshift_cluster(): + if _is_openshift_cluster(): try: api_instance = client.CustomObjectsApi(get_api_client()) routes = api_instance.list_namespaced_custom_object( @@ -442,67 +446,6 @@ def _head_worker_extended_resources_from_rc_dict(rc: Dict) -> Tuple[dict, dict]: return head_extended_resources, worker_extended_resources - def from_k8_cluster_object( - rc, - appwrapper=True, - write_to_file=False, - verify_tls=True, - ): - config_check() - machine_types = ( - rc["metadata"]["labels"]["orderedinstance"].split("_") - if "orderedinstance" in rc["metadata"]["labels"] - else [] - ) - - ( - head_extended_resources, - worker_extended_resources, - ) = Cluster._head_worker_extended_resources_from_rc_dict(rc) - - cluster_config = ClusterConfiguration( - name=rc["metadata"]["name"], - namespace=rc["metadata"]["namespace"], - machine_types=machine_types, - head_cpu_requests=rc["spec"]["headGroupSpec"]["template"]["spec"][ - "containers" - ][0]["resources"]["requests"]["cpu"], - head_cpu_limits=rc["spec"]["headGroupSpec"]["template"]["spec"][ - "containers" - ][0]["resources"]["limits"]["cpu"], - head_memory_requests=rc["spec"]["headGroupSpec"]["template"]["spec"][ - "containers" - ][0]["resources"]["requests"]["memory"], - head_memory_limits=rc["spec"]["headGroupSpec"]["template"]["spec"][ - "containers" - ][0]["resources"]["limits"]["memory"], - num_workers=rc["spec"]["workerGroupSpecs"][0]["minReplicas"], - worker_cpu_requests=rc["spec"]["workerGroupSpecs"][0]["template"]["spec"][ - "containers" - ][0]["resources"]["requests"]["cpu"], - worker_cpu_limits=rc["spec"]["workerGroupSpecs"][0]["template"]["spec"][ - "containers" - ][0]["resources"]["limits"]["cpu"], - worker_memory_requests=rc["spec"]["workerGroupSpecs"][0]["template"][ - "spec" - ]["containers"][0]["resources"]["requests"]["memory"], - worker_memory_limits=rc["spec"]["workerGroupSpecs"][0]["template"]["spec"][ - "containers" - ][0]["resources"]["limits"]["memory"], - worker_extended_resource_requests=worker_extended_resources, - head_extended_resource_requests=head_extended_resources, - image=rc["spec"]["workerGroupSpecs"][0]["template"]["spec"]["containers"][ - 0 - ]["image"], - appwrapper=appwrapper, - write_to_file=write_to_file, - verify_tls=verify_tls, - local_queue=rc["metadata"] - .get("labels", dict()) - .get("kueue.x-k8s.io/queue-name", None), - ) - return Cluster(cluster_config) - def local_client_url(self): ingress_domain = _get_ingress_domain(self) return f"ray://{ingress_domain}" @@ -511,35 +454,22 @@ def _component_resources_up( self, namespace: str, api_instance: client.CustomObjectsApi ): if self.config.write_to_file: - with open(self.app_wrapper_yaml) as f: - yamls = list(yaml.load_all(f, Loader=yaml.FullLoader)) - for resource in yamls: - enable_ingress = ( - resource.get("spec", {}) - .get("headGroupSpec", {}) - .get("enableIngress") - ) - if resource["kind"] == "RayCluster" and enable_ingress is True: - name = resource["metadata"]["name"] - print( - f"Forbidden: RayCluster '{name}' has 'enableIngress' set to 'True'." - ) - return - _create_resources(yamls, namespace, api_instance) + with open(self.resource_yaml) as f: + ray_cluster = yaml.safe_load(f) + _create_resources(ray_cluster, namespace, api_instance) else: - yamls = yaml.load_all(self.app_wrapper_yaml, Loader=yaml.FullLoader) - _create_resources(yamls, namespace, api_instance) + _create_resources(self.resource_yaml, namespace, api_instance) def _component_resources_down( self, namespace: str, api_instance: client.CustomObjectsApi ): cluster_name = self.config.name if self.config.write_to_file: - with open(self.app_wrapper_yaml) as f: + with open(self.resource_yaml) as f: yamls = yaml.load_all(f, Loader=yaml.FullLoader) _delete_resources(yamls, namespace, api_instance, cluster_name) else: - yamls = yaml.safe_load_all(self.app_wrapper_yaml) + yamls = yaml.safe_load_all(self.resource_yaml) _delete_resources(yamls, namespace, api_instance, cluster_name) @@ -595,61 +525,166 @@ def get_current_namespace(): # pragma: no cover def get_cluster( cluster_name: str, namespace: str = "default", - write_to_file: bool = False, verify_tls: bool = True, + write_to_file: bool = False, ): - try: - config_check() - api_instance = client.CustomObjectsApi(get_api_client()) - rcs = api_instance.list_namespaced_custom_object( - group="ray.io", - version="v1", - namespace=namespace, - plural="rayclusters", - ) - except Exception as e: - return _kube_api_error_handling(e) - - for rc in rcs["items"]: - if rc["metadata"]["name"] == cluster_name: - appwrapper = _check_aw_exists(cluster_name, namespace) - return Cluster.from_k8_cluster_object( - rc, - appwrapper=appwrapper, - write_to_file=write_to_file, - verify_tls=verify_tls, + """Returns the given Ray Cluster/AppWrapper as a Cluster Object + + The get_cluster() method is used for retrieving a Ray Cluster that already exists in your K8s Cluster. + Returned is a basic Cluster object which includes the exact yaml for your Ray Cluster under Cluster.resource_yaml. + + Parameters + ---------- + cluster_name : str + The name of the Ray Cluster/AppWrapper + namespace : str + The namespace of the Ray Cluster/AppWrapper + verify_tls : bool + A boolean indicating whether to verify TLS when connecting to the cluster + write_to_file : bool + A boolean indicating whether or not to write the resource to a Yaml file + + Raises + ------ + Exception + If the Ray Cluster/AppWrapper cannot be found/does not exist + """ + config_check() + api_instance = client.CustomObjectsApi(get_api_client()) + # Check/Get the AppWrapper if it exists + is_appwrapper = _check_aw_exists(cluster_name, namespace) + if is_appwrapper: + try: + resource = api_instance.get_namespaced_custom_object( + group="workload.codeflare.dev", + version="v1beta2", + namespace=namespace, + plural="appwrappers", + name=cluster_name, ) - raise FileNotFoundError( - f"Cluster {cluster_name} is not found in {namespace} namespace" - ) - - -# private methods -def _delete_resources( - yamls, namespace: str, api_instance: client.CustomObjectsApi, cluster_name: str -): - for resource in yamls: - if resource["kind"] == "RayCluster": - name = resource["metadata"]["name"] - api_instance.delete_namespaced_custom_object( + resource_extraction = resource["spec"]["components"][0]["template"] + except Exception as e: + return _kube_api_error_handling(e) + else: + # Get the Ray Cluster + try: + resource = api_instance.get_namespaced_custom_object( group="ray.io", version="v1", namespace=namespace, plural="rayclusters", - name=name, + name=cluster_name, ) + resource_extraction = resource + except Exception as e: + return _kube_api_error_handling(e) + + ( + head_extended_resources, + worker_extended_resources, + ) = Cluster._head_worker_extended_resources_from_rc_dict(resource_extraction) + # Create a Cluster Configuration with just the necessary provided parameters + cluster_config = ClusterConfiguration( + name=cluster_name, + namespace=namespace, + verify_tls=verify_tls, + write_to_file=write_to_file, + appwrapper=is_appwrapper, + head_cpu_limits=resource_extraction["spec"]["headGroupSpec"]["template"][ + "spec" + ]["containers"][0]["resources"]["requests"]["cpu"], + head_cpu_requests=resource_extraction["spec"]["headGroupSpec"]["template"][ + "spec" + ]["containers"][0]["resources"]["limits"]["cpu"], + head_memory_limits=resource_extraction["spec"]["headGroupSpec"]["template"][ + "spec" + ]["containers"][0]["resources"]["requests"]["memory"], + head_memory_requests=resource_extraction["spec"]["headGroupSpec"]["template"][ + "spec" + ]["containers"][0]["resources"]["limits"]["memory"], + num_workers=resource_extraction["spec"]["workerGroupSpecs"][0]["minReplicas"], + worker_cpu_limits=resource_extraction["spec"]["workerGroupSpecs"][0][ + "template" + ]["spec"]["containers"][0]["resources"]["limits"]["cpu"], + worker_cpu_requests=resource_extraction["spec"]["workerGroupSpecs"][0][ + "template" + ]["spec"]["containers"][0]["resources"]["requests"]["cpu"], + worker_memory_limits=resource_extraction["spec"]["workerGroupSpecs"][0][ + "template" + ]["spec"]["containers"][0]["resources"]["requests"]["memory"], + worker_memory_requests=resource_extraction["spec"]["workerGroupSpecs"][0][ + "template" + ]["spec"]["containers"][0]["resources"]["limits"]["memory"], + head_extended_resource_requests=head_extended_resources, + worker_extended_resource_requests=worker_extended_resources, + ) + # Ignore the warning here for the lack of a ClusterConfiguration + with warnings.catch_warnings(): + warnings.filterwarnings( + "ignore", + message="Please provide a ClusterConfiguration to initialise the Cluster object", + ) + cluster = Cluster(None) + cluster.config = cluster_config + + # Remove auto-generated fields like creationTimestamp, uid and etc. + remove_autogenerated_fields(resource) + + if write_to_file: + cluster.resource_yaml = write_cluster_to_file(cluster, resource) + else: + # Update the Cluster's resource_yaml to reflect the retrieved Ray Cluster/AppWrapper + cluster.resource_yaml = resource + print(f"Yaml resources loaded for {cluster.config.name}") + + return cluster + + +def remove_autogenerated_fields(resource): + """Recursively remove autogenerated fields from a dictionary.""" + if isinstance(resource, dict): + for key in list(resource.keys()): + if key in [ + "creationTimestamp", + "resourceVersion", + "uid", + "selfLink", + "managedFields", + "finalizers", + "generation", + "status", + "suspend", + "workload.codeflare.dev/user", # AppWrapper field + "workload.codeflare.dev/userid", # AppWrapper field + "podSetInfos", # AppWrapper field + ]: + del resource[key] + else: + remove_autogenerated_fields(resource[key]) + elif isinstance(resource, list): + for item in resource: + remove_autogenerated_fields(item) + + +# private methods +def _delete_resources(name: str, namespace: str, api_instance: client.CustomObjectsApi): + api_instance.delete_namespaced_custom_object( + group="ray.io", + version="v1", + namespace=namespace, + plural="rayclusters", + name=name, + ) def _create_resources(yamls, namespace: str, api_instance: client.CustomObjectsApi): - for resource in yamls: - if resource["kind"] == "RayCluster": - api_instance.create_namespaced_custom_object( - group="ray.io", - version="v1", - namespace=namespace, - plural="rayclusters", - body=resource, - ) + api_instance.create_namespaced_custom_object( + group="ray.io", + version="v1", + namespace=namespace, + plural="rayclusters", + body=yamls, + ) def _check_aw_exists(name: str, namespace: str) -> bool: @@ -680,7 +715,7 @@ def _get_ingress_domain(self): # pragma: no cover namespace = get_current_namespace() domain = None - if is_openshift_cluster(): + if _is_openshift_cluster(): try: api_instance = client.CustomObjectsApi(get_api_client()) @@ -812,7 +847,7 @@ def _map_to_ray_cluster(rc) -> Optional[RayCluster]: status = RayClusterStatus.UNKNOWN config_check() dashboard_url = None - if is_openshift_cluster(): + if _is_openshift_cluster(): try: api_instance = client.CustomObjectsApi(get_api_client()) routes = api_instance.list_namespaced_custom_object( @@ -926,3 +961,17 @@ def _copy_to_ray(cluster: Cluster) -> RayCluster: if ray.status == CodeFlareClusterStatus.READY: ray.status = RayClusterStatus.READY return ray + + +# Check if the routes api exists +def _is_openshift_cluster(): + try: + config_check() + for api in client.ApisApi(get_api_client()).get_api_versions().groups: + for v in api.versions: + if "route.openshift.io/v1" in v.group_version: + return True + else: + return False + except Exception as e: # pragma: no cover + return _kube_api_error_handling(e) diff --git a/src/codeflare_sdk/ray/cluster/config.py b/src/codeflare_sdk/ray/cluster/config.py index a9fe44088..b8b1652e3 100644 --- a/src/codeflare_sdk/ray/cluster/config.py +++ b/src/codeflare_sdk/ray/cluster/config.py @@ -47,19 +47,16 @@ class ClusterConfiguration: Attributes: - name: The name of the cluster. - namespace: The namespace in which the cluster should be created. - - head_info: A list of strings containing information about the head node. - head_cpus: The number of CPUs to allocate to the head node. - head_memory: The amount of memory to allocate to the head node. - head_gpus: The number of GPUs to allocate to the head node. (Deprecated, use head_extended_resource_requests) - head_extended_resource_requests: A dictionary of extended resource requests for the head node. ex: {"nvidia.com/gpu": 1} - - machine_types: A list of machine types to use for the cluster. - min_cpus: The minimum number of CPUs to allocate to each worker. - max_cpus: The maximum number of CPUs to allocate to each worker. - num_workers: The number of workers to create. - min_memory: The minimum amount of memory to allocate to each worker. - max_memory: The maximum amount of memory to allocate to each worker. - num_gpus: The number of GPUs to allocate to each worker. (Deprecated, use worker_extended_resource_requests) - - template: The path to the template file to use for the cluster. - appwrapper: A boolean indicating whether to use an AppWrapper. - envs: A dictionary of environment variables to set for the cluster. - image: The image to use for the cluster. @@ -74,7 +71,6 @@ class ClusterConfiguration: name: str namespace: Optional[str] = None - head_info: List[str] = field(default_factory=list) head_cpu_requests: Union[int, str] = 2 head_cpu_limits: Union[int, str] = 2 head_cpus: Optional[Union[int, str]] = None # Deprecating @@ -85,9 +81,6 @@ class ClusterConfiguration: head_extended_resource_requests: Dict[str, Union[str, int]] = field( default_factory=dict ) - machine_types: List[str] = field( - default_factory=list - ) # ["m4.xlarge", "g4dn.xlarge"] worker_cpu_requests: Union[int, str] = 1 worker_cpu_limits: Union[int, str] = 1 min_cpus: Optional[Union[int, str]] = None # Deprecating @@ -98,7 +91,6 @@ class ClusterConfiguration: min_memory: Optional[Union[int, str]] = None # Deprecating max_memory: Optional[Union[int, str]] = None # Deprecating num_gpus: Optional[int] = None # Deprecating - template: str = f"{dir}/templates/base-template.yaml" appwrapper: bool = False envs: Dict[str, str] = field(default_factory=dict) image: str = "" diff --git a/src/codeflare_sdk/ray/cluster/generate_yaml.py b/src/codeflare_sdk/ray/cluster/generate_yaml.py deleted file mode 100755 index 01823f1df..000000000 --- a/src/codeflare_sdk/ray/cluster/generate_yaml.py +++ /dev/null @@ -1,308 +0,0 @@ -# Copyright 2022 IBM, Red Hat -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -""" -This sub-module exists primarily to be used internally by the Cluster object -(in the cluster sub-module) for AppWrapper generation. -""" - -import json -import sys -import typing -import warnings -import yaml -import os -import uuid -from kubernetes import client -from ...common import _kube_api_error_handling -from ...common.kueue import add_queue_label -from ...common.kubernetes_cluster.auth import ( - get_api_client, - config_check, -) -import codeflare_sdk - -SUPPORTED_PYTHON_VERSIONS = { - "3.9": "quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06", - "3.11": "quay.io/modh/ray@sha256:db667df1bc437a7b0965e8031e905d3ab04b86390d764d120e05ea5a5c18d1b4", -} - - -def read_template(template): - with open(template, "r") as stream: - try: - return yaml.safe_load(stream) - except yaml.YAMLError as exc: - print(exc) - - -def gen_names(name): - if not name: - gen_id = str(uuid.uuid4()) - appwrapper_name = "appwrapper-" + gen_id - cluster_name = "cluster-" + gen_id - return appwrapper_name, cluster_name - else: - return name, name - - -# Check if the routes api exists -def is_openshift_cluster(): - try: - config_check() - for api in client.ApisApi(get_api_client()).get_api_versions().groups: - for v in api.versions: - if "route.openshift.io/v1" in v.group_version: - return True - else: - return False - except Exception as e: # pragma: no cover - return _kube_api_error_handling(e) - - -def is_kind_cluster(): - try: - config_check() - v1 = client.CoreV1Api() - label_selector = "kubernetes.io/hostname=kind-control-plane" - nodes = v1.list_node(label_selector=label_selector) - # If we find one or more nodes with the label, assume it's a KinD cluster - return len(nodes.items) > 0 - except Exception as e: - print(f"Error checking if cluster is KinD: {e}") - return False - - -def update_names( - cluster_yaml: dict, - cluster: "codeflare_sdk.ray.cluster.cluster.Cluster", -): - metadata = cluster_yaml.get("metadata") - metadata["name"] = cluster.config.name - metadata["namespace"] = cluster.config.namespace - - -def update_image(spec, image): - containers = spec.get("containers") - if not image: - python_version = f"{sys.version_info.major}.{sys.version_info.minor}" - if python_version in SUPPORTED_PYTHON_VERSIONS: - image = SUPPORTED_PYTHON_VERSIONS[python_version] - else: - warnings.warn( - f"No default Ray image defined for {python_version}. Please provide your own image or use one of the following python versions: {', '.join(SUPPORTED_PYTHON_VERSIONS.keys())}." - ) - for container in containers: - container["image"] = image - - -def update_image_pull_secrets(spec, image_pull_secrets): - template_secrets = spec.get("imagePullSecrets", []) - spec["imagePullSecrets"] = template_secrets + [ - {"name": x} for x in image_pull_secrets - ] - - -def update_env(spec, env): - containers = spec.get("containers") - for container in containers: - if env: - if "env" in container: - container["env"].extend(env) - else: - container["env"] = env - - -def update_resources( - spec, - cpu_requests, - cpu_limits, - memory_requests, - memory_limits, - custom_resources, -): - container = spec.get("containers") - for resource in container: - requests = resource.get("resources").get("requests") - if requests is not None: - requests["cpu"] = cpu_requests - requests["memory"] = memory_requests - limits = resource.get("resources").get("limits") - if limits is not None: - limits["cpu"] = cpu_limits - limits["memory"] = memory_limits - for k in custom_resources.keys(): - limits[k] = custom_resources[k] - requests[k] = custom_resources[k] - - -def head_worker_gpu_count_from_cluster( - cluster: "codeflare_sdk.ray.cluster.cluster.Cluster", -) -> typing.Tuple[int, int]: - head_gpus = 0 - worker_gpus = 0 - for k in cluster.config.head_extended_resource_requests.keys(): - resource_type = cluster.config.extended_resource_mapping[k] - if resource_type == "GPU": - head_gpus += int(cluster.config.head_extended_resource_requests[k]) - for k in cluster.config.worker_extended_resource_requests.keys(): - resource_type = cluster.config.extended_resource_mapping[k] - if resource_type == "GPU": - worker_gpus += int(cluster.config.worker_extended_resource_requests[k]) - - return head_gpus, worker_gpus - - -FORBIDDEN_CUSTOM_RESOURCE_TYPES = ["GPU", "CPU", "memory"] - - -def head_worker_resources_from_cluster( - cluster: "codeflare_sdk.ray.cluster.cluster.Cluster", -) -> typing.Tuple[dict, dict]: - to_return = {}, {} - for k in cluster.config.head_extended_resource_requests.keys(): - resource_type = cluster.config.extended_resource_mapping[k] - if resource_type in FORBIDDEN_CUSTOM_RESOURCE_TYPES: - continue - to_return[0][resource_type] = cluster.config.head_extended_resource_requests[ - k - ] + to_return[0].get(resource_type, 0) - - for k in cluster.config.worker_extended_resource_requests.keys(): - resource_type = cluster.config.extended_resource_mapping[k] - if resource_type in FORBIDDEN_CUSTOM_RESOURCE_TYPES: - continue - to_return[1][resource_type] = cluster.config.worker_extended_resource_requests[ - k - ] + to_return[1].get(resource_type, 0) - return to_return - - -def update_nodes( - ray_cluster_dict: dict, - cluster: "codeflare_sdk.ray.cluster.cluster.Cluster", -): - head = ray_cluster_dict.get("spec").get("headGroupSpec") - worker = ray_cluster_dict.get("spec").get("workerGroupSpecs")[0] - head_gpus, worker_gpus = head_worker_gpu_count_from_cluster(cluster) - head_resources, worker_resources = head_worker_resources_from_cluster(cluster) - head_resources = json.dumps(head_resources).replace('"', '\\"') - head_resources = f'"{head_resources}"' - worker_resources = json.dumps(worker_resources).replace('"', '\\"') - worker_resources = f'"{worker_resources}"' - head["rayStartParams"]["num-gpus"] = str(head_gpus) - head["rayStartParams"]["resources"] = head_resources - - # Head counts as first worker - worker["replicas"] = cluster.config.num_workers - worker["minReplicas"] = cluster.config.num_workers - worker["maxReplicas"] = cluster.config.num_workers - worker["groupName"] = "small-group-" + cluster.config.name - worker["rayStartParams"]["num-gpus"] = str(worker_gpus) - worker["rayStartParams"]["resources"] = worker_resources - - for comp in [head, worker]: - spec = comp.get("template").get("spec") - update_image_pull_secrets(spec, cluster.config.image_pull_secrets) - update_image(spec, cluster.config.image) - update_env(spec, cluster.config.envs) - if comp == head: - # TODO: Eventually add head node configuration outside of template - update_resources( - spec, - cluster.config.head_cpu_requests, - cluster.config.head_cpu_limits, - cluster.config.head_memory_requests, - cluster.config.head_memory_limits, - cluster.config.head_extended_resource_requests, - ) - else: - update_resources( - spec, - cluster.config.worker_cpu_requests, - cluster.config.worker_cpu_limits, - cluster.config.worker_memory_requests, - cluster.config.worker_memory_limits, - cluster.config.worker_extended_resource_requests, - ) - - -def del_from_list_by_name(l: list, target: typing.List[str]) -> list: - return [x for x in l if x["name"] not in target] - - -def augment_labels(item: dict, labels: dict): - if not "labels" in item["metadata"]: - item["metadata"]["labels"] = {} - item["metadata"]["labels"].update(labels) - - -def notebook_annotations(item: dict): - nb_prefix = os.environ.get("NB_PREFIX") - if nb_prefix: - if not "annotations" in item["metadata"]: - item["metadata"]["annotations"] = {} - item["metadata"]["annotations"].update( - {"app.kubernetes.io/managed-by": nb_prefix} - ) - - -def wrap_cluster(cluster_yaml: dict, appwrapper_name: str, namespace: str): - return { - "apiVersion": "workload.codeflare.dev/v1beta2", - "kind": "AppWrapper", - "metadata": {"name": appwrapper_name, "namespace": namespace}, - "spec": {"components": [{"template": cluster_yaml}]}, - } - - -def write_user_yaml(user_yaml, output_file_name): - # Create the directory if it doesn't exist - directory_path = os.path.dirname(output_file_name) - if not os.path.exists(directory_path): - os.makedirs(directory_path) - - with open(output_file_name, "w") as outfile: - yaml.dump(user_yaml, outfile, default_flow_style=False) - - print(f"Written to: {output_file_name}") - - -def generate_appwrapper(cluster: "codeflare_sdk.ray.cluster.cluster.Cluster"): - cluster_yaml = read_template(cluster.config.template) - appwrapper_name, _ = gen_names(cluster.config.name) - update_names( - cluster_yaml, - cluster, - ) - update_nodes(cluster_yaml, cluster) - augment_labels(cluster_yaml, cluster.config.labels) - notebook_annotations(cluster_yaml) - user_yaml = ( - wrap_cluster(cluster_yaml, appwrapper_name, cluster.config.namespace) - if cluster.config.appwrapper - else cluster_yaml - ) - - add_queue_label(user_yaml, cluster.config.namespace, cluster.config.local_queue) - - if cluster.config.write_to_file: - directory_path = os.path.expanduser("~/.codeflare/resources/") - outfile = os.path.join(directory_path, appwrapper_name + ".yaml") - write_user_yaml(user_yaml, outfile) - return outfile - else: - user_yaml = yaml.dump(user_yaml) - print(f"Yaml resources loaded for {cluster.config.name}") - return user_yaml diff --git a/src/codeflare_sdk/ray/templates/base-template.yaml b/src/codeflare_sdk/ray/templates/base-template.yaml deleted file mode 100644 index b59d2a495..000000000 --- a/src/codeflare_sdk/ray/templates/base-template.yaml +++ /dev/null @@ -1,192 +0,0 @@ -# This config demonstrates KubeRay's Ray autoscaler integration. -# The resource requests and limits in this config are too small for production! -# For an example with more realistic resource configuration, see -# ray-cluster.autoscaler.large.yaml. -apiVersion: ray.io/v1 -kind: RayCluster -metadata: - labels: - controller-tools.k8s.io: "1.0" - # A unique identifier for the head node and workers of this cluster. - name: kuberay-cluster - namespace: default -spec: - # The version of Ray you are using. Make sure all Ray containers are running this version of Ray. - rayVersion: '2.35.0' - # If enableInTreeAutoscaling is true, the autoscaler sidecar will be added to the Ray head pod. - # Ray autoscaler integration is supported only for Ray versions >= 1.11.0 - # Ray autoscaler integration is Beta with KubeRay >= 0.3.0 and Ray >= 2.0.0. - enableInTreeAutoscaling: false - # autoscalerOptions is an OPTIONAL field specifying configuration overrides for the Ray autoscaler. - # The example configuration shown below below represents the DEFAULT values. - # (You may delete autoscalerOptions if the defaults are suitable.) - autoscalerOptions: - # upscalingMode is "Default" or "Aggressive." - # Conservative: Upscaling is rate-limited; the number of pending worker pods is at most the size of the Ray cluster. - # Default: Upscaling is not rate-limited. - # Aggressive: An alias for Default; upscaling is not rate-limited. - upscalingMode: Default - # idleTimeoutSeconds is the number of seconds to wait before scaling down a worker pod which is not using Ray resources. - idleTimeoutSeconds: 60 - # image optionally overrides the autoscaler's container image. - # If instance.spec.rayVersion is at least "2.0.0", the autoscaler will default to the same image as - # the ray container. For older Ray versions, the autoscaler will default to using the Ray 2.0.0 image. - ## image: "my-repo/my-custom-autoscaler-image:tag" - # imagePullPolicy optionally overrides the autoscaler container's image pull policy. - imagePullPolicy: Always - # resources specifies optional resource request and limit overrides for the autoscaler container. - # For large Ray clusters, we recommend monitoring container resource usage to determine if overriding the defaults is required. - resources: - limits: - cpu: "500m" - memory: "512Mi" - requests: - cpu: "500m" - memory: "512Mi" - ######################headGroupSpec################################# - # head group template and specs, (perhaps 'group' is not needed in the name) - headGroupSpec: - # Kubernetes Service Type, valid values are 'ClusterIP', 'NodePort' and 'LoadBalancer' - serviceType: ClusterIP - enableIngress: false - # logical group name, for this called head-group, also can be functional - # pod type head or worker - # rayNodeType: head # Not needed since it is under the headgroup - # the following params are used to complete the ray start: ray start --head --block ... - rayStartParams: - # Flag "no-monitor" will be automatically set when autoscaling is enabled. - dashboard-host: '0.0.0.0' - block: 'true' - # num-cpus: '1' # can be auto-completed from the limits - # Use `resources` to optionally specify custom resource annotations for the Ray node. - # The value of `resources` is a string-integer mapping. - # Currently, `resources` must be provided in the specific format demonstrated below: - # resources: '"{\"Custom1\": 1, \"Custom2\": 5}"' - num-gpus: '0' - #pod template - template: - spec: - containers: - # The Ray head pod - - name: ray-head - imagePullPolicy: Always - ports: - - containerPort: 6379 - name: gcs - - containerPort: 8265 - name: dashboard - - containerPort: 10001 - name: client - lifecycle: - preStop: - exec: - command: ["/bin/sh","-c","ray stop"] - resources: - limits: - cpu: 2 - memory: "8G" - requests: - cpu: 2 - memory: "8G" - volumeMounts: - - mountPath: /etc/pki/tls/certs/odh-trusted-ca-bundle.crt - name: odh-trusted-ca-cert - subPath: odh-trusted-ca-bundle.crt - - mountPath: /etc/ssl/certs/odh-trusted-ca-bundle.crt - name: odh-trusted-ca-cert - subPath: odh-trusted-ca-bundle.crt - - mountPath: /etc/pki/tls/certs/odh-ca-bundle.crt - name: odh-ca-cert - subPath: odh-ca-bundle.crt - - mountPath: /etc/ssl/certs/odh-ca-bundle.crt - name: odh-ca-cert - subPath: odh-ca-bundle.crt - volumes: - - name: odh-trusted-ca-cert - configMap: - name: odh-trusted-ca-bundle - items: - - key: ca-bundle.crt - path: odh-trusted-ca-bundle.crt - optional: true - - name: odh-ca-cert - configMap: - name: odh-trusted-ca-bundle - items: - - key: odh-ca-bundle.crt - path: odh-ca-bundle.crt - optional: true - workerGroupSpecs: - # the pod replicas in this group typed worker - - replicas: 3 - minReplicas: 3 - maxReplicas: 3 - # logical group name, for this called small-group, also can be functional - groupName: small-group - # if worker pods need to be added, we can simply increment the replicas - # if worker pods need to be removed, we decrement the replicas, and populate the podsToDelete list - # the operator will remove pods from the list until the number of replicas is satisfied - # when a pod is confirmed to be deleted, its name will be removed from the list below - #scaleStrategy: - # workersToDelete: - # - raycluster-complete-worker-small-group-bdtwh - # - raycluster-complete-worker-small-group-hv457 - # - raycluster-complete-worker-small-group-k8tj7 - # the following params are used to complete the ray start: ray start --block ... - rayStartParams: - block: 'true' - num-gpus: 1 - #pod template - template: - metadata: - labels: - key: value - # annotations for pod - annotations: - key: value - # finalizers: - # - kubernetes - spec: - containers: - - name: machine-learning # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' - # environment variables to set in the container.Optional. - # Refer to https://kubernetes.io/docs/tasks/inject-data-application/define-environment-variable-container/ - lifecycle: - preStop: - exec: - command: ["/bin/sh","-c","ray stop"] - resources: - limits: - cpu: "2" - memory: "12G" - requests: - cpu: "2" - memory: "12G" - volumeMounts: - - mountPath: /etc/pki/tls/certs/odh-trusted-ca-bundle.crt - name: odh-trusted-ca-cert - subPath: odh-trusted-ca-bundle.crt - - mountPath: /etc/ssl/certs/odh-trusted-ca-bundle.crt - name: odh-trusted-ca-cert - subPath: odh-trusted-ca-bundle.crt - - mountPath: /etc/pki/tls/certs/odh-ca-bundle.crt - name: odh-ca-cert - subPath: odh-ca-bundle.crt - - mountPath: /etc/ssl/certs/odh-ca-bundle.crt - name: odh-ca-cert - subPath: odh-ca-bundle.crt - volumes: - - name: odh-trusted-ca-cert - configMap: - name: odh-trusted-ca-bundle - items: - - key: ca-bundle.crt - path: odh-trusted-ca-bundle.crt - optional: true - - name: odh-ca-cert - configMap: - name: odh-trusted-ca-bundle - items: - - key: odh-ca-bundle.crt - path: odh-ca-bundle.crt - optional: true From 30bf971033c880eedecddeaaa30cdf60d96b2da1 Mon Sep 17 00:00:00 2001 From: Bobbins228 Date: Thu, 31 Oct 2024 16:27:25 +0000 Subject: [PATCH 2/2] test: update unit tests after refactor --- src/codeflare_sdk/common/kueue/test_kueue.py | 12 +- .../common/utils/unit_test_support.py | 36 ++++ .../ray/appwrapper/test_status.py | 7 +- ...rate_yaml.py => test_build_ray_cluster.py} | 13 +- src/codeflare_sdk/ray/cluster/test_cluster.py | 164 +++++----------- src/codeflare_sdk/ray/cluster/test_config.py | 124 +++++------- .../ray/cluster/test_pretty_print.py | 7 +- src/codeflare_sdk/ray/cluster/test_status.py | 13 +- .../appwrapper/test-case-bad.yaml | 3 +- .../appwrapper/unit-test-all-params.yaml | 176 ++++++++++++++++++ tests/test_cluster_yamls/kueue/aw_kueue.yaml | 9 +- .../kueue/ray_cluster_kueue.yaml | 9 +- .../ray/default-appwrapper.yaml | 9 +- .../ray/default-ray-cluster.yaml | 9 +- .../ray/unit-test-all-params.yaml | 25 +-- .../support_clusters/test-aw-a.yaml | 3 +- .../support_clusters/test-aw-b.yaml | 3 +- .../support_clusters/test-rc-a.yaml | 3 +- .../support_clusters/test-rc-b.yaml | 3 +- ui-tests/playwright.config.js | 2 +- 20 files changed, 355 insertions(+), 275 deletions(-) rename src/codeflare_sdk/ray/cluster/{test_generate_yaml.py => test_build_ray_cluster.py} (84%) create mode 100644 tests/test_cluster_yamls/appwrapper/unit-test-all-params.yaml diff --git a/src/codeflare_sdk/common/kueue/test_kueue.py b/src/codeflare_sdk/common/kueue/test_kueue.py index e9de364d8..77095d4d9 100644 --- a/src/codeflare_sdk/common/kueue/test_kueue.py +++ b/src/codeflare_sdk/common/kueue/test_kueue.py @@ -51,8 +51,7 @@ def test_cluster_creation_no_aw_local_queue(mocker): config.write_to_file = True config.local_queue = "local-queue-default" cluster = Cluster(config) - assert cluster.app_wrapper_yaml == f"{aw_dir}unit-test-cluster-kueue.yaml" - assert cluster.app_wrapper_name == "unit-test-cluster-kueue" + assert cluster.resource_yaml == f"{aw_dir}unit-test-cluster-kueue.yaml" assert filecmp.cmp( f"{aw_dir}unit-test-cluster-kueue.yaml", f"{parent}/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml", @@ -65,10 +64,9 @@ def test_cluster_creation_no_aw_local_queue(mocker): config.write_to_file = False cluster = Cluster(config) - test_rc = yaml.load(cluster.app_wrapper_yaml, Loader=yaml.FullLoader) with open(f"{parent}/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml") as f: expected_rc = yaml.load(f, Loader=yaml.FullLoader) - assert test_rc == expected_rc + assert cluster.resource_yaml == expected_rc def test_aw_creation_local_queue(mocker): @@ -87,8 +85,7 @@ def test_aw_creation_local_queue(mocker): config.write_to_file = True config.local_queue = "local-queue-default" cluster = Cluster(config) - assert cluster.app_wrapper_yaml == f"{aw_dir}unit-test-aw-kueue.yaml" - assert cluster.app_wrapper_name == "unit-test-aw-kueue" + assert cluster.resource_yaml == f"{aw_dir}unit-test-aw-kueue.yaml" assert filecmp.cmp( f"{aw_dir}unit-test-aw-kueue.yaml", f"{parent}/tests/test_cluster_yamls/kueue/aw_kueue.yaml", @@ -102,10 +99,9 @@ def test_aw_creation_local_queue(mocker): config.write_to_file = False cluster = Cluster(config) - test_rc = yaml.load(cluster.app_wrapper_yaml, Loader=yaml.FullLoader) with open(f"{parent}/tests/test_cluster_yamls/kueue/aw_kueue.yaml") as f: expected_rc = yaml.load(f, Loader=yaml.FullLoader) - assert test_rc == expected_rc + assert cluster.resource_yaml == expected_rc def test_get_local_queue_exists_fail(mocker): diff --git a/src/codeflare_sdk/common/utils/unit_test_support.py b/src/codeflare_sdk/common/utils/unit_test_support.py index 61a16260c..88b65aa4f 100644 --- a/src/codeflare_sdk/common/utils/unit_test_support.py +++ b/src/codeflare_sdk/common/utils/unit_test_support.py @@ -20,6 +20,7 @@ import yaml from pathlib import Path from kubernetes import client +from unittest.mock import patch parent = Path(__file__).resolve().parents[4] # project directory aw_dir = os.path.expanduser("~/.codeflare/resources/") @@ -381,3 +382,38 @@ def mocked_ingress(port, cluster_name="unit-test-cluster", annotations: dict = N ), ) return mock_ingress + + +@patch.dict("os.environ", {"NB_PREFIX": "test-prefix"}) +def create_cluster_all_config_params(mocker, cluster_name, is_appwrapper) -> Cluster: + mocker.patch( + "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", + return_value=get_local_queue("kueue.x-k8s.io", "v1beta1", "ns", "localqueues"), + ) + + config = ClusterConfiguration( + name=cluster_name, + namespace="ns", + head_cpu_requests=4, + head_cpu_limits=8, + head_memory_requests=12, + head_memory_limits=16, + head_extended_resource_requests={"nvidia.com/gpu": 1, "intel.com/gpu": 2}, + worker_cpu_requests=4, + worker_cpu_limits=8, + num_workers=10, + worker_memory_requests=12, + worker_memory_limits=16, + appwrapper=is_appwrapper, + envs={"key1": "value1", "key2": "value2"}, + image="example/ray:tag", + image_pull_secrets=["secret1", "secret2"], + write_to_file=True, + verify_tls=True, + labels={"key1": "value1", "key2": "value2"}, + worker_extended_resource_requests={"nvidia.com/gpu": 1}, + extended_resource_mapping={"example.com/gpu": "GPU", "intel.com/gpu": "TPU"}, + overwrite_default_resource_mapping=True, + local_queue="local-queue-default", + ) + return Cluster(config) diff --git a/src/codeflare_sdk/ray/appwrapper/test_status.py b/src/codeflare_sdk/ray/appwrapper/test_status.py index 8c693767c..a3fcf8700 100644 --- a/src/codeflare_sdk/ray/appwrapper/test_status.py +++ b/src/codeflare_sdk/ray/appwrapper/test_status.py @@ -19,6 +19,7 @@ ) from codeflare_sdk.ray.appwrapper import AppWrapper, AppWrapperStatus from codeflare_sdk.ray.cluster.status import CodeFlareClusterStatus +from codeflare_sdk.common.utils.unit_test_support import get_local_queue import os aw_dir = os.path.expanduser("~/.codeflare/resources/") @@ -28,8 +29,8 @@ def test_cluster_status(mocker): mocker.patch("kubernetes.client.ApisApi.get_api_versions") mocker.patch("kubernetes.config.load_kube_config", return_value="ignore") mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", + "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", + return_value=get_local_queue("kueue.x-k8s.io", "v1beta1", "ns", "localqueues"), ) fake_aw = AppWrapper("test", AppWrapperStatus.FAILED) @@ -39,7 +40,7 @@ def test_cluster_status(mocker): namespace="ns", write_to_file=True, appwrapper=True, - local_queue="local_default_queue", + local_queue="local-queue-default", ) ) mocker.patch( diff --git a/src/codeflare_sdk/ray/cluster/test_generate_yaml.py b/src/codeflare_sdk/ray/cluster/test_build_ray_cluster.py similarity index 84% rename from src/codeflare_sdk/ray/cluster/test_generate_yaml.py rename to src/codeflare_sdk/ray/cluster/test_build_ray_cluster.py index 606cc950c..7d6d3d0a6 100644 --- a/src/codeflare_sdk/ray/cluster/test_generate_yaml.py +++ b/src/codeflare_sdk/ray/cluster/test_build_ray_cluster.py @@ -13,7 +13,7 @@ # limitations under the License. from collections import namedtuple import sys -from .generate_yaml import gen_names, update_image +from .build_ray_cluster import gen_names, update_image import uuid @@ -39,7 +39,7 @@ def test_gen_names_without_name(mocker): def test_update_image_without_supported_python_version(mocker): # Mock SUPPORTED_PYTHON_VERSIONS mocker.patch.dict( - "codeflare_sdk.ray.cluster.generate_yaml.SUPPORTED_PYTHON_VERSIONS", + "codeflare_sdk.ray.cluster.build_ray_cluster.SUPPORTED_PYTHON_VERSIONS", { "3.9": "ray-py3.9", "3.11": "ray-py3.11", @@ -55,16 +55,13 @@ def test_update_image_without_supported_python_version(mocker): # Mock warnings.warn to check if it gets called warn_mock = mocker.patch("warnings.warn") - # Create a sample spec - spec = {"containers": [{"image": None}]} - # Call the update_image function with no image provided - update_image(spec, None) + image = update_image(None) # Assert that the warning was called with the expected message warn_mock.assert_called_once_with( "No default Ray image defined for 3.8. Please provide your own image or use one of the following python versions: 3.9, 3.11." ) - # Assert that no image was set in the containers since the Python version is not supported - assert spec["containers"][0]["image"] is None + # Assert that no image was set since the Python version is not supported + assert image is None diff --git a/src/codeflare_sdk/ray/cluster/test_cluster.py b/src/codeflare_sdk/ray/cluster/test_cluster.py index 20438bbe3..5e83c82a8 100644 --- a/src/codeflare_sdk/ray/cluster/test_cluster.py +++ b/src/codeflare_sdk/ray/cluster/test_cluster.py @@ -25,21 +25,17 @@ arg_check_apply_effect, get_local_queue, createClusterConfig, - route_list_retrieval, get_ray_obj, - get_aw_obj, - get_named_aw, get_obj_none, get_ray_obj_with_status, get_aw_obj_with_status, ) -from codeflare_sdk.ray.cluster.generate_yaml import ( - is_openshift_cluster, - is_kind_cluster, -) +from codeflare_sdk.ray.cluster.cluster import _is_openshift_cluster from pathlib import Path from unittest.mock import MagicMock from kubernetes import client +import yaml +import filecmp import os parent = Path(__file__).resolve().parents[4] # project directory @@ -205,7 +201,7 @@ def test_local_client_url(mocker): return_value="rayclient-unit-test-cluster-localinter-ns.apps.cluster.awsroute.org", ) mocker.patch( - "codeflare_sdk.ray.cluster.cluster.Cluster.create_app_wrapper", + "codeflare_sdk.ray.cluster.cluster.Cluster.create_resource", return_value="unit-test-cluster-localinter.yaml", ) @@ -225,120 +221,51 @@ def test_local_client_url(mocker): """ -def test_get_cluster_openshift(mocker): +def test_get_cluster_no_appwrapper(mocker): + """ + This test uses the "test all params" unit test file as a comparison + """ + mocker.patch("kubernetes.client.ApisApi.get_api_versions") mocker.patch("kubernetes.config.load_kube_config", return_value="ignore") - # Mock the client.ApisApi function to return a mock object - mock_api = MagicMock() - mock_api.get_api_versions.return_value.groups = [ - MagicMock(versions=[MagicMock(group_version="route.openshift.io/v1")]) - ] - mocker.patch("kubernetes.client.ApisApi", return_value=mock_api) mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", + "codeflare_sdk.ray.cluster.cluster._check_aw_exists", + return_value=False, ) - assert is_openshift_cluster() - - def custom_side_effect(group, version, namespace, plural, **kwargs): - if plural == "routes": - return route_list_retrieval("route.openshift.io", "v1", "ns", "routes") - elif plural == "rayclusters": - return get_ray_obj("ray.io", "v1", "ns", "rayclusters") - elif plural == "appwrappers": - return get_aw_obj("workload.codeflare.dev", "v1beta2", "ns", "appwrappers") - elif plural == "localqueues": - return get_local_queue("kueue.x-k8s.io", "v1beta1", "ns", "localqueues") - - mocker.patch( - "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", get_aw_obj - ) - - mocker.patch( - "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", - side_effect=custom_side_effect, - ) - mocker.patch( - "kubernetes.client.CustomObjectsApi.get_namespaced_custom_object", - return_value=get_named_aw, - ) - mocker.patch( - "kubernetes.client.CustomObjectsApi.get_namespaced_custom_object", - side_effect=route_list_retrieval("route.openshift.io", "v1", "ns", "routes")[ - "items" - ], - ) - mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", - ) - - cluster = get_cluster( - "test-cluster-a", "ns" - ) # see tests/test_cluster_yamls/support_clusters - cluster_config = cluster.config - - assert cluster_config.name == "test-cluster-a" and cluster_config.namespace == "ns" - assert cluster_config.head_cpu_requests == 2 and cluster_config.head_cpu_limits == 2 - assert ( - cluster_config.head_memory_requests == "8G" - and cluster_config.head_memory_limits == "8G" - ) - assert ( - cluster_config.worker_cpu_requests == 1 - and cluster_config.worker_cpu_limits == 1 - ) - assert ( - cluster_config.worker_memory_requests == "2G" - and cluster_config.worker_memory_limits == "2G" - ) - assert cluster_config.num_workers == 1 - assert cluster_config.write_to_file == False - assert cluster_config.local_queue == "local_default_queue" + with open(f"{expected_clusters_dir}/ray/unit-test-all-params.yaml") as f: + expected_rc = yaml.load(f, Loader=yaml.FullLoader) + mocker.patch( + "kubernetes.client.CustomObjectsApi.get_namespaced_custom_object", + return_value=expected_rc, + ) + get_cluster("test-all-params", "ns", write_to_file=True) + assert filecmp.cmp( + f"{aw_dir}test-all-params.yaml", + f"{expected_clusters_dir}/ray/unit-test-all-params.yaml", + shallow=True, + ) -def test_get_cluster(mocker): - # test get_cluster for Kind Clusters +def test_get_cluster_with_appwrapper(mocker): mocker.patch("kubernetes.client.ApisApi.get_api_versions") mocker.patch("kubernetes.config.load_kube_config", return_value="ignore") mocker.patch( - "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", - side_effect=get_ray_obj, - ) - mocker.patch( - "kubernetes.client.CustomObjectsApi.get_namespaced_custom_object", - side_effect=get_named_aw, - ) - mocker.patch( - "kubernetes.client.NetworkingV1Api.list_namespaced_ingress", - return_value=ingress_retrieval(cluster_name="quicktest", client_ing=True), - ) - mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", + "codeflare_sdk.ray.cluster.cluster._check_aw_exists", + return_value=True, ) - cluster = get_cluster( - "test-cluster-a" - ) # see tests/test_cluster_yamls/support_clusters - cluster_config = cluster.config - assert cluster_config.name == "test-cluster-a" and cluster_config.namespace == "ns" - assert cluster_config.head_cpu_requests == 2 and cluster_config.head_cpu_limits == 2 - assert ( - cluster_config.head_memory_requests == "8G" - and cluster_config.head_memory_limits == "8G" - ) - assert ( - cluster_config.worker_cpu_requests == 1 - and cluster_config.worker_cpu_limits == 1 - ) - assert ( - cluster_config.worker_memory_requests == "2G" - and cluster_config.worker_memory_limits == "2G" - ) - assert cluster_config.num_workers == 1 - assert cluster_config.write_to_file == False - assert cluster_config.local_queue == "local_default_queue" + with open(f"{expected_clusters_dir}/appwrapper/unit-test-all-params.yaml") as f: + expected_aw = yaml.load(f, Loader=yaml.FullLoader) + mocker.patch( + "kubernetes.client.CustomObjectsApi.get_namespaced_custom_object", + return_value=expected_aw, + ) + get_cluster("aw-all-params", "ns", write_to_file=True) + assert filecmp.cmp( + f"{aw_dir}aw-all-params.yaml", + f"{expected_clusters_dir}/appwrapper/unit-test-all-params.yaml", + shallow=True, + ) def test_wait_ready(mocker, capsys): @@ -356,10 +283,6 @@ def test_wait_ready(mocker, capsys): mocker.patch( "codeflare_sdk.ray.cluster.cluster._ray_cluster_status", return_value=None ) - mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", - ) mocker.patch.object( client.CustomObjectsApi, "list_namespaced_custom_object", @@ -381,7 +304,6 @@ def test_wait_ready(mocker, capsys): namespace="ns", write_to_file=False, appwrapper=True, - local_queue="local-queue-default", ) ) try: @@ -460,7 +382,7 @@ def test_list_queue_rayclusters(mocker, capsys): ] mocker.patch("kubernetes.client.ApisApi", return_value=mock_api) - assert is_openshift_cluster() == True + assert _is_openshift_cluster() == True mocker.patch( "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", return_value=get_obj_none("ray.io", "v1", "ns", "rayclusters"), @@ -566,7 +488,7 @@ def test_map_to_ray_cluster(mocker): mocker.patch("kubernetes.config.load_kube_config") mocker.patch( - "codeflare_sdk.ray.cluster.cluster.is_openshift_cluster", return_value=True + "codeflare_sdk.ray.cluster.cluster._is_openshift_cluster", return_value=True ) mock_api_client = mocker.MagicMock(spec=client.ApiClient) @@ -608,3 +530,9 @@ def custom_side_effect(group, version, namespace, plural, **kwargs): assert result is not None assert result.dashboard == rc_dashboard + + +# Make sure to always keep this function last +def test_cleanup(): + os.remove(f"{aw_dir}test-all-params.yaml") + os.remove(f"{aw_dir}aw-all-params.yaml") diff --git a/src/codeflare_sdk/ray/cluster/test_config.py b/src/codeflare_sdk/ray/cluster/test_config.py index f1ac53559..1423fc2b5 100644 --- a/src/codeflare_sdk/ray/cluster/test_config.py +++ b/src/codeflare_sdk/ray/cluster/test_config.py @@ -12,10 +12,13 @@ # See the License for the specific language governing permissions and # limitations under the License. -from codeflare_sdk.common.utils.unit_test_support import createClusterWrongType +from codeflare_sdk.common.utils.unit_test_support import ( + createClusterWrongType, + get_local_queue, + create_cluster_all_config_params, +) from codeflare_sdk.ray.cluster.cluster import ClusterConfiguration, Cluster from pathlib import Path -from unittest.mock import patch import filecmp import pytest import yaml @@ -31,17 +34,11 @@ def test_default_cluster_creation(mocker): mocker.patch("kubernetes.client.ApisApi.get_api_versions") mocker.patch("kubernetes.client.CustomObjectsApi.list_namespaced_custom_object") - cluster = Cluster( - ClusterConfiguration( - name="default-cluster", - namespace="ns", - ) - ) + cluster = Cluster(ClusterConfiguration(name="default-cluster", namespace="ns")) - test_rc = yaml.load(cluster.app_wrapper_yaml, Loader=yaml.FullLoader) with open(f"{expected_clusters_dir}/ray/default-ray-cluster.yaml") as f: expected_rc = yaml.load(f, Loader=yaml.FullLoader) - assert test_rc == expected_rc + assert cluster.resource_yaml == expected_rc def test_default_appwrapper_creation(mocker): @@ -53,81 +50,46 @@ def test_default_appwrapper_creation(mocker): ClusterConfiguration(name="default-appwrapper", namespace="ns", appwrapper=True) ) - test_aw = yaml.load(cluster.app_wrapper_yaml, Loader=yaml.FullLoader) with open(f"{expected_clusters_dir}/ray/default-appwrapper.yaml") as f: expected_aw = yaml.load(f, Loader=yaml.FullLoader) - assert test_aw == expected_aw + assert cluster.resource_yaml == expected_aw -@patch.dict("os.environ", {"NB_PREFIX": "test-prefix"}) def test_config_creation_all_parameters(mocker): from codeflare_sdk.ray.cluster.config import DEFAULT_RESOURCE_MAPPING - mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", - ) - extended_resource_mapping = DEFAULT_RESOURCE_MAPPING - extended_resource_mapping.update({"example.com/gpu": "GPU"}) - - config = ClusterConfiguration( - name="test-all-params", - namespace="ns", - head_info=["test1", "test2"], - head_cpu_requests=4, - head_cpu_limits=8, - head_memory_requests=12, - head_memory_limits=16, - head_extended_resource_requests={"nvidia.com/gpu": 1}, - machine_types={"gpu.small", "gpu.large"}, - worker_cpu_requests=4, - worker_cpu_limits=8, - num_workers=10, - worker_memory_requests=12, - worker_memory_limits=16, - template=f"{parent}/src/codeflare_sdk/ray/templates/base-template.yaml", - appwrapper=False, - envs={"key1": "value1", "key2": "value2"}, - image="example/ray:tag", - image_pull_secrets=["secret1", "secret2"], - write_to_file=True, - verify_tls=True, - labels={"key1": "value1", "key2": "value2"}, - worker_extended_resource_requests={"nvidia.com/gpu": 1}, - extended_resource_mapping=extended_resource_mapping, - overwrite_default_resource_mapping=True, - local_queue="local-queue-default", - ) - Cluster(config) - - assert config.name == "test-all-params" and config.namespace == "ns" - assert config.head_info == ["test1", "test2"] - assert config.head_cpu_requests == 4 - assert config.head_cpu_limits == 8 - assert config.head_memory_requests == "12G" - assert config.head_memory_limits == "16G" - assert config.head_extended_resource_requests == {"nvidia.com/gpu": 1} - assert config.machine_types == {"gpu.small", "gpu.large"} - assert config.worker_cpu_requests == 4 - assert config.worker_cpu_limits == 8 - assert config.num_workers == 10 - assert config.worker_memory_requests == "12G" - assert config.worker_memory_limits == "16G" + expected_extended_resource_mapping = DEFAULT_RESOURCE_MAPPING + expected_extended_resource_mapping.update({"example.com/gpu": "GPU"}) + expected_extended_resource_mapping["intel.com/gpu"] = "TPU" + + cluster = create_cluster_all_config_params(mocker, "test-all-params", False) + assert cluster.config.name == "test-all-params" and cluster.config.namespace == "ns" + assert cluster.config.head_cpu_requests == 4 + assert cluster.config.head_cpu_limits == 8 + assert cluster.config.head_memory_requests == "12G" + assert cluster.config.head_memory_limits == "16G" + assert cluster.config.head_extended_resource_requests == { + "nvidia.com/gpu": 1, + "intel.com/gpu": 2, + } + assert cluster.config.worker_cpu_requests == 4 + assert cluster.config.worker_cpu_limits == 8 + assert cluster.config.num_workers == 10 + assert cluster.config.worker_memory_requests == "12G" + assert cluster.config.worker_memory_limits == "16G" + assert cluster.config.appwrapper == False + assert cluster.config.envs == {"key1": "value1", "key2": "value2"} + assert cluster.config.image == "example/ray:tag" + assert cluster.config.image_pull_secrets == ["secret1", "secret2"] + assert cluster.config.write_to_file == True + assert cluster.config.verify_tls == True + assert cluster.config.labels == {"key1": "value1", "key2": "value2"} + assert cluster.config.worker_extended_resource_requests == {"nvidia.com/gpu": 1} assert ( - config.template - == f"{parent}/src/codeflare_sdk/ray/templates/base-template.yaml" + cluster.config.extended_resource_mapping == expected_extended_resource_mapping ) - assert config.appwrapper == False - assert config.envs == {"key1": "value1", "key2": "value2"} - assert config.image == "example/ray:tag" - assert config.image_pull_secrets == ["secret1", "secret2"] - assert config.write_to_file == True - assert config.verify_tls == True - assert config.labels == {"key1": "value1", "key2": "value2"} - assert config.worker_extended_resource_requests == {"nvidia.com/gpu": 1} - assert config.extended_resource_mapping == extended_resource_mapping - assert config.overwrite_default_resource_mapping == True - assert config.local_queue == "local-queue-default" + assert cluster.config.overwrite_default_resource_mapping == True + assert cluster.config.local_queue == "local-queue-default" assert filecmp.cmp( f"{aw_dir}test-all-params.yaml", @@ -136,6 +98,15 @@ def test_config_creation_all_parameters(mocker): ) +def test_all_config_params_aw(mocker): + create_cluster_all_config_params(mocker, "aw-all-params", True) + assert filecmp.cmp( + f"{aw_dir}aw-all-params.yaml", + f"{expected_clusters_dir}/appwrapper/unit-test-all-params.yaml", + shallow=True, + ) + + def test_config_creation_wrong_type(): with pytest.raises(TypeError): createClusterWrongType() @@ -168,3 +139,4 @@ def test_cluster_config_deprecation_conversion(mocker): # Make sure to always keep this function last def test_cleanup(): os.remove(f"{aw_dir}test-all-params.yaml") + os.remove(f"{aw_dir}aw-all-params.yaml") diff --git a/src/codeflare_sdk/ray/cluster/test_pretty_print.py b/src/codeflare_sdk/ray/cluster/test_pretty_print.py index b0da42011..329a1354d 100644 --- a/src/codeflare_sdk/ray/cluster/test_pretty_print.py +++ b/src/codeflare_sdk/ray/cluster/test_pretty_print.py @@ -29,6 +29,7 @@ ClusterConfiguration, _copy_to_ray, ) +from codeflare_sdk.common.utils.unit_test_support import get_local_queue def test_print_no_resources(capsys): @@ -100,15 +101,15 @@ def test_ray_details(mocker, capsys): return_value="", ) mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", + "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", + return_value=get_local_queue("kueue.x-k8s.io", "v1beta1", "ns", "localqueues"), ) cf = Cluster( ClusterConfiguration( name="raytest2", namespace="ns", appwrapper=True, - local_queue="local_default_queue", + local_queue="local-queue-default", ) ) captured = capsys.readouterr() diff --git a/src/codeflare_sdk/ray/cluster/test_status.py b/src/codeflare_sdk/ray/cluster/test_status.py index 146d21901..27eda49ec 100644 --- a/src/codeflare_sdk/ray/cluster/test_status.py +++ b/src/codeflare_sdk/ray/cluster/test_status.py @@ -23,6 +23,7 @@ RayCluster, ) import os +from ...common.utils.unit_test_support import get_local_queue aw_dir = os.path.expanduser("~/.codeflare/resources/") @@ -30,10 +31,6 @@ def test_cluster_status(mocker): mocker.patch("kubernetes.client.ApisApi.get_api_versions") mocker.patch("kubernetes.config.load_kube_config", return_value="ignore") - mocker.patch( - "codeflare_sdk.common.kueue.kueue.local_queue_exists", - return_value="true", - ) fake_ray = RayCluster( name="test", @@ -50,13 +47,19 @@ def test_cluster_status(mocker): head_mem_requests=8, head_mem_limits=8, ) + + mocker.patch( + "kubernetes.client.CustomObjectsApi.list_namespaced_custom_object", + return_value=get_local_queue("kueue.x-k8s.io", "v1beta1", "ns", "localqueues"), + ) + cf = Cluster( ClusterConfiguration( name="test", namespace="ns", write_to_file=True, appwrapper=False, - local_queue="local_default_queue", + local_queue="local-queue-default", ) ) mocker.patch( diff --git a/tests/test_cluster_yamls/appwrapper/test-case-bad.yaml b/tests/test_cluster_yamls/appwrapper/test-case-bad.yaml index a072e9e92..9166eced5 100644 --- a/tests/test_cluster_yamls/appwrapper/test-case-bad.yaml +++ b/tests/test_cluster_yamls/appwrapper/test-case-bad.yaml @@ -18,7 +18,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -44,7 +43,7 @@ spec: fieldRef: fieldPath: status.podIP image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 - imagePullPolicy: Always + imagePullPolicy: IfNotPresent lifecycle: preStop: exec: diff --git a/tests/test_cluster_yamls/appwrapper/unit-test-all-params.yaml b/tests/test_cluster_yamls/appwrapper/unit-test-all-params.yaml new file mode 100644 index 000000000..6d2c5440a --- /dev/null +++ b/tests/test_cluster_yamls/appwrapper/unit-test-all-params.yaml @@ -0,0 +1,176 @@ +apiVersion: workload.codeflare.dev/v1beta2 +kind: AppWrapper +metadata: + labels: + kueue.x-k8s.io/queue-name: local-queue-default + name: aw-all-params + namespace: ns +spec: + components: + - template: + apiVersion: ray.io/v1 + kind: RayCluster + metadata: + annotations: + app.kubernetes.io/managed-by: test-prefix + labels: + controller-tools.k8s.io: '1.0' + key1: value1 + key2: value2 + name: aw-all-params + namespace: ns + spec: + autoscalerOptions: + idleTimeoutSeconds: 60 + resources: + limits: + cpu: 500m + memory: 512Mi + requests: + cpu: 500m + memory: 512Mi + upscalingMode: Default + enableInTreeAutoscaling: false + headGroupSpec: + enableIngress: false + rayStartParams: + block: 'true' + dashboard-host: 0.0.0.0 + num-gpus: '1' + resources: '"{\"TPU\": 2}"' + serviceType: ClusterIP + template: + spec: + containers: + - env: + - name: key1 + value: value1 + - name: key2 + value: value2 + image: example/ray:tag + imagePullPolicy: Always + lifecycle: + preStop: + exec: + command: + - /bin/sh + - -c + - ray stop + name: ray-head + ports: + - containerPort: 6379 + name: gcs + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + resources: + limits: + cpu: 8 + intel.com/gpu: 2 + memory: 16G + nvidia.com/gpu: 1 + requests: + cpu: 4 + intel.com/gpu: 2 + memory: 12G + nvidia.com/gpu: 1 + volumeMounts: + - mountPath: /etc/pki/tls/certs/odh-trusted-ca-bundle.crt + name: odh-trusted-ca-cert + subPath: odh-trusted-ca-bundle.crt + - mountPath: /etc/ssl/certs/odh-trusted-ca-bundle.crt + name: odh-trusted-ca-cert + subPath: odh-trusted-ca-bundle.crt + - mountPath: /etc/pki/tls/certs/odh-ca-bundle.crt + name: odh-ca-cert + subPath: odh-ca-bundle.crt + - mountPath: /etc/ssl/certs/odh-ca-bundle.crt + name: odh-ca-cert + subPath: odh-ca-bundle.crt + imagePullSecrets: + - name: secret1 + - name: secret2 + volumes: + - configMap: + items: + - key: ca-bundle.crt + path: odh-trusted-ca-bundle.crt + name: odh-trusted-ca-bundle + optional: true + name: odh-trusted-ca-cert + - configMap: + items: + - key: odh-ca-bundle.crt + path: odh-ca-bundle.crt + name: odh-trusted-ca-bundle + optional: true + name: odh-ca-cert + rayVersion: 2.35.0 + workerGroupSpecs: + - groupName: small-group-aw-all-params + maxReplicas: 10 + minReplicas: 10 + rayStartParams: + block: 'true' + num-gpus: '1' + resources: '"{}"' + replicas: 10 + template: + spec: + containers: + - env: + - name: key1 + value: value1 + - name: key2 + value: value2 + image: example/ray:tag + imagePullPolicy: Always + lifecycle: + preStop: + exec: + command: + - /bin/sh + - -c + - ray stop + name: machine-learning + resources: + limits: + cpu: 8 + memory: 16G + nvidia.com/gpu: 1 + requests: + cpu: 4 + memory: 12G + nvidia.com/gpu: 1 + volumeMounts: + - mountPath: /etc/pki/tls/certs/odh-trusted-ca-bundle.crt + name: odh-trusted-ca-cert + subPath: odh-trusted-ca-bundle.crt + - mountPath: /etc/ssl/certs/odh-trusted-ca-bundle.crt + name: odh-trusted-ca-cert + subPath: odh-trusted-ca-bundle.crt + - mountPath: /etc/pki/tls/certs/odh-ca-bundle.crt + name: odh-ca-cert + subPath: odh-ca-bundle.crt + - mountPath: /etc/ssl/certs/odh-ca-bundle.crt + name: odh-ca-cert + subPath: odh-ca-bundle.crt + imagePullSecrets: + - name: secret1 + - name: secret2 + volumes: + - configMap: + items: + - key: ca-bundle.crt + path: odh-trusted-ca-bundle.crt + name: odh-trusted-ca-bundle + optional: true + name: odh-trusted-ca-cert + - configMap: + items: + - key: odh-ca-bundle.crt + path: odh-ca-bundle.crt + name: odh-trusted-ca-bundle + optional: true + name: odh-ca-cert diff --git a/tests/test_cluster_yamls/kueue/aw_kueue.yaml b/tests/test_cluster_yamls/kueue/aw_kueue.yaml index 2c6d868ac..402ffb6a6 100644 --- a/tests/test_cluster_yamls/kueue/aw_kueue.yaml +++ b/tests/test_cluster_yamls/kueue/aw_kueue.yaml @@ -18,7 +18,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -76,7 +75,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: @@ -103,14 +101,10 @@ spec: resources: '"{}"' replicas: 2 template: - metadata: - annotations: - key: value - labels: - key: value spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 + imagePullPolicy: Always lifecycle: preStop: exec: @@ -139,7 +133,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: diff --git a/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml b/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml index 0c4efb29a..a5cb36164 100644 --- a/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml +++ b/tests/test_cluster_yamls/kueue/ray_cluster_kueue.yaml @@ -18,7 +18,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -76,7 +75,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: @@ -103,14 +101,10 @@ spec: resources: '"{}"' replicas: 2 template: - metadata: - annotations: - key: value - labels: - key: value spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 + imagePullPolicy: Always lifecycle: preStop: exec: @@ -139,7 +133,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: diff --git a/tests/test_cluster_yamls/ray/default-appwrapper.yaml b/tests/test_cluster_yamls/ray/default-appwrapper.yaml index 60152c1e7..3e97474d4 100644 --- a/tests/test_cluster_yamls/ray/default-appwrapper.yaml +++ b/tests/test_cluster_yamls/ray/default-appwrapper.yaml @@ -16,7 +16,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -74,7 +73,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: @@ -101,14 +99,10 @@ spec: resources: '"{}"' replicas: 1 template: - metadata: - annotations: - key: value - labels: - key: value spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 + imagePullPolicy: Always lifecycle: preStop: exec: @@ -137,7 +131,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: diff --git a/tests/test_cluster_yamls/ray/default-ray-cluster.yaml b/tests/test_cluster_yamls/ray/default-ray-cluster.yaml index 7a3329b6d..34de53d2b 100644 --- a/tests/test_cluster_yamls/ray/default-ray-cluster.yaml +++ b/tests/test_cluster_yamls/ray/default-ray-cluster.yaml @@ -8,7 +8,6 @@ metadata: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -66,7 +65,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: @@ -93,14 +91,10 @@ spec: resources: '"{}"' replicas: 1 template: - metadata: - annotations: - key: value - labels: - key: value spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 + imagePullPolicy: Always lifecycle: preStop: exec: @@ -129,7 +123,6 @@ spec: - mountPath: /etc/ssl/certs/odh-ca-bundle.crt name: odh-ca-cert subPath: odh-ca-bundle.crt - imagePullSecrets: [] volumes: - configMap: items: diff --git a/tests/test_cluster_yamls/ray/unit-test-all-params.yaml b/tests/test_cluster_yamls/ray/unit-test-all-params.yaml index eda7270f6..8426eede6 100644 --- a/tests/test_cluster_yamls/ray/unit-test-all-params.yaml +++ b/tests/test_cluster_yamls/ray/unit-test-all-params.yaml @@ -13,7 +13,6 @@ metadata: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -29,14 +28,16 @@ spec: block: 'true' dashboard-host: 0.0.0.0 num-gpus: '1' - resources: '"{}"' + resources: '"{\"TPU\": 2}"' serviceType: ClusterIP template: spec: containers: - - env: &id001 - key1: value1 - key2: value2 + - env: + - name: key1 + value: value1 + - name: key2 + value: value2 image: example/ray:tag imagePullPolicy: Always lifecycle: @@ -57,10 +58,12 @@ spec: resources: limits: cpu: 8 + intel.com/gpu: 2 memory: 16G nvidia.com/gpu: 1 requests: cpu: 4 + intel.com/gpu: 2 memory: 12G nvidia.com/gpu: 1 volumeMounts: @@ -105,15 +108,15 @@ spec: resources: '"{}"' replicas: 10 template: - metadata: - annotations: - key: value - labels: - key: value spec: containers: - - env: *id001 + - env: + - name: key1 + value: value1 + - name: key2 + value: value2 image: example/ray:tag + imagePullPolicy: Always lifecycle: preStop: exec: diff --git a/tests/test_cluster_yamls/support_clusters/test-aw-a.yaml b/tests/test_cluster_yamls/support_clusters/test-aw-a.yaml index 9b8a647f6..fe26900d7 100644 --- a/tests/test_cluster_yamls/support_clusters/test-aw-a.yaml +++ b/tests/test_cluster_yamls/support_clusters/test-aw-a.yaml @@ -18,7 +18,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -40,7 +39,7 @@ spec: spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 - imagePullPolicy: Always + imagePullPolicy: IfNotPresent lifecycle: preStop: exec: diff --git a/tests/test_cluster_yamls/support_clusters/test-aw-b.yaml b/tests/test_cluster_yamls/support_clusters/test-aw-b.yaml index 763eb5c2a..eed571fe7 100644 --- a/tests/test_cluster_yamls/support_clusters/test-aw-b.yaml +++ b/tests/test_cluster_yamls/support_clusters/test-aw-b.yaml @@ -18,7 +18,6 @@ spec: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -40,7 +39,7 @@ spec: spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 - imagePullPolicy: Always + imagePullPolicy: IfNotPresent lifecycle: preStop: exec: diff --git a/tests/test_cluster_yamls/support_clusters/test-rc-a.yaml b/tests/test_cluster_yamls/support_clusters/test-rc-a.yaml index f12ffde00..5f5d456cb 100644 --- a/tests/test_cluster_yamls/support_clusters/test-rc-a.yaml +++ b/tests/test_cluster_yamls/support_clusters/test-rc-a.yaml @@ -9,7 +9,6 @@ metadata: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -31,7 +30,7 @@ spec: spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 - imagePullPolicy: Always + imagePullPolicy: IfNotPresent lifecycle: preStop: exec: diff --git a/tests/test_cluster_yamls/support_clusters/test-rc-b.yaml b/tests/test_cluster_yamls/support_clusters/test-rc-b.yaml index 1d41e365f..3bf894dbb 100644 --- a/tests/test_cluster_yamls/support_clusters/test-rc-b.yaml +++ b/tests/test_cluster_yamls/support_clusters/test-rc-b.yaml @@ -9,7 +9,6 @@ metadata: spec: autoscalerOptions: idleTimeoutSeconds: 60 - imagePullPolicy: Always resources: limits: cpu: 500m @@ -31,7 +30,7 @@ spec: spec: containers: - image: quay.io/modh/ray@sha256:0d715f92570a2997381b7cafc0e224cfa25323f18b9545acfd23bc2b71576d06 - imagePullPolicy: Always + imagePullPolicy: IfNotPresent lifecycle: preStop: exec: diff --git a/ui-tests/playwright.config.js b/ui-tests/playwright.config.js index 88003f8ea..1ba51f152 100644 --- a/ui-tests/playwright.config.js +++ b/ui-tests/playwright.config.js @@ -2,7 +2,7 @@ const baseConfig = require('@jupyterlab/galata/lib/playwright-config'); module.exports = { ...baseConfig, - timeout: 460000, + timeout: 600000, webServer: { command: 'yarn start', url: 'http://localhost:8888/lab',