From 2b2900d105123d7814947fe4233eaf20260c5976 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Fri, 8 Jul 2022 16:50:35 -0700 Subject: [PATCH 01/11] wip --- .../tests/kuberay/setup/raycluster_test.yaml | 15 +++ .../ray/tests/kuberay/setup/setup_kuberay.py | 10 ++ .../tests/kuberay/setup/teardown_kuberay.py | 8 ++ .../ray/tests/kuberay/test_autoscaling_e2e.py | 64 +---------- python/ray/tests/kuberay/utils.py | 102 ++++++++++++++++-- 5 files changed, 133 insertions(+), 66 deletions(-) create mode 100644 python/ray/tests/kuberay/setup/raycluster_test.yaml create mode 100644 python/ray/tests/kuberay/setup/setup_kuberay.py create mode 100644 python/ray/tests/kuberay/setup/teardown_kuberay.py diff --git a/python/ray/tests/kuberay/setup/raycluster_test.yaml b/python/ray/tests/kuberay/setup/raycluster_test.yaml new file mode 100644 index 0000000000000..b96f59e29498b --- /dev/null +++ b/python/ray/tests/kuberay/setup/raycluster_test.yaml @@ -0,0 +1,15 @@ +--- +apiVersion: ray.io/v1alpha1 +kind: RayCluster +metadata: + name: raycluster-test +spec: + headGroupSpec: + serviceType: ClusterIP + replicas: 1 + rayStartParams: {} + template: + metadata: + spec: + containers: + - name: ray-test diff --git a/python/ray/tests/kuberay/setup/setup_kuberay.py b/python/ray/tests/kuberay/setup/setup_kuberay.py new file mode 100644 index 0000000000000..7c580ea122416 --- /dev/null +++ b/python/ray/tests/kuberay/setup/setup_kuberay.py @@ -0,0 +1,10 @@ +from ray.tests.kuberay.utils import ( + setup_kuberay_operator, + wait_for_raycluster_crd, + setup_logging +) + +if __name__ == "__main__": + setup_logging() + setup_kuberay_operator() + wait_for_raycluster_crd() diff --git a/python/ray/tests/kuberay/setup/teardown_kuberay.py b/python/ray/tests/kuberay/setup/teardown_kuberay.py new file mode 100644 index 0000000000000..f75e4c29f4644 --- /dev/null +++ b/python/ray/tests/kuberay/setup/teardown_kuberay.py @@ -0,0 +1,8 @@ +from ray.tests.kuberay.utils import ( + teardown_kuberay_operator, + setup_logging +) + +if __name__ == "__main__": + setup_logging() + teardown_kuberay_operator() diff --git a/python/ray/tests/kuberay/test_autoscaling_e2e.py b/python/ray/tests/kuberay/test_autoscaling_e2e.py index e0b19f64f2527..ab6c3f634af63 100644 --- a/python/ray/tests/kuberay/test_autoscaling_e2e.py +++ b/python/ray/tests/kuberay/test_autoscaling_e2e.py @@ -1,7 +1,6 @@ import copy import logging import os -import pathlib import tempfile import unittest import subprocess @@ -16,6 +15,7 @@ get_raycluster, ray_client_port_forward, ray_job_submit, + switch_to_ray_parent_dir, kubectl_exec_python_script, kubectl_logs, kubectl_patch, @@ -23,7 +23,6 @@ wait_for_pods, wait_for_pod_to_start, wait_for_ray_health, - wait_for_crd, ) from ray.tests.kuberay.scripts import ( @@ -33,14 +32,10 @@ ) logger = logging.getLogger(__name__) -logging.basicConfig( - level=logging.INFO, - format="[%(levelname)s %(asctime)s] " "%(filename)s: %(lineno)d " "%(message)s", -) # This image will be used for both the Ray nodes and the autoscaler. # The CI should pass an image built from the test branch. -RAY_IMAGE = os.environ.get("RAY_IMAGE", "rayproject/ray:448f52") +RAY_IMAGE = os.environ.get("RAY_IMAGE", "rayproject/ray:nightly") # By default, use the same image for the autoscaler and Ray containers. AUTOSCALER_IMAGE = os.environ.get("AUTOSCALER_IMAGE", RAY_IMAGE) # Set to IfNotPresent in kind CI. @@ -48,11 +43,7 @@ logger.info(f"Using image `{RAY_IMAGE}` for Ray containers.") logger.info(f"Using image `{AUTOSCALER_IMAGE}` for Autoscaler containers.") logger.info(f"Using pull policy `{PULL_POLICY}` for all images.") -# The default "rayproject/ray:413fe0" is the currently pinned autoscaler image -# (to be replaced with rayproject/ray:1.12.0 upon 1.12.0 release). -# Parent directory of Ray repository -RAY_PARENT = str(pathlib.Path(__file__).resolve().parents[5]) # Path to example config rel RAY_PARENT EXAMPLE_CLUSTER_PATH = "ray/python/ray/autoscaler/kuberay/ray-cluster.complete.yaml" @@ -68,38 +59,6 @@ class KubeRayAutoscalingTest(unittest.TestCase): kubectl is used throughout, as that reflects the instructions in the docs. """ - def setUp(self): - """Set up KubeRay operator and Ray autoscaler RBAC.""" - - # Switch to parent of Ray repo, because that's what the doc examples do. - logger.info("Switching to parent of Ray directory.") - os.chdir(RAY_PARENT) - - logger.info("Cloning KubeRay and setting up KubeRay configuration.") - # For faster run-time when triggering the test locally, don't run the init - # script if it has already been run. - subprocess.check_call( - [ - "bash", - "-c", - ( - "ls ray/python/ray/autoscaler/kuberay/config ||" - " ./ray/python/ray/autoscaler/kuberay/init-config.sh" - ), - ] - ) - logger.info("Creating KubeRay operator.") - subprocess.check_call( - [ - "kubectl", - "create", - "-k", - "ray/python/ray/autoscaler/kuberay/config/default", - ] - ) - logger.info("Making sure RayCluster CRD has been registered.") - wait_for_crd("rayclusters.ray.io") - def _get_ray_cr_config( self, min_replicas=0, cpu_replicas=0, gpu_replicas=0 ) -> Dict[str, Any]: @@ -242,6 +201,8 @@ def testAutoscaling(self): The `num-cpus` arg to Ray start is 1 for each Ray container; thus Ray accounts 1 CPU for each Ray node in the test. """ + switch_to_ray_parent_dir() + # Cluster creation logger.info("Creating a RayCluster with no worker pods.") self._apply_ray_cr(min_replicas=0, cpu_replicas=0, gpu_replicas=0) @@ -424,23 +385,6 @@ def testAutoscaling(self): logger.info("Confirming Ray pods are gone.") wait_for_pods(goal_num_pods=0, namespace=RAY_CLUSTER_NAMESPACE) - def tearDown(self): - """Clean resources following the instructions in the docs.""" - - logger.info("Deleting operator.") - subprocess.check_call( - [ - "kubectl", - "delete", - "-k", - "ray/python/ray/autoscaler/kuberay/config/default", - ] - ) - - logger.info("Double-checking no pods left over.") - wait_for_pods(goal_num_pods=0, namespace=RAY_CLUSTER_NAMESPACE) - wait_for_pods(goal_num_pods=0, namespace="ray-system") - if __name__ == "__main__": import pytest diff --git a/python/ray/tests/kuberay/utils.py b/python/ray/tests/kuberay/utils.py index 95820ad0136b4..3840d3ed88fa3 100644 --- a/python/ray/tests/kuberay/utils.py +++ b/python/ray/tests/kuberay/utils.py @@ -10,6 +10,7 @@ import time from typing import Any, Dict, Generator, List, Optional import yaml +import os import ray from ray.job_submission import JobStatus, JobSubmissionClient @@ -18,22 +19,111 @@ logger = logging.getLogger(__name__) SCRIPTS_DIR = pathlib.Path(__file__).resolve().parent / "scripts" +TEST_CR_PATH = pathlib.Path(__file__).resolve().parent / "setup" / "raycluster_test.yaml" +TEST_CLUSTER_NAME = "raycluster-test" +# Parent directory of Ray repository +RAY_PARENT = str(pathlib.Path(__file__).resolve().parents[5]) -def wait_for_crd(crd_name: str, tries=60, backoff_s=5): +RAYCLUSTERS_QUALIFIED = "rayclusters.ray.io" + +LOG_FORMAT = "[%(levelname)s %(asctime)s] " "%(filename)s: %(lineno)d " "%(message)s" + + +def setup_logging(): + logging.basicConfig( + level=logging.INFO, + format=LOG_FORMAT, + ) + +def switch_to_ray_parent_dir(): + # Switch to parent of Ray repo, because that's what the doc examples do. + logger.info("Switching to parent of Ray directory.") + os.chdir(RAY_PARENT) + + +def setup_kuberay_operator(): + """Set up KubeRay operator and Ray autoscaler RBAC.""" + switch_to_ray_parent_dir() + logger.info("Cloning KubeRay and setting up KubeRay configuration.") + # For faster run-time when triggering the test locally, don't run the init + # script if it has already been run. + subprocess.check_call( + [ + "bash", + "-c", + ( + "ls ray/python/ray/autoscaler/kuberay/config ||" + " ./ray/python/ray/autoscaler/kuberay/init-config.sh" + ), + ] + ) + logger.info("Creating KubeRay operator.") + subprocess.check_call( + [ + "kubectl", + "create", + "-k", + "ray/python/ray/autoscaler/kuberay/config/default", + ] + ) + + +def teardown_kuberay_operator(): + logger.info("Switching to parent of Ray directory.") + os.chdir(RAY_PARENT) + + logger.info("Deleting operator.") + subprocess.check_call( + [ + "kubectl", + "delete", + "-k", + "ray/python/ray/autoscaler/kuberay/config/default", + ] + ) + + logger.info("Double-checking no pods left over in namespace ray-system.") + wait_for_pods(goal_num_pods=0, namespace="ray-system") + + +def wait_for_raycluster_crd(tries=60, backoff_s=5): """CRD creation can take a bit of time after the client request. This function waits until the crd with the provided name is registered. """ + switch_to_ray_parent_dir() + logger.info("Making sure RayCluster CRD has been registered.") for i in range(tries): get_crd_output = subprocess.check_output(["kubectl", "get", "crd"]).decode() - if crd_name in get_crd_output: - logger.info(f"Confirmed existence of CRD {crd_name}.") - return + if RAYCLUSTERS_QUALIFIED in get_crd_output: + logger.info("Confirmed existence of RayCluster CRD.") + break elif i < tries - 1: - logger.info(f"Still waiting to register CRD {crd_name}") + logger.info("Still waiting to register RayCluster CRD.") time.sleep(backoff_s) else: - raise Exception(f"Failed to register CRD {crd_name}") + raise Exception("Failed to register RayCluster CRD.") + + # Create a test RayCluster CR to make sure that the CRD is fully registered. + for i in range(tries): + try: + subprocess.check_call(["kubectl", "apply", "-f", TEST_CR_PATH]) + break + except subprocess.CalledProcessError as e: + logger.info("Can't create RayCluster CR.") + if i < tries - 1: + logger.info("Retrying.") + time.sleep(backoff_s) + else: + logger.info("Giving up.") + raise e from None + + # Confirm the test RayCluster exists. + out = subprocess.check_output(["kubectl", "get", "rayclusters.ray.io"]).decode() + assert TEST_CLUSTER_NAME in out, out + + # Delete the test RayCluster + subprocess.check_call(["kubectl", "delete", "-f", TEST_CR_PATH]) def wait_for_pods(goal_num_pods: int, namespace: str, tries=60, backoff_s=5) -> None: From a4a7bf3d5f153d3ace1ee70d4d5773aede69d376 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Fri, 8 Jul 2022 17:35:15 -0700 Subject: [PATCH 02/11] probably-done --- .buildkite/pipeline.yml | 4 ++++ python/ray/tests/kuberay/setup/setup_kuberay.py | 2 +- python/ray/tests/kuberay/setup/teardown_kuberay.py | 5 +---- python/ray/tests/kuberay/test_autoscaling_e2e.py | 4 +++- python/ray/tests/kuberay/utils.py | 5 ++++- 5 files changed, 13 insertions(+), 7 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 0113dc0c1a713..c6fee669a1540 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -442,6 +442,8 @@ - docker tag rayproject/ray:nightly-py37-cpu ray-ci:kuberay-test # Load the image into the kind node. - kind load docker-image ray-ci:kuberay-test + - echo "--- Setting up KubeRay operator." + - python python/ray/tests/kuberay/setup/setup_kuberay.py - echo "--- Running the test." - bazel test --config=ci $(./ci/run/bazel_export_options) --test_tag_filters=kuberay_operator @@ -449,6 +451,8 @@ --test_env=PULL_POLICY=IfNotPresent --test_env=KUBECONFIG=/root/.kube/config python/ray/tests/... + - echo "--- Tearing down KubeRay operator." + - python python/ray/tests/kuberay/setup/teardown_kuberay.py - label: ":python: Ray DAG Tests" conditions: [ diff --git a/python/ray/tests/kuberay/setup/setup_kuberay.py b/python/ray/tests/kuberay/setup/setup_kuberay.py index 7c580ea122416..1d72e99713c91 100644 --- a/python/ray/tests/kuberay/setup/setup_kuberay.py +++ b/python/ray/tests/kuberay/setup/setup_kuberay.py @@ -1,7 +1,7 @@ from ray.tests.kuberay.utils import ( setup_kuberay_operator, wait_for_raycluster_crd, - setup_logging + setup_logging, ) if __name__ == "__main__": diff --git a/python/ray/tests/kuberay/setup/teardown_kuberay.py b/python/ray/tests/kuberay/setup/teardown_kuberay.py index f75e4c29f4644..e7e82c23c7a18 100644 --- a/python/ray/tests/kuberay/setup/teardown_kuberay.py +++ b/python/ray/tests/kuberay/setup/teardown_kuberay.py @@ -1,7 +1,4 @@ -from ray.tests.kuberay.utils import ( - teardown_kuberay_operator, - setup_logging -) +from ray.tests.kuberay.utils import teardown_kuberay_operator, setup_logging if __name__ == "__main__": setup_logging() diff --git a/python/ray/tests/kuberay/test_autoscaling_e2e.py b/python/ray/tests/kuberay/test_autoscaling_e2e.py index ab6c3f634af63..5dee695c1a414 100644 --- a/python/ray/tests/kuberay/test_autoscaling_e2e.py +++ b/python/ray/tests/kuberay/test_autoscaling_e2e.py @@ -15,6 +15,7 @@ get_raycluster, ray_client_port_forward, ray_job_submit, + setup_logging, switch_to_ray_parent_dir, kubectl_exec_python_script, kubectl_logs, @@ -35,7 +36,7 @@ # This image will be used for both the Ray nodes and the autoscaler. # The CI should pass an image built from the test branch. -RAY_IMAGE = os.environ.get("RAY_IMAGE", "rayproject/ray:nightly") +RAY_IMAGE = os.environ.get("RAY_IMAGE", "rayproject/ray:nightly-py38") # By default, use the same image for the autoscaler and Ray containers. AUTOSCALER_IMAGE = os.environ.get("AUTOSCALER_IMAGE", RAY_IMAGE) # Set to IfNotPresent in kind CI. @@ -390,4 +391,5 @@ def testAutoscaling(self): import pytest import sys + setup_logging() sys.exit(pytest.main(["-vv", __file__])) diff --git a/python/ray/tests/kuberay/utils.py b/python/ray/tests/kuberay/utils.py index 3840d3ed88fa3..4ddef26691698 100644 --- a/python/ray/tests/kuberay/utils.py +++ b/python/ray/tests/kuberay/utils.py @@ -19,7 +19,9 @@ logger = logging.getLogger(__name__) SCRIPTS_DIR = pathlib.Path(__file__).resolve().parent / "scripts" -TEST_CR_PATH = pathlib.Path(__file__).resolve().parent / "setup" / "raycluster_test.yaml" +TEST_CR_PATH = ( + pathlib.Path(__file__).resolve().parent / "setup" / "raycluster_test.yaml" +) TEST_CLUSTER_NAME = "raycluster-test" # Parent directory of Ray repository @@ -36,6 +38,7 @@ def setup_logging(): format=LOG_FORMAT, ) + def switch_to_ray_parent_dir(): # Switch to parent of Ray repo, because that's what the doc examples do. logger.info("Switching to parent of Ray directory.") From 797c60d0a09181d11064322fba09f2843ebc8b65 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Fri, 8 Jul 2022 17:45:31 -0700 Subject: [PATCH 03/11] Cleaner setup. --- python/ray/tests/kuberay/utils.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/tests/kuberay/utils.py b/python/ray/tests/kuberay/utils.py index 4ddef26691698..911474873e91e 100644 --- a/python/ray/tests/kuberay/utils.py +++ b/python/ray/tests/kuberay/utils.py @@ -125,8 +125,10 @@ def wait_for_raycluster_crd(tries=60, backoff_s=5): out = subprocess.check_output(["kubectl", "get", "rayclusters.ray.io"]).decode() assert TEST_CLUSTER_NAME in out, out - # Delete the test RayCluster + # Delete the test RayCluster. subprocess.check_call(["kubectl", "delete", "-f", TEST_CR_PATH]) + # Make sure the associated resources are gone before proceeding. + wait_for_pods(goal_num_pods=0, namespace="default") def wait_for_pods(goal_num_pods: int, namespace: str, tries=60, backoff_s=5) -> None: From 3f6af5bbd45d4408bc69b191056aad1e18aec1db Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Sat, 9 Jul 2022 10:32:06 -0700 Subject: [PATCH 04/11] Use the constant. --- python/ray/tests/kuberay/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tests/kuberay/utils.py b/python/ray/tests/kuberay/utils.py index 911474873e91e..d8e9b2f4b9af7 100644 --- a/python/ray/tests/kuberay/utils.py +++ b/python/ray/tests/kuberay/utils.py @@ -122,7 +122,7 @@ def wait_for_raycluster_crd(tries=60, backoff_s=5): raise e from None # Confirm the test RayCluster exists. - out = subprocess.check_output(["kubectl", "get", "rayclusters.ray.io"]).decode() + out = subprocess.check_output(["kubectl", "get", RAYCLUSTERS_QUALIFIED]).decode() assert TEST_CLUSTER_NAME in out, out # Delete the test RayCluster. From e5a70bb2c36fc1f11bf0ab59fdc91440fd438cfa Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 11:29:15 -0700 Subject: [PATCH 05/11] Add redis dependency. --- docker/ray-deps/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/ray-deps/Dockerfile b/docker/ray-deps/Dockerfile index c25c4957837ff..13f78b37a8f1d 100644 --- a/docker/ray-deps/Dockerfile +++ b/docker/ray-deps/Dockerfile @@ -12,6 +12,7 @@ COPY $FIND_LINKS_PATH $FIND_LINKS_PATH RUN $HOME/anaconda3/bin/pip --no-cache-dir install --find-links $FIND_LINKS_PATH \ $(basename $WHEEL_PATH)[all] \ $(if [ "$AUTOSCALER" = "autoscaler" ]; then echo \ + "redis >= 5.5.0, < 4.0.0" \ "six==1.13.0" \ "boto3==1.4.8" \ "google-api-python-client==1.7.8" \ From 01d1f5bb91febe54182b0e9b911d80e8f0366a95 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 11:29:39 -0700 Subject: [PATCH 06/11] Revert "Add redis dependency." This reverts commit e5a70bb2c36fc1f11bf0ab59fdc91440fd438cfa. --- docker/ray-deps/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/ray-deps/Dockerfile b/docker/ray-deps/Dockerfile index 13f78b37a8f1d..c25c4957837ff 100644 --- a/docker/ray-deps/Dockerfile +++ b/docker/ray-deps/Dockerfile @@ -12,7 +12,6 @@ COPY $FIND_LINKS_PATH $FIND_LINKS_PATH RUN $HOME/anaconda3/bin/pip --no-cache-dir install --find-links $FIND_LINKS_PATH \ $(basename $WHEEL_PATH)[all] \ $(if [ "$AUTOSCALER" = "autoscaler" ]; then echo \ - "redis >= 5.5.0, < 4.0.0" \ "six==1.13.0" \ "boto3==1.4.8" \ "google-api-python-client==1.7.8" \ From ed63dc646cc16bddf2f7b44d78b7ed362977f24e Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 12:22:56 -0700 Subject: [PATCH 07/11] Add --ignore-not-found to make idempotent. --- python/ray/tests/kuberay/utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tests/kuberay/utils.py b/python/ray/tests/kuberay/utils.py index d8e9b2f4b9af7..5ccad6bb56197 100644 --- a/python/ray/tests/kuberay/utils.py +++ b/python/ray/tests/kuberay/utils.py @@ -81,6 +81,7 @@ def teardown_kuberay_operator(): [ "kubectl", "delete", + "--ignore-not-found", "-k", "ray/python/ray/autoscaler/kuberay/config/default", ] From 58936d50e7b15c5c8e98234a1dc7f13b758b5c3c Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 15:14:26 -0700 Subject: [PATCH 08/11] Move teardown into cleanup. --- .buildkite/pipeline.yml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index c6fee669a1540..38acd1976618a 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -424,7 +424,12 @@ - label: ":kubernetes: operator" conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] commands: - - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - | + cleanup() { + if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi + python python/ray/tests/kuberay/setup/teardown_kuberay.py + } + trap cleanup EXIT - echo "--- Setting up Python 3.7 environment." - PYTHON=3.7 ./ci/env/install-dependencies.sh # Specifying PYTHON=3.7 above somehow messes up the Ray install. @@ -451,8 +456,6 @@ --test_env=PULL_POLICY=IfNotPresent --test_env=KUBECONFIG=/root/.kube/config python/ray/tests/... - - echo "--- Tearing down KubeRay operator." - - python python/ray/tests/kuberay/setup/teardown_kuberay.py - label: ":python: Ray DAG Tests" conditions: [ From 891eb295063a5b6975f8aca707d761207f583ea5 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 18:15:29 -0700 Subject: [PATCH 09/11] Delete kind cluster in cleanup. --- .buildkite/pipeline.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 38acd1976618a..9f933ace15411 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -428,6 +428,7 @@ cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi python python/ray/tests/kuberay/setup/teardown_kuberay.py + kind delete cluster } trap cleanup EXIT - echo "--- Setting up Python 3.7 environment." From 0b1ea1ddf83ff1cdf9399a0f9c098121824a515b Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 18:19:38 -0700 Subject: [PATCH 10/11] More cleanup logic. --- ci/k8s/prep-k8s-environment.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ci/k8s/prep-k8s-environment.sh b/ci/k8s/prep-k8s-environment.sh index d9b6f7b37f1ca..2f62d9d43bd84 100755 --- a/ci/k8s/prep-k8s-environment.sh +++ b/ci/k8s/prep-k8s-environment.sh @@ -16,6 +16,9 @@ chmod +x kubectl mv ./kubectl /usr/bin/kubectl kubectl version --client +# Delete dangling clusters +kind delete clusters --all + # Create the cluster time kind create cluster --wait 120s --config ./ci/k8s/kind.config.yaml docker ps From 11f0699540353174f81814520b85323455a81e40 Mon Sep 17 00:00:00 2001 From: Dmitri Gekhtman Date: Mon, 11 Jul 2022 22:31:49 -0700 Subject: [PATCH 11/11] Extend timeout by setting test size to large. --- python/ray/tests/BUILD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 1fcb1e453cfcf..54a77782f0f33 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -387,7 +387,7 @@ py_test( py_test( name = "kuberay/test_autoscaling_e2e", - size = "medium", + size = "large", srcs = SRCS + ["kuberay/test_autoscaling_e2e.py"], tags = ["exclusive", "kuberay_operator", "team:serverless"], deps = ["//:ray_lib"],