From c4871d85e938b3e586c6a423939c92c4d400c6a8 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 26 Sep 2022 22:56:53 +0800 Subject: [PATCH 001/121] init Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 224 +++++++++++++++++++++++++++++++++++ python/ray/spark/utils.py | 153 ++++++++++++++++++++++++ 2 files changed, 377 insertions(+) create mode 100644 python/ray/spark/__init__.py create mode 100644 python/ray/spark/utils.py diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py new file mode 100644 index 000000000000..7525216a88e4 --- /dev/null +++ b/python/ray/spark/__init__.py @@ -0,0 +1,224 @@ +import os +import sys +from urllib.parse import urlparse +import time +import threading +import logging + +from .utils import ( + exec_cmd, + check_port_open, + get_safe_port, + get_spark_session, + get_spark_driver_hostname, + is_in_databricks_runtime +) + + +def _create_ray_tmp_dir(prefix): + import tempfile + + return tempfile.mkdtemp(prefix=prefix) + + +def wait_ray_node_available(hostname, port, timeout, error_on_failure): + # Wait Ray head node spin up. + for _ in range(timeout): + time.sleep(1) + if check_port_open(hostname, port): + break + + if not check_port_open(hostname, port): + raise RuntimeError(error_on_failure) + + +def _get_spark_task_assigned_physical_gpus(task_context): + resources = task_context.resources() + if "gpu" not in resources: + raise RuntimeError( + "Couldn't get the gpu id, Please check the GPU resource configuration" + ) + gpu_addr_list = [int(addr.strip()) for addr in resources["gpu"].addresses] + + if is_in_databricks_runtime(): + visible_cuda_dev_list = [ + int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") + ] + return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] + else: + return gpu_addr_list + + +class RayClusterOnSpark: + + def __init__(self, address, head_proc, spark_job_group_id): + self.address = address + self.head_proc = head_proc + self.spark_job_group_id = spark_job_group_id + + def shutdown(self): + get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) + self.head_proc.kill() + + +def init_cluster(num_spark_tasks): + # TODO: + # support more ray head / worker arguments. + # support more ray.init arguments. + import ray + from pyspark.util import inheritable_thread_target + + spark = get_spark_session() + ray_head_hostname = get_spark_driver_hostname(spark) + ray_head_port = get_safe_port(ray_head_hostname) + + logging.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + + ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") + + ray_head_tmp_dir = _create_ray_tmp_dir(f"ray-head-port-{ray_head_port}-tmp-") + ray_head_node_cmd = [ + ray_exec_path, + "start", + f"--temp-dir={ray_head_tmp_dir}", + f"--num-cpus=0", # disallow ray tasks scheduled to ray head node. + "--block", + "--head", + f"--port={ray_head_port}", + "--include-dashboard=false", + ] + + logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") + ray_node_proc = exec_cmd( + ray_head_node_cmd, + synchronous=False, + capture_output=False, + stream_output=False, + ) + + # wait ray head node spin up. + wait_ray_node_available( + ray_head_hostname, ray_head_port, 40, + "Start Ray head node failed!" + ) + + logging.info("Ray head node started.") + + num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) + num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) + + def ray_cluster_job_mapper(_): + from pyspark.taskcontext import BarrierTaskContext + + context = BarrierTaskContext.get() + context.barrier() + task_id = context.partitionId() + + ray_worker_tmp_dir = _create_ray_tmp_dir( + f"ray-worker-{task_id}-head-{ray_head_hostname}:{ray_head_port}-tmp-" + ) + + ray_worker_cmd = [ + ray_exec_path, + "start", + f"--temp-dir={ray_worker_tmp_dir}", + f"--num-cpus={num_spark_task_cpus}", + "--block", + f"--address={ray_head_hostname}:{ray_head_port}", + ] + + ray_worker_extra_envs = {} + + if num_spark_task_gpus > 0: + available_physical_gpus = _get_spark_task_assigned_physical_gpus(context) + ray_worker_cmd.append( + f"--num-gpus={len(available_physical_gpus)}", + ) + ray_worker_extra_envs['CUDA_VISIBLE_DEVICES'] = ",".join([ + str(gpu_id) for gpu_id in num_spark_task_gpus + ]) + + if sys.platform.startswith("linux"): + + def setup_sigterm_on_parent_death(): + """ + Uses prctl to automatically send SIGTERM to the command process when its parent is + dead. + + This handles the case when the parent is a PySpark worker process. + If a user cancels the PySpark job, the worker process gets killed, regardless of + PySpark daemon and worker reuse settings. + We use prctl to ensure the command process receives SIGTERM after spark job + cancellation. + The command process itself should handle SIGTERM properly. + This is a no-op on macOS because prctl is not supported. + + Note: + When a pyspark job canceled, the UDF python process are killed by signal "SIGKILL", + This case neither "atexit" nor signal handler can capture SIGKILL signal. + prctl is the only way to capture SIGKILL signal. + """ + try: + import ctypes + import signal + + libc = ctypes.CDLL("libc.so.6") + # Set the parent process death signal of the command process to SIGTERM. + libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h + except OSError as e: + logging.warning(f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}.") + + else: + setup_sigterm_on_parent_death = None + + # TODO: Add a thread to redirect subprocess logs + # and collect tail logs and raise error if subprocess failed. + logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + + # TODO: Add memory control + # Q: When Ray head node killed, will ray worker node exit as well ? + exec_cmd( + ray_worker_cmd, + synchronous=True, + capture_output=False, + stream_output=False, + extra_env=ray_worker_extra_envs, + preexec_fn=setup_sigterm_on_parent_death, + ) + + # NB: Not reachable. + yield 0 + + spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + + # TODO: redirect background thread output. + def backgroud_job_thread_fn(): + spark.sparkContext.setJobGroup( + spark_job_group_id, + "This job group is for spark job which runs the Ray cluster with ray head node " + f"{ray_head_hostname}:{ray_head_port}" + ) + spark.sparkContext.parallelize( + list(range(num_spark_tasks)), num_spark_tasks + ).barrier().mapPartitions( + ray_cluster_job_mapper + ).collect()[0] + + threading.Thread( + target=inheritable_thread_target(backgroud_job_thread_fn), + args=() + ).start() + + # Waiting all ray workers spin up. + time.sleep(10) + + # discover the ray cluster. + ray.init(address=f"{ray_head_hostname}:{ray_head_port}") + + # TODO: + # Register databricks REPL detach event hook to shutdown the ray cluster. + return RayClusterOnSpark( + address=f"{ray_head_hostname}:{ray_head_port}", + head_proc=ray_node_proc, + spark_job_group_id=spark_job_group_id + ) diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py new file mode 100644 index 000000000000..26c67e7cdbab --- /dev/null +++ b/python/ray/spark/utils.py @@ -0,0 +1,153 @@ +import subprocess +from urllib.parse import urlparse +import os +import sys + + +def is_in_databricks_runtime(): + return "DATABRICKS_RUNTIME_VERSION" in os.environ + + +class ShellCommandException(Exception): + @classmethod + def from_completed_process(cls, process): + lines = [ + f"Non-zero exit code: {process.returncode}", + f"Command: {process.args}", + ] + if process.stdout: + lines += [ + "", + "STDOUT:", + process.stdout, + ] + if process.stderr: + lines += [ + "", + "STDERR:", + process.stderr, + ] + return cls("\n".join(lines)) + + +def exec_cmd( + cmd, + *, + throw_on_error=True, + extra_env=None, + capture_output=True, + synchronous=True, + stream_output=False, + **kwargs, +): + """ + A convenience wrapper of `subprocess.Popen` for running a command from a Python script. + + :param cmd: The command to run, as a list of strings. + :param throw_on_error: If True, raises an Exception if the exit code of the program is nonzero. + :param extra_env: Extra environment variables to be defined when running the child process. + If this argument is specified, `kwargs` cannot contain `env`. + :param capture_output: If True, stdout and stderr will be captured and included in an exception + message on failure; if False, these streams won't be captured. + :param synchronous: If True, wait for the command to complete and return a CompletedProcess + instance, If False, does not wait for the command to complete and return + a Popen instance, and ignore the `throw_on_error` argument. + :param stream_output: If True, stream the command's stdout and stderr to `sys.stdout` + as a unified stream during execution. + If False, do not stream the command's stdout and stderr to `sys.stdout`. + :param kwargs: Keyword arguments (except `text`) passed to `subprocess.Popen`. + :return: If synchronous is True, return a `subprocess.CompletedProcess` instance, + otherwise return a Popen instance. + """ + illegal_kwargs = set(kwargs.keys()).intersection({"text"}) + if illegal_kwargs: + raise ValueError(f"`kwargs` cannot contain {list(illegal_kwargs)}") + + env = kwargs.pop("env", None) + if extra_env is not None and env is not None: + raise ValueError("`extra_env` and `env` cannot be used at the same time") + + if capture_output and stream_output: + raise ValueError( + "`capture_output=True` and `stream_output=True` cannot be specified at the same time" + ) + + env = env if extra_env is None else {**os.environ, **extra_env} + + # In Python < 3.8, `subprocess.Popen` doesn't accept a command containing path-like + # objects (e.g. `["ls", pathlib.Path("abc")]`) on Windows. To avoid this issue, + # stringify all elements in `cmd`. Note `str(pathlib.Path("abc"))` returns 'abc'. + cmd = list(map(str, cmd)) + + if capture_output or stream_output: + if kwargs.get("stdout") is not None or kwargs.get("stderr") is not None: + raise ValueError( + "stdout and stderr arguments may not be used with capture_output or stream_output" + ) + kwargs["stdout"] = subprocess.PIPE + if capture_output: + kwargs["stderr"] = subprocess.PIPE + elif stream_output: + # Redirect stderr to stdout in order to combine the streams for unified printing to + # `sys.stdout`, as documented in + # https://docs.python.org/3/library/subprocess.html#subprocess.run + kwargs["stderr"] = subprocess.STDOUT + + process = subprocess.Popen( + cmd, + env=env, + text=True, + **kwargs, + ) + if not synchronous: + return process + + if stream_output: + for output_char in iter(lambda: process.stdout.read(1), ""): + sys.stdout.write(output_char) + + stdout, stderr = process.communicate() + returncode = process.poll() + comp_process = subprocess.CompletedProcess( + process.args, + returncode=returncode, + stdout=stdout, + stderr=stderr, + ) + if throw_on_error and returncode != 0: + raise ShellCommandException.from_completed_process(comp_process) + return comp_process + + +def get_safe_port(ip): + import socket + """Returns an ephemeral port that is very likely to be free to bind to.""" + sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + sock.bind((ip, 0)) + port = sock.getsockname()[1] + sock.close() + return port + + +def check_port_open(host, port): + import socket + from contextlib import closing + with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as sock: + return sock.connect_ex((host, port)) == 0 + + +def get_spark_session(): + from pyspark.sql import SparkSession + return SparkSession.builder.getOrCreate() + + +def get_spark_driver_hostname(spark_session): + spark_master_url = spark_session.conf.get("spark.master") + if spark_master_url.lower().startswith("local"): + return "127.0.0.1" + else: + parsed_spark_master_url = urlparse(spark_master_url) + if parsed_spark_master_url.scheme.lower() != "spark" or \ + not parsed_spark_master_url.hostname: + raise ValueError(f"Unsupported spark.master URL: {spark_master_url}") + return parsed_spark_master_url.hostname From 93f527a0c22ed4b9b9a4e65388e827ce329d9ce6 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 27 Sep 2022 16:48:09 +0800 Subject: [PATCH 002/121] memory control Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 25 ++++------------- python/ray/spark/utils.py | 54 ++++++++++++++++++++++++++++++------ 2 files changed, 51 insertions(+), 28 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 7525216a88e4..2e87c1580bd8 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -11,7 +11,9 @@ get_safe_port, get_spark_session, get_spark_driver_hostname, - is_in_databricks_runtime + is_in_databricks_runtime, + get_spark_task_assigned_physical_gpus, + get_per_spark_task_memory, ) @@ -32,22 +34,6 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): raise RuntimeError(error_on_failure) -def _get_spark_task_assigned_physical_gpus(task_context): - resources = task_context.resources() - if "gpu" not in resources: - raise RuntimeError( - "Couldn't get the gpu id, Please check the GPU resource configuration" - ) - gpu_addr_list = [int(addr.strip()) for addr in resources["gpu"].addresses] - - if is_in_databricks_runtime(): - visible_cuda_dev_list = [ - int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") - ] - return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] - else: - return gpu_addr_list - class RayClusterOnSpark: @@ -107,6 +93,7 @@ def init_cluster(num_spark_tasks): num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) + ray_worker_memory_in_bytes = get_per_spark_task_memory() def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext @@ -125,12 +112,13 @@ def ray_cluster_job_mapper(_): f"--num-cpus={num_spark_task_cpus}", "--block", f"--address={ray_head_hostname}:{ray_head_port}", + f"--memory={ray_worker_memory_in_bytes}", ] ray_worker_extra_envs = {} if num_spark_task_gpus > 0: - available_physical_gpus = _get_spark_task_assigned_physical_gpus(context) + available_physical_gpus = get_spark_task_assigned_physical_gpus(context) ray_worker_cmd.append( f"--num-gpus={len(available_physical_gpus)}", ) @@ -175,7 +163,6 @@ def setup_sigterm_on_parent_death(): # and collect tail logs and raise error if subprocess failed. logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - # TODO: Add memory control # Q: When Ray head node killed, will ray worker node exit as well ? exec_cmd( ray_worker_cmd, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 26c67e7cdbab..76bafacca1d0 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -141,13 +141,49 @@ def get_spark_session(): return SparkSession.builder.getOrCreate() -def get_spark_driver_hostname(spark_session): - spark_master_url = spark_session.conf.get("spark.master") - if spark_master_url.lower().startswith("local"): - return "127.0.0.1" +def get_spark_driver_hostname(spark): + return spark.sparkContext._jsc.sc().env().blockManager().master().driverEndpoint().address().host() + + +def get_max_num_concurrent_tasks(spark_context): + """Gets the current max number of concurrent tasks.""" + # pylint: disable=protected-access + # spark 3.1 and above has a different API for fetching max concurrent tasks + if spark_context._jsc.sc().version() >= "3.1": + return spark_context._jsc.sc().maxNumConcurrentTasks( + spark_context._jsc.sc().resourceProfileManager().resourceProfileFromId(0) + ) + return spark_context._jsc.sc().maxNumConcurrentTasks() + + +def get_per_spark_task_memory(spark): + """ + Return the memory size in bytes allocated to each spark task evenly. + """ + sc = spark.sparkContext + spark_mem_fraction = float(sc.getConf().get("spark.memory.fraction", "0.6")) + executor_mem_status = sc._jvm.scala.collection.JavaConversions.mapAsJavaMap( + sc._jsc.sc().env().blockManager().master().getMemoryStatus() + ) + total_executor_mem = sum( + v._1() for k, v in executor_mem_status.items() if k.executorId_() != "driver" + ) + max_num_concurrent_spark_tasks = get_max_num_concurrent_tasks(sc) + return int(total_executor_mem / spark_mem_fraction / max_num_concurrent_spark_tasks) + + +def get_spark_task_assigned_physical_gpus(task_context): + resources = task_context.resources() + if "gpu" not in resources: + raise RuntimeError( + "Couldn't get the gpu id, Please check the GPU resource configuration" + ) + gpu_addr_list = [int(addr.strip()) for addr in resources["gpu"].addresses] + + if is_in_databricks_runtime(): + visible_cuda_dev_list = [ + int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") + ] + return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] else: - parsed_spark_master_url = urlparse(spark_master_url) - if parsed_spark_master_url.scheme.lower() != "spark" or \ - not parsed_spark_master_url.hostname: - raise ValueError(f"Unsupported spark.master URL: {spark_master_url}") - return parsed_spark_master_url.hostname + return gpu_addr_list From 06d41256368572df69fd26b2add074c8eae924da Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 29 Sep 2022 21:39:44 +0800 Subject: [PATCH 003/121] check spark version Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 2e87c1580bd8..a4b351311148 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -1,9 +1,9 @@ import os import sys -from urllib.parse import urlparse import time import threading import logging +from packaging.version import Version from .utils import ( exec_cmd, @@ -16,6 +16,14 @@ get_per_spark_task_memory, ) +_spark_dependency_error = "ray.spark module requires pyspark >= 3.3" +try: + import pyspark + if Version(pyspark.__version__) < Version("3.3"): + raise RuntimeError(_spark_dependency_error) +except ImportError: + raise RuntimeError(_spark_dependency_error) + def _create_ray_tmp_dir(prefix): import tempfile @@ -94,6 +102,7 @@ def init_cluster(num_spark_tasks): num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) ray_worker_memory_in_bytes = get_per_spark_task_memory() + def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext From 3a010d01782e22f7a0a000c681512373b5616bc8 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 29 Sep 2022 21:46:34 +0800 Subject: [PATCH 004/121] db set background job --- python/ray/spark/__init__.py | 12 ++++++++++-- python/ray/spark/utils.py | 20 +++++++++++++++++++- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index a4b351311148..d340115901d4 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -14,6 +14,7 @@ is_in_databricks_runtime, get_spark_task_assigned_physical_gpus, get_per_spark_task_memory, + get_dbutils, ) _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" @@ -211,8 +212,15 @@ def backgroud_job_thread_fn(): # discover the ray cluster. ray.init(address=f"{ray_head_hostname}:{ray_head_port}") - # TODO: - # Register databricks REPL detach event hook to shutdown the ray cluster. + if is_in_databricks_runtime(): + try: + get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) + except Exception: + logging.warning( + "Register ray cluster spark job as background job failed. You need to manually " + "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " + "python REPL." + ) return RayClusterOnSpark( address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_node_proc, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 76bafacca1d0..a6cd247c88dd 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -1,5 +1,4 @@ import subprocess -from urllib.parse import urlparse import os import sys @@ -8,6 +7,25 @@ def is_in_databricks_runtime(): return "DATABRICKS_RUNTIME_VERSION" in os.environ +class _NoDbutilsError(Exception): + pass + + +def get_dbutils(): + try: + import IPython + + ip_shell = IPython.get_ipython() + if ip_shell is None: + raise _NoDbutilsError + return ip_shell.ns_table["user_global"]["dbutils"] + except ImportError: + raise _NoDbutilsError + except KeyError: + raise _NoDbutilsError + + + class ShellCommandException(Exception): @classmethod def from_completed_process(cls, process): From 8e706361e4806aefb902e1049fd5469c76035814 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 29 Sep 2022 21:48:53 +0800 Subject: [PATCH 005/121] setup extra dep --- python/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/setup.py b/python/setup.py index 756c30620b28..c9eb1819587d 100644 --- a/python/setup.py +++ b/python/setup.py @@ -248,6 +248,7 @@ def get_packages(self): "opentelemetry-sdk==1.1.0", "opentelemetry-exporter-otlp==1.1.0", ], + "spark": ["pyspark >= 3.3.0"], } # Ray Serve depends on the Ray dashboard components. From fb6712ebcc09a20b85a4063ee37296689570f827 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 29 Sep 2022 22:02:59 +0800 Subject: [PATCH 006/121] support ray node options --- python/ray/spark/__init__.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d340115901d4..93d3e8ac91a2 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -56,13 +56,17 @@ def shutdown(self): self.head_proc.kill() -def init_cluster(num_spark_tasks): - # TODO: - # support more ray head / worker arguments. - # support more ray.init arguments. +def _convert_ray_node_options(options): + return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] + + +def init_cluster(num_spark_tasks, head_options=None, worker_options=None): import ray from pyspark.util import inheritable_thread_target + head_options = head_options or {} + worker_options = worker_options or {} + spark = get_spark_session() ray_head_hostname = get_spark_driver_hostname(spark) ray_head_port = get_safe_port(ray_head_hostname) @@ -81,6 +85,7 @@ def init_cluster(num_spark_tasks): "--head", f"--port={ray_head_port}", "--include-dashboard=false", + *_convert_ray_node_options(head_options) ] logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") @@ -123,6 +128,7 @@ def ray_cluster_job_mapper(_): "--block", f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_memory_in_bytes}", + *_convert_ray_node_options(worker_options) ] ray_worker_extra_envs = {} @@ -210,6 +216,7 @@ def backgroud_job_thread_fn(): time.sleep(10) # discover the ray cluster. + # Q: We connect to an existing ray cluster, shall we set other arguments in `ray.init` ? ray.init(address=f"{ray_head_hostname}:{ray_head_port}") if is_in_databricks_runtime(): From e6e443b4493f6f465469d1000b421f65b78e3f17 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 29 Sep 2022 22:11:02 +0800 Subject: [PATCH 007/121] doc Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 93d3e8ac91a2..12436d16d49d 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -43,8 +43,11 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): raise RuntimeError(error_on_failure) - class RayClusterOnSpark: + """ + The class is the type of instance returned by `init_cluster` API. + It can be used to shutdown the cluster. + """ def __init__(self, address, head_proc, spark_job_group_id): self.address = address @@ -52,6 +55,9 @@ def __init__(self, address, head_proc, spark_job_group_id): self.spark_job_group_id = spark_job_group_id def shutdown(self): + """ + Shutdown the ray cluster created by `init_cluster` API. + """ get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) self.head_proc.kill() @@ -61,6 +67,18 @@ def _convert_ray_node_options(options): def init_cluster(num_spark_tasks, head_options=None, worker_options=None): + """ + Initialize a ray cluster on the spark cluster, via creating a background spark barrier + mode job and each spark task running a ray worker node, and in spark driver side + a ray head node is started. And then connect to the created ray cluster. + + Args + num_spark_tasks: Specify the spark task number the spark job will create. + This argument controls how many resources (CPU / GPU / memory) the ray cluster + can use. + head_options: A dict representing Ray head node options. + worker_options: A dict representing Ray worker node options. + """ import ray from pyspark.util import inheritable_thread_target From 45c5b3779c23322f8adfc7fa12ab20eee2b32f16 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 8 Oct 2022 15:55:58 +0800 Subject: [PATCH 008/121] add object store mem setting Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 7 +++--- python/ray/spark/utils.py | 42 ++++++++++++++++++++++++++++++++++-- 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 12436d16d49d..18c7e7e376b3 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -13,7 +13,7 @@ get_spark_driver_hostname, is_in_databricks_runtime, get_spark_task_assigned_physical_gpus, - get_per_spark_task_memory, + get_avail_mem_per_ray_worker, get_dbutils, ) @@ -125,7 +125,7 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) - ray_worker_memory_in_bytes = get_per_spark_task_memory() + ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes = get_avail_mem_per_ray_worker(spark) def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext @@ -145,7 +145,8 @@ def ray_cluster_job_mapper(_): f"--num-cpus={num_spark_task_cpus}", "--block", f"--address={ray_head_hostname}:{ray_head_port}", - f"--memory={ray_worker_memory_in_bytes}", + f"--memory={ray_worker_heap_mem_bytes}", + f"--object-store-memory={ray_worker_object_store_mem_bytes}", *_convert_ray_node_options(worker_options) ] diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index a6cd247c88dd..06edbb2033d2 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -174,9 +174,40 @@ def get_max_num_concurrent_tasks(spark_context): return spark_context._jsc.sc().maxNumConcurrentTasks() -def get_per_spark_task_memory(spark): +def get_avail_mem_per_ray_worker(spark): + num_cpus_per_spark_task = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) + + def mapper(_): + try: + import multiprocessing + import psutil + import shutil + + num_cpus = multiprocessing.cpu_count() + num_task_slots = num_cpus // num_cpus_per_spark_task + + physical_mem_bytes = psutil.virtual_memory().total + shared_mem_bytes = shutil.disk_usage('/dev/shm').total + + ray_worker_object_store_bytes = int(shared_mem_bytes / num_task_slots * 0.8) + ray_worker_heap_mem_bytes = int((physical_mem_bytes - shared_mem_bytes) / num_task_slots * 0.8) + + return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes, None + except Exception as e: + return -1, -1, repr(e) + + inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes, err = \ + spark.sparkContext.parallelize([1], 1).map(mapper).collect()[0] + + if err is not None: + raise RuntimeError(f"Inferring ray worker available memory failed, error: {err}") + return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes + + +def get_ray_worker_heap_memory(spark): """ - Return the memory size in bytes allocated to each spark task evenly. + Return the available heap memory size for each ray node. + NB: We have one ray node per spark task. """ sc = spark.sparkContext spark_mem_fraction = float(sc.getConf().get("spark.memory.fraction", "0.6")) @@ -190,6 +221,13 @@ def get_per_spark_task_memory(spark): return int(total_executor_mem / spark_mem_fraction / max_num_concurrent_spark_tasks) +def get_ray_worker_object_store_memory(): + """ + Return the available object store memory for each ray node. + NB: We have one ray node per spark task. + """ + + def get_spark_task_assigned_physical_gpus(task_context): resources = task_context.resources() if "gpu" not in resources: From 97359ec83022a754dc8c44f34ff5f0ef5ec0314a Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 8 Oct 2022 16:30:27 +0800 Subject: [PATCH 009/121] disconnect when shutdown Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 18c7e7e376b3..d750dbe8003f 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -49,15 +49,17 @@ class RayClusterOnSpark: It can be used to shutdown the cluster. """ - def __init__(self, address, head_proc, spark_job_group_id): + def __init__(self, address, head_proc, spark_job_group_id, ray_context): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id + self.ray_context = ray_context def shutdown(self): """ Shutdown the ray cluster created by `init_cluster` API. """ + self.ray_context.disconnect() get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) self.head_proc.kill() @@ -234,9 +236,8 @@ def backgroud_job_thread_fn(): # Waiting all ray workers spin up. time.sleep(10) - # discover the ray cluster. - # Q: We connect to an existing ray cluster, shall we set other arguments in `ray.init` ? - ray.init(address=f"{ray_head_hostname}:{ray_head_port}") + # connect to the ray cluster. + ray_context = ray.init(address=f"{ray_head_hostname}:{ray_head_port}") if is_in_databricks_runtime(): try: @@ -250,5 +251,6 @@ def backgroud_job_thread_fn(): return RayClusterOnSpark( address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_node_proc, - spark_job_group_id=spark_job_group_id + spark_job_group_id=spark_job_group_id, + ray_context=ray_context, ) From f1d5a84fea494b69c205c99883b1a37ce1cc7525 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 15:29:29 +0800 Subject: [PATCH 010/121] misc udpates Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 91 ++++++++++++++++------------ python/ray/spark/utils.py | 66 ++++++++++---------- python/ray/tests/spark/test_spark.py | 57 +++++++++++++++++ 3 files changed, 141 insertions(+), 73 deletions(-) create mode 100644 python/ray/tests/spark/test_spark.py diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d750dbe8003f..226f0b3ebce7 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -17,6 +17,9 @@ get_dbutils, ) +if not sys.platform.startswith("linux"): + raise RuntimeError("Ray on spark functionality only supports Linux system.") + _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: import pyspark @@ -26,12 +29,6 @@ raise RuntimeError(_spark_dependency_error) -def _create_ray_tmp_dir(prefix): - import tempfile - - return tempfile.mkdtemp(prefix=prefix) - - def wait_ray_node_available(hostname, port, timeout, error_on_failure): # Wait Ray head node spin up. for _ in range(timeout): @@ -63,6 +60,12 @@ def shutdown(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) self.head_proc.kill() + def __enter__(self): + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + self.shutdown() + def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] @@ -93,23 +96,33 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): logging.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) + num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) + + ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes = get_avail_mem_per_ray_worker(spark) + ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_head_tmp_dir = _create_ray_tmp_dir(f"ray-head-port-{ray_head_port}-tmp-") + ray_head_tmp_dir = f"/tmp/ray-temp/ray-head-port-{ray_head_port}" + ray_head_node_cmd = [ ray_exec_path, "start", f"--temp-dir={ray_head_tmp_dir}", - f"--num-cpus=0", # disallow ray tasks scheduled to ray head node. "--block", "--head", f"--port={ray_head_port}", "--include-dashboard=false", + f"--num-cpus=0", # disallow ray tasks scheduled to ray head node. + # limit the memory usage of head node because no task running on it. + f"--memory={128 * 1024 * 1024}", + # limit the object store memory usage of head node because no task running on it. + f"--object-store-memory={128 * 1024 * 1024}", *_convert_ray_node_options(head_options) ] logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - ray_node_proc = exec_cmd( + ray_head_proc = exec_cmd( ray_head_node_cmd, synchronous=False, capture_output=False, @@ -124,11 +137,6 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): logging.info("Ray head node started.") - num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) - num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) - - ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes = get_avail_mem_per_ray_worker(spark) - def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext @@ -136,9 +144,8 @@ def ray_cluster_job_mapper(_): context.barrier() task_id = context.partitionId() - ray_worker_tmp_dir = _create_ray_tmp_dir( - f"ray-worker-{task_id}-head-{ray_head_hostname}:{ray_head_port}-tmp-" - ) + # TODO: remove temp dir when ray worker exits. + ray_worker_tmp_dir = f"/tmp/ray-temp/ray-worker-{task_id}-head-{ray_head_hostname}-{ray_head_port}" ray_worker_cmd = [ ray_exec_path, @@ -213,8 +220,6 @@ def setup_sigterm_on_parent_death(): # NB: Not reachable. yield 0 - spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" - # TODO: redirect background thread output. def backgroud_job_thread_fn(): spark.sparkContext.setJobGroup( @@ -228,29 +233,35 @@ def backgroud_job_thread_fn(): ray_cluster_job_mapper ).collect()[0] + spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + threading.Thread( target=inheritable_thread_target(backgroud_job_thread_fn), args=() ).start() - # Waiting all ray workers spin up. - time.sleep(10) - - # connect to the ray cluster. - ray_context = ray.init(address=f"{ray_head_hostname}:{ray_head_port}") - - if is_in_databricks_runtime(): - try: - get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) - except Exception: - logging.warning( - "Register ray cluster spark job as background job failed. You need to manually " - "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " - "python REPL." - ) - return RayClusterOnSpark( - address=f"{ray_head_hostname}:{ray_head_port}", - head_proc=ray_node_proc, - spark_job_group_id=spark_job_group_id, - ray_context=ray_context, - ) + try: + # Waiting all ray workers spin up. + time.sleep(10) + + # connect to the ray cluster. + ray_context = ray.init(address=f"{ray_head_hostname}:{ray_head_port}") + + if is_in_databricks_runtime(): + try: + get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) + except Exception: + logging.warning( + "Register ray cluster spark job as background job failed. You need to manually " + "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " + "python REPL." + ) + return RayClusterOnSpark( + address=f"{ray_head_hostname}:{ray_head_port}", + head_proc=ray_head_proc, + spark_job_group_id=spark_job_group_id, + ray_context=ray_context, + ) + finally: + # If init cluster raise exception, kill the ray head proc. + ray_head_proc.kill() diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 06edbb2033d2..e5fd62012cda 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -1,4 +1,5 @@ import subprocess +from urllib.parse import urlparse import os import sys @@ -25,7 +26,6 @@ def get_dbutils(): raise _NoDbutilsError - class ShellCommandException(Exception): @classmethod def from_completed_process(cls, process): @@ -160,7 +160,15 @@ def get_spark_session(): def get_spark_driver_hostname(spark): - return spark.sparkContext._jsc.sc().env().blockManager().master().driverEndpoint().address().host() + spark_master_url = spark.conf.get("spark.master") + if spark_master_url.lower().startswith("local"): + return "127.0.0.1" + else: + parsed_spark_master_url = urlparse(spark_master_url) + if parsed_spark_master_url.scheme.lower() != "spark" or \ + not parsed_spark_master_url.hostname: + raise ValueError(f"Unsupported spark.master URL: {spark_master_url}") + return parsed_spark_master_url.hostname def get_max_num_concurrent_tasks(spark_context): @@ -174,20 +182,36 @@ def get_max_num_concurrent_tasks(spark_context): return spark_context._jsc.sc().maxNumConcurrentTasks() +def _get_total_phyisical_memory(): + import psutil + return psutil.virtual_memory().total + + +def _get_total_shared_memory(): + import shutil + return shutil.disk_usage('/dev/shm').total + + +def _get_cpu_cores(): + import multiprocessing + return multiprocessing.cpu_count() + + def get_avail_mem_per_ray_worker(spark): + """ + Return the available heap memory and object store memory for each ray worker. + NB: We have one ray node per spark task. + """ + # TODO: add a option of heap memory / object store memory ratio. num_cpus_per_spark_task = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) def mapper(_): try: - import multiprocessing - import psutil - import shutil - - num_cpus = multiprocessing.cpu_count() + num_cpus = _get_cpu_cores() num_task_slots = num_cpus // num_cpus_per_spark_task - physical_mem_bytes = psutil.virtual_memory().total - shared_mem_bytes = shutil.disk_usage('/dev/shm').total + physical_mem_bytes = _get_total_phyisical_memory() + shared_mem_bytes = _get_total_shared_memory() ray_worker_object_store_bytes = int(shared_mem_bytes / num_task_slots * 0.8) ray_worker_heap_mem_bytes = int((physical_mem_bytes - shared_mem_bytes) / num_task_slots * 0.8) @@ -204,30 +228,6 @@ def mapper(_): return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes -def get_ray_worker_heap_memory(spark): - """ - Return the available heap memory size for each ray node. - NB: We have one ray node per spark task. - """ - sc = spark.sparkContext - spark_mem_fraction = float(sc.getConf().get("spark.memory.fraction", "0.6")) - executor_mem_status = sc._jvm.scala.collection.JavaConversions.mapAsJavaMap( - sc._jsc.sc().env().blockManager().master().getMemoryStatus() - ) - total_executor_mem = sum( - v._1() for k, v in executor_mem_status.items() if k.executorId_() != "driver" - ) - max_num_concurrent_spark_tasks = get_max_num_concurrent_tasks(sc) - return int(total_executor_mem / spark_mem_fraction / max_num_concurrent_spark_tasks) - - -def get_ray_worker_object_store_memory(): - """ - Return the available object store memory for each ray node. - NB: We have one ray node per spark task. - """ - - def get_spark_task_assigned_physical_gpus(task_context): resources = task_context.resources() if "gpu" not in resources: diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py new file mode 100644 index 000000000000..1dbdb5aa9560 --- /dev/null +++ b/python/ray/tests/spark/test_spark.py @@ -0,0 +1,57 @@ +import pytest +import ray + +from ray import spark as ray_spark +from pyspark.sql import SparkSession +import time + + +class RayOnSparkTestBase: + + spark = None + num_total_cpus = None + num_total_gpus = None + num_cpus_per_spark_task = None + num_gpus_per_spark_task = None + max_spark_tasks = None + + @classmethod + def setup_class(cls): + raise NotImplementedError() + + @classmethod + def teardown_class(cls): + cls.spark.stop() + + @staticmethod + def get_ray_worker_resources_list(): + wr_list = [] + for node in ray.nodes(): + # exclude dead node and head node (with 0 CPU resource) + if node['Alive'] and node['Resources'].get('CPU', 0) > 0: + wr_list.append(node['Resources']) + return wr_list + + def test_cpu_allocation(self): + for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: + with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: + time.sleep(5) + worker_res_list = self.get_ray_worker_resources_list() + assert len(worker_res_list) == num_spark_tasks + for worker_res in worker_res_list: + assert worker_res['CPU'] == self.num_cpus_per_spark_task + + +class TestBasicSparkCluster(RayOnSparkTestBase): + + @classmethod + def setup_class(cls): + cls.num_total_cpus = 2 + cls.num_total_gpus = 0 + cls.num_cpus_per_spark_task = 1 + cls.num_gpus_per_spark_task = 0 + cls.max_spark_tasks = 2 + cls.spark = SparkSession.builder \ + .config("master", "local-cluster[1, 2, 1024]") \ + .config("spark.task.cpus", "1") \ + .getOrCreate() From f4f73a4415b3aba8c381d04f70a3017e15318d17 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 17:10:07 +0800 Subject: [PATCH 011/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 54 ++++++++++++++++++++-------- python/ray/spark/utils.py | 8 +++-- python/ray/tests/spark/test_spark.py | 6 ++-- 3 files changed, 49 insertions(+), 19 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 226f0b3ebce7..0340974cd040 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -1,4 +1,5 @@ import os +import subprocess import sys import time import threading @@ -18,7 +19,10 @@ ) if not sys.platform.startswith("linux"): - raise RuntimeError("Ray on spark functionality only supports Linux system.") + logging.warning( + "Ray on spark running on non-linux systems cannot be shutdown correctly, you need to " + "manually kill ray node on spark worker side." + ) _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: @@ -71,7 +75,12 @@ def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] -def init_cluster(num_spark_tasks, head_options=None, worker_options=None): +def init_cluster( + num_spark_tasks, + head_options=None, + worker_options=None, + ray_node_log_root_path="/tmp/ray/logs" +): """ Initialize a ray cluster on the spark cluster, via creating a background spark barrier mode job and each spark task running a ray worker node, and in spark driver side @@ -103,7 +112,10 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_head_tmp_dir = f"/tmp/ray-temp/ray-head-port-{ray_head_port}" + ray_head_tmp_dir = f"/tmp/ray/ray-temp/ray-head-port-{ray_head_port}" + + ray_node_log_path = os.path.join(ray_node_log_root_path, f"cluster-{ray_head_hostname}-{ray_head_port}") + os.makedirs(ray_node_log_path, exist_ok=True) ray_head_node_cmd = [ ray_exec_path, @@ -111,6 +123,7 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): f"--temp-dir={ray_head_tmp_dir}", "--block", "--head", + "--disable-usage-stats", f"--port={ray_head_port}", "--include-dashboard=false", f"--num-cpus=0", # disallow ray tasks scheduled to ray head node. @@ -121,7 +134,7 @@ def init_cluster(num_spark_tasks, head_options=None, worker_options=None): *_convert_ray_node_options(head_options) ] - logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") + print(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") ray_head_proc = exec_cmd( ray_head_node_cmd, synchronous=False, @@ -145,7 +158,8 @@ def ray_cluster_job_mapper(_): task_id = context.partitionId() # TODO: remove temp dir when ray worker exits. - ray_worker_tmp_dir = f"/tmp/ray-temp/ray-worker-{task_id}-head-{ray_head_hostname}-{ray_head_port}" + ray_worker_tmp_dir = f"/tmp/ray/ray-temp/ray-worker-{task_id}-head-{ray_head_hostname}-{ray_head_port}" + os.makedirs(ray_worker_tmp_dir, exist_ok=True) ray_worker_cmd = [ ray_exec_path, @@ -153,12 +167,15 @@ def ray_cluster_job_mapper(_): f"--temp-dir={ray_worker_tmp_dir}", f"--num-cpus={num_spark_task_cpus}", "--block", + "--disable-usage-stats", f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", *_convert_ray_node_options(worker_options) ] + print(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + ray_worker_extra_envs = {} if num_spark_task_gpus > 0: @@ -207,15 +224,21 @@ def setup_sigterm_on_parent_death(): # and collect tail logs and raise error if subprocess failed. logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - # Q: When Ray head node killed, will ray worker node exit as well ? - exec_cmd( - ray_worker_cmd, - synchronous=True, - capture_output=False, - stream_output=False, - extra_env=ray_worker_extra_envs, - preexec_fn=setup_sigterm_on_parent_death, + ray_worker_log_file = os.path.join( + ray_node_log_path, + f"ray-worker-{task_id}.log" ) + with open(ray_worker_log_file, "w", buffering=1) as log_fp: + exec_cmd( + ray_worker_cmd, + synchronous=True, + capture_output=False, + stream_output=False, + extra_env=ray_worker_extra_envs, + preexec_fn=setup_sigterm_on_parent_death, + stdout=log_fp, + stderr=subprocess.STDOUT, + ) # NB: Not reachable. yield 0 @@ -231,7 +254,7 @@ def backgroud_job_thread_fn(): list(range(num_spark_tasks)), num_spark_tasks ).barrier().mapPartitions( ray_cluster_job_mapper - ).collect()[0] + ).collect() spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" @@ -262,6 +285,7 @@ def backgroud_job_thread_fn(): spark_job_group_id=spark_job_group_id, ray_context=ray_context, ) - finally: + except Exception: # If init cluster raise exception, kill the ray head proc. ray_head_proc.kill() + raise diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index e5fd62012cda..75093a888401 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -188,8 +188,12 @@ def _get_total_phyisical_memory(): def _get_total_shared_memory(): - import shutil - return shutil.disk_usage('/dev/shm').total + try: + import shutil + return shutil.disk_usage('/dev/shm').total + except Exception: + # fallback + return int(_get_total_phyisical_memory() * 0.3) def _get_cpu_cores(): diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 1dbdb5aa9560..f739c046348e 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -33,9 +33,10 @@ def get_ray_worker_resources_list(): return wr_list def test_cpu_allocation(self): - for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: + for num_spark_tasks in [self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: - time.sleep(5) + print(f"cluster addr: {cluster.address}") + time.sleep(60) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -54,4 +55,5 @@ def setup_class(cls): cls.spark = SparkSession.builder \ .config("master", "local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ + .config("spark.task.maxFailures", 1) \ .getOrCreate() From 29cd053d2901e7cc48924d0bc474035ef0084ba9 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 21:23:00 +0800 Subject: [PATCH 012/121] add test Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 34 +++++++++---------- python/ray/spark/utils.py | 8 ++--- python/ray/tests/spark/test_spark.py | 49 +++++++++++++++++++++++++--- 3 files changed, 62 insertions(+), 29 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 0340974cd040..88c6755c369e 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -19,10 +19,7 @@ ) if not sys.platform.startswith("linux"): - logging.warning( - "Ray on spark running on non-linux systems cannot be shutdown correctly, you need to " - "manually kill ray node on spark worker side." - ) + raise RuntimeError("Ray on spark ony supports linux system.") _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: @@ -134,18 +131,22 @@ def init_cluster( *_convert_ray_node_options(head_options) ] - print(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - ray_head_proc = exec_cmd( - ray_head_node_cmd, - synchronous=False, - capture_output=False, - stream_output=False, - ) + logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") + + with open(os.path.join(ray_node_log_path, "ray-head.log"), "w", buffering=1) as head_log_fp: + ray_head_proc = exec_cmd( + ray_head_node_cmd, + synchronous=False, + capture_output=False, + stream_output=False, + stdout=head_log_fp, + stderr=subprocess.STDOUT, + ) # wait ray head node spin up. wait_ray_node_available( ray_head_hostname, ray_head_port, 40, - "Start Ray head node failed!" + error_on_failure="Start Ray head node failed!" ) logging.info("Ray head node started.") @@ -174,7 +175,7 @@ def ray_cluster_job_mapper(_): *_convert_ray_node_options(worker_options) ] - print(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") ray_worker_extra_envs = {} @@ -220,15 +221,13 @@ def setup_sigterm_on_parent_death(): else: setup_sigterm_on_parent_death = None - # TODO: Add a thread to redirect subprocess logs - # and collect tail logs and raise error if subprocess failed. logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") ray_worker_log_file = os.path.join( ray_node_log_path, f"ray-worker-{task_id}.log" ) - with open(ray_worker_log_file, "w", buffering=1) as log_fp: + with open(ray_worker_log_file, "w", buffering=1) as worker_log_fp: exec_cmd( ray_worker_cmd, synchronous=True, @@ -236,14 +235,13 @@ def setup_sigterm_on_parent_death(): stream_output=False, extra_env=ray_worker_extra_envs, preexec_fn=setup_sigterm_on_parent_death, - stdout=log_fp, + stdout=worker_log_fp, stderr=subprocess.STDOUT, ) # NB: Not reachable. yield 0 - # TODO: redirect background thread output. def backgroud_job_thread_fn(): spark.sparkContext.setJobGroup( spark_job_group_id, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 75093a888401..e5fd62012cda 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -188,12 +188,8 @@ def _get_total_phyisical_memory(): def _get_total_shared_memory(): - try: - import shutil - return shutil.disk_usage('/dev/shm').total - except Exception: - # fallback - return int(_get_total_phyisical_memory() * 0.3) + import shutil + return shutil.disk_usage('/dev/shm').total def _get_cpu_cores(): diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index f739c046348e..43297f1ae001 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -33,15 +33,36 @@ def get_ray_worker_resources_list(): return wr_list def test_cpu_allocation(self): - for num_spark_tasks in [self.max_spark_tasks]: - with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: - print(f"cluster addr: {cluster.address}") - time.sleep(60) + for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: + with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): + time.sleep(2) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: assert worker_res['CPU'] == self.num_cpus_per_spark_task + def test_gpu_allocation(self): + if self.num_gpus_per_spark_task == 0: + pytest.skip("Skip GPU test on cluster without GPU configured.") + + for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: + with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): + time.sleep(2) + worker_res_list = self.get_ray_worker_resources_list() + assert len(worker_res_list) == num_spark_tasks + for worker_res in worker_res_list: + assert worker_res['GPU'] == self.num_gpus_per_spark_task + + def test_basic_ray_app(self): + with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + @ray.remote + def f(x): + return x * x + + futures = [f.remote(i) for i in range(32)] + results = ray.get(futures) + assert results == [i * i for i in range(32)] + class TestBasicSparkCluster(RayOnSparkTestBase): @@ -55,5 +76,23 @@ def setup_class(cls): cls.spark = SparkSession.builder \ .config("master", "local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ - .config("spark.task.maxFailures", 1) \ + .config("spark.task.maxFailures", "1") \ + .getOrCreate() + + +class TestBasicSparkGPUCluster(RayOnSparkTestBase): + + @classmethod + def setup_class(cls): + cls.num_total_cpus = 2 + cls.num_total_gpus = 2 + cls.num_cpus_per_spark_task = 1 + cls.num_gpus_per_spark_task = 2 + cls.max_spark_tasks = 2 + cls.spark = SparkSession.builder \ + .config("master", "local-cluster[1, 2, 1024]") \ + .config("spark.task.cpus", "1") \ + .config("spark.worker.resource.gpu.amount", "1") \ + .config("spark.executor.resource.gpu.amount", "2") \ + .config("spark.task.maxFailures", "1") \ .getOrCreate() From dc63cc6204d0822c4a80fb4aff9fade0ea67353f Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 21:45:01 +0800 Subject: [PATCH 013/121] fix Signed-off-by: Weichen Xu --- python/ray/tests/spark/discover_2_gpu.sh | 4 ++++ python/ray/tests/spark/test_spark.py | 10 +++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 python/ray/tests/spark/discover_2_gpu.sh diff --git a/python/ray/tests/spark/discover_2_gpu.sh b/python/ray/tests/spark/discover_2_gpu.sh new file mode 100644 index 000000000000..3a111e1f577b --- /dev/null +++ b/python/ray/tests/spark/discover_2_gpu.sh @@ -0,0 +1,4 @@ +#!/bin/bash + +# This script is used in spark GPU cluster config for discovering available GPU. +echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\",\"2\",\"3\"]}" diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 43297f1ae001..d4b42e4ed5cd 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -1,3 +1,5 @@ +import os.path + import pytest import ray @@ -46,11 +48,12 @@ def test_gpu_allocation(self): pytest.skip("Skip GPU test on cluster without GPU configured.") for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): + with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: time.sleep(2) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: + breakpoint() assert worker_res['GPU'] == self.num_gpus_per_spark_task def test_basic_ray_app(self): @@ -89,10 +92,15 @@ def setup_class(cls): cls.num_cpus_per_spark_task = 1 cls.num_gpus_per_spark_task = 2 cls.max_spark_tasks = 2 + gpu_discovery_script_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), + "discover_2_gpu.sh" + ) cls.spark = SparkSession.builder \ .config("master", "local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ .config("spark.worker.resource.gpu.amount", "1") \ .config("spark.executor.resource.gpu.amount", "2") \ .config("spark.task.maxFailures", "1") \ + .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ .getOrCreate() From 6d0060002379c1bbdefe9addbb5dcc80c5cd9d46 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 22:20:01 +0800 Subject: [PATCH 014/121] update test Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 5 ++- python/ray/tests/spark/discover_2_gpu.sh | 2 +- python/ray/tests/spark/discover_4_gpu.sh | 4 +++ python/ray/tests/spark/test_spark.py | 39 +++++++++++++++++++++--- 4 files changed, 43 insertions(+), 7 deletions(-) mode change 100644 => 100755 python/ray/tests/spark/discover_2_gpu.sh create mode 100755 python/ray/tests/spark/discover_4_gpu.sh diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 88c6755c369e..403f09eddf7d 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -112,6 +112,9 @@ def init_cluster( ray_head_tmp_dir = f"/tmp/ray/ray-temp/ray-head-port-{ray_head_port}" ray_node_log_path = os.path.join(ray_node_log_root_path, f"cluster-{ray_head_hostname}-{ray_head_port}") + + logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_path}") + os.makedirs(ray_node_log_path, exist_ok=True) ray_head_node_cmd = [ @@ -185,7 +188,7 @@ def ray_cluster_job_mapper(_): f"--num-gpus={len(available_physical_gpus)}", ) ray_worker_extra_envs['CUDA_VISIBLE_DEVICES'] = ",".join([ - str(gpu_id) for gpu_id in num_spark_task_gpus + str(gpu_id) for gpu_id in available_physical_gpus ]) if sys.platform.startswith("linux"): diff --git a/python/ray/tests/spark/discover_2_gpu.sh b/python/ray/tests/spark/discover_2_gpu.sh old mode 100644 new mode 100755 index 3a111e1f577b..6491bacaecb1 --- a/python/ray/tests/spark/discover_2_gpu.sh +++ b/python/ray/tests/spark/discover_2_gpu.sh @@ -1,4 +1,4 @@ #!/bin/bash # This script is used in spark GPU cluster config for discovering available GPU. -echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\",\"2\",\"3\"]}" +echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\"]}" diff --git a/python/ray/tests/spark/discover_4_gpu.sh b/python/ray/tests/spark/discover_4_gpu.sh new file mode 100755 index 000000000000..3a111e1f577b --- /dev/null +++ b/python/ray/tests/spark/discover_4_gpu.sh @@ -0,0 +1,4 @@ +#!/bin/bash + +# This script is used in spark GPU cluster config for discovering available GPU. +echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\",\"2\",\"3\"]}" diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index d4b42e4ed5cd..0fabbb0eadc3 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -53,7 +53,6 @@ def test_gpu_allocation(self): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: - breakpoint() assert worker_res['GPU'] == self.num_gpus_per_spark_task def test_basic_ray_app(self): @@ -76,8 +75,9 @@ def setup_class(cls): cls.num_cpus_per_spark_task = 1 cls.num_gpus_per_spark_task = 0 cls.max_spark_tasks = 2 + os.environ["SPARK_WORKER_CORES"] = 2 cls.spark = SparkSession.builder \ - .config("master", "local-cluster[1, 2, 1024]") \ + .master("local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ .config("spark.task.maxFailures", "1") \ .getOrCreate() @@ -90,17 +90,46 @@ def setup_class(cls): cls.num_total_cpus = 2 cls.num_total_gpus = 2 cls.num_cpus_per_spark_task = 1 - cls.num_gpus_per_spark_task = 2 + cls.num_gpus_per_spark_task = 1 cls.max_spark_tasks = 2 gpu_discovery_script_path = os.path.join( os.path.dirname(os.path.abspath(__file__)), "discover_2_gpu.sh" ) + os.environ["SPARK_WORKER_CORES"] = 2 cls.spark = SparkSession.builder \ - .config("master", "local-cluster[1, 2, 1024]") \ + .master("local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ - .config("spark.worker.resource.gpu.amount", "1") \ + .config("spark.task.resource.gpu.amount", "1") \ + .config("spark.executor.cores", "2") \ + .config("spark.worker.resource.gpu.amount", "2") \ .config("spark.executor.resource.gpu.amount", "2") \ .config("spark.task.maxFailures", "1") \ .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ .getOrCreate() + + +class TestMultiCoresPerTaskCluster(RayOnSparkTestBase): + + @classmethod + def setup_class(cls): + cls.num_total_cpus = 4 + cls.num_total_gpus = 4 + cls.num_cpus_per_spark_task = 2 + cls.num_gpus_per_spark_task = 2 + cls.max_spark_tasks = 2 + gpu_discovery_script_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), + "discover_4_gpu.sh" + ) + os.environ["SPARK_WORKER_CORES"] = 4 + cls.spark = SparkSession.builder \ + .master("local-cluster[1, 4, 1024]") \ + .config("spark.task.cpus", "2") \ + .config("spark.task.resource.gpu.amount", "2") \ + .config("spark.executor.cores", "4") \ + .config("spark.worker.resource.gpu.amount", "4") \ + .config("spark.executor.resource.gpu.amount", "4") \ + .config("spark.task.maxFailures", "1") \ + .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ + .getOrCreate() From 3a5b01bed5c0431981a951e00716819389872ccb Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 23:08:01 +0800 Subject: [PATCH 015/121] fix Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 6 ++++++ python/ray/tests/spark/test_spark.py | 6 +++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 403f09eddf7d..6c1de83c3b87 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -114,6 +114,12 @@ def init_cluster( ray_node_log_path = os.path.join(ray_node_log_root_path, f"cluster-{ray_head_hostname}-{ray_head_port}") logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_path}") + if is_in_databricks_runtime() and not ray_node_log_root_path.startswith("/dbfs"): + logging.warning( + "We recommend you to set `ray_node_log_root_path` argument to be a path under '/dbfs/', " + "because for all spark cluster nodes '/dbfs/' path is mounted with a shared disk, " + "so that you can check ray worker logs on spark driver node." + ) os.makedirs(ray_node_log_path, exist_ok=True) diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 0fabbb0eadc3..1dfb6c6c0038 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -75,7 +75,7 @@ def setup_class(cls): cls.num_cpus_per_spark_task = 1 cls.num_gpus_per_spark_task = 0 cls.max_spark_tasks = 2 - os.environ["SPARK_WORKER_CORES"] = 2 + os.environ["SPARK_WORKER_CORES"] = "2" cls.spark = SparkSession.builder \ .master("local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ @@ -96,7 +96,7 @@ def setup_class(cls): os.path.dirname(os.path.abspath(__file__)), "discover_2_gpu.sh" ) - os.environ["SPARK_WORKER_CORES"] = 2 + os.environ["SPARK_WORKER_CORES"] = "4" cls.spark = SparkSession.builder \ .master("local-cluster[1, 2, 1024]") \ .config("spark.task.cpus", "1") \ @@ -122,7 +122,7 @@ def setup_class(cls): os.path.dirname(os.path.abspath(__file__)), "discover_4_gpu.sh" ) - os.environ["SPARK_WORKER_CORES"] = 4 + os.environ["SPARK_WORKER_CORES"] = "4" cls.spark = SparkSession.builder \ .master("local-cluster[1, 4, 1024]") \ .config("spark.task.cpus", "2") \ From c84733a5ec6afa0f232d9f12e5ec654fbe7cdb3d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 23:29:06 +0800 Subject: [PATCH 016/121] add tests Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 5 ++- python/ray/tests/spark/test_spark.py | 64 ++++++++++++++++++++++------ 2 files changed, 56 insertions(+), 13 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 6c1de83c3b87..ddb8ae628909 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -53,12 +53,15 @@ def __init__(self, address, head_proc, spark_job_group_id, ray_context): self.spark_job_group_id = spark_job_group_id self.ray_context = ray_context + def _cancel_background_spark_job(self): + get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) + def shutdown(self): """ Shutdown the ray cluster created by `init_cluster` API. """ self.ray_context.disconnect() - get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) + self._cancel_background_spark_job() self.head_proc.kill() def __enter__(self): diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 1dfb6c6c0038..7e2891f2e5b7 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -1,14 +1,17 @@ -import os.path +import os +from abc import ABC import pytest import ray from ray import spark as ray_spark +from ray.spark import check_port_open from pyspark.sql import SparkSession import time +import functools -class RayOnSparkTestBase: +class RayOnSparkCPUClusterTestBase(ABC): spark = None num_total_cpus = None @@ -43,6 +46,34 @@ def test_cpu_allocation(self): for worker_res in worker_res_list: assert worker_res['CPU'] == self.num_cpus_per_spark_task + def test_basic_ray_app(self): + with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + @ray.remote + def f(x): + return x * x + + futures = [f.remote(i) for i in range(32)] + results = ray.get(futures) + assert results == [i * i for i in range(32)] + + def test_ray_cluster_shutdown(self): + with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + time.sleep(2) + worker_res_list = self.get_ray_worker_resources_list() + assert len(worker_res_list) == self.max_spark_tasks + + # cancel background spark job will cause all ray worker nodes exit. + cluster._cancel_background_spark_job() + time.sleep(5) + assert len(worker_res_list) == 0 + + time.sleep(2) # wait ray head node exit. + # assert ray head node exit by checking head port being closed. + assert not check_port_open(int(cluster.address.split(":")[1])) + + +class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): + def test_gpu_allocation(self): if self.num_gpus_per_spark_task == 0: pytest.skip("Skip GPU test on cluster without GPU configured.") @@ -55,18 +86,27 @@ def test_gpu_allocation(self): for worker_res in worker_res_list: assert worker_res['GPU'] == self.num_gpus_per_spark_task - def test_basic_ray_app(self): - with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - @ray.remote - def f(x): - return x * x + def test_basic_ray_app_using_gpu(self): - futures = [f.remote(i) for i in range(32)] + with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + @ray.remote(num_gpus=1) + def f(_): + # Add a sleep to avoid the task finishing too fast, + # so that it can make all ray tasks concurrently running in all idle task slots. + time.sleep(5) + return [ + int(gpu_id) + for gpu_id in os.environ['CUDA_VISIBLE_DEVICES'].split(",") + ] + + futures = [f.remote(i) for i in range(self.max_spark_tasks)] results = ray.get(futures) - assert results == [i * i for i in range(32)] + merged_results = functools.reduce(lambda x, y: x + y, results) + # Test all ray tasks are assigned with different GPUs. + assert sorted(merged_results) == list(range(self.num_total_gpus)) -class TestBasicSparkCluster(RayOnSparkTestBase): +class TestBasicSparkCluster(RayOnSparkCPUClusterTestBase): @classmethod def setup_class(cls): @@ -83,7 +123,7 @@ def setup_class(cls): .getOrCreate() -class TestBasicSparkGPUCluster(RayOnSparkTestBase): +class TestBasicSparkGPUCluster(RayOnSparkGPUClusterTestBase): @classmethod def setup_class(cls): @@ -109,7 +149,7 @@ def setup_class(cls): .getOrCreate() -class TestMultiCoresPerTaskCluster(RayOnSparkTestBase): +class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): @classmethod def setup_class(cls): From 82b573578a33c3530515532212f48416d0825b17 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 9 Oct 2022 23:47:17 +0800 Subject: [PATCH 017/121] fix Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_spark.py | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 7e2891f2e5b7..2674a43bfc53 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -15,9 +15,7 @@ class RayOnSparkCPUClusterTestBase(ABC): spark = None num_total_cpus = None - num_total_gpus = None num_cpus_per_spark_task = None - num_gpus_per_spark_task = None max_spark_tasks = None @classmethod @@ -59,24 +57,24 @@ def f(x): def test_ray_cluster_shutdown(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: time.sleep(2) - worker_res_list = self.get_ray_worker_resources_list() - assert len(worker_res_list) == self.max_spark_tasks + assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # cancel background spark job will cause all ray worker nodes exit. cluster._cancel_background_spark_job() time.sleep(5) - assert len(worker_res_list) == 0 + assert len(self.get_ray_worker_resources_list()) == 0 time.sleep(2) # wait ray head node exit. # assert ray head node exit by checking head port being closed. - assert not check_port_open(int(cluster.address.split(":")[1])) + assert not check_port_open("127.0.0.1", int(cluster.address.split(":")[1])) class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): + num_total_gpus = None + num_gpus_per_spark_task = None + def test_gpu_allocation(self): - if self.num_gpus_per_spark_task == 0: - pytest.skip("Skip GPU test on cluster without GPU configured.") for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: @@ -89,7 +87,7 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - @ray.remote(num_gpus=1) + @ray.remote(num_cpus=1, num_gpus=1) def f(_): # Add a sleep to avoid the task finishing too fast, # so that it can make all ray tasks concurrently running in all idle task slots. @@ -99,7 +97,7 @@ def f(_): for gpu_id in os.environ['CUDA_VISIBLE_DEVICES'].split(",") ] - futures = [f.remote(i) for i in range(self.max_spark_tasks)] + futures = [f.remote(i) for i in range(self.num_total_gpus)] results = ray.get(futures) merged_results = functools.reduce(lambda x, y: x + y, results) # Test all ray tasks are assigned with different GPUs. From e3463eaf76810f5236340f87d1b1f4fc22c3eeb3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 14:44:24 +0800 Subject: [PATCH 018/121] add param Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 40 +++++++++++++++++++++++------------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index ddb8ae628909..30223d6d9463 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -1,4 +1,5 @@ import os +import shutil import subprocess import sys import time @@ -47,11 +48,12 @@ class RayClusterOnSpark: It can be used to shutdown the cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, ray_context): + def __init__(self, address, head_proc, spark_job_group_id, ray_context, ray_head_temp_dir): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id self.ray_context = ray_context + self.ray_head_temp_dir = ray_head_temp_dir def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -63,6 +65,7 @@ def shutdown(self): self.ray_context.disconnect() self._cancel_background_spark_job() self.head_proc.kill() + shutil.rmtree(self.ray_head_temp_dir, ignore_errors=True) def __enter__(self): return self @@ -79,7 +82,8 @@ def init_cluster( num_spark_tasks, head_options=None, worker_options=None, - ray_node_log_root_path="/tmp/ray/logs" + ray_temp_dir="/tmp/ray/temp", + ray_node_log_dir="/tmp/ray/logs" ): """ Initialize a ray cluster on the spark cluster, via creating a background spark barrier @@ -92,6 +96,11 @@ def init_cluster( can use. head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. + ray_temp_dir: A local disk path to store the ray temporary data. + ray_node_log_dir: A local disk path to store the ray head / worker nodes logs. + On databricks runtime, we recommend to use path under `/dbfs/` that is mounted + with DBFS shared by all spark cluster nodes, so that we can check all ray worker + logs from driver side easily. """ import ray from pyspark.util import inheritable_thread_target @@ -112,24 +121,26 @@ def init_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_head_tmp_dir = f"/tmp/ray/ray-temp/ray-head-port-{ray_head_port}" - - ray_node_log_path = os.path.join(ray_node_log_root_path, f"cluster-{ray_head_hostname}-{ray_head_port}") + ray_node_log_dir = os.path.join(ray_node_log_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_path}") - if is_in_databricks_runtime() and not ray_node_log_root_path.startswith("/dbfs"): + if is_in_databricks_runtime() and not ray_node_log_dir.startswith("/dbfs"): logging.warning( "We recommend you to set `ray_node_log_root_path` argument to be a path under '/dbfs/', " "because for all spark cluster nodes '/dbfs/' path is mounted with a shared disk, " "so that you can check ray worker logs on spark driver node." ) - os.makedirs(ray_node_log_path, exist_ok=True) + os.makedirs(ray_node_log_dir, exist_ok=True) + + ray_temp_dir = os.path.join(ray_temp_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") + ray_head_temp_dir = os.path.join(ray_temp_dir, "head") + os.makedirs(ray_head_temp_dir, exist_ok=True) ray_head_node_cmd = [ ray_exec_path, "start", - f"--temp-dir={ray_head_tmp_dir}", + f"--temp-dir={ray_head_temp_dir}", "--block", "--head", "--disable-usage-stats", @@ -145,7 +156,7 @@ def init_cluster( logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - with open(os.path.join(ray_node_log_path, "ray-head.log"), "w", buffering=1) as head_log_fp: + with open(os.path.join(ray_node_log_dir, "ray-head.log"), "w", buffering=1) as head_log_fp: ray_head_proc = exec_cmd( ray_head_node_cmd, synchronous=False, @@ -171,13 +182,13 @@ def ray_cluster_job_mapper(_): task_id = context.partitionId() # TODO: remove temp dir when ray worker exits. - ray_worker_tmp_dir = f"/tmp/ray/ray-temp/ray-worker-{task_id}-head-{ray_head_hostname}-{ray_head_port}" - os.makedirs(ray_worker_tmp_dir, exist_ok=True) + ray_worker_temp_dir = os.path.join(ray_temp_dir, f"worker-{task_id}") + os.makedirs(ray_worker_temp_dir, exist_ok=True) ray_worker_cmd = [ ray_exec_path, "start", - f"--temp-dir={ray_worker_tmp_dir}", + f"--temp-dir={ray_worker_temp_dir}", f"--num-cpus={num_spark_task_cpus}", "--block", "--disable-usage-stats", @@ -236,7 +247,7 @@ def setup_sigterm_on_parent_death(): logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") ray_worker_log_file = os.path.join( - ray_node_log_path, + ray_node_log_dir, f"ray-worker-{task_id}.log" ) with open(ray_worker_log_file, "w", buffering=1) as worker_log_fp: @@ -294,8 +305,9 @@ def backgroud_job_thread_fn(): head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, ray_context=ray_context, + ray_head_temp_dir=ray_head_temp_dir, ) except Exception: - # If init cluster raise exception, kill the ray head proc. + # If init ray cluster raise exception, kill the ray head proc. ray_head_proc.kill() raise From ef8dbf25c3fedf6c1b5574957355a3638d74f90e Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 15:31:20 +0800 Subject: [PATCH 019/121] update test Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_spark.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_spark.py index 2674a43bfc53..e08751356b42 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_spark.py @@ -38,7 +38,7 @@ def get_ray_worker_resources_list(): def test_cpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): - time.sleep(2) + time.sleep(5) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -56,7 +56,7 @@ def f(x): def test_ray_cluster_shutdown(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - time.sleep(2) + time.sleep(5) assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # cancel background spark job will cause all ray worker nodes exit. @@ -64,7 +64,7 @@ def test_ray_cluster_shutdown(self): time.sleep(5) assert len(self.get_ray_worker_resources_list()) == 0 - time.sleep(2) # wait ray head node exit. + time.sleep(3) # wait ray head node exit. # assert ray head node exit by checking head port being closed. assert not check_port_open("127.0.0.1", int(cluster.address.split(":")[1])) @@ -78,7 +78,7 @@ def test_gpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: - time.sleep(2) + time.sleep(5) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -87,6 +87,8 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + time.sleep(5) + @ray.remote(num_cpus=1, num_gpus=1) def f(_): # Add a sleep to avoid the task finishing too fast, From cebae2d1200ce84a85e519fc366899c5bf31a847 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 15:47:24 +0800 Subject: [PATCH 020/121] update setup-dev Signed-off-by: Weichen Xu --- python/ray/setup-dev.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/setup-dev.py b/python/ray/setup-dev.py index b62d2a800f1a..a8e42c24007e 100755 --- a/python/ray/setup-dev.py +++ b/python/ray/setup-dev.py @@ -101,6 +101,7 @@ def do_link(package, force=False, skip_list=None, local_path=None): do_link("workflow", force=args.yes, skip_list=args.skip) do_link("dag", force=args.yes, skip_list=args.skip) do_link("widgets", force=args.yes, skip_list=args.skip) + do_link("spark", force=args.yes, skip_list=args.skip) do_link("cluster_utils.py", force=args.yes, skip_list=args.skip) do_link("_private", force=args.yes, skip_list=args.skip) # Link package's `dashboard` directly to local (repo's) dashboard. From d127e34b99792607f7fd74b8b848a4f7ab36eb41 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 15:50:47 +0800 Subject: [PATCH 021/121] fix Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 30223d6d9463..a141868e00a4 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -123,7 +123,7 @@ def init_cluster( ray_node_log_dir = os.path.join(ray_node_log_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") - logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_path}") + logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_dir}") if is_in_databricks_runtime() and not ray_node_log_dir.startswith("/dbfs"): logging.warning( "We recommend you to set `ray_node_log_root_path` argument to be a path under '/dbfs/', " From 02dbc155acf9fa5fdbac1f5ba9b6ce8a14e3c237 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 16:47:35 +0800 Subject: [PATCH 022/121] add logger Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index a141868e00a4..15bdede2002b 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -22,6 +22,8 @@ if not sys.platform.startswith("linux"): raise RuntimeError("Ray on spark ony supports linux system.") +_logger = logging.getLogger("ray.spark") + _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: import pyspark @@ -112,7 +114,7 @@ def init_cluster( ray_head_hostname = get_spark_driver_hostname(spark) ray_head_port = get_safe_port(ray_head_hostname) - logging.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) @@ -123,9 +125,9 @@ def init_cluster( ray_node_log_dir = os.path.join(ray_node_log_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") - logging.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_dir}") + _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_dir}") if is_in_databricks_runtime() and not ray_node_log_dir.startswith("/dbfs"): - logging.warning( + _logger.warning( "We recommend you to set `ray_node_log_root_path` argument to be a path under '/dbfs/', " "because for all spark cluster nodes '/dbfs/' path is mounted with a shared disk, " "so that you can check ray worker logs on spark driver node." @@ -154,7 +156,7 @@ def init_cluster( *_convert_ray_node_options(head_options) ] - logging.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") + _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") with open(os.path.join(ray_node_log_dir, "ray-head.log"), "w", buffering=1) as head_log_fp: ray_head_proc = exec_cmd( @@ -172,10 +174,11 @@ def init_cluster( error_on_failure="Start Ray head node failed!" ) - logging.info("Ray head node started.") + _logger.info("Ray head node started.") def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext + _worker_logger = logging.getLogger("ray.spark.worker") context = BarrierTaskContext.get() context.barrier() @@ -198,8 +201,6 @@ def ray_cluster_job_mapper(_): *_convert_ray_node_options(worker_options) ] - logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - ray_worker_extra_envs = {} if num_spark_task_gpus > 0: @@ -239,12 +240,12 @@ def setup_sigterm_on_parent_death(): # Set the parent process death signal of the command process to SIGTERM. libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h except OSError as e: - logging.warning(f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}.") + _worker_logger.warning(f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}.") else: setup_sigterm_on_parent_death = None - logging.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") ray_worker_log_file = os.path.join( ray_node_log_dir, @@ -295,7 +296,7 @@ def backgroud_job_thread_fn(): try: get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) except Exception: - logging.warning( + _logger.warning( "Register ray cluster spark job as background job failed. You need to manually " "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " "python REPL." From 5f5649c7eeeb9808d9237ed91014147a185e462f Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 17:16:40 +0800 Subject: [PATCH 023/121] add tests Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 2 +- python/ray/spark/utils.py | 23 +++++++++----- .../{test_spark.py => test_ray_on_spark.py} | 3 +- python/ray/tests/spark/test_utils.py | 31 +++++++++++++++++++ 4 files changed, 49 insertions(+), 10 deletions(-) rename python/ray/tests/spark/{test_spark.py => test_ray_on_spark.py} (98%) create mode 100644 python/ray/tests/spark/test_utils.py diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 15bdede2002b..c27abbda7b36 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -204,7 +204,7 @@ def ray_cluster_job_mapper(_): ray_worker_extra_envs = {} if num_spark_task_gpus > 0: - available_physical_gpus = get_spark_task_assigned_physical_gpus(context) + available_physical_gpus = get_spark_task_assigned_physical_gpus(context.resources()) ray_worker_cmd.append( f"--num-gpus={len(available_physical_gpus)}", ) diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index e5fd62012cda..2916b369aa21 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -197,6 +197,12 @@ def _get_cpu_cores(): return multiprocessing.cpu_count() +def _calc_mem_per_ray_worker(num_task_slots, physical_mem_bytes, shared_mem_bytes): + ray_worker_object_store_bytes = int(shared_mem_bytes / num_task_slots * 0.8) + ray_worker_heap_mem_bytes = int((physical_mem_bytes - shared_mem_bytes) / num_task_slots * 0.8) + return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes + + def get_avail_mem_per_ray_worker(spark): """ Return the available heap memory and object store memory for each ray worker. @@ -213,9 +219,11 @@ def mapper(_): physical_mem_bytes = _get_total_phyisical_memory() shared_mem_bytes = _get_total_shared_memory() - ray_worker_object_store_bytes = int(shared_mem_bytes / num_task_slots * 0.8) - ray_worker_heap_mem_bytes = int((physical_mem_bytes - shared_mem_bytes) / num_task_slots * 0.8) - + ray_worker_heap_mem_bytes, ray_worker_object_store_bytes = _calc_mem_per_ray_worker( + num_task_slots, + physical_mem_bytes, + shared_mem_bytes, + ) return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes, None except Exception as e: return -1, -1, repr(e) @@ -228,15 +236,14 @@ def mapper(_): return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes -def get_spark_task_assigned_physical_gpus(task_context): - resources = task_context.resources() - if "gpu" not in resources: +def get_spark_task_assigned_physical_gpus(task_resources): + if "gpu" not in task_resources: raise RuntimeError( "Couldn't get the gpu id, Please check the GPU resource configuration" ) - gpu_addr_list = [int(addr.strip()) for addr in resources["gpu"].addresses] + gpu_addr_list = [int(addr.strip()) for addr in task_resources["gpu"].addresses] - if is_in_databricks_runtime(): + if 'CUDA_VISIBLE_DEVICES' in os.environ: visible_cuda_dev_list = [ int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") ] diff --git a/python/ray/tests/spark/test_spark.py b/python/ray/tests/spark/test_ray_on_spark.py similarity index 98% rename from python/ray/tests/spark/test_spark.py rename to python/ray/tests/spark/test_ray_on_spark.py index e08751356b42..9ae5600462db 100644 --- a/python/ray/tests/spark/test_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -66,7 +66,8 @@ def test_ray_cluster_shutdown(self): time.sleep(3) # wait ray head node exit. # assert ray head node exit by checking head port being closed. - assert not check_port_open("127.0.0.1", int(cluster.address.split(":")[1])) + hostname, port = cluster.address.split(":") + assert not check_port_open(hostname, int(port)) class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py new file mode 100644 index 000000000000..427b0c1a2fdd --- /dev/null +++ b/python/ray/tests/spark/test_utils.py @@ -0,0 +1,31 @@ +from unittest.mock import patch +import os + +from ray.spark import ( + get_spark_task_assigned_physical_gpus, + get_spark_driver_hostname, +) + +from ray.spark.utils import _calc_mem_per_ray_worker + + +def test_get_spark_task_assigned_physical_gpus(): + with patch.dict(os.environ, {}, clear=True): + assert get_spark_task_assigned_physical_gpus({"gpu": [2, 5]}) == [2, 5] + + with patch.dict(os.environ, {"CUDA_VISIBLE_DEVICES": "2,3,6"}, clear=True): + assert get_spark_task_assigned_physical_gpus({"gpu": [0, 1]}) == [2, 3] + assert get_spark_task_assigned_physical_gpus({"gpu": [0, 2]}) == [2, 6] + + +def test_get_spark_driver_hostname(): + assert get_spark_driver_hostname("local") == "127.0.0.1" + assert get_spark_driver_hostname("local[4]") == "127.0.0.1" + assert get_spark_driver_hostname("local-cluster[1, 4, 1024]") == "127.0.0.1" + assert get_spark_driver_hostname("spark://23.195.26.187:7077") == "23.195.26.187" + assert get_spark_driver_hostname("spark://aa.xx.yy:7077") == "aa.xx.yy" + + +def test_calc_mem_per_ray_worker(): + assert _calc_mem_per_ray_worker(4, 1000000, 400000) == (120000, 80000) + assert _calc_mem_per_ray_worker(6, 1000000, 400000) == (80000, 53333) From df85ef140b698e07d2f98ebb9856409c8058b0e6 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 21:05:54 +0800 Subject: [PATCH 024/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 118 ++++++++++++++++++++++++--- python/ray/spark/utils.py | 13 +-- python/ray/tests/spark/test_utils.py | 6 +- 3 files changed, 112 insertions(+), 25 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index c27abbda7b36..3713dc88bb56 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -5,6 +5,7 @@ import time import threading import logging +import math from packaging.version import Version from .utils import ( @@ -17,6 +18,7 @@ get_spark_task_assigned_physical_gpus, get_avail_mem_per_ray_worker, get_dbutils, + get_max_num_concurrent_tasks, ) if not sys.platform.startswith("linux"): @@ -81,11 +83,15 @@ def _convert_ray_node_options(options): def init_cluster( - num_spark_tasks, - head_options=None, - worker_options=None, - ray_temp_dir="/tmp/ray/temp", - ray_node_log_dir="/tmp/ray/logs" + num_spark_tasks=None, + total_cpus=None, + total_gpus=None, + total_heap_memory_bytes=None, + total_object_store_memory_bytes=None, + head_options=None, + worker_options=None, + ray_temp_dir="/tmp/ray/temp", + ray_node_log_dir="/tmp/ray/logs" ): """ Initialize a ray cluster on the spark cluster, via creating a background spark barrier @@ -94,8 +100,18 @@ def init_cluster( Args num_spark_tasks: Specify the spark task number the spark job will create. - This argument controls how many resources (CPU / GPU / memory) the ray cluster - can use. + This argument represents how many concurrent spark tasks it will use to create the + ray cluster, and the ray cluster total available resources (CPU / GPU / memory) + equals to the resources allocated to these spark tasks. + You can specify num_spark_tasks to -1, representing the ray cluster uses all + available spark tasks slots, if you want to create a shared ray cluster + and use the whole spark cluster resources, simply set it to -1. + total_cpus: Specify the total cpus resources the ray cluster requests. + total_gpus: Specify the total gpus resources the ray cluster requests. + total_heap_memory_bytes: Specify the total heap memory resources (in bytes) + the ray cluster requests. + total_object_store_memory_bytes: Specify the total object store memory resources (in bytes) + the ray cluster requests. head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_dir: A local disk path to store the ray temporary data. @@ -110,17 +126,85 @@ def init_cluster( head_options = head_options or {} worker_options = worker_options or {} - spark = get_spark_session() - ray_head_hostname = get_spark_driver_hostname(spark) - ray_head_port = get_safe_port(ray_head_hostname) + num_spark_tasks_specified = num_spark_tasks is not None + total_resources_req_specified = ( + total_cpus is not None or + total_gpus is not None or + total_heap_memory_bytes is not None or + total_object_store_memory_bytes is not None + ) - _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + if (num_spark_tasks_specified and total_resources_req_specified) or \ + (not num_spark_tasks_specified and not total_resources_req_specified): + raise ValueError( + "You should specify either 'num_spark_tasks' argument or argument group of " + "'total_cpus', 'total_gpus', 'total_heap_memory_bytes' and 'total_object_store_memory_bytes'." + ) + + spark = get_spark_session() num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) - ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes = get_avail_mem_per_ray_worker(spark) + if total_gpus is not None and num_spark_task_gpus == 0: + raise ValueError( + "The spark cluster is without GPU configuration, so you cannot specify 'total_gpus' " + "argument" + ) + + if num_spark_tasks is not None: + if num_spark_tasks == -1: + # num_spark_tasks=-1 represents using all spark task slots + num_spark_tasks = get_max_num_concurrent_tasks(spark.sparkContext) + elif num_spark_tasks <= 0: + raise ValueError( + "You should specify 'num_spark_tasks' argument to a positive integer or -1." + ) + else: + num_spark_tasks = 1 + if total_cpus is not None: + if total_cpus <= 0: + raise ValueError( + "You should specify 'total_cpus' argument to a positive integer." + ) + num_spark_tasks_for_cpus_req = int(math.ceil(total_cpus / num_spark_task_cpus)) + if num_spark_tasks_for_cpus_req > num_spark_tasks: + num_spark_tasks = num_spark_tasks_for_cpus_req + + if total_gpus is not None: + if total_gpus <= 0: + raise ValueError( + "You should specify 'total_gpus' argument to a positive integer." + ) + num_spark_tasks_for_gpus_req = int(math.ceil(total_gpus / num_spark_task_gpus)) + if num_spark_tasks_for_gpus_req > num_spark_tasks: + num_spark_tasks = num_spark_tasks_for_gpus_req + + if total_heap_memory_bytes is not None: + if total_heap_memory_bytes <= 0: + raise ValueError( + "You should specify 'total_heap_memory_bytes' argument to a positive integer." + ) + num_spark_tasks_for_heap_mem_req = int(math.ceil(total_heap_memory_bytes / ray_worker_heap_mem_bytes)) + if num_spark_tasks_for_heap_mem_req > num_spark_tasks: + num_spark_tasks = num_spark_tasks_for_heap_mem_req + + if total_object_store_memory_bytes is not None: + if total_object_store_memory_bytes <= 0: + raise ValueError( + "You should specify 'total_object_store_memory_bytes' argument to a positive integer." + ) + num_spark_tasks_for_object_store_mem_req = \ + int(math.ceil(total_object_store_memory_bytes / ray_worker_object_store_mem_bytes)) + if num_spark_tasks_for_object_store_mem_req > num_spark_tasks: + num_spark_tasks = num_spark_tasks_for_object_store_mem_req + + ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) + ray_head_port = get_safe_port(ray_head_hostname) + + _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") ray_node_log_dir = os.path.join(ray_node_log_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") @@ -204,7 +288,15 @@ def ray_cluster_job_mapper(_): ray_worker_extra_envs = {} if num_spark_task_gpus > 0: - available_physical_gpus = get_spark_task_assigned_physical_gpus(context.resources()) + task_resources = context.resources() + + if "gpu" not in task_resources: + raise RuntimeError( + "Couldn't get the gpu id, Please check the GPU resource configuration" + ) + gpu_addr_list = [int(addr.strip()) for addr in task_resources["gpu"].addresses] + + available_physical_gpus = get_spark_task_assigned_physical_gpus(gpu_addr_list) ray_worker_cmd.append( f"--num-gpus={len(available_physical_gpus)}", ) diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 2916b369aa21..946f489b6ca8 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -159,8 +159,7 @@ def get_spark_session(): return SparkSession.builder.getOrCreate() -def get_spark_driver_hostname(spark): - spark_master_url = spark.conf.get("spark.master") +def get_spark_driver_hostname(spark_master_url): if spark_master_url.lower().startswith("local"): return "127.0.0.1" else: @@ -228,6 +227,8 @@ def mapper(_): except Exception as e: return -1, -1, repr(e) + # running inferring memory routine on spark executor side. + # because spark worker node might have different machine shape with spark driver node. inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes, err = \ spark.sparkContext.parallelize([1], 1).map(mapper).collect()[0] @@ -236,13 +237,7 @@ def mapper(_): return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes -def get_spark_task_assigned_physical_gpus(task_resources): - if "gpu" not in task_resources: - raise RuntimeError( - "Couldn't get the gpu id, Please check the GPU resource configuration" - ) - gpu_addr_list = [int(addr.strip()) for addr in task_resources["gpu"].addresses] - +def get_spark_task_assigned_physical_gpus(gpu_addr_list): if 'CUDA_VISIBLE_DEVICES' in os.environ: visible_cuda_dev_list = [ int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index 427b0c1a2fdd..a104fd9fd0fc 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -11,11 +11,11 @@ def test_get_spark_task_assigned_physical_gpus(): with patch.dict(os.environ, {}, clear=True): - assert get_spark_task_assigned_physical_gpus({"gpu": [2, 5]}) == [2, 5] + assert get_spark_task_assigned_physical_gpus([2, 5]) == [2, 5] with patch.dict(os.environ, {"CUDA_VISIBLE_DEVICES": "2,3,6"}, clear=True): - assert get_spark_task_assigned_physical_gpus({"gpu": [0, 1]}) == [2, 3] - assert get_spark_task_assigned_physical_gpus({"gpu": [0, 2]}) == [2, 6] + assert get_spark_task_assigned_physical_gpus([0, 1]) == [2, 3] + assert get_spark_task_assigned_physical_gpus([0, 2]) == [2, 6] def test_get_spark_driver_hostname(): From 0119fe52995ce98e207258669ca02c1e2ebfe321 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 21:18:40 +0800 Subject: [PATCH 025/121] add warning Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 3713dc88bb56..28c13ff7b662 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -200,6 +200,23 @@ def init_cluster( if num_spark_tasks_for_object_store_mem_req > num_spark_tasks: num_spark_tasks = num_spark_tasks_for_object_store_mem_req + if num_spark_task_cpus < 4: + _logger.warning( + f"Each ray worker node will be assigned with {num_spark_task_cpus} CPU cores, less than " + "recommended value 4, because ray worker node cpu cors aligns with the cpu cores assigned to " + "a spark task, you can increase 'spark.task.cpus' config value to address it." + ) + + if ray_worker_heap_mem_bytes < 10 * 1024 * 1024 * 1024: + _logger.warning( + f"Each ray worker node will be assigned with {ray_worker_heap_mem_bytes} bytes heap memory, " + "less than recommended value 10GB, the ray worker node heap memory size is calculated by " + "(SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / num_local_spark_task_slots * 0.8, " + "so you can increase spark cluster worker machine memory, or reduce spark task slots " + "number on spark cluster worker, or reduce spark worker machine /dev/shm quota to " + "address it." + ) + ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) ray_head_port = get_safe_port(ray_head_hostname) From cab7ff0c55040891fdee46a5e944533acc61b92e Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 22:14:01 +0800 Subject: [PATCH 026/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 28c13ff7b662..d4c16ba469c9 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -289,6 +289,10 @@ def ray_cluster_job_mapper(_): ray_worker_temp_dir = os.path.join(ray_temp_dir, f"worker-{task_id}") os.makedirs(ray_worker_temp_dir, exist_ok=True) + # Ray worker might run on a machine different with the head node, so create the + # local log dir again. + os.makedirs(ray_node_log_dir, exist_ok=True) + ray_worker_cmd = [ ray_exec_path, "start", From 5869b82ecd13b3afa0658e4c19a41b15af4fb6ed Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 10 Oct 2022 22:36:46 +0800 Subject: [PATCH 027/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d4c16ba469c9..76d781b874a1 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -224,7 +224,7 @@ def init_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_node_log_dir = os.path.join(ray_node_log_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") + ray_node_log_dir = os.path.join(ray_node_log_dir, f"ray-{ray_head_port}") _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_dir}") if is_in_databricks_runtime() and not ray_node_log_dir.startswith("/dbfs"): @@ -236,7 +236,7 @@ def init_cluster( os.makedirs(ray_node_log_dir, exist_ok=True) - ray_temp_dir = os.path.join(ray_temp_dir, f"cluster-{ray_head_hostname}-{ray_head_port}") + ray_temp_dir = os.path.join(ray_temp_dir, f"ray-{ray_head_port}") ray_head_temp_dir = os.path.join(ray_temp_dir, "head") os.makedirs(ray_head_temp_dir, exist_ok=True) From 98a7d6219b46ba78031356cc087023c5bfa5910b Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 13:29:05 +0800 Subject: [PATCH 028/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 43 +++++++++++++----------------------- 1 file changed, 15 insertions(+), 28 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 76d781b874a1..29f3a2bf2617 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -91,7 +91,7 @@ def init_cluster( head_options=None, worker_options=None, ray_temp_dir="/tmp/ray/temp", - ray_node_log_dir="/tmp/ray/logs" + ray_log_dir="/tmp/ray/logs" ): """ Initialize a ray cluster on the spark cluster, via creating a background spark barrier @@ -115,10 +115,7 @@ def init_cluster( head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_dir: A local disk path to store the ray temporary data. - ray_node_log_dir: A local disk path to store the ray head / worker nodes logs. - On databricks runtime, we recommend to use path under `/dbfs/` that is mounted - with DBFS shared by all spark cluster nodes, so that we can check all ray worker - logs from driver side easily. + ray_log_dir: A local disk path to store ray processes logs. """ import ray from pyspark.util import inheritable_thread_target @@ -224,26 +221,18 @@ def init_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_node_log_dir = os.path.join(ray_node_log_dir, f"ray-{ray_head_port}") + ray_log_dir = os.path.join(ray_log_dir, f"ray-{ray_head_port}") + os.makedirs(ray_log_dir, exist_ok=True) - _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_node_log_dir}") - if is_in_databricks_runtime() and not ray_node_log_dir.startswith("/dbfs"): - _logger.warning( - "We recommend you to set `ray_node_log_root_path` argument to be a path under '/dbfs/', " - "because for all spark cluster nodes '/dbfs/' path is mounted with a shared disk, " - "so that you can check ray worker logs on spark driver node." - ) - - os.makedirs(ray_node_log_dir, exist_ok=True) + _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_log_dir}") ray_temp_dir = os.path.join(ray_temp_dir, f"ray-{ray_head_port}") - ray_head_temp_dir = os.path.join(ray_temp_dir, "head") - os.makedirs(ray_head_temp_dir, exist_ok=True) + os.makedirs(ray_temp_dir, exist_ok=True) ray_head_node_cmd = [ ray_exec_path, "start", - f"--temp-dir={ray_head_temp_dir}", + f"--temp-dir={ray_temp_dir}", "--block", "--head", "--disable-usage-stats", @@ -259,7 +248,7 @@ def init_cluster( _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - with open(os.path.join(ray_node_log_dir, "ray-head.log"), "w", buffering=1) as head_log_fp: + with open(os.path.join(ray_log_dir, "ray-start-head.log"), "w", buffering=1) as head_log_fp: ray_head_proc = exec_cmd( ray_head_node_cmd, synchronous=False, @@ -285,18 +274,16 @@ def ray_cluster_job_mapper(_): context.barrier() task_id = context.partitionId() - # TODO: remove temp dir when ray worker exits. - ray_worker_temp_dir = os.path.join(ray_temp_dir, f"worker-{task_id}") - os.makedirs(ray_worker_temp_dir, exist_ok=True) - + # TODO: remove worker side ray temp dir when ray worker exits. # Ray worker might run on a machine different with the head node, so create the - # local log dir again. - os.makedirs(ray_node_log_dir, exist_ok=True) + # local log dir and temp dir again. + os.makedirs(ray_temp_dir, exist_ok=True) + os.makedirs(ray_log_dir, exist_ok=True) ray_worker_cmd = [ ray_exec_path, "start", - f"--temp-dir={ray_worker_temp_dir}", + f"--temp-dir={ray_temp_dir}", f"--num-cpus={num_spark_task_cpus}", "--block", "--disable-usage-stats", @@ -361,8 +348,8 @@ def setup_sigterm_on_parent_death(): _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") ray_worker_log_file = os.path.join( - ray_node_log_dir, - f"ray-worker-{task_id}.log" + ray_log_dir, + f"ray-start-worker-{task_id}.log" ) with open(ray_worker_log_file, "w", buffering=1) as worker_log_fp: exec_cmd( From e0d46210659f856bc1af4108cd039b24f91134c7 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 13:50:01 +0800 Subject: [PATCH 029/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 29f3a2bf2617..ce56e42b1ac1 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -52,12 +52,12 @@ class RayClusterOnSpark: It can be used to shutdown the cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, ray_context, ray_head_temp_dir): + def __init__(self, address, head_proc, spark_job_group_id, ray_context, ray_temp_dir): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id self.ray_context = ray_context - self.ray_head_temp_dir = ray_head_temp_dir + self.ray_temp_dir = ray_temp_dir def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -69,7 +69,7 @@ def shutdown(self): self.ray_context.disconnect() self._cancel_background_spark_job() self.head_proc.kill() - shutil.rmtree(self.ray_head_temp_dir, ignore_errors=True) + shutil.rmtree(self.ray_temp_dir, ignore_errors=True) def __enter__(self): return self @@ -248,7 +248,10 @@ def init_cluster( _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - with open(os.path.join(ray_log_dir, "ray-start-head.log"), "w", buffering=1) as head_log_fp: + with open( + os.path.join(ray_log_dir, "ray-start-head.log"), + "w", buffering=1 + ) as head_log_fp: ray_head_proc = exec_cmd( ray_head_node_cmd, synchronous=False, @@ -347,11 +350,10 @@ def setup_sigterm_on_parent_death(): _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - ray_worker_log_file = os.path.join( - ray_log_dir, - f"ray-start-worker-{task_id}.log" - ) - with open(ray_worker_log_file, "w", buffering=1) as worker_log_fp: + with open( + os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), + "w", buffering=1 + ) as worker_log_fp: exec_cmd( ray_worker_cmd, synchronous=True, @@ -406,7 +408,7 @@ def backgroud_job_thread_fn(): head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, ray_context=ray_context, - ray_head_temp_dir=ray_head_temp_dir, + ray_temp_dir=ray_temp_dir, ) except Exception: # If init ray cluster raise exception, kill the ray head proc. From 779abc2202337d7693abf8923e21f085956564f2 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 14:12:53 +0800 Subject: [PATCH 030/121] add connect / disconnect api Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 45 ++++++++++++++++++++++++++---------- 1 file changed, 33 insertions(+), 12 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index ce56e42b1ac1..f7aaf91aa62e 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -49,29 +49,49 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): class RayClusterOnSpark: """ The class is the type of instance returned by `init_cluster` API. - It can be used to shutdown the cluster. + It can be used to connect / disconnect / shut down the cluster. + It can be also used as a python context manager, + when entering the `RayClusterOnSpark` context, connect to the ray cluster, + when exiting the `RayClusterOnSpark` context, disconnect from the ray cluster and + shut down the cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, ray_context, ray_temp_dir): + def __init__(self, address, head_proc, spark_job_group_id, ray_temp_dir): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id - self.ray_context = ray_context self.ray_temp_dir = ray_temp_dir + self.ray_context = None def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) + def connect(self): + import ray + if self.ray_context is None: + # connect to the ray cluster. + self.ray_context = ray.init(address=self.address) + else: + _logger.warning("Already connected to this ray cluster.") + + def disconnect(self): + if self.ray_context is not None: + self.ray_context.disconnect() + self.ray_context = None + else: + _logger.warning("Already disconnected from this ray cluster.") + def shutdown(self): """ Shutdown the ray cluster created by `init_cluster` API. """ - self.ray_context.disconnect() + self.disconnect() self._cancel_background_spark_job() self.head_proc.kill() shutil.rmtree(self.ray_temp_dir, ignore_errors=True) def __enter__(self): + self.connect() return self def __exit__(self, exc_type, exc_val, exc_tb): @@ -94,9 +114,13 @@ def init_cluster( ray_log_dir="/tmp/ray/logs" ): """ - Initialize a ray cluster on the spark cluster, via creating a background spark barrier - mode job and each spark task running a ray worker node, and in spark driver side - a ray head node is started. And then connect to the created ray cluster. + Initialize a ray cluster on the spark cluster, via starting a ray head node + in spark drive side and creating a background spark barrier mode job and each + spark task running a ray worker node, returns an instance of `RayClusterOnSpark` type. + The returned instance can be used to connect / disconnect / shut down the ray cluster. + We can also use `with` statement like `with init_cluster(...):` , when entering + the managed scope, the ray cluster is initiated and connected, and when exiting the + scope, the ray cluster is disconnected and shut down. Args num_spark_tasks: Specify the spark task number the spark job will create. @@ -391,9 +415,6 @@ def backgroud_job_thread_fn(): # Waiting all ray workers spin up. time.sleep(10) - # connect to the ray cluster. - ray_context = ray.init(address=f"{ray_head_hostname}:{ray_head_port}") - if is_in_databricks_runtime(): try: get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) @@ -407,10 +428,10 @@ def backgroud_job_thread_fn(): address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, - ray_context=ray_context, ray_temp_dir=ray_temp_dir, ) except Exception: - # If init ray cluster raise exception, kill the ray head proc. + # If init ray cluster raise exception, kill the ray head proc and the background spark job. ray_head_proc.kill() + spark.sparkContext.cancelJobGroup(spark_job_group_id) raise From 597059d1a4ee625ff9e03b813bb75bd4464d39ba Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 16:03:55 +0800 Subject: [PATCH 031/121] add comment Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 120 +++++++++++++++++++++++++++-------- 1 file changed, 93 insertions(+), 27 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index f7aaf91aa62e..ae11ec95585e 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -62,6 +62,7 @@ def __init__(self, address, head_proc, spark_job_group_id, ray_temp_dir): self.spark_job_group_id = spark_job_group_id self.ray_temp_dir = ray_temp_dir self.ray_context = None + self.is_shutdown = False def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -76,7 +77,11 @@ def connect(self): def disconnect(self): if self.ray_context is not None: - self.ray_context.disconnect() + try: + self.ray_context.disconnect() + except Exception as e: + # swallow exception. + _logger.warning(f"Error happens during disconnecting: {repr(e)}") self.ray_context = None else: _logger.warning("Already disconnected from this ray cluster.") @@ -85,10 +90,27 @@ def shutdown(self): """ Shutdown the ray cluster created by `init_cluster` API. """ - self.disconnect() - self._cancel_background_spark_job() - self.head_proc.kill() - shutil.rmtree(self.ray_temp_dir, ignore_errors=True) + if not self.is_shutdown: + if self.ray_context is not None: + self.disconnect() + try: + self._cancel_background_spark_job() + except Exception as e: + # swallow exception. + _logger.warning( + f"Error happens during cancelling ray cluster background spark job: {repr(e)}" + ) + try: + self.head_proc.kill() + except Exception as e: + # swallow exception. + _logger.warning( + f"Error happens during killing ray head node: {repr(e)}" + ) + shutil.rmtree(self.ray_temp_dir, ignore_errors=True) + self.is_shutdown = True + else: + _logger.warning("The cluster has been shut down.") def __enter__(self): self.connect() @@ -250,6 +272,13 @@ def init_cluster( _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_log_dir}") + # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/logs", + # We should update "ray start" scirpt to add a new option "ray_log_dir", and output logs + # to a different directory specified by "ray_log_dir", instead of using "{ray_temp_dir}/logs", + # The reason is, for ray on spark, user is hard to access log files on spark worker machines, + # (especially on databricks runtime), so we'd better set the log output dir to be a + # path mounted with NFS shared by all spark cluster nodes, so that the user can access + # these remote log files from spark drive side easily. ray_temp_dir = os.path.join(ray_temp_dir, f"ray-{ray_head_port}") os.makedirs(ray_temp_dir, exist_ok=True) @@ -293,6 +322,27 @@ def init_cluster( _logger.info("Ray head node started.") + # NB: + # In order to start ray worker nodes on spark cluster worker machines, + # We launch a background spark job: + # 1. it is a barrier mode spark job, i.e. all spark tasks in the job runs concurrently. + # if the spark cluster resources are not sufficient to launch all these tasks concurrently, + # the spark job will hang and retry, if exceeding maximum retries, it will fail. + # 2. Each spark task launches one ray worker node. This design ensures all ray worker nodes + # has the same shape (same cpus / gpus / memory configuration). If ray worker nodes have + # different shape, the Ray cluster setup will be nondeterministic, and you could get very + # strange results with bad luck on the node sizing. + # 3. It starts ray worker node via `ray start` CLI. In each spark task, it creates a + # child process and run `ray start ...` command in blocking mode. + # 4. When shut down ray cluster, killing these ray worker nodes is implemented by: + # First, it installs a PR_SET_PDEATHSIG signal for the `ray start ...` child processes + # so that when parent process (pyspark task) dead, the child processes + # (`ray start ...` processes) will receive SIGTERM signal. + # When we need to shut down the ray cluster, call `sparkContext.cancelJobGroup` + # to cancel the background spark job, and it sends SIGKILL signal to all spark tasks, + # so this make spark task processes dead and triggers sending SIGTERM to `ray start ...` + # child processes. + def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext _worker_logger = logging.getLogger("ray.spark.worker") @@ -355,7 +405,7 @@ def setup_sigterm_on_parent_death(): This is a no-op on macOS because prctl is not supported. Note: - When a pyspark job canceled, the UDF python process are killed by signal "SIGKILL", + When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", This case neither "atexit" nor signal handler can capture SIGKILL signal. prctl is the only way to capture SIGKILL signal. """ @@ -392,20 +442,40 @@ def setup_sigterm_on_parent_death(): # NB: Not reachable. yield 0 - def backgroud_job_thread_fn(): - spark.sparkContext.setJobGroup( - spark_job_group_id, - "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_hostname}:{ray_head_port}" - ) - spark.sparkContext.parallelize( - list(range(num_spark_tasks)), num_spark_tasks - ).barrier().mapPartitions( - ray_cluster_job_mapper - ).collect() - spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + ray_cluster_handler = RayClusterOnSpark( + address=f"{ray_head_hostname}:{ray_head_port}", + head_proc=ray_head_proc, + spark_job_group_id=spark_job_group_id, + ray_temp_dir=ray_temp_dir, + ) + + def backgroud_job_thread_fn(): + try: + spark.sparkContext.setJobGroup( + spark_job_group_id, + "This job group is for spark job which runs the Ray cluster with ray head node " + f"{ray_head_hostname}:{ray_head_port}" + ) + spark.sparkContext.parallelize( + list(range(num_spark_tasks)), num_spark_tasks + ).barrier().mapPartitions( + ray_cluster_job_mapper + ).collect() + finally: + # NB: + # The background spark job is designed to running forever until it is killed, + # So this `finally` block is reachable only when: + # 1. The background job raises unexpected exception (i.e. ray worker nodes failed unexpectedly) + # 2. User explicitly orders shutting down the ray cluster. + # 3. On databricks runtime, when notebook detached, it triggers python REPL onCancel event and + # it cancelled the background running spark job + # For case 1 and 3, only ray workers are killed, but driver side ray head might be still + # running, and ray context might be in connected status, we need to disconnect and kill ray + # head node, so call `ray_cluster_handler.shutdown()` here. + ray_cluster_handler.shutdown() + threading.Thread( target=inheritable_thread_target(backgroud_job_thread_fn), args=() @@ -424,14 +494,10 @@ def backgroud_job_thread_fn(): "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " "python REPL." ) - return RayClusterOnSpark( - address=f"{ray_head_hostname}:{ray_head_port}", - head_proc=ray_head_proc, - spark_job_group_id=spark_job_group_id, - ray_temp_dir=ray_temp_dir, - ) + return ray_cluster_handler except Exception: - # If init ray cluster raise exception, kill the ray head proc and the background spark job. - ray_head_proc.kill() - spark.sparkContext.cancelJobGroup(spark_job_group_id) + # If driver side setup ray-cluster routine raises exception, it might result in part of ray + # processes has been launched (e.g. ray head or some ray workers have been launched), + # calling `ray_cluster_handler.shutdown()` to kill them and clean status. + ray_cluster_handler.shutdown() raise From 8fffcf220d0e40bdf04832817796569d0b71127e Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 16:17:01 +0800 Subject: [PATCH 032/121] add test Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 25 +++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 9ae5600462db..9ce1fd95785f 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -1,7 +1,8 @@ import os from abc import ABC -import pytest +from unittest.mock import patch + import ray from ray import spark as ray_spark @@ -59,9 +60,14 @@ def test_ray_cluster_shutdown(self): time.sleep(5) assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks - # cancel background spark job will cause all ray worker nodes exit. - cluster._cancel_background_spark_job() - time.sleep(5) + # Test: cancel background spark job will cause all ray worker nodes exit. + def fake_shutdown(_): + pass + + with patch.object(ray_spark.RayClusterOnSpark, fake_shutdown): + cluster._cancel_background_spark_job() + time.sleep(5) + assert len(self.get_ray_worker_resources_list()) == 0 time.sleep(3) # wait ray head node exit. @@ -69,6 +75,17 @@ def test_ray_cluster_shutdown(self): hostname, port = cluster.address.split(":") assert not check_port_open(hostname, int(port)) + def test_background_spark_job_exit_trigger_ray_head_exit(self): + with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + time.sleep(5) + + cluster._cancel_background_spark_job() + time.sleep(5) + + # assert ray head node exit by checking head port being closed. + hostname, port = cluster.address.split(":") + assert not check_port_open(hostname, int(port)) + class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): From 3ef3cf5318b48c4c26e6c80fa5f88d1380ce957f Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 17:10:14 +0800 Subject: [PATCH 033/121] update comment Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 14 +++++++++----- python/ray/tests/spark/test_ray_on_spark.py | 2 +- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index ae11ec95585e..d1f51ab7d837 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -161,7 +161,7 @@ def init_cluster( head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_dir: A local disk path to store the ray temporary data. - ray_log_dir: A local disk path to store ray processes logs. + ray_log_dir: A local disk path to store "ray start" script logs. """ import ray from pyspark.util import inheritable_thread_target @@ -270,11 +270,15 @@ def init_cluster( ray_log_dir = os.path.join(ray_log_dir, f"ray-{ray_head_port}") os.makedirs(ray_log_dir, exist_ok=True) - _logger.warning(f"You can check ray head / worker nodes logs under local disk path {ray_log_dir}") + _logger.warning( + f"You can check ray head / worker starting script logs under local disk path {ray_log_dir}, " + f"and you can check ray processes logs under local disk path {ray_temp_dir}/session_latest/logs." + ) - # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/logs", - # We should update "ray start" scirpt to add a new option "ray_log_dir", and output logs - # to a different directory specified by "ray_log_dir", instead of using "{ray_temp_dir}/logs", + # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/session_latest/logs", + # Proposal: Update "ray start" scirpt to add a new option "ray_log_dir", and output logs + # to a different directory specified by "ray_log_dir", instead of using + # "{ray_temp_dir}/session_latest/logs". # The reason is, for ray on spark, user is hard to access log files on spark worker machines, # (especially on databricks runtime), so we'd better set the log output dir to be a # path mounted with NFS shared by all spark cluster nodes, so that the user can access diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 9ce1fd95785f..6bd72d2682cc 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -64,7 +64,7 @@ def test_ray_cluster_shutdown(self): def fake_shutdown(_): pass - with patch.object(ray_spark.RayClusterOnSpark, fake_shutdown): + with patch.object(ray_spark.RayClusterOnSpark, "shutdown", fake_shutdown): cluster._cancel_background_spark_job() time.sleep(5) From 8eaecd605cb27bf80f7a041910de6beb7d3b3121 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 17:35:42 +0800 Subject: [PATCH 034/121] use safe port Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d1f51ab7d837..93150833b9e5 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -263,6 +263,9 @@ def init_cluster( ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) ray_head_port = get_safe_port(ray_head_hostname) + ray_head_node_manager_port = get_safe_port(ray_head_hostname) + ray_head_object_manager_port = get_safe_port(ray_head_hostname) + _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") @@ -300,6 +303,8 @@ def init_cluster( f"--memory={128 * 1024 * 1024}", # limit the object store memory usage of head node because no task running on it. f"--object-store-memory={128 * 1024 * 1024}", + f"--node-manager-port={ray_head_node_manager_port}", + f"--object-manager-port={ray_head_object_manager_port}", *_convert_ray_node_options(head_options) ] @@ -352,15 +357,19 @@ def ray_cluster_job_mapper(_): _worker_logger = logging.getLogger("ray.spark.worker") context = BarrierTaskContext.get() - context.barrier() task_id = context.partitionId() + worker_hostname = context.getTaskInfos()[task_id].address.split(":")[0].strip() + # TODO: remove worker side ray temp dir when ray worker exits. # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) + ray_worker_node_manager_port = get_safe_port(ray_head_hostname) + ray_worker_object_manager_port = get_safe_port(ray_head_hostname) + ray_worker_cmd = [ ray_exec_path, "start", @@ -368,9 +377,12 @@ def ray_cluster_job_mapper(_): f"--num-cpus={num_spark_task_cpus}", "--block", "--disable-usage-stats", + "--include-dashboard=false", f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", + f"--node-manager-port={ray_worker_node_manager_port}", + f"--object-manager-port={ray_worker_object_manager_port}", *_convert_ray_node_options(worker_options) ] From 8d4ddd02ea306aa90f818663963113dbf40bf9c4 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 17:36:34 +0800 Subject: [PATCH 035/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 93150833b9e5..d55652b46450 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -367,8 +367,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_safe_port(ray_head_hostname) - ray_worker_object_manager_port = get_safe_port(ray_head_hostname) + ray_worker_node_manager_port = get_safe_port(worker_hostname) + ray_worker_object_manager_port = get_safe_port(worker_hostname) ray_worker_cmd = [ ray_exec_path, From 32a53a071af628bc43114bfa0935f2d8d5e43a34 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 17:44:58 +0800 Subject: [PATCH 036/121] fix Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d55652b46450..5d69b7848124 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -377,7 +377,6 @@ def ray_cluster_job_mapper(_): f"--num-cpus={num_spark_task_cpus}", "--block", "--disable-usage-stats", - "--include-dashboard=false", f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", From a699c629ff2834b840ffa95f990d4f1b24c21cc3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 18:26:55 +0800 Subject: [PATCH 037/121] use random port Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 26 +++++++++++++++----------- python/ray/spark/utils.py | 11 +++++++++++ 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 5d69b7848124..c2ebae617828 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -11,7 +11,7 @@ from .utils import ( exec_cmd, check_port_open, - get_safe_port, + get_random_safe_port, get_spark_session, get_spark_driver_hostname, is_in_databricks_runtime, @@ -69,6 +69,10 @@ def _cancel_background_spark_job(self): def connect(self): import ray + if self.is_shutdown: + raise RuntimeError( + "The ray cluster has been shut down or it failed to start." + ) if self.ray_context is None: # connect to the ray cluster. self.ray_context = ray.init(address=self.address) @@ -261,10 +265,10 @@ def init_cluster( ) ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) - ray_head_port = get_safe_port(ray_head_hostname) + ray_head_port = get_random_safe_port(ray_head_hostname) - ray_head_node_manager_port = get_safe_port(ray_head_hostname) - ray_head_object_manager_port = get_safe_port(ray_head_hostname) + ray_head_node_manager_port = get_random_safe_port(ray_head_hostname) + ray_head_object_manager_port = get_random_safe_port(ray_head_hostname) _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") @@ -273,11 +277,6 @@ def init_cluster( ray_log_dir = os.path.join(ray_log_dir, f"ray-{ray_head_port}") os.makedirs(ray_log_dir, exist_ok=True) - _logger.warning( - f"You can check ray head / worker starting script logs under local disk path {ray_log_dir}, " - f"and you can check ray processes logs under local disk path {ray_temp_dir}/session_latest/logs." - ) - # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/session_latest/logs", # Proposal: Update "ray start" scirpt to add a new option "ray_log_dir", and output logs # to a different directory specified by "ray_log_dir", instead of using @@ -289,6 +288,11 @@ def init_cluster( ray_temp_dir = os.path.join(ray_temp_dir, f"ray-{ray_head_port}") os.makedirs(ray_temp_dir, exist_ok=True) + _logger.warning( + f"You can check ray head / worker starting script logs under local disk path {ray_log_dir}, " + f"and you can check ray processes logs under local disk path {ray_temp_dir}/session_latest/logs." + ) + ray_head_node_cmd = [ ray_exec_path, "start", @@ -367,8 +371,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_safe_port(worker_hostname) - ray_worker_object_manager_port = get_safe_port(worker_hostname) + ray_worker_node_manager_port = get_random_safe_port(worker_hostname) + ray_worker_object_manager_port = get_random_safe_port(worker_hostname) ray_worker_cmd = [ ray_exec_path, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 946f489b6ca8..20cfafc7fb96 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -2,6 +2,8 @@ from urllib.parse import urlparse import os import sys +import random +import time def is_in_databricks_runtime(): @@ -147,6 +149,15 @@ def get_safe_port(ip): return port +def get_random_safe_port(host, min_port, max_port, max_retries=200): + random.seed(int(time.time() * 1000)) + for _ in range(max_retries): + port = random.randint(min_port, max_port) + if not check_port_open(host, port): + return port + raise RuntimeError("Get random safe port failed.") + + def check_port_open(host, port): import socket from contextlib import closing From ce9635a06a17a2ca1aa67047f6e29e10b1915146 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 18:28:35 +0800 Subject: [PATCH 038/121] update Signed-off-by: Weichen Xu --- python/ray/spark/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 20cfafc7fb96..5f4090790cb6 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -149,7 +149,7 @@ def get_safe_port(ip): return port -def get_random_safe_port(host, min_port, max_port, max_retries=200): +def get_random_safe_port(host, min_port=10000, max_port=60000, max_retries=200): random.seed(int(time.time() * 1000)) for _ in range(max_retries): port = random.randint(min_port, max_port) From 3f4634ad3145f31c267c1c7b8496858f718b00ec Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 19:01:36 +0800 Subject: [PATCH 039/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 2 +- python/ray/spark/utils.py | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index c2ebae617828..cdcd7bc96877 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -270,7 +270,7 @@ def init_cluster( ray_head_node_manager_port = get_random_safe_port(ray_head_hostname) ray_head_object_manager_port = get_random_safe_port(ray_head_hostname) - _logger.info(f"Ray head hostanme {ray_head_hostname}, port {ray_head_port}") + _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 5f4090790cb6..842233485cb9 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -139,6 +139,7 @@ def exec_cmd( return comp_process +# In my test this method might choose conflicted ports. Use get_random_safe_port instead. def get_safe_port(ip): import socket """Returns an ephemeral port that is very likely to be free to bind to.""" @@ -150,9 +151,9 @@ def get_safe_port(ip): def get_random_safe_port(host, min_port=10000, max_port=60000, max_retries=200): - random.seed(int(time.time() * 1000)) + rng = random.SystemRandom() for _ in range(max_retries): - port = random.randint(min_port, max_port) + port = rng.randint(min_port, max_port) if not check_port_open(host, port): return port raise RuntimeError("Get random safe port failed.") From 488a878068b0327c478c6f903cb36e67c28a8da5 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 19:11:12 +0800 Subject: [PATCH 040/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index cdcd7bc96877..e873756be8b1 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -111,7 +111,6 @@ def shutdown(self): _logger.warning( f"Error happens during killing ray head node: {repr(e)}" ) - shutil.rmtree(self.ray_temp_dir, ignore_errors=True) self.is_shutdown = True else: _logger.warning("The cluster has been shut down.") From 6b013299c21d57c22f800e122254278f18f7d834 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 19:27:00 +0800 Subject: [PATCH 041/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 12 ++++++------ python/ray/spark/utils.py | 13 ++++++------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index e873756be8b1..0e07948b7a2d 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -11,7 +11,7 @@ from .utils import ( exec_cmd, check_port_open, - get_random_safe_port, + get_safe_port, get_spark_session, get_spark_driver_hostname, is_in_databricks_runtime, @@ -264,10 +264,10 @@ def init_cluster( ) ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) - ray_head_port = get_random_safe_port(ray_head_hostname) + ray_head_port = get_safe_port() - ray_head_node_manager_port = get_random_safe_port(ray_head_hostname) - ray_head_object_manager_port = get_random_safe_port(ray_head_hostname) + ray_head_node_manager_port = get_safe_port() + ray_head_object_manager_port = get_safe_port() _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") @@ -370,8 +370,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_random_safe_port(worker_hostname) - ray_worker_object_manager_port = get_random_safe_port(worker_hostname) + ray_worker_node_manager_port = get_safe_port() + ray_worker_object_manager_port = get_safe_port() ray_worker_cmd = [ ray_exec_path, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 842233485cb9..b42085efdad1 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -139,17 +139,15 @@ def exec_cmd( return comp_process -# In my test this method might choose conflicted ports. Use get_random_safe_port instead. -def get_safe_port(ip): +def get_safe_port(): import socket """Returns an ephemeral port that is very likely to be free to bind to.""" - sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - sock.bind((ip, 0)) - port = sock.getsockname()[1] - sock.close() - return port + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as sock: + sock.bind(("0.0.0.0", 0)) + return sock.getsockname()[1] +""" def get_random_safe_port(host, min_port=10000, max_port=60000, max_retries=200): rng = random.SystemRandom() for _ in range(max_retries): @@ -157,6 +155,7 @@ def get_random_safe_port(host, min_port=10000, max_port=60000, max_retries=200): if not check_port_open(host, port): return port raise RuntimeError("Get random safe port failed.") +""" def check_port_open(host, port): From 7b6605993374050ed94ba2d9f5a65bcd796c467c Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 19:59:03 +0800 Subject: [PATCH 042/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 12 ++++++------ python/ray/spark/utils.py | 10 ++-------- 2 files changed, 8 insertions(+), 14 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 0e07948b7a2d..d8dc44aba461 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -11,7 +11,7 @@ from .utils import ( exec_cmd, check_port_open, - get_safe_port, + get_random_safe_port, get_spark_session, get_spark_driver_hostname, is_in_databricks_runtime, @@ -264,10 +264,10 @@ def init_cluster( ) ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) - ray_head_port = get_safe_port() + ray_head_port = get_random_safe_port() - ray_head_node_manager_port = get_safe_port() - ray_head_object_manager_port = get_safe_port() + ray_head_node_manager_port = get_random_safe_port() + ray_head_object_manager_port = get_random_safe_port() _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") @@ -370,8 +370,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_safe_port() - ray_worker_object_manager_port = get_safe_port() + ray_worker_node_manager_port = get_random_safe_port() + ray_worker_object_manager_port = get_random_safe_port() ray_worker_cmd = [ ray_exec_path, diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index b42085efdad1..7a519f0d99ae 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -147,15 +147,9 @@ def get_safe_port(): return sock.getsockname()[1] -""" -def get_random_safe_port(host, min_port=10000, max_port=60000, max_retries=200): +def get_random_safe_port(min_port=20000, max_port=60000): rng = random.SystemRandom() - for _ in range(max_retries): - port = rng.randint(min_port, max_port) - if not check_port_open(host, port): - return port - raise RuntimeError("Get random safe port failed.") -""" + return rng.randint(min_port, max_port) def check_port_open(host, port): From 8054b858dff98708ae6055a9647a873e89866101 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 20:36:43 +0800 Subject: [PATCH 043/121] debug Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d8dc44aba461..eeda6e8ea1b2 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -362,6 +362,8 @@ def ray_cluster_job_mapper(_): context = BarrierTaskContext.get() task_id = context.partitionId() + time.sleep(task_id * 1) # debug. + worker_hostname = context.getTaskInfos()[task_id].address.split(":")[0].strip() # TODO: remove worker side ray temp dir when ray worker exits. @@ -383,8 +385,8 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", - f"--node-manager-port={ray_worker_node_manager_port}", - f"--object-manager-port={ray_worker_object_manager_port}", + # f"--node-manager-port={ray_worker_node_manager_port}", + # f"--object-manager-port={ray_worker_object_manager_port}", *_convert_ray_node_options(worker_options) ] From 9c42646ab8cecb0aea995eae9d61ed23af24814d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 21:03:30 +0800 Subject: [PATCH 044/121] add sleep Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 27 +++++++++++++++++---------- python/ray/spark/utils.py | 13 +++++++++++-- 2 files changed, 28 insertions(+), 12 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index eeda6e8ea1b2..9f5fbc6ee585 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -11,7 +11,7 @@ from .utils import ( exec_cmd, check_port_open, - get_random_safe_port, + get_safe_port, get_spark_session, get_spark_driver_hostname, is_in_databricks_runtime, @@ -19,6 +19,7 @@ get_avail_mem_per_ray_worker, get_dbutils, get_max_num_concurrent_tasks, + get_spark_task_local_rank, ) if not sys.platform.startswith("linux"): @@ -264,10 +265,10 @@ def init_cluster( ) ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) - ray_head_port = get_random_safe_port() + ray_head_port = get_safe_port() - ray_head_node_manager_port = get_random_safe_port() - ray_head_object_manager_port = get_random_safe_port() + ray_head_node_manager_port = get_safe_port() + ray_head_object_manager_port = get_safe_port() _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") @@ -362,9 +363,15 @@ def ray_cluster_job_mapper(_): context = BarrierTaskContext.get() task_id = context.partitionId() - time.sleep(task_id * 1) # debug. + task_ip_list = [info.address.split(":")[0] for info in context.getTaskInfos()] + task_local_rank = get_spark_task_local_rank(task_id, task_ip_list) - worker_hostname = context.getTaskInfos()[task_id].address.split(":")[0].strip() + # NB: If we launch multiple ray worker node at the same time, + # it might cause Raylet uses conflicted port. + # probably race conditions issues in ray implementation. + # as a workaround, I add a sleep here to make different local tasks runs starting from + # different time. + time.sleep(task_local_rank * 2.0) # TODO: remove worker side ray temp dir when ray worker exits. # Ray worker might run on a machine different with the head node, so create the @@ -372,8 +379,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_random_safe_port() - ray_worker_object_manager_port = get_random_safe_port() + ray_worker_node_manager_port = get_safe_port() + ray_worker_object_manager_port = get_safe_port() ray_worker_cmd = [ ray_exec_path, @@ -385,8 +392,8 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", - # f"--node-manager-port={ray_worker_node_manager_port}", - # f"--object-manager-port={ray_worker_object_manager_port}", + f"--node-manager-port={ray_worker_node_manager_port}", + f"--object-manager-port={ray_worker_object_manager_port}", *_convert_ray_node_options(worker_options) ] diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 7a519f0d99ae..3e7fca10beb4 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -140,14 +140,14 @@ def exec_cmd( def get_safe_port(): - import socket """Returns an ephemeral port that is very likely to be free to bind to.""" + import socket with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as sock: sock.bind(("0.0.0.0", 0)) return sock.getsockname()[1] -def get_random_safe_port(min_port=20000, max_port=60000): +def get_random_port(min_port=20000, max_port=60000): rng = random.SystemRandom() return rng.randint(min_port, max_port) @@ -250,3 +250,12 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] else: return gpu_addr_list + + +def get_spark_task_local_rank(task_id, task_ip_list): + node_ip = task_ip_list[task_id] + rank = 0 + for i in range(task_id): + if task_ip_list[i] == node_ip: + rank += 1 + return rank From 28f7cef54789e07c0dd1f4696a7ea044abb81bb5 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 11 Oct 2022 22:30:28 +0800 Subject: [PATCH 045/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 9f5fbc6ee585..6dba2ae806de 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -371,7 +371,7 @@ def ray_cluster_job_mapper(_): # probably race conditions issues in ray implementation. # as a workaround, I add a sleep here to make different local tasks runs starting from # different time. - time.sleep(task_local_rank * 2.0) + time.sleep(task_local_rank * 10.0) # TODO: remove worker side ray temp dir when ray worker exits. # Ray worker might run on a machine different with the head node, so create the From 6ed296d7d02b0b8fae999018de4def8d119cc824 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 11:11:16 +0800 Subject: [PATCH 046/121] misc udpates Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 54 +++++++++++++++------ python/ray/tests/spark/test_ray_on_spark.py | 6 --- 2 files changed, 40 insertions(+), 20 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 6dba2ae806de..87dac02dccb8 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -1,11 +1,11 @@ import os -import shutil import subprocess import sys import time import threading import logging import math +import uuid from packaging.version import Version from .utils import ( @@ -57,13 +57,13 @@ class RayClusterOnSpark: shut down the cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, ray_temp_dir): + def __init__(self, address, head_proc, spark_job_group_id, num_ray_workers): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id - self.ray_temp_dir = ray_temp_dir self.ray_context = None self.is_shutdown = False + self.num_ray_workers = num_ray_workers def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -77,6 +77,33 @@ def connect(self): if self.ray_context is None: # connect to the ray cluster. self.ray_context = ray.init(address=self.address) + + last_alive_worker_count = 0 + last_progress_move_time = time.time() + while True: + time.sleep(10) + cur_alive_worker_count = len([ + node for node in ray.nodes() + if node['Alive'] and node['Resources'].get('CPU', 0) > 0] + ) + if cur_alive_worker_count == self.num_ray_workers: + return + + if cur_alive_worker_count > last_alive_worker_count: + last_alive_worker_count = cur_alive_worker_count + last_progress_move_time = time.time() + _logger.info( + "Ray worker nodes are starting, progress: " + f"({cur_alive_worker_count} / {self.num_ray_workers})" + ) + else: + if time.time() - last_progress_move_time > 120: + _logger.warning( + "Waiting all ray workers starting timeout, progress: " + f"({cur_alive_worker_count} / {self.num_ray_workers}), " + "Please check ray logs to see why some ray workers haven't start." + ) + return else: _logger.warning("Already connected to this ray cluster.") @@ -136,8 +163,8 @@ def init_cluster( total_object_store_memory_bytes=None, head_options=None, worker_options=None, - ray_temp_dir="/tmp/ray/temp", - ray_log_dir="/tmp/ray/logs" + ray_temp_root_dir="/tmp", + ray_log_root_dir="/tmp" ): """ Initialize a ray cluster on the spark cluster, via starting a ray head node @@ -164,10 +191,11 @@ def init_cluster( the ray cluster requests. head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. - ray_temp_dir: A local disk path to store the ray temporary data. - ray_log_dir: A local disk path to store "ray start" script logs. + ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster create + a subdirectory "ray-temp-{head_port}_{random_suffix}" under it. + ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster create + a subdirectory "ray-logs-{head_port}_{random_suffix}" under it. """ - import ray from pyspark.util import inheritable_thread_target head_options = head_options or {} @@ -274,7 +302,8 @@ def init_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - ray_log_dir = os.path.join(ray_log_dir, f"ray-{ray_head_port}") + temp_dir_unique_suffix = uuid.uuid4().hex[:4] + ray_log_dir = os.path.join(ray_log_root_dir, f"ray-logs-{ray_head_port}-{temp_dir_unique_suffix}") os.makedirs(ray_log_dir, exist_ok=True) # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/session_latest/logs", @@ -285,7 +314,7 @@ def init_cluster( # (especially on databricks runtime), so we'd better set the log output dir to be a # path mounted with NFS shared by all spark cluster nodes, so that the user can access # these remote log files from spark drive side easily. - ray_temp_dir = os.path.join(ray_temp_dir, f"ray-{ray_head_port}") + ray_temp_dir = os.path.join(ray_temp_root_dir, f"ray-temp-{ray_head_port}-{temp_dir_unique_suffix}") os.makedirs(ray_temp_dir, exist_ok=True) _logger.warning( @@ -475,7 +504,7 @@ def setup_sigterm_on_parent_death(): address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, - ray_temp_dir=ray_temp_dir, + num_ray_workers=num_spark_tasks ) def backgroud_job_thread_fn(): @@ -509,9 +538,6 @@ def backgroud_job_thread_fn(): ).start() try: - # Waiting all ray workers spin up. - time.sleep(10) - if is_in_databricks_runtime(): try: get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 6bd72d2682cc..ff5068cfef9e 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -39,7 +39,6 @@ def get_ray_worker_resources_list(): def test_cpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): - time.sleep(5) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -57,7 +56,6 @@ def f(x): def test_ray_cluster_shutdown(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - time.sleep(5) assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. @@ -77,8 +75,6 @@ def fake_shutdown(_): def test_background_spark_job_exit_trigger_ray_head_exit(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - time.sleep(5) - cluster._cancel_background_spark_job() time.sleep(5) @@ -96,7 +92,6 @@ def test_gpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: - time.sleep(5) worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -105,7 +100,6 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: - time.sleep(5) @ray.remote(num_cpus=1, num_gpus=1) def f(_): From 884d913c1fd7a48a1d367f72158802f3baed72a3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 11:28:22 +0800 Subject: [PATCH 047/121] add test Signed-off-by: Weichen Xu --- python/ray/tests/spark/discover_8_gpu.sh | 4 ++++ python/ray/tests/spark/test_ray_on_spark.py | 26 +++++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100755 python/ray/tests/spark/discover_8_gpu.sh diff --git a/python/ray/tests/spark/discover_8_gpu.sh b/python/ray/tests/spark/discover_8_gpu.sh new file mode 100755 index 000000000000..8d9957a7d6d7 --- /dev/null +++ b/python/ray/tests/spark/discover_8_gpu.sh @@ -0,0 +1,4 @@ +#!/bin/bash + +# This script is used in spark GPU cluster config for discovering available GPU. +echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\",\"2\",\"3\",\"4\",\"5\",\"6\",\"7\"]}" diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index ff5068cfef9e..a8cbd173c0c2 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -185,3 +185,29 @@ def setup_class(cls): .config("spark.task.maxFailures", "1") \ .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ .getOrCreate() + + +class TestLargeCoresCluster(RayOnSparkGPUClusterTestBase): + + @classmethod + def setup_class(cls): + cls.num_total_cpus = 8 + cls.num_total_gpus = 8 + cls.num_cpus_per_spark_task = 1 + cls.num_gpus_per_spark_task = 1 + cls.max_spark_tasks = 8 + gpu_discovery_script_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), + "discover_8_gpu.sh" + ) + os.environ["SPARK_WORKER_CORES"] = "8" + cls.spark = SparkSession.builder \ + .master("local-cluster[1, 8, 8192]") \ + .config("spark.task.cpus", "1") \ + .config("spark.task.resource.gpu.amount", "1") \ + .config("spark.executor.cores", "8") \ + .config("spark.worker.resource.gpu.amount", "8") \ + .config("spark.executor.resource.gpu.amount", "8") \ + .config("spark.task.maxFailures", "1") \ + .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ + .getOrCreate() From 39692b006a8f22f973acd1910a9e3b5e498688d3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 11:45:36 +0800 Subject: [PATCH 048/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 2 +- python/ray/tests/spark/test_ray_on_spark.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 87dac02dccb8..d2d96c828c70 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -92,7 +92,7 @@ def connect(self): if cur_alive_worker_count > last_alive_worker_count: last_alive_worker_count = cur_alive_worker_count last_progress_move_time = time.time() - _logger.info( + _logger.warning( "Ray worker nodes are starting, progress: " f"({cur_alive_worker_count} / {self.num_ray_workers})" ) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index a8cbd173c0c2..1c9b662a7639 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -64,7 +64,7 @@ def fake_shutdown(_): with patch.object(ray_spark.RayClusterOnSpark, "shutdown", fake_shutdown): cluster._cancel_background_spark_job() - time.sleep(5) + time.sleep(20) assert len(self.get_ray_worker_resources_list()) == 0 From 6849e10b9b2ea038a476b3e71e0db2479c1b59a0 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 12:23:06 +0800 Subject: [PATCH 049/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index d2d96c828c70..3c0c8d9ce383 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -514,6 +514,7 @@ def backgroud_job_thread_fn(): "This job group is for spark job which runs the Ray cluster with ray head node " f"{ray_head_hostname}:{ray_head_port}" ) + time.sleep(1) spark.sparkContext.parallelize( list(range(num_spark_tasks)), num_spark_tasks ).barrier().mapPartitions( @@ -532,12 +533,14 @@ def backgroud_job_thread_fn(): # head node, so call `ray_cluster_handler.shutdown()` here. ray_cluster_handler.shutdown() - threading.Thread( - target=inheritable_thread_target(backgroud_job_thread_fn), - args=() - ).start() - try: + threading.Thread( + target=inheritable_thread_target(backgroud_job_thread_fn), + args=() + ).start() + + time.sleep(5) # wait background spark task starting. + if is_in_databricks_runtime(): try: get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) From 00fef7c6ed5491b9ee433aff0be173e73ecb3e73 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 12:30:18 +0800 Subject: [PATCH 050/121] update Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 3c0c8d9ce383..f4b55e6147df 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -514,7 +514,6 @@ def backgroud_job_thread_fn(): "This job group is for spark job which runs the Ray cluster with ray head node " f"{ray_head_hostname}:{ray_head_port}" ) - time.sleep(1) spark.sparkContext.parallelize( list(range(num_spark_tasks)), num_spark_tasks ).barrier().mapPartitions( From 39d683185f637ada18215aa0bd847186c1b17c41 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 12 Oct 2022 13:19:06 +0800 Subject: [PATCH 051/121] update Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 26 --------------------- 1 file changed, 26 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 1c9b662a7639..4a5e4e66e7e4 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -185,29 +185,3 @@ def setup_class(cls): .config("spark.task.maxFailures", "1") \ .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ .getOrCreate() - - -class TestLargeCoresCluster(RayOnSparkGPUClusterTestBase): - - @classmethod - def setup_class(cls): - cls.num_total_cpus = 8 - cls.num_total_gpus = 8 - cls.num_cpus_per_spark_task = 1 - cls.num_gpus_per_spark_task = 1 - cls.max_spark_tasks = 8 - gpu_discovery_script_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), - "discover_8_gpu.sh" - ) - os.environ["SPARK_WORKER_CORES"] = "8" - cls.spark = SparkSession.builder \ - .master("local-cluster[1, 8, 8192]") \ - .config("spark.task.cpus", "1") \ - .config("spark.task.resource.gpu.amount", "1") \ - .config("spark.executor.cores", "8") \ - .config("spark.worker.resource.gpu.amount", "8") \ - .config("spark.executor.resource.gpu.amount", "8") \ - .config("spark.task.maxFailures", "1") \ - .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ - .getOrCreate() From c7ff65f83b2d164287264ab1781b4d33a75a6f85 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Tue, 18 Oct 2022 15:31:07 -0400 Subject: [PATCH 052/121] sizing logic change and test add Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 353 ++++++++++++++------------- python/ray/spark/utils.py | 160 ++++++++++-- python/ray/tests/spark/test_utils.py | 75 +++++- 3 files changed, 401 insertions(+), 187 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index f4b55e6147df..88f4bd025e66 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -1,10 +1,10 @@ import os +import shutil import subprocess import sys import time import threading import logging -import math import uuid from packaging.version import Version @@ -20,6 +20,9 @@ get_dbutils, get_max_num_concurrent_tasks, get_spark_task_local_rank, + get_target_spark_tasks, + _HEAP_TO_SHARED_RATIO, + _resolve_target_spark_tasks, ) if not sys.platform.startswith("linux"): @@ -30,6 +33,7 @@ _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: import pyspark + if Version(pyspark.__version__) < Version("3.3"): raise RuntimeError(_spark_dependency_error) except ImportError: @@ -49,12 +53,17 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): class RayClusterOnSpark: """ - The class is the type of instance returned by `init_cluster` API. - It can be used to connect / disconnect / shut down the cluster. - It can be also used as a python context manager, - when entering the `RayClusterOnSpark` context, connect to the ray cluster, - when exiting the `RayClusterOnSpark` context, disconnect from the ray cluster and - shut down the cluster. + This class is the type of instance returned by the `init_cluster` API. + Its main functionality is to: + Connect to, disconnect from, and shutdown the Ray cluster running on Apache Spark. + Serve as a Python context manager for the `RayClusterOnSpark` instance. + + Args + address: The url for the ray head node (defined as the hostname and unused port on + Spark driver node) + head_proc: Ray head process + spark_job_group_id: The Spark job id for a submitted ray job + num_ray_workers: The number of workers in the ray cluster. """ def __init__(self, address, head_proc, spark_job_group_id, num_ray_workers): @@ -70,6 +79,7 @@ def _cancel_background_spark_job(self): def connect(self): import ray + if self.is_shutdown: raise RuntimeError( "The ray cluster has been shut down or it failed to start." @@ -82,9 +92,12 @@ def connect(self): last_progress_move_time = time.time() while True: time.sleep(10) - cur_alive_worker_count = len([ - node for node in ray.nodes() - if node['Alive'] and node['Resources'].get('CPU', 0) > 0] + cur_alive_worker_count = len( + [ + node + for node in ray.nodes() + if node["Alive"] and node["Resources"].get("CPU", 0) > 0 + ] ) if cur_alive_worker_count == self.num_ray_workers: return @@ -93,15 +106,15 @@ def connect(self): last_alive_worker_count = cur_alive_worker_count last_progress_move_time = time.time() _logger.warning( - "Ray worker nodes are starting, progress: " + "Ray worker nodes are starting. Progress: " f"({cur_alive_worker_count} / {self.num_ray_workers})" ) else: if time.time() - last_progress_move_time > 120: _logger.warning( - "Waiting all ray workers starting timeout, progress: " - f"({cur_alive_worker_count} / {self.num_ray_workers}), " - "Please check ray logs to see why some ray workers haven't start." + "Timeout in waiting for all ray workers to start. Started / Total " + f"requested: ({cur_alive_worker_count} / {self.num_ray_workers}). " + "Please check ray logs to see why some ray workers failed to start." ) return else: @@ -113,14 +126,17 @@ def disconnect(self): self.ray_context.disconnect() except Exception as e: # swallow exception. - _logger.warning(f"Error happens during disconnecting: {repr(e)}") + _logger.warning( + f"An error occurred while disconnecting from the ray cluster: " + f"{repr(e)}" + ) self.ray_context = None else: _logger.warning("Already disconnected from this ray cluster.") def shutdown(self): """ - Shutdown the ray cluster created by `init_cluster` API. + Shutdown the ray cluster created by the `init_cluster` API. """ if not self.is_shutdown: if self.ray_context is not None: @@ -130,14 +146,15 @@ def shutdown(self): except Exception as e: # swallow exception. _logger.warning( - f"Error happens during cancelling ray cluster background spark job: {repr(e)}" + f"An error occurred while cancelling the ray cluster background spark job: " + f"{repr(e)}" ) try: self.head_proc.kill() except Exception as e: # swallow exception. _logger.warning( - f"Error happens during killing ray head node: {repr(e)}" + f"An Error occurred during shutdown of ray head node: {repr(e)}" ) self.is_shutdown = True else: @@ -161,40 +178,46 @@ def init_cluster( total_gpus=None, total_heap_memory_bytes=None, total_object_store_memory_bytes=None, + heap_to_object_store_memory_ratio=None, head_options=None, worker_options=None, ray_temp_root_dir="/tmp", - ray_log_root_dir="/tmp" + ray_log_root_dir="/tmp", ): """ - Initialize a ray cluster on the spark cluster, via starting a ray head node - in spark drive side and creating a background spark barrier mode job and each - spark task running a ray worker node, returns an instance of `RayClusterOnSpark` type. - The returned instance can be used to connect / disconnect / shut down the ray cluster. - We can also use `with` statement like `with init_cluster(...):` , when entering - the managed scope, the ray cluster is initiated and connected, and when exiting the - scope, the ray cluster is disconnected and shut down. + Initialize a ray cluster on the spark cluster by starting a ray head node in the spark driver + side node. After creating the head node, a background spark barrier mode job is created that + generates an instance of `RayClusterOnSpark` that contains configuration for the ray cluster + that will run on the Spark cluster's worker nodes. + The returned instance can be used to connect to, disconnect from and shutdown the ray cluster. + This instance can also be used as a context manager (used by encapsulating operations within + `with init_cluster(...):`). Upon entering the managed scope, the ray cluster is initiated + and connected to. When exiting the scope, the ray cluster is disconnected and shut down. Args - num_spark_tasks: Specify the spark task number the spark job will create. - This argument represents how many concurrent spark tasks it will use to create the - ray cluster, and the ray cluster total available resources (CPU / GPU / memory) - equals to the resources allocated to these spark tasks. - You can specify num_spark_tasks to -1, representing the ray cluster uses all - available spark tasks slots, if you want to create a shared ray cluster - and use the whole spark cluster resources, simply set it to -1. - total_cpus: Specify the total cpus resources the ray cluster requests. - total_gpus: Specify the total gpus resources the ray cluster requests. - total_heap_memory_bytes: Specify the total heap memory resources (in bytes) - the ray cluster requests. - total_object_store_memory_bytes: Specify the total object store memory resources (in bytes) - the ray cluster requests. + num_spark_tasks: The number of spark tasks that the spark job will create. + This argument represents how many concurrent spark tasks will be used to create the + ray cluster. The ray cluster's total available resources (memory, CPU and/or GPU) + is equal to the quantity of resources allocated within these spark tasks. + Specifying the `num_spark_tasks` as `-1` represents a ray cluster configuration that + will use all available spark tasks slots on the spark cluster. + To create a spark cluster that is intended to be used exclusively as a shared ray + cluster, it is recommended to set this argument to `-1`. + total_cpus: The total cpu core count for the ray cluster to utilize. + total_gpus: The total gpu count for the ray cluster to utilize. + total_heap_memory_bytes: The total amount of heap memory (in bytes) for the ray cluster + to utilize. + total_object_store_memory_bytes: The total amount of object store memory (in bytes) for + the ray cluster to utilize. + heap_to_object_store_memory_ratio: The ratio of per-ray worker node available memory to the + size of the `/dev/shm` capacity per worker on the spark worker. Without modification, + this ratio is 0.4. head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. - ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster create - a subdirectory "ray-temp-{head_port}_{random_suffix}" under it. - ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster create - a subdirectory "ray-logs-{head_port}_{random_suffix}" under it. + ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster + will create a subdirectory "ray-temp-{head_port}_{random_suffix}" beneath this path. + ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster + will create a subdirectory "ray-logs-{head_port}_{random_suffix}" beneath this path. """ from pyspark.util import inheritable_thread_target @@ -203,93 +226,74 @@ def init_cluster( num_spark_tasks_specified = num_spark_tasks is not None total_resources_req_specified = ( - total_cpus is not None or - total_gpus is not None or - total_heap_memory_bytes is not None or - total_object_store_memory_bytes is not None + total_cpus is not None + or total_gpus is not None + or total_heap_memory_bytes is not None + or total_object_store_memory_bytes is not None ) - if (num_spark_tasks_specified and total_resources_req_specified) or \ - (not num_spark_tasks_specified and not total_resources_req_specified): + if (num_spark_tasks_specified and total_resources_req_specified) or ( + not num_spark_tasks_specified and not total_resources_req_specified + ): raise ValueError( - "You should specify either 'num_spark_tasks' argument or argument group of " - "'total_cpus', 'total_gpus', 'total_heap_memory_bytes' and 'total_object_store_memory_bytes'." + "You should specify either 'num_spark_tasks' argument or an argument group of " + "'total_cpus', 'total_gpus', 'total_heap_memory_bytes' and " + "'total_object_store_memory_bytes'." ) spark = get_spark_session() + # Environment configurations within the Spark Session that dictate how many cpus and gpus to + # use for each submitted spark task. num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) - num_spark_task_gpus = int(spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0")) - ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes = get_avail_mem_per_ray_worker(spark) + num_spark_task_gpus = int( + spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0") + ) + + if heap_to_object_store_memory_ratio is None: + heap_to_object_store_memory_ratio = _HEAP_TO_SHARED_RATIO + + ( + ray_worker_heap_mem_bytes, + ray_worker_object_store_mem_bytes, + ) = get_avail_mem_per_ray_worker(spark, heap_to_object_store_memory_ratio) if total_gpus is not None and num_spark_task_gpus == 0: raise ValueError( - "The spark cluster is without GPU configuration, so you cannot specify 'total_gpus' " - "argument" + "The spark cluster is not configured with available GPUs. Start a GPU instance cluster " + "to set the 'total_gpus' argument" ) - if num_spark_tasks is not None: - if num_spark_tasks == -1: - # num_spark_tasks=-1 represents using all spark task slots - num_spark_tasks = get_max_num_concurrent_tasks(spark.sparkContext) - elif num_spark_tasks <= 0: - raise ValueError( - "You should specify 'num_spark_tasks' argument to a positive integer or -1." - ) - else: - num_spark_tasks = 1 - if total_cpus is not None: - if total_cpus <= 0: - raise ValueError( - "You should specify 'total_cpus' argument to a positive integer." - ) - num_spark_tasks_for_cpus_req = int(math.ceil(total_cpus / num_spark_task_cpus)) - if num_spark_tasks_for_cpus_req > num_spark_tasks: - num_spark_tasks = num_spark_tasks_for_cpus_req - - if total_gpus is not None: - if total_gpus <= 0: - raise ValueError( - "You should specify 'total_gpus' argument to a positive integer." - ) - num_spark_tasks_for_gpus_req = int(math.ceil(total_gpus / num_spark_task_gpus)) - if num_spark_tasks_for_gpus_req > num_spark_tasks: - num_spark_tasks = num_spark_tasks_for_gpus_req - - if total_heap_memory_bytes is not None: - if total_heap_memory_bytes <= 0: - raise ValueError( - "You should specify 'total_heap_memory_bytes' argument to a positive integer." - ) - num_spark_tasks_for_heap_mem_req = int(math.ceil(total_heap_memory_bytes / ray_worker_heap_mem_bytes)) - if num_spark_tasks_for_heap_mem_req > num_spark_tasks: - num_spark_tasks = num_spark_tasks_for_heap_mem_req - - if total_object_store_memory_bytes is not None: - if total_object_store_memory_bytes <= 0: - raise ValueError( - "You should specify 'total_object_store_memory_bytes' argument to a positive integer." - ) - num_spark_tasks_for_object_store_mem_req = \ - int(math.ceil(total_object_store_memory_bytes / ray_worker_object_store_mem_bytes)) - if num_spark_tasks_for_object_store_mem_req > num_spark_tasks: - num_spark_tasks = num_spark_tasks_for_object_store_mem_req + max_concurrent_tasks = get_max_num_concurrent_tasks(spark.sparkContext) + + num_spark_tasks = get_target_spark_tasks( + max_concurrent_tasks, + num_spark_task_cpus, + num_spark_task_gpus, + ray_worker_heap_mem_bytes, + ray_worker_object_store_mem_bytes, + num_spark_tasks, + total_cpus, + total_gpus, + total_heap_memory_bytes, + total_object_store_memory_bytes, + ) if num_spark_task_cpus < 4: _logger.warning( - f"Each ray worker node will be assigned with {num_spark_task_cpus} CPU cores, less than " - "recommended value 4, because ray worker node cpu cors aligns with the cpu cores assigned to " - "a spark task, you can increase 'spark.task.cpus' config value to address it." + f"Each ray worker node will be assigned with {num_spark_task_cpus} CPU cores. This is " + "less than the recommended value of `4` CPUs. Increasing the spark configuration " + "'spark.task.cpus' to a minimum of `4` is recommended." ) if ray_worker_heap_mem_bytes < 10 * 1024 * 1024 * 1024: _logger.warning( - f"Each ray worker node will be assigned with {ray_worker_heap_mem_bytes} bytes heap memory, " - "less than recommended value 10GB, the ray worker node heap memory size is calculated by " - "(SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / num_local_spark_task_slots * 0.8, " - "so you can increase spark cluster worker machine memory, or reduce spark task slots " - "number on spark cluster worker, or reduce spark worker machine /dev/shm quota to " - "address it." + f"Each ray worker node will be assigned with {ray_worker_heap_mem_bytes} bytes heap " + "memory. This is less than the recommended value of 10GB. The ray worker node heap " + "memory size is calculated by (SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / " + "num_local_spark_task_slots * 0.8. To increase the heap space available, either " + "increase the memory in the spark cluster, reduce the target `num_spark_tasks`, or " + "apply a lower `heap_to_object_store_memory_ratio`." ) ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) @@ -303,7 +307,9 @@ def init_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") temp_dir_unique_suffix = uuid.uuid4().hex[:4] - ray_log_dir = os.path.join(ray_log_root_dir, f"ray-logs-{ray_head_port}-{temp_dir_unique_suffix}") + ray_log_dir = os.path.join( + ray_log_root_dir, f"ray-logs-{ray_head_port}-{temp_dir_unique_suffix}" + ) os.makedirs(ray_log_dir, exist_ok=True) # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/session_latest/logs", @@ -314,12 +320,15 @@ def init_cluster( # (especially on databricks runtime), so we'd better set the log output dir to be a # path mounted with NFS shared by all spark cluster nodes, so that the user can access # these remote log files from spark drive side easily. - ray_temp_dir = os.path.join(ray_temp_root_dir, f"ray-temp-{ray_head_port}-{temp_dir_unique_suffix}") + ray_temp_dir = os.path.join( + ray_temp_root_dir, f"ray-temp-{ray_head_port}-{temp_dir_unique_suffix}" + ) os.makedirs(ray_temp_dir, exist_ok=True) _logger.warning( - f"You can check ray head / worker starting script logs under local disk path {ray_log_dir}, " - f"and you can check ray processes logs under local disk path {ray_temp_dir}/session_latest/logs." + "Logs for ray head / worker starting script can be found in local disk path " + f"{ray_log_dir}. Logs for ray processes can be found in local disk path " + f"{ray_temp_dir}/session_latest/logs." ) ray_head_node_cmd = [ @@ -338,14 +347,13 @@ def init_cluster( f"--object-store-memory={128 * 1024 * 1024}", f"--node-manager-port={ray_head_node_manager_port}", f"--object-manager-port={ray_head_object_manager_port}", - *_convert_ray_node_options(head_options) + *_convert_ray_node_options(head_options), ] _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") with open( - os.path.join(ray_log_dir, "ray-start-head.log"), - "w", buffering=1 + os.path.join(ray_log_dir, "ray-start-head.log"), "w", buffering=1 ) as head_log_fp: ray_head_proc = exec_cmd( ray_head_node_cmd, @@ -358,8 +366,10 @@ def init_cluster( # wait ray head node spin up. wait_ray_node_available( - ray_head_hostname, ray_head_port, 40, - error_on_failure="Start Ray head node failed!" + ray_head_hostname, + ray_head_port, + 40, + error_on_failure="Start Ray head node failed!", ) _logger.info("Ray head node started.") @@ -367,26 +377,28 @@ def init_cluster( # NB: # In order to start ray worker nodes on spark cluster worker machines, # We launch a background spark job: - # 1. it is a barrier mode spark job, i.e. all spark tasks in the job runs concurrently. - # if the spark cluster resources are not sufficient to launch all these tasks concurrently, - # the spark job will hang and retry, if exceeding maximum retries, it will fail. + # 1. it is a barrier mode spark job, i.e. all spark tasks in the job run concurrently. + # If the spark cluster resources are not sufficient to launch all tasks concurrently, + # the spark job will hang and retry. If spark exceeds the maximum retry count, the + # submitted operation will fail. # 2. Each spark task launches one ray worker node. This design ensures all ray worker nodes - # has the same shape (same cpus / gpus / memory configuration). If ray worker nodes have - # different shape, the Ray cluster setup will be nondeterministic, and you could get very - # strange results with bad luck on the node sizing. - # 3. It starts ray worker node via `ray start` CLI. In each spark task, it creates a - # child process and run `ray start ...` command in blocking mode. - # 4. When shut down ray cluster, killing these ray worker nodes is implemented by: - # First, it installs a PR_SET_PDEATHSIG signal for the `ray start ...` child processes - # so that when parent process (pyspark task) dead, the child processes - # (`ray start ...` processes) will receive SIGTERM signal. - # When we need to shut down the ray cluster, call `sparkContext.cancelJobGroup` - # to cancel the background spark job, and it sends SIGKILL signal to all spark tasks, - # so this make spark task processes dead and triggers sending SIGTERM to `ray start ...` - # child processes. + # have the same shape (same cpus / gpus / memory configuration). If ray worker nodes have a + # non-uniform shape, the Ray cluster setup will be non-deterministic and could create + # issues with node sizing. + # 3. A ray worker node is started via the `ray start` CLI. In each spark task, a child + # process is started and will execute a `ray start ...` command in blocking mode. + # 4. When the ray cluster is shutdown, killing ray worker nodes is implemented by: + # Installing a PR_SET_PDEATHSIG signal for the `ray start ...` child processes + # so that when parent process (pyspark task) is killed, the child processes + # (`ray start ...` processes) will receive a SIGTERM signal, killing it. + # Shutting down the ray cluster is performed by calling `sparkContext.cancelJobGroup` + # to cancel the background spark job, sending a SIGKILL signal to all spark tasks. + # Once the spark tasks are killed, this triggers the sending of a SIGTERM to the child + # processes spawned by the `ray_start ...` process. def ray_cluster_job_mapper(_): from pyspark.taskcontext import BarrierTaskContext + _worker_logger = logging.getLogger("ray.spark.worker") context = BarrierTaskContext.get() @@ -395,14 +407,12 @@ def ray_cluster_job_mapper(_): task_ip_list = [info.address.split(":")[0] for info in context.getTaskInfos()] task_local_rank = get_spark_task_local_rank(task_id, task_ip_list) - # NB: If we launch multiple ray worker node at the same time, - # it might cause Raylet uses conflicted port. - # probably race conditions issues in ray implementation. - # as a workaround, I add a sleep here to make different local tasks runs starting from - # different time. + # NB: If we launch multiple ray worker nodes at the same time, + # it might cause Raylet to have a port conflict, likely due to a race condition. + # A sleep is added here to attempt to avoid resource allocation contention with available + # ports. time.sleep(task_local_rank * 10.0) - # TODO: remove worker side ray temp dir when ray worker exits. # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. os.makedirs(ray_temp_dir, exist_ok=True) @@ -423,7 +433,7 @@ def ray_cluster_job_mapper(_): f"--object-store-memory={ray_worker_object_store_mem_bytes}", f"--node-manager-port={ray_worker_node_manager_port}", f"--object-manager-port={ray_worker_object_manager_port}", - *_convert_ray_node_options(worker_options) + *_convert_ray_node_options(worker_options), ] ray_worker_extra_envs = {} @@ -435,15 +445,19 @@ def ray_cluster_job_mapper(_): raise RuntimeError( "Couldn't get the gpu id, Please check the GPU resource configuration" ) - gpu_addr_list = [int(addr.strip()) for addr in task_resources["gpu"].addresses] + gpu_addr_list = [ + int(addr.strip()) for addr in task_resources["gpu"].addresses + ] - available_physical_gpus = get_spark_task_assigned_physical_gpus(gpu_addr_list) + available_physical_gpus = get_spark_task_assigned_physical_gpus( + gpu_addr_list + ) ray_worker_cmd.append( f"--num-gpus={len(available_physical_gpus)}", ) - ray_worker_extra_envs['CUDA_VISIBLE_DEVICES'] = ",".join([ - str(gpu_id) for gpu_id in available_physical_gpus - ]) + ray_worker_extra_envs["CUDA_VISIBLE_DEVICES"] = ",".join( + [str(gpu_id) for gpu_id in available_physical_gpus] + ) if sys.platform.startswith("linux"): @@ -473,16 +487,20 @@ def setup_sigterm_on_parent_death(): # Set the parent process death signal of the command process to SIGTERM. libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h except OSError as e: - _worker_logger.warning(f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}.") + _worker_logger.warning( + f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." + ) else: setup_sigterm_on_parent_death = None _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + # TODO: write to NFS mount with open( os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), - "w", buffering=1 + "w", + buffering=1, ) as worker_log_fp: exec_cmd( ray_worker_cmd, @@ -495,16 +513,23 @@ def setup_sigterm_on_parent_death(): stderr=subprocess.STDOUT, ) + # Delete the worker temp and log directories at the conclusion of running the + # submitted task. + shutil.rmtree(ray_temp_dir, ignore_errors=True) + shutil.rmtree(ray_log_dir, ignore_errors=True) + # NB: Not reachable. yield 0 - spark_job_group_id = f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + spark_job_group_id = ( + f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + ) ray_cluster_handler = RayClusterOnSpark( address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, - num_ray_workers=num_spark_tasks + num_ray_workers=num_spark_tasks, ) def backgroud_job_thread_fn(): @@ -512,37 +537,37 @@ def backgroud_job_thread_fn(): spark.sparkContext.setJobGroup( spark_job_group_id, "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_hostname}:{ray_head_port}" + f"{ray_head_hostname}:{ray_head_port}", ) spark.sparkContext.parallelize( list(range(num_spark_tasks)), num_spark_tasks - ).barrier().mapPartitions( - ray_cluster_job_mapper - ).collect() + ).barrier().mapPartitions(ray_cluster_job_mapper).collect() finally: # NB: # The background spark job is designed to running forever until it is killed, # So this `finally` block is reachable only when: - # 1. The background job raises unexpected exception (i.e. ray worker nodes failed unexpectedly) + # 1. The background job raises unexpected exception (i.e. ray worker nodes + # failed unexpectedly) # 2. User explicitly orders shutting down the ray cluster. - # 3. On databricks runtime, when notebook detached, it triggers python REPL onCancel event and - # it cancelled the background running spark job - # For case 1 and 3, only ray workers are killed, but driver side ray head might be still - # running, and ray context might be in connected status, we need to disconnect and kill ray - # head node, so call `ray_cluster_handler.shutdown()` here. + # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL + # `onCancel` event, cancelling the background running spark job + # For case 1 and 3, only ray workers are killed, but driver side ray head might still + # be running and the ray context might be in connected status. In order to disconnect + # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. ray_cluster_handler.shutdown() try: threading.Thread( - target=inheritable_thread_target(backgroud_job_thread_fn), - args=() + target=inheritable_thread_target(backgroud_job_thread_fn), args=() ).start() time.sleep(5) # wait background spark task starting. if is_in_databricks_runtime(): try: - get_dbutils().entry_point.registerBackgroundSparkJobGroup(spark_job_group_id) + get_dbutils().entry_point.registerBackgroundSparkJobGroup( + spark_job_group_id + ) except Exception: _logger.warning( "Register ray cluster spark job as background job failed. You need to manually " diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 3e7fca10beb4..85bb6f4a350f 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -3,7 +3,11 @@ import os import sys import random -import time +import math + + +_MEMORY_BUFFER_OFFSET = 0.8 +_HEAP_TO_SHARED_RATIO = 0.4 def is_in_databricks_runtime(): @@ -142,6 +146,7 @@ def exec_cmd( def get_safe_port(): """Returns an ephemeral port that is very likely to be free to bind to.""" import socket + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as sock: sock.bind(("0.0.0.0", 0)) return sock.getsockname()[1] @@ -155,12 +160,14 @@ def get_random_port(min_port=20000, max_port=60000): def check_port_open(host, port): import socket from contextlib import closing + with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as sock: return sock.connect_ex((host, port)) == 0 def get_spark_session(): from pyspark.sql import SparkSession + return SparkSession.builder.getOrCreate() @@ -169,8 +176,10 @@ def get_spark_driver_hostname(spark_master_url): return "127.0.0.1" else: parsed_spark_master_url = urlparse(spark_master_url) - if parsed_spark_master_url.scheme.lower() != "spark" or \ - not parsed_spark_master_url.hostname: + if ( + parsed_spark_master_url.scheme.lower() != "spark" + or not parsed_spark_master_url.hostname + ): raise ValueError(f"Unsupported spark.master URL: {spark_master_url}") return parsed_spark_master_url.hostname @@ -178,7 +187,7 @@ def get_spark_driver_hostname(spark_master_url): def get_max_num_concurrent_tasks(spark_context): """Gets the current max number of concurrent tasks.""" # pylint: disable=protected-access - # spark 3.1 and above has a different API for fetching max concurrent tasks + # spark version 3.1 and above have a different API for fetching max concurrent tasks if spark_context._jsc.sc().version() >= "3.1": return spark_context._jsc.sc().maxNumConcurrentTasks( spark_context._jsc.sc().resourceProfileManager().resourceProfileFromId(0) @@ -186,66 +195,177 @@ def get_max_num_concurrent_tasks(spark_context): return spark_context._jsc.sc().maxNumConcurrentTasks() -def _get_total_phyisical_memory(): +def _get_total_physical_memory(): import psutil + return psutil.virtual_memory().total def _get_total_shared_memory(): import shutil - return shutil.disk_usage('/dev/shm').total + + return shutil.disk_usage("/dev/shm").total def _get_cpu_cores(): import multiprocessing + return multiprocessing.cpu_count() -def _calc_mem_per_ray_worker(num_task_slots, physical_mem_bytes, shared_mem_bytes): - ray_worker_object_store_bytes = int(shared_mem_bytes / num_task_slots * 0.8) - ray_worker_heap_mem_bytes = int((physical_mem_bytes - shared_mem_bytes) / num_task_slots * 0.8) +def _calc_mem_per_ray_worker( + num_task_slots, physical_mem_bytes, shared_mem_bytes, heap_to_object_store_ratio +): + available_physical_mem_per_node = int( + physical_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET + ) + available_shared_mem_per_node = int( + shared_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET + ) + + ray_worker_object_store_bytes = int( + min( + available_physical_mem_per_node * heap_to_object_store_ratio, + available_shared_mem_per_node, + ) + ) + ray_worker_heap_mem_bytes = ( + available_physical_mem_per_node - ray_worker_object_store_bytes + ) return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes -def get_avail_mem_per_ray_worker(spark): +def _resolve_target_spark_tasks(calculated_limits): + """ + Return the max value of a list of spark task total count calculations based on the + provided configuration arguments to `init_cluster`. + Args: + calculated_limits: A list of calculated values wherein the highest value based on + spark cluster worker instance sizes and user-specified ray cluster configuration is + taken to ensure that spark cluster limits are not exceeded. + + Returns: The maximum calculated number of spark tasks for the configured ray cluster. + + """ + return max(*calculated_limits) + + +def get_target_spark_tasks( + max_concurrent_tasks, + num_spark_task_cpus, + num_spark_task_gpus, + ray_worker_heap_memory_bytes, + ray_worker_object_store_memory_bytes, + num_spark_tasks, + total_cpus, + total_gpus, + total_heap_memory_bytes, + total_object_store_memory_bytes, +): + + if num_spark_tasks is not None: + if num_spark_tasks == -1: + # num_spark_tasks=-1 represents using all available spark task slots + num_spark_tasks = max_concurrent_tasks + elif num_spark_tasks <= 0: + raise ValueError( + "The value of 'num_spark_tasks' argument must be either a positive integer or -1." + ) + else: + calculated_tasks = [1] + if total_cpus is not None: + if total_cpus <= 0: + raise ValueError( + "The value of 'total_cpus' argument must be a positive integer." + ) + + calculated_tasks.append(int(math.ceil(total_cpus / num_spark_task_cpus))) + + if total_gpus is not None: + if total_gpus <= 0: + raise ValueError( + "The value of 'total_gpus' argument must be a positive integer." + ) + + calculated_tasks.append(int(math.ceil(total_gpus / num_spark_task_gpus))) + + if total_heap_memory_bytes is not None: + if total_heap_memory_bytes <= 0: + raise ValueError( + "The value of 'total_heap_memory_bytes' argument must be a positive integer." + ) + + calculated_tasks.append( + int(math.ceil(total_heap_memory_bytes / ray_worker_heap_memory_bytes)) + ) + + if total_object_store_memory_bytes is not None: + if total_object_store_memory_bytes <= 0: + raise ValueError( + "The value of 'total_object_store_memory_bytes' argument must be a " + "positive integer." + ) + + calculated_tasks.append( + int( + math.ceil( + total_object_store_memory_bytes + / ray_worker_object_store_memory_bytes + ) + ) + ) + + num_spark_tasks = _resolve_target_spark_tasks(calculated_tasks) + return num_spark_tasks + + +def get_avail_mem_per_ray_worker(spark, heap_to_object_store_ratio): """ Return the available heap memory and object store memory for each ray worker. NB: We have one ray node per spark task. """ - # TODO: add a option of heap memory / object store memory ratio. - num_cpus_per_spark_task = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) + num_cpus_per_spark_task = int( + spark.sparkContext.getConf().get("spark.task.cpus", "1") + ) def mapper(_): try: num_cpus = _get_cpu_cores() num_task_slots = num_cpus // num_cpus_per_spark_task - physical_mem_bytes = _get_total_phyisical_memory() + physical_mem_bytes = _get_total_physical_memory() shared_mem_bytes = _get_total_shared_memory() - ray_worker_heap_mem_bytes, ray_worker_object_store_bytes = _calc_mem_per_ray_worker( + ( + ray_worker_heap_mem_bytes, + ray_worker_object_store_bytes, + ) = _calc_mem_per_ray_worker( num_task_slots, physical_mem_bytes, shared_mem_bytes, + heap_to_object_store_ratio, ) return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes, None except Exception as e: return -1, -1, repr(e) - # running inferring memory routine on spark executor side. - # because spark worker node might have different machine shape with spark driver node. - inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes, err = \ + # Running memory inference routine on spark executor side since the spark worker nodes may + # have a different machine configuration compared to the spark driver node. + inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes, err = ( spark.sparkContext.parallelize([1], 1).map(mapper).collect()[0] + ) if err is not None: - raise RuntimeError(f"Inferring ray worker available memory failed, error: {err}") + raise RuntimeError( + f"Inferring ray worker available memory failed, error: {err}" + ) return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes def get_spark_task_assigned_physical_gpus(gpu_addr_list): - if 'CUDA_VISIBLE_DEVICES' in os.environ: + if "CUDA_VISIBLE_DEVICES" in os.environ: visible_cuda_dev_list = [ - int(dev.strip()) for dev in os.environ['CUDA_VISIBLE_DEVICES'].split(",") + int(dev.strip()) for dev in os.environ["CUDA_VISIBLE_DEVICES"].split(",") ] return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] else: diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index a104fd9fd0fc..61601df863d0 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -1,12 +1,13 @@ from unittest.mock import patch import os +import pytest from ray.spark import ( get_spark_task_assigned_physical_gpus, get_spark_driver_hostname, ) -from ray.spark.utils import _calc_mem_per_ray_worker +from ray.spark.utils import _calc_mem_per_ray_worker, get_target_spark_tasks def test_get_spark_task_assigned_physical_gpus(): @@ -27,5 +28,73 @@ def test_get_spark_driver_hostname(): def test_calc_mem_per_ray_worker(): - assert _calc_mem_per_ray_worker(4, 1000000, 400000) == (120000, 80000) - assert _calc_mem_per_ray_worker(6, 1000000, 400000) == (80000, 53333) + assert _calc_mem_per_ray_worker(4, 1000000, 400000, 0.4) == (120000, 80000) + assert _calc_mem_per_ray_worker(6, 1000000, 400000, 0.4) == (80000, 53333) + assert _calc_mem_per_ray_worker(4, 800000, 600000, 0.2) == (128000, 32000) + assert _calc_mem_per_ray_worker(4, 800000, 600000, 0.5) == (80000, 80000) + assert _calc_mem_per_ray_worker(8, 2000000, 600000, 0.3) == (140000, 60000) + + +def test_target_spark_tasks(): + def _mem_in_gbs(gb): + return 1024 * 1024 * 1024 * gb + + # CPU availability sets the task count + cpu_defined_task_count = get_target_spark_tasks( + max_concurrent_tasks=400, + num_spark_task_cpus=4, + num_spark_task_gpus=None, + ray_worker_heap_memory_bytes=_mem_in_gbs(10), + ray_worker_object_store_memory_bytes=_mem_in_gbs(2), + num_spark_tasks=None, + total_cpus=400, + total_gpus=None, + total_heap_memory_bytes=_mem_in_gbs(800), + total_object_store_memory_bytes=_mem_in_gbs(100), + ) + assert cpu_defined_task_count == 100 + + # Heap memory sets the task count + heap_defined_task_count = get_target_spark_tasks( + max_concurrent_tasks=1600, + num_spark_task_cpus=8, + num_spark_task_gpus=None, + ray_worker_heap_memory_bytes=_mem_in_gbs(20), + ray_worker_object_store_memory_bytes=_mem_in_gbs(4), + num_spark_tasks=None, + total_cpus=1600, + total_gpus=None, + total_heap_memory_bytes=_mem_in_gbs(8000), + total_object_store_memory_bytes=_mem_in_gbs(400), + ) + assert heap_defined_task_count == 400 + + # GPU + gpu_defined_task_count = get_target_spark_tasks( + max_concurrent_tasks=400, + num_spark_task_cpus=None, + num_spark_task_gpus=4, + ray_worker_heap_memory_bytes=_mem_in_gbs(40), + ray_worker_object_store_memory_bytes=_mem_in_gbs(8), + num_spark_tasks=None, + total_cpus=None, + total_gpus=80, + total_heap_memory_bytes=_mem_in_gbs(400), + total_object_store_memory_bytes=_mem_in_gbs(80), + ) + assert gpu_defined_task_count == 20 + + # Invalid configuration raises + with pytest.raises(ValueError, match="The value of `num_spark_tasks argument"): + get_target_spark_tasks( + max_concurrent_tasks=400, + num_spark_task_cpus=None, + num_spark_task_gpus=4, + ray_worker_heap_memory_bytes=_mem_in_gbs(40), + ray_worker_object_store_memory_bytes=_mem_in_gbs(8), + num_spark_tasks=-2, + total_cpus=None, + total_gpus=80, + total_heap_memory_bytes=_mem_in_gbs(400), + total_object_store_memory_bytes=_mem_in_gbs(80), + ) From 824c4b3e3f880857199613eba6eb79f43efc1487 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 19 Oct 2022 13:34:44 +0800 Subject: [PATCH 053/121] update ray head host Signed-off-by: Weichen Xu --- python/ray/spark/__init__.py | 8 ++++---- python/ray/spark/utils.py | 11 ++--------- python/ray/tests/spark/test_utils.py | 9 --------- 3 files changed, 6 insertions(+), 22 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index f4b55e6147df..1a86f4ee3481 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -13,7 +13,7 @@ check_port_open, get_safe_port, get_spark_session, - get_spark_driver_hostname, + get_spark_application_driver_host, is_in_databricks_runtime, get_spark_task_assigned_physical_gpus, get_avail_mem_per_ray_worker, @@ -168,7 +168,7 @@ def init_cluster( ): """ Initialize a ray cluster on the spark cluster, via starting a ray head node - in spark drive side and creating a background spark barrier mode job and each + in spark application drive side and creating a background spark barrier mode job and each spark task running a ray worker node, returns an instance of `RayClusterOnSpark` type. The returned instance can be used to connect / disconnect / shut down the ray cluster. We can also use `with` statement like `with init_cluster(...):` , when entering @@ -182,7 +182,7 @@ def init_cluster( equals to the resources allocated to these spark tasks. You can specify num_spark_tasks to -1, representing the ray cluster uses all available spark tasks slots, if you want to create a shared ray cluster - and use the whole spark cluster resources, simply set it to -1. + and use all the resources allocated to the spark application, simply set it to -1. total_cpus: Specify the total cpus resources the ray cluster requests. total_gpus: Specify the total gpus resources the ray cluster requests. total_heap_memory_bytes: Specify the total heap memory resources (in bytes) @@ -292,7 +292,7 @@ def init_cluster( "address it." ) - ray_head_hostname = get_spark_driver_hostname(spark.conf.get("spark.master")) + ray_head_hostname = get_spark_application_driver_host(spark) ray_head_port = get_safe_port() ray_head_node_manager_port = get_safe_port() diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 3e7fca10beb4..671d422c9f17 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -164,15 +164,8 @@ def get_spark_session(): return SparkSession.builder.getOrCreate() -def get_spark_driver_hostname(spark_master_url): - if spark_master_url.lower().startswith("local"): - return "127.0.0.1" - else: - parsed_spark_master_url = urlparse(spark_master_url) - if parsed_spark_master_url.scheme.lower() != "spark" or \ - not parsed_spark_master_url.hostname: - raise ValueError(f"Unsupported spark.master URL: {spark_master_url}") - return parsed_spark_master_url.hostname +def get_spark_application_driver_host(spark): + return spark.conf.get("spark.driver.host") def get_max_num_concurrent_tasks(spark_context): diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index a104fd9fd0fc..a08770ec9b24 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -3,7 +3,6 @@ from ray.spark import ( get_spark_task_assigned_physical_gpus, - get_spark_driver_hostname, ) from ray.spark.utils import _calc_mem_per_ray_worker @@ -18,14 +17,6 @@ def test_get_spark_task_assigned_physical_gpus(): assert get_spark_task_assigned_physical_gpus([0, 2]) == [2, 6] -def test_get_spark_driver_hostname(): - assert get_spark_driver_hostname("local") == "127.0.0.1" - assert get_spark_driver_hostname("local[4]") == "127.0.0.1" - assert get_spark_driver_hostname("local-cluster[1, 4, 1024]") == "127.0.0.1" - assert get_spark_driver_hostname("spark://23.195.26.187:7077") == "23.195.26.187" - assert get_spark_driver_hostname("spark://aa.xx.yy:7077") == "aa.xx.yy" - - def test_calc_mem_per_ray_worker(): assert _calc_mem_per_ray_worker(4, 1000000, 400000) == (120000, 80000) assert _calc_mem_per_ray_worker(6, 1000000, 400000) == (80000, 53333) From 260a0a70b2d56500c918dbf6ba023f444f48ff1c Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 19 Oct 2022 16:11:28 +0800 Subject: [PATCH 054/121] use active spark session Signed-off-by: Weichen Xu --- python/ray/spark/utils.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index 671d422c9f17..7e5f06dad136 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -161,7 +161,13 @@ def check_port_open(host, port): def get_spark_session(): from pyspark.sql import SparkSession - return SparkSession.builder.getOrCreate() + spark_session = SparkSession.getActiveSession() + if spark_session is None: + raise RuntimeError( + "Spark session haven't been initiated yet. Please use `SparkSession.builder` to " + "create a spark session and connect to a spark cluster." + ) + return spark_session def get_spark_application_driver_host(spark): From 809c34776ab3fb1515747c65349aa65e8787b90e Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Wed, 19 Oct 2022 18:32:02 -0400 Subject: [PATCH 055/121] fix safe_mode logic Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index e36010f37430..4de946bc3ac5 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -316,9 +316,9 @@ def init_cluster( ) if insufficient_resources: if safe_mode: - _logger.warning("\n".join(insufficient_resources)) + raise (ValueError, "\n".join(insufficient_resources)) else: - raise(ValueError, "\n".join(insufficient_resources)) + _logger.warning("\n".join(insufficient_resources)) ray_head_hostname = get_spark_application_driver_host(spark) ray_head_port = get_safe_port() From da78fcbd647212e918a1a811f33352488882c361 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Fri, 21 Oct 2022 18:57:40 -0400 Subject: [PATCH 056/121] add initial retry logic for failed barrier execution Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 73 ++++++++++++++++++++---------------- 1 file changed, 40 insertions(+), 33 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 4de946bc3ac5..c12a42274d8f 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -6,7 +6,6 @@ import threading import logging import uuid -import warnings from packaging.version import Version @@ -323,9 +322,6 @@ def init_cluster( ray_head_hostname = get_spark_application_driver_host(spark) ray_head_port = get_safe_port() - ray_head_node_manager_port = get_safe_port() - ray_head_object_manager_port = get_safe_port() - _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") @@ -369,8 +365,6 @@ def init_cluster( f"--memory={128 * 1024 * 1024}", # limit the object store memory usage of head node because no task running on it. f"--object-store-memory={128 * 1024 * 1024}", - f"--node-manager-port={ray_head_node_manager_port}", - f"--object-manager-port={ray_head_object_manager_port}", *_convert_ray_node_options(head_options), ] @@ -442,9 +436,6 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - ray_worker_node_manager_port = get_safe_port() - ray_worker_object_manager_port = get_safe_port() - ray_worker_cmd = [ ray_exec_path, "start", @@ -455,8 +446,6 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", - f"--node-manager-port={ray_worker_node_manager_port}", - f"--object-manager-port={ray_worker_object_manager_port}", *_convert_ray_node_options(worker_options), ] @@ -557,28 +546,46 @@ def setup_sigterm_on_parent_death(): ) def backgroud_job_thread_fn(): - try: - spark.sparkContext.setJobGroup( - spark_job_group_id, - "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_hostname}:{ray_head_port}", - ) - spark.sparkContext.parallelize( - list(range(num_spark_tasks)), num_spark_tasks - ).barrier().mapPartitions(ray_cluster_job_mapper).collect() - finally: - # NB: - # The background spark job is designed to running forever until it is killed, - # So this `finally` block is reachable only when: - # 1. The background job raises unexpected exception (i.e. ray worker nodes - # failed unexpectedly) - # 2. User explicitly orders shutting down the ray cluster. - # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL - # `onCancel` event, cancelling the background running spark job - # For case 1 and 3, only ray workers are killed, but driver side ray head might still - # be running and the ray context might be in connected status. In order to disconnect - # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. - ray_cluster_handler.shutdown() + from pyspark.sql.utils import Py4JJavaError + + _spark_log4j = spark.sparkContext._jvm.org.apache.log4j.LogManager.getLogger("ray.spark") + + # TODO: decide whether to expose these as configurable env variables or args + max_retries = 5 + retry_wait_seconds = 30 + + for restart in range(max_retries + 1): + + try: + spark.sparkContext.setJobGroup( + spark_job_group_id, + "This job group is for spark job which runs the Ray cluster with ray head node " + f"{ray_head_hostname}:{ray_head_port}", + ) + spark.sparkContext.parallelize( + list(range(num_spark_tasks)), num_spark_tasks + ).barrier().mapPartitions(ray_cluster_job_mapper).collect() + except Py4JJavaError as e: + if restart == max_retries: + raise Exception("The Ray cluster has entered an unhealthy state and will not " + "restart again. Max retries exceeded.") from e + _spark_log4j.warn(f"A failure occurred in the Spark process running Ray. Retrying " + f"in {retry_wait_seconds} seconds. {restart} of {max_retries} " + f"retries") + time.sleep(retry_wait_seconds) + finally: + # NB: + # The background spark job is designed to running forever until it is killed, + # So this `finally` block is reachable only when: + # 1. The background job raises unexpected exception (i.e. ray worker nodes + # failed unexpectedly) + # 2. User explicitly orders shutting down the ray cluster. + # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL + # `onCancel` event, cancelling the background running spark job + # For case 1 and 3, only ray workers are killed, but driver side ray head might still + # be running and the ray context might be in connected status. In order to disconnect + # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. + ray_cluster_handler.shutdown() try: threading.Thread( From 7c59952733ce037d48d762d037e39a7be25b1ba6 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Mon, 24 Oct 2022 10:18:37 -0400 Subject: [PATCH 057/121] switch to standard task context instead of barrier context execution Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 61 +++++++++++++----------------------- 1 file changed, 22 insertions(+), 39 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index c12a42274d8f..054a82886dee 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -546,46 +546,29 @@ def setup_sigterm_on_parent_death(): ) def backgroud_job_thread_fn(): - from pyspark.sql.utils import Py4JJavaError - _spark_log4j = spark.sparkContext._jvm.org.apache.log4j.LogManager.getLogger("ray.spark") - - # TODO: decide whether to expose these as configurable env variables or args - max_retries = 5 - retry_wait_seconds = 30 - - for restart in range(max_retries + 1): - - try: - spark.sparkContext.setJobGroup( - spark_job_group_id, - "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_hostname}:{ray_head_port}", - ) - spark.sparkContext.parallelize( - list(range(num_spark_tasks)), num_spark_tasks - ).barrier().mapPartitions(ray_cluster_job_mapper).collect() - except Py4JJavaError as e: - if restart == max_retries: - raise Exception("The Ray cluster has entered an unhealthy state and will not " - "restart again. Max retries exceeded.") from e - _spark_log4j.warn(f"A failure occurred in the Spark process running Ray. Retrying " - f"in {retry_wait_seconds} seconds. {restart} of {max_retries} " - f"retries") - time.sleep(retry_wait_seconds) - finally: - # NB: - # The background spark job is designed to running forever until it is killed, - # So this `finally` block is reachable only when: - # 1. The background job raises unexpected exception (i.e. ray worker nodes - # failed unexpectedly) - # 2. User explicitly orders shutting down the ray cluster. - # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL - # `onCancel` event, cancelling the background running spark job - # For case 1 and 3, only ray workers are killed, but driver side ray head might still - # be running and the ray context might be in connected status. In order to disconnect - # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. - ray_cluster_handler.shutdown() + try: + spark.sparkContext.setJobGroup( + spark_job_group_id, + "This job group is for spark job which runs the Ray cluster with ray head node " + f"{ray_head_hostname}:{ray_head_port}", + ) + spark.sparkContext.parallelize( + list(range(num_spark_tasks)), num_spark_tasks + ).mapPartitions(ray_cluster_job_mapper).collect() + finally: + # NB: + # The background spark job is designed to running forever until it is killed, + # So this `finally` block is reachable only when: + # 1. The background job raises unexpected exception (i.e. ray worker nodes + # failed unexpectedly) + # 2. User explicitly orders shutting down the ray cluster. + # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL + # `onCancel` event, cancelling the background running spark job + # For case 1 and 3, only ray workers are killed, but driver side ray head might still + # be running and the ray context might be in connected status. In order to disconnect + # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. + ray_cluster_handler.shutdown() try: threading.Thread( From 81a7a94209f9cc2ae4f378eda9c87864fc9dae69 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Mon, 24 Oct 2022 11:01:32 -0400 Subject: [PATCH 058/121] use TaskContext and adjust port timing to use partition_id Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 10 +++------- python/ray/spark/utils.py | 9 --------- 2 files changed, 3 insertions(+), 16 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 054a82886dee..5dabcec0b539 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -20,7 +20,6 @@ get_avail_mem_per_ray_worker, get_dbutils, get_max_num_concurrent_tasks, - get_spark_task_local_rank, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, _resolve_target_spark_tasks, @@ -415,21 +414,18 @@ def init_cluster( # processes spawned by the `ray_start ...` process. def ray_cluster_job_mapper(_): - from pyspark.taskcontext import BarrierTaskContext + from pyspark.taskcontext import TaskContext _worker_logger = logging.getLogger("ray.spark.worker") - context = BarrierTaskContext.get() + context = TaskContext.get() task_id = context.partitionId() - task_ip_list = [info.address.split(":")[0] for info in context.getTaskInfos()] - task_local_rank = get_spark_task_local_rank(task_id, task_ip_list) - # NB: If we launch multiple ray worker nodes at the same time, # it might cause Raylet to have a port conflict, likely due to a race condition. # A sleep is added here to attempt to avoid resource allocation contention with available # ports. - time.sleep(task_local_rank * 10.0) + time.sleep(task_id + 5.0) # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index b9b706f1af2f..d0cac99ac274 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -367,12 +367,3 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] else: return gpu_addr_list - - -def get_spark_task_local_rank(task_id, task_ip_list): - node_ip = task_ip_list[task_id] - rank = 0 - for i in range(task_id): - if task_ip_list[i] == node_ip: - rank += 1 - return rank From a990970b30af7029a8cfc8ec37514ee81e7bb859 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Tue, 25 Oct 2022 10:22:51 -0400 Subject: [PATCH 059/121] add worker file lock for race condition port acquisition Signed-off-by: Ben Wilson --- python/ray/spark/__init__.py | 22 +++++++++---------- python/ray/spark/utils.py | 42 ++++++++++++++++++++++++++++++++++++ 2 files changed, 53 insertions(+), 11 deletions(-) diff --git a/python/ray/spark/__init__.py b/python/ray/spark/__init__.py index 5dabcec0b539..a66a477c4197 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/spark/__init__.py @@ -22,7 +22,7 @@ get_max_num_concurrent_tasks, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, - _resolve_target_spark_tasks, + _resolve_target_spark_tasks, _port_acquisition_delay, ) if not sys.platform.startswith("linux"): @@ -188,7 +188,7 @@ def init_cluster( """ Initialize a ray cluster on the spark cluster by starting a ray head node in the spark application's driver side node. - After creating the head node, a background spark barrier mode job is created that + After creating the head node, a background spark job is created that generates an instance of `RayClusterOnSpark` that contains configuration for the ray cluster that will run on the Spark cluster's worker nodes. The returned instance can be used to connect to, disconnect from and shutdown the ray cluster. @@ -394,16 +394,15 @@ def init_cluster( # NB: # In order to start ray worker nodes on spark cluster worker machines, # We launch a background spark job: - # 1. it is a barrier mode spark job, i.e. all spark tasks in the job run concurrently. - # If the spark cluster resources are not sufficient to launch all tasks concurrently, - # the spark job will hang and retry. If spark exceeds the maximum retry count, the - # submitted operation will fail. - # 2. Each spark task launches one ray worker node. This design ensures all ray worker nodes + # 1. Each spark task launches one ray worker node. This design ensures all ray worker nodes # have the same shape (same cpus / gpus / memory configuration). If ray worker nodes have a # non-uniform shape, the Ray cluster setup will be non-deterministic and could create # issues with node sizing. - # 3. A ray worker node is started via the `ray start` CLI. In each spark task, a child + # 2. A ray worker node is started via the `ray start` CLI. In each spark task, a child # process is started and will execute a `ray start ...` command in blocking mode. + # 3. Each task will acquire a file lock for 10s to ensure that the ray worker init will + # acquire a port connection to the ray head node that does not contend with other + # worker processes on the same Spark worker node. # 4. When the ray cluster is shutdown, killing ray worker nodes is implemented by: # Installing a PR_SET_PDEATHSIG signal for the `ray start ...` child processes # so that when parent process (pyspark task) is killed, the child processes @@ -425,7 +424,8 @@ def ray_cluster_job_mapper(_): # it might cause Raylet to have a port conflict, likely due to a race condition. # A sleep is added here to attempt to avoid resource allocation contention with available # ports. - time.sleep(task_id + 5.0) + if sys.platform.startswith("linux"): + _port_acquisition_delay() # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -556,8 +556,8 @@ def backgroud_job_thread_fn(): # NB: # The background spark job is designed to running forever until it is killed, # So this `finally` block is reachable only when: - # 1. The background job raises unexpected exception (i.e. ray worker nodes - # failed unexpectedly) + # 1. The background job raises unexpected exception (i.e. ray cluster dies + # unexpectedly) # 2. User explicitly orders shutting down the ray cluster. # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL # `onCancel` event, cancelling the background running spark job diff --git a/python/ray/spark/utils.py b/python/ray/spark/utils.py index d0cac99ac274..76891e5aa47f 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/spark/utils.py @@ -4,6 +4,8 @@ import sys import random import math +import time +import fcntl _MEMORY_BUFFER_OFFSET = 0.8 @@ -367,3 +369,43 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return [visible_cuda_dev_list[addr] for addr in gpu_addr_list] else: return gpu_addr_list + + +def _port_acquisition_delay(): + """ + Use a file lock to delay the worker processes to ensure that port acquisition does not + create a resource contention issue due to a race condition + + Returns: None + """ + + def acquire_lock(file): + mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC + file_ref = os.open(file, mode) + locked_file_ref = None + lock_time = 10.0 + start = current = time.time() + while current < start + lock_time: + try: + fcntl.flock(file_ref, fcntl.LOCK_EX | fcntl.LOCK_NB) + except (IOError, OSError): + pass + else: + locked_file_ref = file_ref + break + time.sleep(1.0) + current = time.time() + if locked_file_ref is None: + os.close(file_ref) + return locked_file_ref + + def release_lock(locked_file): + fcntl.flock(locked_file, fcntl.LOCK_UN) + os.close(locked_file) + + file_lock = acquire_lock("/tmp/port_waiting_lock.lock") + if file_lock is None: + raise RuntimeError( + "Failed to acquire lock for port assignment contention resolution." + ) + release_lock(file_lock) From d13392f83506090c53cc137ebc7846233067f18e Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Fri, 28 Oct 2022 13:23:55 -0400 Subject: [PATCH 060/121] PR feedback Signed-off-by: Ben Wilson --- python/__init__.py | 0 python/ray/setup-dev.py | 1 - python/ray/tests/spark/test_ray_on_spark.py | 18 +-- python/ray/tests/spark/test_utils.py | 8 +- python/ray/util/spark/__init__.py | 6 + .../spark/cluster_init.py} | 119 ++++++++++++------ python/ray/{ => util}/spark/utils.py | 18 +-- python/setup.py | 1 - 8 files changed, 106 insertions(+), 65 deletions(-) create mode 100644 python/__init__.py create mode 100644 python/ray/util/spark/__init__.py rename python/ray/{spark/__init__.py => util/spark/cluster_init.py} (86%) rename python/ray/{ => util}/spark/utils.py (95%) diff --git a/python/__init__.py b/python/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/setup-dev.py b/python/ray/setup-dev.py index a8e42c24007e..b62d2a800f1a 100755 --- a/python/ray/setup-dev.py +++ b/python/ray/setup-dev.py @@ -101,7 +101,6 @@ def do_link(package, force=False, skip_list=None, local_path=None): do_link("workflow", force=args.yes, skip_list=args.skip) do_link("dag", force=args.yes, skip_list=args.skip) do_link("widgets", force=args.yes, skip_list=args.skip) - do_link("spark", force=args.yes, skip_list=args.skip) do_link("cluster_utils.py", force=args.yes, skip_list=args.skip) do_link("_private", force=args.yes, skip_list=args.skip) # Link package's `dashboard` directly to local (repo's) dashboard. diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 4a5e4e66e7e4..55359746b82e 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -5,8 +5,8 @@ import ray -from ray import spark as ray_spark -from ray.spark import check_port_open +from ray.util.spark import init_ray_cluster, RayClusterOnSpark +from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time import functools @@ -38,14 +38,14 @@ def get_ray_worker_resources_list(): def test_cpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks): + with init_ray_cluster(num_spark_tasks=num_spark_tasks): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: assert worker_res['CPU'] == self.num_cpus_per_spark_task def test_basic_ray_app(self): - with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_spark_tasks=self.max_spark_tasks): @ray.remote def f(x): return x * x @@ -55,14 +55,14 @@ def f(x): assert results == [i * i for i in range(32)] def test_ray_cluster_shutdown(self): - with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. def fake_shutdown(_): pass - with patch.object(ray_spark.RayClusterOnSpark, "shutdown", fake_shutdown): + with patch.object(RayClusterOnSpark, "shutdown", fake_shutdown): cluster._cancel_background_spark_job() time.sleep(20) @@ -74,7 +74,7 @@ def fake_shutdown(_): assert not check_port_open(hostname, int(port)) def test_background_spark_job_exit_trigger_ray_head_exit(self): - with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: cluster._cancel_background_spark_job() time.sleep(5) @@ -91,7 +91,7 @@ class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): def test_gpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with ray_spark.init_cluster(num_spark_tasks=num_spark_tasks) as cluster: + with init_ray_cluster(num_spark_tasks=num_spark_tasks): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -99,7 +99,7 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): - with ray_spark.init_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_spark_tasks=self.max_spark_tasks): @ray.remote(num_cpus=1, num_gpus=1) def f(_): diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index 0ebddac34e14..f2af0273b8f0 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -2,12 +2,12 @@ import os import pytest -from ray.spark import ( +from ray.util.spark.utils import ( get_spark_task_assigned_physical_gpus, + _calc_mem_per_ray_worker, + get_target_spark_tasks, ) -from ray.spark.utils import _calc_mem_per_ray_worker, get_target_spark_tasks - def test_get_spark_task_assigned_physical_gpus(): with patch.dict(os.environ, {}, clear=True): @@ -76,7 +76,7 @@ def _mem_in_gbs(gb): assert gpu_defined_task_count == 20 # Invalid configuration raises - with pytest.raises(ValueError, match="The value of `num_spark_tasks argument"): + with pytest.raises(ValueError, match="The value of `num_worker_nodes` argument"): get_target_spark_tasks( max_concurrent_tasks=400, num_spark_task_cpus=None, diff --git a/python/ray/util/spark/__init__.py b/python/ray/util/spark/__init__.py new file mode 100644 index 000000000000..ae4c61c8feb3 --- /dev/null +++ b/python/ray/util/spark/__init__.py @@ -0,0 +1,6 @@ +from ray.util.spark.cluster_init import RayClusterOnSpark, init_ray_cluster + +__all__ = [ + "RayClusterOnSpark", + "init_ray_cluster", +] diff --git a/python/ray/spark/__init__.py b/python/ray/util/spark/cluster_init.py similarity index 86% rename from python/ray/spark/__init__.py rename to python/ray/util/spark/cluster_init.py index a66a477c4197..a930b52a087e 100644 --- a/python/ray/spark/__init__.py +++ b/python/ray/util/spark/cluster_init.py @@ -6,9 +6,10 @@ import threading import logging import uuid - from packaging.version import Version +from ray.util.annotations import PublicAPI + from .utils import ( exec_cmd, check_port_open, @@ -22,13 +23,13 @@ get_max_num_concurrent_tasks, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, - _resolve_target_spark_tasks, _port_acquisition_delay, + _port_acquisition_delay, ) if not sys.platform.startswith("linux"): raise RuntimeError("Ray on spark ony supports linux system.") -_logger = logging.getLogger("ray.spark") +_logger = logging.getLogger("ray.util.spark") _spark_dependency_error = "ray.spark module requires pyspark >= 3.3" try: @@ -41,7 +42,7 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): - # Wait Ray head node spin up. + # Wait for the Ray head node to start. for _ in range(timeout): time.sleep(1) if check_port_open(hostname, port): @@ -51,9 +52,10 @@ def wait_ray_node_available(hostname, port, timeout, error_on_failure): raise RuntimeError(error_on_failure) +@PublicAPI(stability="alpha") class RayClusterOnSpark: """ - This class is the type of instance returned by the `init_cluster` API. + This class is the type of instance returned by the `init_ray_cluster` API. Its main functionality is to: Connect to, disconnect from, and shutdown the Ray cluster running on Apache Spark. Serve as a Python context manager for the `RayClusterOnSpark` instance. @@ -96,16 +98,18 @@ def connect(self): [ node for node in ray.nodes() - if node["Alive"] and node["Resources"].get("CPU", 0) > 0 + if node["Alive"] ] ) - if cur_alive_worker_count == self.num_ray_workers: + # The head node is included. If alive count is greater than worker count, + # the ray cluster is fully initialized + if cur_alive_worker_count > self.num_ray_workers: return if cur_alive_worker_count > last_alive_worker_count: last_alive_worker_count = cur_alive_worker_count last_progress_move_time = time.time() - _logger.warning( + _logger.info( "Ray worker nodes are starting. Progress: " f"({cur_alive_worker_count} / {self.num_ray_workers})" ) @@ -136,7 +140,7 @@ def disconnect(self): def shutdown(self): """ - Shutdown the ray cluster created by the `init_cluster` API. + Shutdown the ray cluster created by the `init_ray_cluster` API. """ if not self.is_shutdown: if self.ray_context is not None: @@ -172,8 +176,9 @@ def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] -def init_cluster( - num_spark_tasks=None, +@PublicAPI(stability="alpha") +def init_ray_cluster( + num_worker_nodes=None, total_cpus=None, total_gpus=None, total_heap_memory_bytes=None, @@ -193,15 +198,16 @@ def init_cluster( that will run on the Spark cluster's worker nodes. The returned instance can be used to connect to, disconnect from and shutdown the ray cluster. This instance can also be used as a context manager (used by encapsulating operations within - `with init_cluster(...):`). Upon entering the managed scope, the ray cluster is initiated + `with init_ray_cluster(...):`). Upon entering the managed scope, the ray cluster is initiated and connected to. When exiting the scope, the ray cluster is disconnected and shut down. Args - num_spark_tasks: The number of spark tasks that the spark job will create. - This argument represents how many concurrent spark tasks will be used to create the - ray cluster. The ray cluster's total available resources (memory, CPU and/or GPU) + num_worker_nodes: The number of spark worker nodes that the spark job will be submitted to. + This argument represents how many concurrent spark tasks will be available in the + creation of the ray cluster. The ray cluster's total available resources + (memory, CPU and/or GPU) is equal to the quantity of resources allocated within these spark tasks. - Specifying the `num_spark_tasks` as `-1` represents a ray cluster configuration that + Specifying the `num_worker_nodes` as `-1` represents a ray cluster configuration that will use all available spark tasks slots (and resources allocated to the spark application) on the spark cluster. To create a spark cluster that is intended to be used exclusively as a shared ray @@ -223,7 +229,7 @@ def init_cluster( will create a subdirectory "ray-logs-{head_port}_{random_suffix}" beneath this path. safe_mode: Boolean flag to fast-fail initialization of the ray cluster if the available spark cluster does not have sufficient resources to fulfill the resource allocation - for both memory and (cpu or gpu). When set to true, if the requested resources are + for memory, cpu and gpu. When set to true, if the requested resources are not available for minimum recommended functionality, an exception will be raised that details the inadequate spark cluster configuration settings. If overridden as `False`, a warning is raised. @@ -233,7 +239,7 @@ def init_cluster( head_options = head_options or {} worker_options = worker_options or {} - num_spark_tasks_specified = num_spark_tasks is not None + num_worker_nodes_specified = num_worker_nodes is not None total_resources_req_specified = ( total_cpus is not None or total_gpus is not None @@ -241,11 +247,11 @@ def init_cluster( or total_object_store_memory_bytes is not None ) - if (num_spark_tasks_specified and total_resources_req_specified) or ( - not num_spark_tasks_specified and not total_resources_req_specified + if (num_worker_nodes_specified and total_resources_req_specified) or ( + not num_worker_nodes_specified and not total_resources_req_specified ): raise ValueError( - "You should specify either 'num_spark_tasks' argument or an argument group of " + "You should specify either 'num_worker_nodes' argument or an argument group of " "'total_cpus', 'total_gpus', 'total_heap_memory_bytes' and " "'total_object_store_memory_bytes'." ) @@ -275,13 +281,13 @@ def init_cluster( max_concurrent_tasks = get_max_num_concurrent_tasks(spark.sparkContext) - num_spark_tasks = get_target_spark_tasks( + num_worker_nodes = get_target_spark_tasks( max_concurrent_tasks, num_spark_task_cpus, num_spark_task_gpus, ray_worker_heap_mem_bytes, ray_worker_object_store_mem_bytes, - num_spark_tasks, + num_worker_nodes, total_cpus, total_gpus, total_heap_memory_bytes, @@ -309,7 +315,7 @@ def init_cluster( "memory size is calculated by (SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / " "num_local_spark_task_slots * 0.8. To increase the heap space available, " "increase the memory in the spark cluster by changing instance types or worker count, " - "reduce the target `num_spark_tasks`, or apply a lower " + "reduce the target `num_worker_nodes`, or apply a lower " "`heap_to_object_store_memory_ratio`." ) if insufficient_resources: @@ -359,10 +365,13 @@ def init_cluster( "--disable-usage-stats", f"--port={ray_head_port}", "--include-dashboard=false", - f"--num-cpus=0", # disallow ray tasks scheduled to ray head node. - # limit the memory usage of head node because no task running on it. + # disallow ray tasks with cpu requirements from being scheduled on the head node. + f"--num-cpus=0", + # limit the memory allocation to the head node (actual usage may increase beyond this + # for processing of tasks and actors). f"--memory={128 * 1024 * 1024}", - # limit the object store memory usage of head node because no task running on it. + # limit the object store memory allocation to the head node (actual usage may increase + # beyond this for processing of tasks and actors). f"--object-store-memory={128 * 1024 * 1024}", *_convert_ray_node_options(head_options), ] @@ -415,7 +424,7 @@ def init_cluster( def ray_cluster_job_mapper(_): from pyspark.taskcontext import TaskContext - _worker_logger = logging.getLogger("ray.spark.worker") + _worker_logger = logging.getLogger("ray.util.spark.worker") context = TaskContext.get() task_id = context.partitionId() @@ -424,8 +433,9 @@ def ray_cluster_job_mapper(_): # it might cause Raylet to have a port conflict, likely due to a race condition. # A sleep is added here to attempt to avoid resource allocation contention with available # ports. - if sys.platform.startswith("linux"): - _port_acquisition_delay() + port_acquisition_thread = threading.Thread( + target=_port_acquisition_delay, args=[] + ) # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -505,22 +515,49 @@ def setup_sigterm_on_parent_death(): _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + def _worker_exec_cmd( + cmd, + synchronous, + capture_output, + stream_output, + extra_env, + preexec_fn, + stdout, + stderr, + ): + exec_cmd( + cmd, + synchronous=synchronous, + capture_output=capture_output, + stream_output=stream_output, + extra_env=extra_env, + preexec_fn=preexec_fn, + stdout=stdout, + stderr=stderr, + ) + # TODO: write to NFS mount with open( os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), "w", buffering=1, ) as worker_log_fp: - exec_cmd( - ray_worker_cmd, - synchronous=True, - capture_output=False, - stream_output=False, - extra_env=ray_worker_extra_envs, - preexec_fn=setup_sigterm_on_parent_death, - stdout=worker_log_fp, - stderr=subprocess.STDOUT, + worker_thread = threading.Thread( + target=_worker_exec_cmd, + args=( + ray_worker_cmd, + True, + False, + False, + ray_worker_extra_envs, + setup_sigterm_on_parent_death, + worker_log_fp, + subprocess.STDOUT, + ), ) + port_acquisition_thread.start() + worker_thread.start() + port_acquisition_thread.join(timeout=60 * 4) # Delete the worker temp and log directories at the conclusion of running the # submitted task. @@ -538,7 +575,7 @@ def setup_sigterm_on_parent_death(): address=f"{ray_head_hostname}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, - num_ray_workers=num_spark_tasks, + num_ray_workers=num_worker_nodes, ) def backgroud_job_thread_fn(): @@ -550,7 +587,7 @@ def backgroud_job_thread_fn(): f"{ray_head_hostname}:{ray_head_port}", ) spark.sparkContext.parallelize( - list(range(num_spark_tasks)), num_spark_tasks + list(range(num_worker_nodes)), num_worker_nodes ).mapPartitions(ray_cluster_job_mapper).collect() finally: # NB: diff --git a/python/ray/spark/utils.py b/python/ray/util/spark/utils.py similarity index 95% rename from python/ray/spark/utils.py rename to python/ray/util/spark/utils.py index 76891e5aa47f..e1b888bfc4b7 100644 --- a/python/ray/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -1,5 +1,4 @@ import subprocess -from urllib.parse import urlparse import os import sys import random @@ -237,7 +236,7 @@ def _calc_mem_per_ray_worker( def _resolve_target_spark_tasks(calculated_limits): """ Return the max value of a list of spark task total count calculations based on the - provided configuration arguments to `init_cluster`. + provided configuration arguments to `init_ray_cluster`. Args: calculated_limits: A list of calculated values wherein the highest value based on spark cluster worker instance sizes and user-specified ray cluster configuration is @@ -264,11 +263,11 @@ def get_target_spark_tasks( if num_spark_tasks is not None: if num_spark_tasks == -1: - # num_spark_tasks=-1 represents using all available spark task slots + # num_worker_nodes=-1 represents using all available spark task slots num_spark_tasks = max_concurrent_tasks elif num_spark_tasks <= 0: raise ValueError( - "The value of 'num_spark_tasks' argument must be either a positive integer or -1." + "The value of 'num_worker_nodes' argument must be either a positive integer or -1." ) else: calculated_tasks = [1] @@ -383,18 +382,17 @@ def acquire_lock(file): mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC file_ref = os.open(file, mode) locked_file_ref = None - lock_time = 10.0 - start = current = time.time() - while current < start + lock_time: + # Allow for retrying getting a file lock a maximum number of seconds + max_lock_iter = 60 * 2 + for _ in range(max_lock_iter): try: - fcntl.flock(file_ref, fcntl.LOCK_EX | fcntl.LOCK_NB) + fcntl.flock(file_ref, fcntl.LOCK_EX) except (IOError, OSError): pass else: locked_file_ref = file_ref break time.sleep(1.0) - current = time.time() if locked_file_ref is None: os.close(file_ref) return locked_file_ref @@ -408,4 +406,6 @@ def release_lock(locked_file): raise RuntimeError( "Failed to acquire lock for port assignment contention resolution." ) + # Set a sleep while the file is locked to allow for port acquisition for each ray worker + time.sleep(10) release_lock(file_lock) diff --git a/python/setup.py b/python/setup.py index aaf4636e87d4..c3fa4fd04fa7 100644 --- a/python/setup.py +++ b/python/setup.py @@ -248,7 +248,6 @@ def get_packages(self): "opentelemetry-sdk==1.1.0", "opentelemetry-exporter-otlp==1.1.0", ], - "spark": ["pyspark >= 3.3.0"], } # Ray Serve depends on the Ray dashboard components. From 74fed3186735609fb6355111a86f3b4eeffa9479 Mon Sep 17 00:00:00 2001 From: Ben Wilson Date: Fri, 28 Oct 2022 13:25:29 -0400 Subject: [PATCH 061/121] remove auto-added init Signed-off-by: Ben Wilson --- python/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 python/__init__.py diff --git a/python/__init__.py b/python/__init__.py deleted file mode 100644 index e69de29bb2d1..000000000000 From 23a54ddc1b2702dac405ee335b29c353c10d0583 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 21 Nov 2022 20:42:34 +0800 Subject: [PATCH 062/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 67 +++++++--------------- python/ray/util/spark/utils.py | 80 +++++++++++++++------------ 2 files changed, 67 insertions(+), 80 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index a930b52a087e..da892f1fe3b5 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -23,7 +23,7 @@ get_max_num_concurrent_tasks, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, - _port_acquisition_delay, + _ray_worker_startup_barrier, ) if not sys.platform.startswith("linux"): @@ -31,7 +31,7 @@ _logger = logging.getLogger("ray.util.spark") -_spark_dependency_error = "ray.spark module requires pyspark >= 3.3" +_spark_dependency_error = "ray.util.spark module requires pyspark >= 3.3" try: import pyspark @@ -100,10 +100,10 @@ def connect(self): for node in ray.nodes() if node["Alive"] ] - ) + ) - 1 # The head node is included. If alive count is greater than worker count, # the ray cluster is fully initialized - if cur_alive_worker_count > self.num_ray_workers: + if cur_alive_worker_count == self.num_ray_workers: return if cur_alive_worker_count > last_alive_worker_count: @@ -515,54 +515,29 @@ def setup_sigterm_on_parent_death(): _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - def _worker_exec_cmd( - cmd, - synchronous, - capture_output, - stream_output, - extra_env, - preexec_fn, - stdout, - stderr, - ): - exec_cmd( - cmd, - synchronous=synchronous, - capture_output=capture_output, - stream_output=stream_output, - extra_env=extra_env, - preexec_fn=preexec_fn, - stdout=stdout, - stderr=stderr, - ) + _ray_worker_startup_barrier() - # TODO: write to NFS mount with open( os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), "w", - buffering=1, + buffering=1 ) as worker_log_fp: - worker_thread = threading.Thread( - target=_worker_exec_cmd, - args=( - ray_worker_cmd, - True, - False, - False, - ray_worker_extra_envs, - setup_sigterm_on_parent_death, - worker_log_fp, - subprocess.STDOUT, - ), + exec_cmd( + ray_worker_cmd, + synchronous=True, + capture_output=False, + stream_output=False, + extra_env=ray_worker_extra_envs, + preexec_fn=setup_sigterm_on_parent_death, + stdout=worker_log_fp, + stderr=subprocess.STDOUT, ) - port_acquisition_thread.start() - worker_thread.start() - port_acquisition_thread.join(timeout=60 * 4) - - # Delete the worker temp and log directories at the conclusion of running the - # submitted task. - shutil.rmtree(ray_temp_dir, ignore_errors=True) - shutil.rmtree(ray_log_dir, ignore_errors=True) + + # TODO: Delete the worker temp and log directories at the conclusion of running the + # submitted task. + # Currently all workers uses the same ray_temp_dir as head side, + # and one machine might run mulitple ray workers concurrently, + # so we cannot directly delele the temp dir here. # NB: Not reachable. yield 0 diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index e1b888bfc4b7..b418af7d4977 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -5,6 +5,7 @@ import math import time import fcntl +import threading _MEMORY_BUFFER_OFFSET = 0.8 @@ -370,42 +371,53 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return gpu_addr_list -def _port_acquisition_delay(): +def _ray_worker_startup_barrier(): """ - Use a file lock to delay the worker processes to ensure that port acquisition does not - create a resource contention issue due to a race condition + If we start multiple ray workers on a machine concurrently, some ray worker processes + might fail due to ray port conflicts, this is because race condition on getting free + port and opening the free port. + To address the issue, this function use an exclusive file lock to delay the worker processes + to ensure that port acquisition does not create a resource contention issue due to a race + condition. Returns: None """ - - def acquire_lock(file): + def acquire_lock(file_path): mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC - file_ref = os.open(file, mode) - locked_file_ref = None - # Allow for retrying getting a file lock a maximum number of seconds - max_lock_iter = 60 * 2 - for _ in range(max_lock_iter): - try: - fcntl.flock(file_ref, fcntl.LOCK_EX) - except (IOError, OSError): - pass - else: - locked_file_ref = file_ref - break - time.sleep(1.0) - if locked_file_ref is None: - os.close(file_ref) - return locked_file_ref - - def release_lock(locked_file): - fcntl.flock(locked_file, fcntl.LOCK_UN) - os.close(locked_file) - - file_lock = acquire_lock("/tmp/port_waiting_lock.lock") - if file_lock is None: - raise RuntimeError( - "Failed to acquire lock for port assignment contention resolution." - ) - # Set a sleep while the file is locked to allow for port acquisition for each ray worker - time.sleep(10) - release_lock(file_lock) + try: + fd = os.open(file_path, mode) + # Allow for retrying getting a file lock a maximum number of seconds + max_lock_iter = 600 + for _ in range(max_lock_iter): + try: + fcntl.flock(fd, fcntl.LOCK_EX | fcntl.LOCK_NB) + except BlockingIOError: + # Lock is used by other processes, continue loop to wait for lock available + pass + else: + # Acquire lock successfully. + return fd + time.sleep(1.0) + raise TimeoutError(f"Acquiring lock on file {file_path} timeout.") + except Exception: + os.close(fd) + + lock_file_path = "/tmp/ray_worker_startup_barrier_lock.lock" + try: + lock_fd = acquire_lock(lock_file_path) + except TimeoutError: + # The file might be locked forever because that previous process locked the file + # and then crashed. + # So here remove the existing lock file and acquire file lock on the file again. + try: + os.remove(lock_file_path) + except Exception: + pass + lock_fd = acquire_lock(lock_file_path) + + def hold_lock_for_10s_and_release(): + time.sleep(10) + fcntl.flock(lock_fd, fcntl.LOCK_UN) + os.close(lock_fd) + + threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() From be9a411621c794bc278a054edfb9cb72517b4522 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 21 Nov 2022 20:54:44 +0800 Subject: [PATCH 063/121] add comments Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index da892f1fe3b5..eeb69cf23ddc 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -561,6 +561,24 @@ def backgroud_job_thread_fn(): "This job group is for spark job which runs the Ray cluster with ray head node " f"{ray_head_hostname}:{ray_head_port}", ) + + # Starting a normal spark job (not barrier spark job) to run ray workers, the design + # purpose is: + # 1. Using normal spark job, spark tasks can automatically retry individually, we don't + # need to write additional retry logic, But, in barrier mode, if one spark task fails, + # it will cause all other spark tasks killed. + # 2. Using normal spark job, we can support failover when a spark worker physical + # machine crashes. (spark will try to re-schedule the spark task to other spark worker + # nodes) + # 3. Using barrier mode job, if the cluster resources does not satisfy + # "idle spark task slots >= argument num_spark_task", then the barrier job gets stuck + # and waits until enough idle task slots available, this behavior is not user-friendly, + # on a shared spark cluster, user is hard to estimate how many idle tasks available at + # a time, But, if using normal spark job, it can launch job with less spark tasks + # (i.e. user will see a ray cluster setup with less worker number initially), and when + # more task slots become available, it continues to launch tasks on new available + # slots, and user can see the ray cluster worker number increases when more slots + # available. spark.sparkContext.parallelize( list(range(num_worker_nodes)), num_worker_nodes ).mapPartitions(ray_cluster_job_mapper).collect() From 28083a48c9411402f753badcdbcc27c4534ecc03 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 21 Nov 2022 20:59:21 +0800 Subject: [PATCH 064/121] updates Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 73 +++++++++++---------------- 1 file changed, 30 insertions(+), 43 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index eeb69cf23ddc..cf22c4359f7c 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -429,13 +429,7 @@ def ray_cluster_job_mapper(_): context = TaskContext.get() task_id = context.partitionId() - # NB: If we launch multiple ray worker nodes at the same time, - # it might cause Raylet to have a port conflict, likely due to a race condition. - # A sleep is added here to attempt to avoid resource allocation contention with available - # ports. - port_acquisition_thread = threading.Thread( - target=_port_acquisition_delay, args=[] - ) + _ray_worker_startup_barrier() # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -478,44 +472,37 @@ def ray_cluster_job_mapper(_): [str(gpu_id) for gpu_id in available_physical_gpus] ) - if sys.platform.startswith("linux"): - - def setup_sigterm_on_parent_death(): - """ - Uses prctl to automatically send SIGTERM to the command process when its parent is - dead. - - This handles the case when the parent is a PySpark worker process. - If a user cancels the PySpark job, the worker process gets killed, regardless of - PySpark daemon and worker reuse settings. - We use prctl to ensure the command process receives SIGTERM after spark job - cancellation. - The command process itself should handle SIGTERM properly. - This is a no-op on macOS because prctl is not supported. - - Note: - When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", - This case neither "atexit" nor signal handler can capture SIGKILL signal. - prctl is the only way to capture SIGKILL signal. - """ - try: - import ctypes - import signal - - libc = ctypes.CDLL("libc.so.6") - # Set the parent process death signal of the command process to SIGTERM. - libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h - except OSError as e: - _worker_logger.warning( - f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." - ) - - else: - setup_sigterm_on_parent_death = None - _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") - _ray_worker_startup_barrier() + def setup_sigterm_on_parent_death(): + """ + Uses prctl to automatically send SIGTERM to the command process when its parent is + dead. + + This handles the case when the parent is a PySpark worker process. + If a user cancels the PySpark job, the worker process gets killed, regardless of + PySpark daemon and worker reuse settings. + We use prctl to ensure the command process receives SIGTERM after spark job + cancellation. + The command process itself should handle SIGTERM properly. + This is a no-op on macOS because prctl is not supported. + + Note: + When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", + This case neither "atexit" nor signal handler can capture SIGKILL signal. + prctl is the only way to capture SIGKILL signal. + """ + try: + import ctypes + import signal + + libc = ctypes.CDLL("libc.so.6") + # Set the parent process death signal of the command process to SIGTERM. + libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h + except OSError as e: + _worker_logger.warning( + f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." + ) with open( os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), From c60044930b0f0a0ecf044aef33fbf6abe2407ede Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 21 Nov 2022 22:29:35 +0800 Subject: [PATCH 065/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index b418af7d4977..a7d7fd2825af 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -402,7 +402,7 @@ def acquire_lock(file_path): except Exception: os.close(fd) - lock_file_path = "/tmp/ray_worker_startup_barrier_lock.lock" + lock_file_path = "/tmp/ray_on_spark_worker_startup_barrier_lock.lock" try: lock_fd = acquire_lock(lock_file_path) except TimeoutError: From fef37d5d88ab52bc0bb0e99e74dc5aa47638ac6c Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 22 Nov 2022 20:06:51 +0800 Subject: [PATCH 066/121] update Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 12 ++++++------ python/ray/util/spark/cluster_init.py | 13 +++++++++---- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 55359746b82e..6d4f502a98e4 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -38,14 +38,14 @@ def get_ray_worker_resources_list(): def test_cpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with init_ray_cluster(num_spark_tasks=num_spark_tasks): + with init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: assert worker_res['CPU'] == self.num_cpus_per_spark_task def test_basic_ray_app(self): - with init_ray_cluster(num_spark_tasks=self.max_spark_tasks): + with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): @ray.remote def f(x): return x * x @@ -55,7 +55,7 @@ def f(x): assert results == [i * i for i in range(32)] def test_ray_cluster_shutdown(self): - with init_ray_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. @@ -74,7 +74,7 @@ def fake_shutdown(_): assert not check_port_open(hostname, int(port)) def test_background_spark_job_exit_trigger_ray_head_exit(self): - with init_ray_cluster(num_spark_tasks=self.max_spark_tasks) as cluster: + with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: cluster._cancel_background_spark_job() time.sleep(5) @@ -91,7 +91,7 @@ class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): def test_gpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with init_ray_cluster(num_spark_tasks=num_spark_tasks): + with init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -99,7 +99,7 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): - with init_ray_cluster(num_spark_tasks=self.max_spark_tasks): + with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): @ray.remote(num_cpus=1, num_gpus=1) def f(_): diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index cf22c4359f7c..ca162fdad470 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -30,6 +30,7 @@ raise RuntimeError("Ray on spark ony supports linux system.") _logger = logging.getLogger("ray.util.spark") +_logger.setLevel(logging.INFO) _spark_dependency_error = "ray.util.spark module requires pyspark >= 3.3" try: @@ -301,9 +302,8 @@ def init_ray_cluster( f"The provided CPU resources for each ray worker are inadequate to start a ray " f"cluster. Based on the total cpu resources available and the configured task sizing, " f"each ray worker would start with {num_spark_task_cpus} CPU cores. This is " - "less than the recommended value of `4` CPUs per worker. Either Increasing the spark " - "configuration 'spark.task.cpus' to a minimum of `4` or starting more " - "spark worker nodes is recommended." + "less than the recommended value of `4` CPUs per worker. Increasing the spark " + "configuration 'spark.task.cpus' to a minimum of `4` addresses it." ) if ray_worker_heap_mem_bytes < 10 * 1024 * 1024 * 1024: @@ -320,7 +320,12 @@ def init_ray_cluster( ) if insufficient_resources: if safe_mode: - raise (ValueError, "\n".join(insufficient_resources)) + raise ValueError( + "You are creating ray cluster on spark with safe mode (it can be disabled by " + "setting argument 'safe_mode=False' when calling API 'init_ray_cluster'), " + "safe mode requires the spark cluster config satisfying following criterion: " + "\n".join(insufficient_resources) + ) else: _logger.warning("\n".join(insufficient_resources)) From f280f7727adc34741b2872c999b7ea7416cb28ff Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 23 Nov 2022 21:50:41 +0800 Subject: [PATCH 067/121] dashboard Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 17 +++++++++++++---- python/ray/util/spark/utils.py | 25 ++++++++++++++++++++++--- 2 files changed, 35 insertions(+), 7 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ca162fdad470..f40f00f0924b 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -24,6 +24,7 @@ get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, _ray_worker_startup_barrier, + _display_databricks_driver_proxy_url, ) if not sys.platform.startswith("linux"): @@ -101,9 +102,8 @@ def connect(self): for node in ray.nodes() if node["Alive"] ] - ) - 1 - # The head node is included. If alive count is greater than worker count, - # the ray cluster is fully initialized + ) - 1 # Minus 1 means excluding the head node. + if cur_alive_worker_count == self.num_ray_workers: return @@ -361,6 +361,7 @@ def init_ray_cluster( f"{ray_temp_dir}/session_latest/logs." ) + dashboard_port = 8899 ray_head_node_cmd = [ ray_exec_path, "start", @@ -369,7 +370,8 @@ def init_ray_cluster( "--head", "--disable-usage-stats", f"--port={ray_head_port}", - "--include-dashboard=false", + "--include-dashboard=true", + f"--dashboard-port={dashboard_port}", # disallow ray tasks with cpu requirements from being scheduled on the head node. f"--num-cpus=0", # limit the memory allocation to the head node (actual usage may increase beyond this @@ -383,6 +385,13 @@ def init_ray_cluster( _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") + if is_in_databricks_runtime(): + _display_databricks_driver_proxy_url( + spark.sparkContext, + dashboard_port, + "Ray Cluster Dashboard" + ) + with open( os.path.join(ray_log_dir, "ray-start-head.log"), "w", buffering=1 ) as head_log_fp: diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index a7d7fd2825af..fee7cfffae18 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -406,9 +406,10 @@ def acquire_lock(file_path): try: lock_fd = acquire_lock(lock_file_path) except TimeoutError: - # The file might be locked forever because that previous process locked the file - # and then crashed. - # So here remove the existing lock file and acquire file lock on the file again. + # If timeout happens, the file lock might be hold by another process and that process + # does not release the lock in time by some unexpected reason. + # In this case, remove the existing lock file and create the file again, and then + # acquire file lock on the new file. try: os.remove(lock_file_path) except Exception: @@ -421,3 +422,21 @@ def hold_lock_for_10s_and_release(): os.close(lock_fd) threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() + + +def _display_databricks_driver_proxy_url(spark_context, port, title): + driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal + commandContextTags = driverLocal.commandContext().get().toStringMap().apply("tags") + orgId = commandContextTags.apply("orgId") + clusterId = commandContextTags.apply("clusterId") + + template = "/driver-proxy/o/{orgId}/{clusterId}/{port}/" + proxy_url = template.format(orgId=orgId, clusterId=clusterId, port=port) + + displayHTML(f""" + + """) From 957fb86602ec2d9af08974c6fd7f854d84dea89e Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Wed, 23 Nov 2022 21:53:45 +0800 Subject: [PATCH 068/121] fix Signed-off-by: Weichen Xu --- python/ray/util/spark/utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index fee7cfffae18..686cc5ad3c6c 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -425,6 +425,7 @@ def hold_lock_for_10s_and_release(): def _display_databricks_driver_proxy_url(spark_context, port, title): + from dbruntime.display import displayHTML driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal commandContextTags = driverLocal.commandContext().get().toStringMap().apply("tags") orgId = commandContextTags.apply("orgId") From 268bacc40dfef575f7ebfcb69d64f2555c9e407d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 24 Nov 2022 21:36:34 +0800 Subject: [PATCH 069/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 1 + python/ray/util/spark/utils.py | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index f40f00f0924b..1d2d011d50ca 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -371,6 +371,7 @@ def init_ray_cluster( "--disable-usage-stats", f"--port={ray_head_port}", "--include-dashboard=true", + "--dashboard-host=0.0.0.0", f"--dashboard-port={dashboard_port}", # disallow ray tasks with cpu requirements from being scheduled on the head node. f"--num-cpus=0", diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 686cc5ad3c6c..9d385097f87a 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -162,7 +162,6 @@ def get_random_port(min_port=20000, max_port=60000): def check_port_open(host, port): import socket from contextlib import closing - with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as sock: return sock.connect_ex((host, port)) == 0 From 7e8ff81f0d02e1801458ef475710f1f37b621b0d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 25 Nov 2022 21:24:53 +0800 Subject: [PATCH 070/121] simplify api Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 14 +- python/ray/util/spark/cluster_init.py | 139 ++++++++++++++------ 2 files changed, 106 insertions(+), 47 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 6d4f502a98e4..290c01ad32fa 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -5,7 +5,7 @@ import ray -from ray.util.spark import init_ray_cluster, RayClusterOnSpark +from ray.util.spark import _init_ray_cluster, RayClusterOnSpark from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time @@ -38,14 +38,14 @@ def get_ray_worker_resources_list(): def test_cpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): + with _init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: assert worker_res['CPU'] == self.num_cpus_per_spark_task def test_basic_ray_app(self): - with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): @ray.remote def f(x): return x * x @@ -55,7 +55,7 @@ def f(x): assert results == [i * i for i in range(32)] def test_ray_cluster_shutdown(self): - with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. @@ -74,7 +74,7 @@ def fake_shutdown(_): assert not check_port_open(hostname, int(port)) def test_background_spark_job_exit_trigger_ray_head_exit(self): - with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: cluster._cancel_background_spark_job() time.sleep(5) @@ -91,7 +91,7 @@ class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): def test_gpu_allocation(self): for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): + with _init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: @@ -99,7 +99,7 @@ def test_gpu_allocation(self): def test_basic_ray_app_using_gpu(self): - with init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): @ray.remote(num_cpus=1, num_gpus=1) def f(_): diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 1d2d011d50ca..63aebc5d4775 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -177,8 +177,7 @@ def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] -@PublicAPI(stability="alpha") -def init_ray_cluster( +def _init_ray_cluster( num_worker_nodes=None, total_cpus=None, total_gpus=None, @@ -192,48 +191,13 @@ def init_ray_cluster( safe_mode=True, ): """ - Initialize a ray cluster on the spark cluster by starting a ray head node in the spark - application's driver side node. - After creating the head node, a background spark job is created that - generates an instance of `RayClusterOnSpark` that contains configuration for the ray cluster - that will run on the Spark cluster's worker nodes. + This function is used in testing, it has the same arguments with + `ray.util.spark.init_ray_cluster` API, but it returns a `RayClusterOnSpark` instance instead. + The returned instance can be used to connect to, disconnect from and shutdown the ray cluster. This instance can also be used as a context manager (used by encapsulating operations within `with init_ray_cluster(...):`). Upon entering the managed scope, the ray cluster is initiated and connected to. When exiting the scope, the ray cluster is disconnected and shut down. - - Args - num_worker_nodes: The number of spark worker nodes that the spark job will be submitted to. - This argument represents how many concurrent spark tasks will be available in the - creation of the ray cluster. The ray cluster's total available resources - (memory, CPU and/or GPU) - is equal to the quantity of resources allocated within these spark tasks. - Specifying the `num_worker_nodes` as `-1` represents a ray cluster configuration that - will use all available spark tasks slots (and resources allocated to the spark - application) on the spark cluster. - To create a spark cluster that is intended to be used exclusively as a shared ray - cluster, it is recommended to set this argument to `-1`. - total_cpus: The total cpu core count for the ray cluster to utilize. - total_gpus: The total gpu count for the ray cluster to utilize. - total_heap_memory_bytes: The total amount of heap memory (in bytes) for the ray cluster - to utilize. - total_object_store_memory_bytes: The total amount of object store memory (in bytes) for - the ray cluster to utilize. - heap_to_object_store_memory_ratio: The ratio of per-ray worker node available memory to the - size of the `/dev/shm` capacity per worker on the spark worker. Without modification, - this ratio is 0.4. - head_options: A dict representing Ray head node options. - worker_options: A dict representing Ray worker node options. - ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster - will create a subdirectory "ray-temp-{head_port}_{random_suffix}" beneath this path. - ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster - will create a subdirectory "ray-logs-{head_port}_{random_suffix}" beneath this path. - safe_mode: Boolean flag to fast-fail initialization of the ray cluster if the available - spark cluster does not have sufficient resources to fulfill the resource allocation - for memory, cpu and gpu. When set to true, if the requested resources are - not available for minimum recommended functionality, an exception will be raised that - details the inadequate spark cluster configuration settings. If overridden as `False`, - a warning is raised. """ from pyspark.util import inheritable_thread_target @@ -623,3 +587,98 @@ def backgroud_job_thread_fn(): # calling `ray_cluster_handler.shutdown()` to kill them and clean status. ray_cluster_handler.shutdown() raise + + +_active_ray_cluster = None + + +@PublicAPI(stability="alpha") +def init_ray_cluster( + num_worker_nodes=None, + total_cpus=None, + total_gpus=None, + total_heap_memory_bytes=None, + total_object_store_memory_bytes=None, + heap_to_object_store_memory_ratio=None, + head_options=None, + worker_options=None, + ray_temp_root_dir="/tmp", + ray_log_root_dir="/tmp", + safe_mode=True, +): + """ + Initialize a ray cluster on the spark cluster by starting a ray head node in the spark + application's driver side node. + After creating the head node, a background spark job is created that + generates an instance of `RayClusterOnSpark` that contains configuration for the ray cluster + that will run on the Spark cluster's worker nodes. + After a ray cluster initialized, your python process automatically connect to the ray cluster, + you can call `ray.util.spark.shutdown_ray_cluster` to shut down the ray cluster. + Note: If the active ray cluster haven't shut down, you cannot create a new ray cluster. + + Args + num_worker_nodes: The number of spark worker nodes that the spark job will be submitted to. + This argument represents how many concurrent spark tasks will be available in the + creation of the ray cluster. The ray cluster's total available resources + (memory, CPU and/or GPU) + is equal to the quantity of resources allocated within these spark tasks. + Specifying the `num_worker_nodes` as `-1` represents a ray cluster configuration that + will use all available spark tasks slots (and resources allocated to the spark + application) on the spark cluster. + To create a spark cluster that is intended to be used exclusively as a shared ray + cluster, it is recommended to set this argument to `-1`. + total_cpus: The total cpu core count for the ray cluster to utilize. + total_gpus: The total gpu count for the ray cluster to utilize. + total_heap_memory_bytes: The total amount of heap memory (in bytes) for the ray cluster + to utilize. + total_object_store_memory_bytes: The total amount of object store memory (in bytes) for + the ray cluster to utilize. + heap_to_object_store_memory_ratio: The ratio of per-ray worker node available memory to the + size of the `/dev/shm` capacity per worker on the spark worker. Without modification, + this ratio is 0.4. + head_options: A dict representing Ray head node options. + worker_options: A dict representing Ray worker node options. + ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster + will create a subdirectory "ray-temp-{head_port}_{random_suffix}" beneath this path. + ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster + will create a subdirectory "ray-logs-{head_port}_{random_suffix}" beneath this path. + safe_mode: Boolean flag to fast-fail initialization of the ray cluster if the available + spark cluster does not have sufficient resources to fulfill the resource allocation + for memory, cpu and gpu. When set to true, if the requested resources are + not available for minimum recommended functionality, an exception will be raised that + details the inadequate spark cluster configuration settings. If overridden as `False`, + a warning is raised. + """ + global _active_ray_cluster + if _active_ray_cluster is not None: + raise RuntimeError( + "Current active ray cluster on spark haven't shut down. You cannot create a new ray " + "cluster." + ) + _active_ray_cluster = _init_ray_cluster( + num_worker_nodes=num_worker_nodes, + total_cpus=total_cpus, + total_gpus=total_gpus, + total_heap_memory_bytes=total_heap_memory_bytes, + total_object_store_memory_bytes=total_object_store_memory_bytes, + heap_to_object_store_memory_ratio=heap_to_object_store_memory_ratio, + head_options=head_options, + worker_options=worker_options, + ray_temp_root_dir=ray_temp_root_dir, + ray_log_root_dir=ray_log_root_dir, + safe_mode=safe_mode, + ) + _active_ray_cluster.connect() + + +@PublicAPI(stability="alpha") +def shutdown_ray_cluster(): + """ + Shut down the active ray cluster. + """ + global _active_ray_cluster + if _active_ray_cluster is None: + raise RuntimeError("No active ray cluster to shut down.") + + _active_ray_cluster.shutdown() + _active_ray_cluster = None From d6859fdc1a977312530cf89fe73cc3f8962c1490 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 14:10:41 +0800 Subject: [PATCH 071/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/__init__.py b/python/ray/util/spark/__init__.py index ae4c61c8feb3..8705a0475a39 100644 --- a/python/ray/util/spark/__init__.py +++ b/python/ray/util/spark/__init__.py @@ -1,6 +1,6 @@ from ray.util.spark.cluster_init import RayClusterOnSpark, init_ray_cluster __all__ = [ - "RayClusterOnSpark", "init_ray_cluster", + "shutdown_ray_cluster", ] From 0b52e64636e8a569e52664abdf48cff18d4ab9ac Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 17:53:08 +0800 Subject: [PATCH 072/121] set object manager port Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 10 +++++++++- python/ray/util/spark/utils.py | 21 ++++++++++++++++----- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 63aebc5d4775..7bfe918bc3ae 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -22,6 +22,7 @@ get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, + get_safe_port_in_range, _HEAP_TO_SHARED_RATIO, _ray_worker_startup_barrier, _display_databricks_driver_proxy_url, @@ -294,7 +295,7 @@ def _init_ray_cluster( _logger.warning("\n".join(insufficient_resources)) ray_head_hostname = get_spark_application_driver_host(spark) - ray_head_port = get_safe_port() + ray_head_port = get_safe_port(ray_head_hostname) _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") @@ -415,6 +416,12 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) + # although we start the ray node by 10 seconds interval, + # the ray worker object manager port might still cause conflicts. + # so allocate ray object manager port in advance to alleviate it. + ray_worker_object_manager_port = get_safe_port_in_range( + "127.0.0.1", min_port=10000, max_port=20000, + ) ray_worker_cmd = [ ray_exec_path, "start", @@ -425,6 +432,7 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", + f"--object-manager-port={ray_worker_object_manager_port}", *_convert_ray_node_options(worker_options), ] diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 9d385097f87a..438b2384a63a 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -145,18 +145,20 @@ def exec_cmd( return comp_process -def get_safe_port(): +def get_safe_port(host): """Returns an ephemeral port that is very likely to be free to bind to.""" import socket with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as sock: - sock.bind(("0.0.0.0", 0)) + sock.bind((host, 0)) return sock.getsockname()[1] -def get_random_port(min_port=20000, max_port=60000): - rng = random.SystemRandom() - return rng.randint(min_port, max_port) +_rand_generator = random.SystemRandom() + + +def get_random_port(min_port, max_port): + return _rand_generator.randint(min_port, max_port) def check_port_open(host, port): @@ -166,6 +168,15 @@ def check_port_open(host, port): return sock.connect_ex((host, port)) == 0 +def get_safe_port_in_range(host, min_port, max_port, max_retries=100): + for _ in range(max_retries): + port = get_random_port(min_port, max_port) + if not check_port_open(host, port): + return port + time.sleep(0.1) + raise RuntimeError(f"Get available port between range {min_port} and {max_port} failed.") + + def get_spark_session(): from pyspark.sql import SparkSession From 7b1814c95ca2aa016399092cff41149f29ecfc02 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 18:06:03 +0800 Subject: [PATCH 073/121] dont capture output Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 40 ++++++++++----------------- 1 file changed, 14 insertions(+), 26 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 7bfe918bc3ae..9c20287f5d37 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -358,17 +358,12 @@ def _init_ray_cluster( "Ray Cluster Dashboard" ) - with open( - os.path.join(ray_log_dir, "ray-start-head.log"), "w", buffering=1 - ) as head_log_fp: - ray_head_proc = exec_cmd( - ray_head_node_cmd, - synchronous=False, - capture_output=False, - stream_output=False, - stdout=head_log_fp, - stderr=subprocess.STDOUT, - ) + ray_head_proc = exec_cmd( + ray_head_node_cmd, + synchronous=False, + capture_output=False, + stream_output=False, + ) # wait ray head node spin up. wait_ray_node_available( @@ -491,21 +486,14 @@ def setup_sigterm_on_parent_death(): f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." ) - with open( - os.path.join(ray_log_dir, f"ray-start-worker-{task_id}.log"), - "w", - buffering=1 - ) as worker_log_fp: - exec_cmd( - ray_worker_cmd, - synchronous=True, - capture_output=False, - stream_output=False, - extra_env=ray_worker_extra_envs, - preexec_fn=setup_sigterm_on_parent_death, - stdout=worker_log_fp, - stderr=subprocess.STDOUT, - ) + exec_cmd( + ray_worker_cmd, + synchronous=True, + capture_output=False, + stream_output=False, + extra_env=ray_worker_extra_envs, + preexec_fn=setup_sigterm_on_parent_death, + ) # TODO: Delete the worker temp and log directories at the conclusion of running the # submitted task. From f55723d4bf09ccfe3d548d3f0cdb93642eac7028 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 18:26:54 +0800 Subject: [PATCH 074/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 9c20287f5d37..2ca3cc7c0e85 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -415,7 +415,7 @@ def ray_cluster_job_mapper(_): # the ray worker object manager port might still cause conflicts. # so allocate ray object manager port in advance to alleviate it. ray_worker_object_manager_port = get_safe_port_in_range( - "127.0.0.1", min_port=10000, max_port=20000, + "127.0.0.1", min_port=20001, max_port=23000, ) ray_worker_cmd = [ ray_exec_path, From 2dc2a67992c29e0d65e549fe7520d383624a13e9 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 19:34:04 +0800 Subject: [PATCH 075/121] update Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 2ca3cc7c0e85..839afdc2d8be 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -163,8 +163,6 @@ def shutdown(self): f"An Error occurred during shutdown of ray head node: {repr(e)}" ) self.is_shutdown = True - else: - _logger.warning("The cluster has been shut down.") def __enter__(self): self.connect() @@ -202,6 +200,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target + _logger.warning("Test version 003.") head_options = head_options or {} worker_options = worker_options or {} From 48ace3d690283860cc7b801ada38a9bb2cbfe5f3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 20:13:52 +0800 Subject: [PATCH 076/121] worker port range Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 839afdc2d8be..a87613932bd8 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -200,7 +200,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 003.") + _logger.warning("Test version 006.") head_options = head_options or {} worker_options = worker_options or {} @@ -410,12 +410,8 @@ def ray_cluster_job_mapper(_): os.makedirs(ray_temp_dir, exist_ok=True) os.makedirs(ray_log_dir, exist_ok=True) - # although we start the ray node by 10 seconds interval, - # the ray worker object manager port might still cause conflicts. - # so allocate ray object manager port in advance to alleviate it. - ray_worker_object_manager_port = get_safe_port_in_range( - "127.0.0.1", min_port=20001, max_port=23000, - ) + min_worker_port = 20000 + task_id * 1000 + max_worker_port = min_worker_port + 999 ray_worker_cmd = [ ray_exec_path, "start", @@ -426,7 +422,8 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_hostname}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", - f"--object-manager-port={ray_worker_object_manager_port}", + f"--min-worker-port={min_worker_port}", + f"--max-worker-port={max_worker_port}", *_convert_ray_node_options(worker_options), ] From 181b2c7704243e96c164ca35c7a5a6165085da5b Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 26 Nov 2022 22:34:03 +0800 Subject: [PATCH 077/121] set ray head ip Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index a87613932bd8..754fbbeae4b3 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -1,5 +1,6 @@ import os import shutil +import socket import subprocess import sys import time @@ -22,7 +23,6 @@ get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, - get_safe_port_in_range, _HEAP_TO_SHARED_RATIO, _ray_worker_startup_barrier, _display_databricks_driver_proxy_url, @@ -200,7 +200,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 006.") + _logger.warning("Test version 007.") head_options = head_options or {} worker_options = worker_options or {} @@ -293,10 +293,11 @@ def _init_ray_cluster( else: _logger.warning("\n".join(insufficient_resources)) - ray_head_hostname = get_spark_application_driver_host(spark) - ray_head_port = get_safe_port(ray_head_hostname) + ray_head_ip = socket.gethostbyname(get_spark_application_driver_host(spark)) - _logger.info(f"Ray head hostname {ray_head_hostname}, port {ray_head_port}") + ray_head_port = get_safe_port(ray_head_ip) + + _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") @@ -333,6 +334,7 @@ def _init_ray_cluster( "--block", "--head", "--disable-usage-stats", + f"--node-ip-address={ray_head_ip}", f"--port={ray_head_port}", "--include-dashboard=true", "--dashboard-host=0.0.0.0", @@ -366,7 +368,7 @@ def _init_ray_cluster( # wait ray head node spin up. wait_ray_node_available( - ray_head_hostname, + ray_head_ip, ray_head_port, 40, error_on_failure="Start Ray head node failed!", @@ -419,7 +421,7 @@ def ray_cluster_job_mapper(_): f"--num-cpus={num_spark_task_cpus}", "--block", "--disable-usage-stats", - f"--address={ray_head_hostname}:{ray_head_port}", + f"--address={ray_head_ip}:{ray_head_port}", f"--memory={ray_worker_heap_mem_bytes}", f"--object-store-memory={ray_worker_object_store_mem_bytes}", f"--min-worker-port={min_worker_port}", @@ -501,11 +503,11 @@ def setup_sigterm_on_parent_death(): yield 0 spark_job_group_id = ( - f"ray-cluster-job-head-{ray_head_hostname}-port-{ray_head_port}" + f"ray-cluster-job-head-{ray_head_ip}-port-{ray_head_port}" ) ray_cluster_handler = RayClusterOnSpark( - address=f"{ray_head_hostname}:{ray_head_port}", + address=f"{ray_head_ip}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, num_ray_workers=num_worker_nodes, @@ -517,7 +519,7 @@ def backgroud_job_thread_fn(): spark.sparkContext.setJobGroup( spark_job_group_id, "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_hostname}:{ray_head_port}", + f"{ray_head_ip}:{ray_head_port}", ) # Starting a normal spark job (not barrier spark job) to run ray workers, the design From a69f241e7810302fd8477acc8911059c87fc7a23 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 27 Nov 2022 13:47:54 +0800 Subject: [PATCH 078/121] refactor Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 56 +++++++++++------------ python/ray/util/spark/utils.py | 64 ++++++++++++--------------- 2 files changed, 56 insertions(+), 64 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 754fbbeae4b3..76c6915a9c48 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -14,17 +14,17 @@ from .utils import ( exec_cmd, check_port_open, - get_safe_port, + get_random_unused_port, get_spark_session, get_spark_application_driver_host, is_in_databricks_runtime, get_spark_task_assigned_physical_gpus, - get_avail_mem_per_ray_worker, + get_avail_mem_per_ray_worker_node, get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, - _ray_worker_startup_barrier, + _acquire_lock_for_ray_worker_node_startup, _display_databricks_driver_proxy_url, ) @@ -68,16 +68,16 @@ class RayClusterOnSpark: Spark driver node) head_proc: Ray head process spark_job_group_id: The Spark job id for a submitted ray job - num_ray_workers: The number of workers in the ray cluster. + num_workers_node: The number of workers in the ray cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, num_ray_workers): + def __init__(self, address, head_proc, spark_job_group_id, num_workers_node): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id self.ray_context = None self.is_shutdown = False - self.num_ray_workers = num_ray_workers + self.num_worker_nodes = num_workers_node def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -105,7 +105,7 @@ def connect(self): ] ) - 1 # Minus 1 means excluding the head node. - if cur_alive_worker_count == self.num_ray_workers: + if cur_alive_worker_count == self.num_worker_nodes: return if cur_alive_worker_count > last_alive_worker_count: @@ -113,13 +113,13 @@ def connect(self): last_progress_move_time = time.time() _logger.info( "Ray worker nodes are starting. Progress: " - f"({cur_alive_worker_count} / {self.num_ray_workers})" + f"({cur_alive_worker_count} / {self.num_worker_nodes})" ) else: if time.time() - last_progress_move_time > 120: _logger.warning( "Timeout in waiting for all ray workers to start. Started / Total " - f"requested: ({cur_alive_worker_count} / {self.num_ray_workers}). " + f"requested: ({cur_alive_worker_count} / {self.num_worker_nodes}). " "Please check ray logs to see why some ray workers failed to start." ) return @@ -234,9 +234,9 @@ def _init_ray_cluster( heap_to_object_store_memory_ratio = _HEAP_TO_SHARED_RATIO ( - ray_worker_heap_mem_bytes, - ray_worker_object_store_mem_bytes, - ) = get_avail_mem_per_ray_worker(spark, heap_to_object_store_memory_ratio) + ray_worker_node_heap_mem_bytes, + ray_worker_node_object_store_mem_bytes, + ) = get_avail_mem_per_ray_worker_node(spark, heap_to_object_store_memory_ratio) if total_gpus is not None and num_spark_task_gpus == 0: raise ValueError( @@ -250,8 +250,8 @@ def _init_ray_cluster( max_concurrent_tasks, num_spark_task_cpus, num_spark_task_gpus, - ray_worker_heap_mem_bytes, - ray_worker_object_store_mem_bytes, + ray_worker_node_heap_mem_bytes, + ray_worker_node_object_store_mem_bytes, num_worker_nodes, total_cpus, total_gpus, @@ -270,11 +270,11 @@ def _init_ray_cluster( "configuration 'spark.task.cpus' to a minimum of `4` addresses it." ) - if ray_worker_heap_mem_bytes < 10 * 1024 * 1024 * 1024: + if ray_worker_node_heap_mem_bytes < 10 * 1024 * 1024 * 1024: insufficient_resources.append( f"The provided memory resources for each ray worker are inadequate. Based on the total " f"memory available on the spark cluster and the configured task sizing, each ray " - f"worker would start with {ray_worker_heap_mem_bytes} bytes heap " + f"worker would start with {ray_worker_node_heap_mem_bytes} bytes heap " "memory. This is less than the recommended value of 10GB. The ray worker node heap " "memory size is calculated by (SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / " "num_local_spark_task_slots * 0.8. To increase the heap space available, " @@ -295,7 +295,7 @@ def _init_ray_cluster( ray_head_ip = socket.gethostbyname(get_spark_application_driver_host(spark)) - ray_head_port = get_safe_port(ray_head_ip) + ray_head_port = get_random_unused_port(ray_head_ip) _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") @@ -405,7 +405,7 @@ def ray_cluster_job_mapper(_): context = TaskContext.get() task_id = context.partitionId() - _ray_worker_startup_barrier() + _acquire_lock_for_ray_worker_node_startup() # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -414,7 +414,7 @@ def ray_cluster_job_mapper(_): min_worker_port = 20000 + task_id * 1000 max_worker_port = min_worker_port + 999 - ray_worker_cmd = [ + ray_worker_node_cmd = [ ray_exec_path, "start", f"--temp-dir={ray_temp_dir}", @@ -422,14 +422,14 @@ def ray_cluster_job_mapper(_): "--block", "--disable-usage-stats", f"--address={ray_head_ip}:{ray_head_port}", - f"--memory={ray_worker_heap_mem_bytes}", - f"--object-store-memory={ray_worker_object_store_mem_bytes}", + f"--memory={ray_worker_node_heap_mem_bytes}", + f"--object-store-memory={ray_worker_node_object_store_mem_bytes}", f"--min-worker-port={min_worker_port}", f"--max-worker-port={max_worker_port}", *_convert_ray_node_options(worker_options), ] - ray_worker_extra_envs = {} + ray_worker_node_extra_envs = {} if num_spark_task_gpus > 0: task_resources = context.resources() @@ -445,14 +445,14 @@ def ray_cluster_job_mapper(_): available_physical_gpus = get_spark_task_assigned_physical_gpus( gpu_addr_list ) - ray_worker_cmd.append( + ray_worker_node_cmd.append( f"--num-gpus={len(available_physical_gpus)}", ) - ray_worker_extra_envs["CUDA_VISIBLE_DEVICES"] = ",".join( + ray_worker_node_extra_envs["CUDA_VISIBLE_DEVICES"] = ",".join( [str(gpu_id) for gpu_id in available_physical_gpus] ) - _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_cmd)}") + _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_node_cmd)}") def setup_sigterm_on_parent_death(): """ @@ -485,11 +485,11 @@ def setup_sigterm_on_parent_death(): ) exec_cmd( - ray_worker_cmd, + ray_worker_node_cmd, synchronous=True, capture_output=False, stream_output=False, - extra_env=ray_worker_extra_envs, + extra_env=ray_worker_node_extra_envs, preexec_fn=setup_sigterm_on_parent_death, ) @@ -510,7 +510,7 @@ def setup_sigterm_on_parent_death(): address=f"{ray_head_ip}:{ray_head_port}", head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, - num_ray_workers=num_worker_nodes, + num_workers_node=num_worker_nodes, ) def backgroud_job_thread_fn(): diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 438b2384a63a..66f5e72d9a90 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -145,22 +145,6 @@ def exec_cmd( return comp_process -def get_safe_port(host): - """Returns an ephemeral port that is very likely to be free to bind to.""" - import socket - - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as sock: - sock.bind((host, 0)) - return sock.getsockname()[1] - - -_rand_generator = random.SystemRandom() - - -def get_random_port(min_port, max_port): - return _rand_generator.randint(min_port, max_port) - - def check_port_open(host, port): import socket from contextlib import closing @@ -168,12 +152,20 @@ def check_port_open(host, port): return sock.connect_ex((host, port)) == 0 -def get_safe_port_in_range(host, min_port, max_port, max_retries=100): +def get_random_unused_port(host, min_port=1024, max_port=65535, max_retries=100, exclude_list=None): + """ + Get random unused port. + """ + # Use true random generator + rng = random.SystemRandom() + + exclude_list = exclude_list or [] for _ in range(max_retries): - port = get_random_port(min_port, max_port) + port = rng.randint(min_port, max_port) + if port in exclude_list: + continue if not check_port_open(host, port): return port - time.sleep(0.1) raise RuntimeError(f"Get available port between range {min_port} and {max_port} failed.") @@ -222,7 +214,7 @@ def _get_cpu_cores(): return multiprocessing.cpu_count() -def _calc_mem_per_ray_worker( +def _calc_mem_per_ray_worker_node( num_task_slots, physical_mem_bytes, shared_mem_bytes, heap_to_object_store_ratio ): available_physical_mem_per_node = int( @@ -232,16 +224,16 @@ def _calc_mem_per_ray_worker( shared_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET ) - ray_worker_object_store_bytes = int( + object_store_bytes = int( min( available_physical_mem_per_node * heap_to_object_store_ratio, available_shared_mem_per_node, ) ) - ray_worker_heap_mem_bytes = ( - available_physical_mem_per_node - ray_worker_object_store_bytes + heap_mem_bytes = ( + available_physical_mem_per_node - object_store_bytes ) - return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes + return heap_mem_bytes, object_store_bytes def _resolve_target_spark_tasks(calculated_limits): @@ -263,8 +255,8 @@ def get_target_spark_tasks( max_concurrent_tasks, num_spark_task_cpus, num_spark_task_gpus, - ray_worker_heap_memory_bytes, - ray_worker_object_store_memory_bytes, + ray_worker_node_heap_memory_bytes, + ray_worker_node_object_store_memory_bytes, num_spark_tasks, total_cpus, total_gpus, @@ -305,7 +297,7 @@ def get_target_spark_tasks( ) calculated_tasks.append( - int(math.ceil(total_heap_memory_bytes / ray_worker_heap_memory_bytes)) + int(math.ceil(total_heap_memory_bytes / ray_worker_node_heap_memory_bytes)) ) if total_object_store_memory_bytes is not None: @@ -319,7 +311,7 @@ def get_target_spark_tasks( int( math.ceil( total_object_store_memory_bytes - / ray_worker_object_store_memory_bytes + / ray_worker_node_object_store_memory_bytes ) ) ) @@ -328,7 +320,7 @@ def get_target_spark_tasks( return num_spark_tasks -def get_avail_mem_per_ray_worker(spark, heap_to_object_store_ratio): +def get_avail_mem_per_ray_worker_node(spark, heap_to_object_store_ratio): """ Return the available heap memory and object store memory for each ray worker. NB: We have one ray node per spark task. @@ -346,21 +338,21 @@ def mapper(_): shared_mem_bytes = _get_total_shared_memory() ( - ray_worker_heap_mem_bytes, - ray_worker_object_store_bytes, - ) = _calc_mem_per_ray_worker( + ray_worker_node_heap_mem_bytes, + ray_worker_node_object_store_bytes, + ) = _calc_mem_per_ray_worker_node( num_task_slots, physical_mem_bytes, shared_mem_bytes, heap_to_object_store_ratio, ) - return ray_worker_heap_mem_bytes, ray_worker_object_store_bytes, None + return ray_worker_node_heap_mem_bytes, ray_worker_node_object_store_bytes, None except Exception as e: return -1, -1, repr(e) # Running memory inference routine on spark executor side since the spark worker nodes may # have a different machine configuration compared to the spark driver node. - inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes, err = ( + inferred_ray_worker_node_heap_mem_bytes, inferred_ray_worker_node_object_store_bytes, err = ( spark.sparkContext.parallelize([1], 1).map(mapper).collect()[0] ) @@ -368,7 +360,7 @@ def mapper(_): raise RuntimeError( f"Inferring ray worker available memory failed, error: {err}" ) - return inferred_ray_worker_heap_mem_bytes, inferred_ray_worker_object_store_bytes + return inferred_ray_worker_node_heap_mem_bytes, inferred_ray_worker_node_object_store_bytes def get_spark_task_assigned_physical_gpus(gpu_addr_list): @@ -381,7 +373,7 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return gpu_addr_list -def _ray_worker_startup_barrier(): +def _acquire_lock_for_ray_worker_node_startup(): """ If we start multiple ray workers on a machine concurrently, some ray worker processes might fail due to ray port conflicts, this is because race condition on getting free From 9a120d4d66b6ebf096f7e13839e32a0872144d49 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 27 Nov 2022 14:21:30 +0800 Subject: [PATCH 079/121] dynamic dashboard port Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 76c6915a9c48..4538b3f730fa 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -200,7 +200,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 007.") + _logger.warning("Test version 008.") head_options = head_options or {} worker_options = worker_options or {} @@ -295,7 +295,10 @@ def _init_ray_cluster( ray_head_ip = socket.gethostbyname(get_spark_application_driver_host(spark)) - ray_head_port = get_random_unused_port(ray_head_ip) + ray_head_port = get_random_unused_port(ray_head_ip, min_port=9000, max_port=10000) + ray_dashboard_port = get_random_unused_port( + ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port] + ) _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") @@ -326,7 +329,6 @@ def _init_ray_cluster( f"{ray_temp_dir}/session_latest/logs." ) - dashboard_port = 8899 ray_head_node_cmd = [ ray_exec_path, "start", @@ -338,7 +340,7 @@ def _init_ray_cluster( f"--port={ray_head_port}", "--include-dashboard=true", "--dashboard-host=0.0.0.0", - f"--dashboard-port={dashboard_port}", + f"--dashboard-port={ray_dashboard_port}", # disallow ray tasks with cpu requirements from being scheduled on the head node. f"--num-cpus=0", # limit the memory allocation to the head node (actual usage may increase beyond this @@ -355,7 +357,7 @@ def _init_ray_cluster( if is_in_databricks_runtime(): _display_databricks_driver_proxy_url( spark.sparkContext, - dashboard_port, + ray_dashboard_port, "Ray Cluster Dashboard" ) From 6a74ae1944926e4a049d68b694f21f595106660a Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 27 Nov 2022 15:03:41 +0800 Subject: [PATCH 080/121] update temp dir Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 41 ++++++++------------------- 1 file changed, 12 insertions(+), 29 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 4538b3f730fa..b679055939e3 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -185,8 +185,7 @@ def _init_ray_cluster( heap_to_object_store_memory_ratio=None, head_options=None, worker_options=None, - ray_temp_root_dir="/tmp", - ray_log_root_dir="/tmp", + ray_temp_root_dir=None, safe_mode=True, ): """ @@ -305,30 +304,19 @@ def _init_ray_cluster( ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") temp_dir_unique_suffix = uuid.uuid4().hex[:4] - ray_log_dir = os.path.join( - ray_log_root_dir, f"ray-logs-{ray_head_port}-{temp_dir_unique_suffix}" - ) - os.makedirs(ray_log_dir, exist_ok=True) - - # TODO: Many ray processes logs are outputted under "{ray_temp_dir}/session_latest/logs", - # Proposal: Update "ray start" scirpt to add a new option "ray_log_dir", and output logs - # to a different directory specified by "ray_log_dir", instead of using - # "{ray_temp_dir}/session_latest/logs". - # The reason is, for ray on spark, user is hard to access log files on spark worker machines, - # (especially on databricks runtime), so we'd better set the log output dir to be a - # path mounted with NFS shared by all spark cluster nodes, so that the user can access - # these remote log files from spark drive side easily. + + # TODO: Set individual temp dir for ray worker nodes, and auto GC temp data when ray node exits + # See https://github.com/ray-project/ray/issues/28876#issuecomment-1322016494 + if ray_temp_root_dir is None: + if is_in_databricks_runtime(): + ray_temp_root_dir = "/local_disk0/tmp" + else: + ray_temp_root_dir = "/tmp" ray_temp_dir = os.path.join( - ray_temp_root_dir, f"ray-temp-{ray_head_port}-{temp_dir_unique_suffix}" + ray_temp_root_dir, f"ray-{ray_head_port}-{temp_dir_unique_suffix}" ) os.makedirs(ray_temp_dir, exist_ok=True) - _logger.warning( - "Logs for ray head / worker starting script can be found in local disk path " - f"{ray_log_dir}. Logs for ray processes can be found in local disk path " - f"{ray_temp_dir}/session_latest/logs." - ) - ray_head_node_cmd = [ ray_exec_path, "start", @@ -412,7 +400,6 @@ def ray_cluster_job_mapper(_): # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. os.makedirs(ray_temp_dir, exist_ok=True) - os.makedirs(ray_log_dir, exist_ok=True) min_worker_port = 20000 + task_id * 1000 max_worker_port = min_worker_port + 999 @@ -598,8 +585,7 @@ def init_ray_cluster( heap_to_object_store_memory_ratio=None, head_options=None, worker_options=None, - ray_temp_root_dir="/tmp", - ray_log_root_dir="/tmp", + ray_temp_root_dir=None, safe_mode=True, ): """ @@ -635,9 +621,7 @@ def init_ray_cluster( head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster - will create a subdirectory "ray-temp-{head_port}_{random_suffix}" beneath this path. - ray_log_root_dir: A local disk path to store "ray start" script logs. The created cluster - will create a subdirectory "ray-logs-{head_port}_{random_suffix}" beneath this path. + will create a subdirectory "ray-{head_port}_{random_suffix}" beneath this path. safe_mode: Boolean flag to fast-fail initialization of the ray cluster if the available spark cluster does not have sufficient resources to fulfill the resource allocation for memory, cpu and gpu. When set to true, if the requested resources are @@ -661,7 +645,6 @@ def init_ray_cluster( head_options=head_options, worker_options=worker_options, ray_temp_root_dir=ray_temp_root_dir, - ray_log_root_dir=ray_log_root_dir, safe_mode=safe_mode, ) _active_ray_cluster.connect() From fb8299c5022bfb2bd5d4d29689a646ff1f41b0a4 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 29 Nov 2022 19:45:47 +0800 Subject: [PATCH 081/121] exception includes ray node tail output Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 51 +++++------ python/ray/util/spark/utils.py | 116 ++++++++------------------ 2 files changed, 61 insertions(+), 106 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index b679055939e3..ab1c2768f365 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -26,6 +26,7 @@ _HEAP_TO_SHARED_RATIO, _acquire_lock_for_ray_worker_node_startup, _display_databricks_driver_proxy_url, + gen_cmd_exec_failure_msg, ) if not sys.platform.startswith("linux"): @@ -44,17 +45,6 @@ raise RuntimeError(_spark_dependency_error) -def wait_ray_node_available(hostname, port, timeout, error_on_failure): - # Wait for the Ray head node to start. - for _ in range(timeout): - time.sleep(1) - if check_port_open(hostname, port): - break - - if not check_port_open(hostname, port): - raise RuntimeError(error_on_failure) - - @PublicAPI(stability="alpha") class RayClusterOnSpark: """ @@ -199,7 +189,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 008.") + _logger.warning("Test version 009.") head_options = head_options or {} worker_options = worker_options or {} @@ -309,7 +299,9 @@ def _init_ray_cluster( # See https://github.com/ray-project/ray/issues/28876#issuecomment-1322016494 if ray_temp_root_dir is None: if is_in_databricks_runtime(): - ray_temp_root_dir = "/local_disk0/tmp" + # TODO: Change `ray_temp_root_dir` to use "/local_disk0/tmp" once + # https://github.com/ray-project/ray/issues/30677 is fixed. + ray_temp_root_dir = "/tmp" else: ray_temp_root_dir = "/tmp" ray_temp_dir = os.path.join( @@ -349,20 +341,33 @@ def _init_ray_cluster( "Ray Cluster Dashboard" ) - ray_head_proc = exec_cmd( + ray_head_proc, tail_output_deque = exec_cmd( ray_head_node_cmd, synchronous=False, - capture_output=False, - stream_output=False, ) # wait ray head node spin up. - wait_ray_node_available( - ray_head_ip, - ray_head_port, - 40, - error_on_failure="Start Ray head node failed!", - ) + for _ in range(40): + time.sleep(1) + if ray_head_proc.poll() is not None: + # ray head node process terminated. + break + if check_port_open(ray_head_ip, ray_head_port): + break + + if not check_port_open(ray_head_ip, ray_head_port): + if ray_head_proc.poll() is None: + # Ray head GCS service is down. Kill ray head node. + ray_head_proc.kill() + # wait killing complete. + time.sleep(0.5) + + cmd_exec_failure_msg = gen_cmd_exec_failure_msg( + ray_head_node_cmd, ray_head_proc.returncode, tail_output_deque + ) + raise RuntimeError( + "Start Ray head node failed!\n" + cmd_exec_failure_msg + ) _logger.info("Ray head node started.") @@ -476,8 +481,6 @@ def setup_sigterm_on_parent_death(): exec_cmd( ray_worker_node_cmd, synchronous=True, - capture_output=False, - stream_output=False, extra_env=ray_worker_node_extra_envs, preexec_fn=setup_sigterm_on_parent_death, ) diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 66f5e72d9a90..52124fd4fa83 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -6,6 +6,7 @@ import time import fcntl import threading +import collections _MEMORY_BUFFER_OFFSET = 0.8 @@ -34,58 +35,31 @@ def get_dbutils(): raise _NoDbutilsError -class ShellCommandException(Exception): - @classmethod - def from_completed_process(cls, process): - lines = [ - f"Non-zero exit code: {process.returncode}", - f"Command: {process.args}", - ] - if process.stdout: - lines += [ - "", - "STDOUT:", - process.stdout, - ] - if process.stderr: - lines += [ - "", - "STDERR:", - process.stderr, - ] - return cls("\n".join(lines)) +def gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque): + cmd_str = " ".join(cmd) + tail_output = "".join(tail_output_deque) + return ( + f"Command {cmd_str} failed with return code {return_code}, tail output are included " + f"below.\n{tail_output}\n" + ) def exec_cmd( cmd, *, - throw_on_error=True, extra_env=None, - capture_output=True, synchronous=True, - stream_output=False, **kwargs, ): """ A convenience wrapper of `subprocess.Popen` for running a command from a Python script. - - :param cmd: The command to run, as a list of strings. - :param throw_on_error: If True, raises an Exception if the exit code of the program is nonzero. - :param extra_env: Extra environment variables to be defined when running the child process. - If this argument is specified, `kwargs` cannot contain `env`. - :param capture_output: If True, stdout and stderr will be captured and included in an exception - message on failure; if False, these streams won't be captured. - :param synchronous: If True, wait for the command to complete and return a CompletedProcess - instance, If False, does not wait for the command to complete and return - a Popen instance, and ignore the `throw_on_error` argument. - :param stream_output: If True, stream the command's stdout and stderr to `sys.stdout` - as a unified stream during execution. - If False, do not stream the command's stdout and stderr to `sys.stdout`. - :param kwargs: Keyword arguments (except `text`) passed to `subprocess.Popen`. - :return: If synchronous is True, return a `subprocess.CompletedProcess` instance, - otherwise return a Popen instance. + If `synchronous` is True, wait until the process terminated and if subprocess return code + is not 0, raise error containing last 100 lines output. + If `synchronous` is False, return an `Popen` instance and a deque instance holding tail + outputs. + The subprocess stdout / stderr output will be streamly redirected to current process stdout. """ - illegal_kwargs = set(kwargs.keys()).intersection({"text"}) + illegal_kwargs = set(kwargs.keys()).intersection({"text", "stdout", "stderr"}) if illegal_kwargs: raise ValueError(f"`kwargs` cannot contain {list(illegal_kwargs)}") @@ -93,56 +67,34 @@ def exec_cmd( if extra_env is not None and env is not None: raise ValueError("`extra_env` and `env` cannot be used at the same time") - if capture_output and stream_output: - raise ValueError( - "`capture_output=True` and `stream_output=True` cannot be specified at the same time" - ) - env = env if extra_env is None else {**os.environ, **extra_env} - # In Python < 3.8, `subprocess.Popen` doesn't accept a command containing path-like - # objects (e.g. `["ls", pathlib.Path("abc")]`) on Windows. To avoid this issue, - # stringify all elements in `cmd`. Note `str(pathlib.Path("abc"))` returns 'abc'. - cmd = list(map(str, cmd)) - - if capture_output or stream_output: - if kwargs.get("stdout") is not None or kwargs.get("stderr") is not None: - raise ValueError( - "stdout and stderr arguments may not be used with capture_output or stream_output" - ) - kwargs["stdout"] = subprocess.PIPE - if capture_output: - kwargs["stderr"] = subprocess.PIPE - elif stream_output: - # Redirect stderr to stdout in order to combine the streams for unified printing to - # `sys.stdout`, as documented in - # https://docs.python.org/3/library/subprocess.html#subprocess.run - kwargs["stderr"] = subprocess.STDOUT - process = subprocess.Popen( cmd, env=env, text=True, - **kwargs, + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT, ) + + tail_output_deque = collections.deque(maxlen=100) + + def redirect_log_thread_fn(): + for line in process.stdout: + # collect tail logs by `tail_output_deque` + tail_output_deque.append(line) + + # redirect to stdout. + sys.stdout.write(line) + + threading.Thread(target=redirect_log_thread_fn, args=()).start() + if not synchronous: - return process - - if stream_output: - for output_char in iter(lambda: process.stdout.read(1), ""): - sys.stdout.write(output_char) - - stdout, stderr = process.communicate() - returncode = process.poll() - comp_process = subprocess.CompletedProcess( - process.args, - returncode=returncode, - stdout=stdout, - stderr=stderr, - ) - if throw_on_error and returncode != 0: - raise ShellCommandException.from_completed_process(comp_process) - return comp_process + return process, tail_output_deque + + return_code = process.wait() + if return_code != 0: + raise RuntimeError(gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque)) def check_port_open(host, port): From a813f32fea54625446687c5e1004b44c93e3c9a7 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 29 Nov 2022 20:19:50 +0800 Subject: [PATCH 082/121] improve worker node error exception Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ab1c2768f365..0b0892a8ab05 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -68,6 +68,7 @@ def __init__(self, address, head_proc, spark_job_group_id, num_workers_node): self.ray_context = None self.is_shutdown = False self.num_worker_nodes = num_workers_node + self.background_job_exception = None def _cancel_background_spark_job(self): get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) @@ -75,6 +76,9 @@ def _cancel_background_spark_job(self): def connect(self): import ray + if self.background_job_exception is not None: + raise RuntimeError("Starting ray workers failed.") from self.background_job_exception + if self.is_shutdown: raise RuntimeError( "The ray cluster has been shut down or it failed to start." @@ -332,14 +336,7 @@ def _init_ray_cluster( *_convert_ray_node_options(head_options), ] - _logger.info(f"Start Ray head, command: {' '.join(ray_head_node_cmd)}") - - if is_in_databricks_runtime(): - _display_databricks_driver_proxy_url( - spark.sparkContext, - ray_dashboard_port, - "Ray Cluster Dashboard" - ) + _logger.info(f"Starting Ray head, command: {' '.join(ray_head_node_cmd)}") ray_head_proc, tail_output_deque = exec_cmd( ray_head_node_cmd, @@ -371,6 +368,13 @@ def _init_ray_cluster( _logger.info("Ray head node started.") + if is_in_databricks_runtime(): + _display_databricks_driver_proxy_url( + spark.sparkContext, + ray_dashboard_port, + "Ray Cluster Dashboard" + ) + # NB: # In order to start ray worker nodes on spark cluster worker machines, # We launch a background spark job: @@ -534,10 +538,10 @@ def backgroud_job_thread_fn(): spark.sparkContext.parallelize( list(range(num_worker_nodes)), num_worker_nodes ).mapPartitions(ray_cluster_job_mapper).collect() - finally: + except Exception as e: # NB: # The background spark job is designed to running forever until it is killed, - # So this `finally` block is reachable only when: + # The exception might be raised in following cases: # 1. The background job raises unexpected exception (i.e. ray cluster dies # unexpectedly) # 2. User explicitly orders shutting down the ray cluster. @@ -546,6 +550,7 @@ def backgroud_job_thread_fn(): # For case 1 and 3, only ray workers are killed, but driver side ray head might still # be running and the ray context might be in connected status. In order to disconnect # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. + ray_cluster_handler.background_job_exception = e ray_cluster_handler.shutdown() try: @@ -553,7 +558,7 @@ def backgroud_job_thread_fn(): target=inheritable_thread_target(backgroud_job_thread_fn), args=() ).start() - time.sleep(5) # wait background spark task starting. + time.sleep(30) # wait background spark task starting. if is_in_databricks_runtime(): try: From 7583cc827b657b14c43142a5b963a8f4f2f4a585 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 29 Nov 2022 20:23:03 +0800 Subject: [PATCH 083/121] improve worker node error exception Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 0b0892a8ab05..318b14e8a704 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -77,7 +77,7 @@ def connect(self): import ray if self.background_job_exception is not None: - raise RuntimeError("Starting ray workers failed.") from self.background_job_exception + raise RuntimeError("Ray workers has exited.") from self.background_job_exception if self.is_shutdown: raise RuntimeError( From e95e4ebf3869eb478b664ec7dabe7ccb5ae23e2a Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 29 Nov 2022 21:19:20 +0800 Subject: [PATCH 084/121] address comments Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 318b14e8a704..d85814b2f075 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -30,7 +30,7 @@ ) if not sys.platform.startswith("linux"): - raise RuntimeError("Ray on spark ony supports linux system.") + raise RuntimeError("Ray on spark only supports running on Linux.") _logger = logging.getLogger("ray.util.spark") _logger.setLevel(logging.INFO) @@ -90,7 +90,7 @@ def connect(self): last_alive_worker_count = 0 last_progress_move_time = time.time() while True: - time.sleep(10) + time.sleep(_RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL) cur_alive_worker_count = len( [ node @@ -170,6 +170,11 @@ def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] +_RAY_HEAD_STARTUP_TIMEOUT = 40 +_BACKGROUND_JOB_STARTUP_TIMEOUT = 30 +_RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 + + def _init_ray_cluster( num_worker_nodes=None, total_cpus=None, @@ -319,7 +324,6 @@ def _init_ray_cluster( f"--temp-dir={ray_temp_dir}", "--block", "--head", - "--disable-usage-stats", f"--node-ip-address={ray_head_ip}", f"--port={ray_head_port}", "--include-dashboard=true", @@ -344,7 +348,7 @@ def _init_ray_cluster( ) # wait ray head node spin up. - for _ in range(40): + for _ in range(_RAY_HEAD_STARTUP_TIMEOUT): time.sleep(1) if ray_head_proc.poll() is not None: # ray head node process terminated. @@ -509,7 +513,7 @@ def setup_sigterm_on_parent_death(): num_workers_node=num_worker_nodes, ) - def backgroud_job_thread_fn(): + def background_job_thread_fn(): try: spark.sparkContext.setJobGroup( @@ -555,10 +559,10 @@ def backgroud_job_thread_fn(): try: threading.Thread( - target=inheritable_thread_target(backgroud_job_thread_fn), args=() + target=inheritable_thread_target(background_job_thread_fn), args=() ).start() - time.sleep(30) # wait background spark task starting. + time.sleep(_BACKGROUND_JOB_STARTUP_TIMEOUT) # wait background spark task starting. if is_in_databricks_runtime(): try: From 47381ddcf77483ed4b66a4db544005036a213aee Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 1 Dec 2022 21:45:46 +0800 Subject: [PATCH 085/121] worker port range allocation Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 12 ++-- python/ray/util/spark/utils.py | 89 +++++++++++++++++++++++++-- 2 files changed, 89 insertions(+), 12 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index d85814b2f075..a61e76d963f3 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -24,7 +24,7 @@ get_max_num_concurrent_tasks, get_target_spark_tasks, _HEAP_TO_SHARED_RATIO, - _acquire_lock_for_ray_worker_node_startup, + _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup, _display_databricks_driver_proxy_url, gen_cmd_exec_failure_msg, ) @@ -406,16 +406,14 @@ def ray_cluster_job_mapper(_): _worker_logger = logging.getLogger("ray.util.spark.worker") context = TaskContext.get() - task_id = context.partitionId() - _acquire_lock_for_ray_worker_node_startup() + worker_port_range_begin, worker_port_range_end = \ + _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup() # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. os.makedirs(ray_temp_dir, exist_ok=True) - min_worker_port = 20000 + task_id * 1000 - max_worker_port = min_worker_port + 999 ray_worker_node_cmd = [ ray_exec_path, "start", @@ -426,8 +424,8 @@ def ray_cluster_job_mapper(_): f"--address={ray_head_ip}:{ray_head_port}", f"--memory={ray_worker_node_heap_mem_bytes}", f"--object-store-memory={ray_worker_node_object_store_mem_bytes}", - f"--min-worker-port={min_worker_port}", - f"--max-worker-port={max_worker_port}", + f"--min-worker-port={worker_port_range_begin}", + f"--max-worker-port={worker_port_range_end - 1}", *_convert_ray_node_options(worker_options), ] diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 52124fd4fa83..0e6a76063fbf 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -325,7 +325,7 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return gpu_addr_list -def _acquire_lock_for_ray_worker_node_startup(): +def _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup(): """ If we start multiple ray workers on a machine concurrently, some ray worker processes might fail due to ray port conflicts, this is because race condition on getting free @@ -334,12 +334,31 @@ def _acquire_lock_for_ray_worker_node_startup(): to ensure that port acquisition does not create a resource contention issue due to a race condition. - Returns: None + After acquiring lock, it will allocate port range for worker ports + (for ray node config --min-worker-port and --max-worker-port). + Because on a spark cluster, multiple ray cluster might be created, so on one spark worker + machine, there might be multiple ray worker nodes running, these worker nodes might belong + to different ray cluster, and we must ensure these ray nodes on the same machine using + non-overlapping worker port range, to achieve this, in this function, it creates a file + `/tmp/ray_on_spark_worker_port_allocation.txt` file, the file format is composed of multiple + lines, each line contains 2 number: `pid` and `port_range_slot_index`, + each port range slot allocates 1000 ports, and corresponding port range is: + range_begin (inclusive): 20000 + port_range_slot_index * 1000 + range_end (exclusive): range_begin + 1000 + In this function, it first scans `/tmp/ray_on_spark_worker_port_allocation.txt` file, + removing lines that containing dead process pid, then find the first unused + port_range_slot_index, then regenerate this file, and return the allocated port range. + + Returns: Allocated port range for current worker ports """ + import psutil + def acquire_lock(file_path): mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC try: fd = os.open(file_path, mode) + # The lock file must be readable / writable to all users. + os.chmod(file_path, 0o0777) # Allow for retrying getting a file lock a maximum number of seconds max_lock_iter = 600 for _ in range(max_lock_iter): @@ -351,7 +370,7 @@ def acquire_lock(file_path): else: # Acquire lock successfully. return fd - time.sleep(1.0) + time.sleep(10) raise TimeoutError(f"Acquiring lock on file {file_path} timeout.") except Exception: os.close(fd) @@ -370,13 +389,73 @@ def acquire_lock(file_path): pass lock_fd = acquire_lock(lock_file_path) - def hold_lock_for_10s_and_release(): - time.sleep(10) + def release_lock(): fcntl.flock(lock_fd, fcntl.LOCK_UN) os.close(lock_fd) + try: + port_alloc_file = "/tmp/ray_on_spark_worker_port_allocation.txt" + + # NB: reading / writing `port_alloc_file` is protected by exclusive lock + # on file `lock_file_path` + if os.path.exists(port_alloc_file): + with open(port_alloc_file, mode="r") as fp: + port_alloc_data = fp.read() + port_alloc_table = [line.split(" ") for line in port_alloc_data.strip().split("\n")] + port_alloc_table = [ + (int(pid_str), int(slot_index_str)) + for pid_str, slot_index_str in port_alloc_table + ] + else: + port_alloc_table = [] + with open(port_alloc_file, mode="w"): + pass + # The port range allocation file must be readable / writable to all users. + os.chmod(port_alloc_file, 0o0777) + + port_alloc_map = { + pid: slot_index + for pid, slot_index in port_alloc_table + if psutil.pid_exists(pid) # remove slot used by dead process + } + + allocated_slot_set = set(port_alloc_map.values()) + + if len(allocated_slot_set) == 0: + new_slot_index = 0 + else: + new_slot_index = max(allocated_slot_set) + 1 + for index in range(new_slot_index): + if index not in allocated_slot_set: + new_slot_index = index + break + + port_alloc_map[os.getpid()] = new_slot_index + + with open(port_alloc_file, mode="w") as fp: + for pid, slot_index in port_alloc_map.items(): + fp.write(f"{pid} {slot_index}\n") + + worker_port_range_begin = 20000 + new_slot_index * 1000 + worker_port_range_end = worker_port_range_begin + 1000 + + if worker_port_range_end > 65536: + raise RuntimeError( + "Too many ray worker nodes are running on this machine, cannot " + "allocate worker port range for new ray worker node." + ) + except Exception: + release_lock() + raise + + def hold_lock_for_10s_and_release(): + time.sleep(10) + release_lock() + threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() + return worker_port_range_begin, worker_port_range_end + def _display_databricks_driver_proxy_url(spark_context, port, title): from dbruntime.display import displayHTML From 8989f06d1a1137c64547e89db551f41476700fcd Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 2 Dec 2022 18:34:41 +0800 Subject: [PATCH 086/121] set dashboard agent port Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index a61e76d963f3..f1b79abe8227 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -198,7 +198,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 009.") + _logger.warning("Test version 010.") head_options = head_options or {} worker_options = worker_options or {} @@ -297,6 +297,9 @@ def _init_ray_cluster( ray_dashboard_port = get_random_unused_port( ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port] ) + ray_dashboard_agent_port = get_random_unused_port( + ray_head_ip, min_port=10000, max_port=11000, + ) _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") @@ -329,6 +332,7 @@ def _init_ray_cluster( "--include-dashboard=true", "--dashboard-host=0.0.0.0", f"--dashboard-port={ray_dashboard_port}", + f"--dashboard-agent-listen-port={ray_dashboard_agent_port}", # disallow ray tasks with cpu requirements from being scheduled on the head node. f"--num-cpus=0", # limit the memory allocation to the head node (actual usage may increase beyond this From 2c7c0cd7c1130209d9e434a447729c4d2072891b Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 2 Dec 2022 18:52:52 +0800 Subject: [PATCH 087/121] fix Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index f1b79abe8227..2f2749079e68 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -298,7 +298,7 @@ def _init_ray_cluster( ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port] ) ray_dashboard_agent_port = get_random_unused_port( - ray_head_ip, min_port=10000, max_port=11000, + ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port, ray_dashboard_port] ) _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") From f2ab57dbdce6b48d3884b72988e57634e8043be0 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 2 Dec 2022 20:18:04 +0800 Subject: [PATCH 088/121] update worker dashboard_agent_port Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 2f2749079e68..0ee0c4c31bb2 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -198,7 +198,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 010.") + _logger.warning("Test version 011.") head_options = head_options or {} worker_options = worker_options or {} @@ -418,6 +418,9 @@ def ray_cluster_job_mapper(_): # local log dir and temp dir again. os.makedirs(ray_temp_dir, exist_ok=True) + ray_worker_node_dashboard_agent_port = get_random_unused_port( + ray_head_ip, min_port=10000, max_port=20000 + ) ray_worker_node_cmd = [ ray_exec_path, "start", @@ -430,6 +433,7 @@ def ray_cluster_job_mapper(_): f"--object-store-memory={ray_worker_node_object_store_mem_bytes}", f"--min-worker-port={worker_port_range_begin}", f"--max-worker-port={worker_port_range_end - 1}", + f"--dashboard-agent-listen-port={ray_worker_node_dashboard_agent_port}", *_convert_ray_node_options(worker_options), ] From c32aeb4a678776cbaa7810b597ab2bb50fe04a9d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 5 Dec 2022 20:59:20 +0800 Subject: [PATCH 089/121] auto clean temp dir --- python/ray/util/spark/cluster_init.py | 32 +++++++-------- python/ray/util/spark/start_ray_node.py | 53 +++++++++++++++++++++++++ 2 files changed, 68 insertions(+), 17 deletions(-) create mode 100644 python/ray/util/spark/start_ray_node.py diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 0ee0c4c31bb2..513a9053a9a8 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -150,7 +150,7 @@ def shutdown(self): f"{repr(e)}" ) try: - self.head_proc.kill() + self.head_proc.terminate() except Exception as e: # swallow exception. _logger.warning( @@ -171,7 +171,7 @@ def _convert_ray_node_options(options): _RAY_HEAD_STARTUP_TIMEOUT = 40 -_BACKGROUND_JOB_STARTUP_TIMEOUT = 30 +_BACKGROUND_JOB_STARTUP_WAIT = 30 _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 @@ -198,7 +198,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 011.") + _logger.warning("Test version 012.") head_options = head_options or {} worker_options = worker_options or {} @@ -303,17 +303,13 @@ def _init_ray_cluster( _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") - ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - - temp_dir_unique_suffix = uuid.uuid4().hex[:4] + temp_dir_unique_suffix = uuid.uuid4().hex[:8] # TODO: Set individual temp dir for ray worker nodes, and auto GC temp data when ray node exits # See https://github.com/ray-project/ray/issues/28876#issuecomment-1322016494 if ray_temp_root_dir is None: if is_in_databricks_runtime(): - # TODO: Change `ray_temp_root_dir` to use "/local_disk0/tmp" once - # https://github.com/ray-project/ray/issues/30677 is fixed. - ray_temp_root_dir = "/tmp" + ray_temp_root_dir = "/local_disk0/tmp" else: ray_temp_root_dir = "/tmp" ray_temp_dir = os.path.join( @@ -322,8 +318,9 @@ def _init_ray_cluster( os.makedirs(ray_temp_dir, exist_ok=True) ray_head_node_cmd = [ - ray_exec_path, - "start", + sys.executable, + "-m", + "ray.util.spark.start_ray_node", f"--temp-dir={ray_temp_dir}", "--block", "--head", @@ -363,7 +360,7 @@ def _init_ray_cluster( if not check_port_open(ray_head_ip, ray_head_port): if ray_head_proc.poll() is None: # Ray head GCS service is down. Kill ray head node. - ray_head_proc.kill() + ray_head_proc.terminate() # wait killing complete. time.sleep(0.5) @@ -422,8 +419,9 @@ def ray_cluster_job_mapper(_): ray_head_ip, min_port=10000, max_port=20000 ) ray_worker_node_cmd = [ - ray_exec_path, - "start", + sys.executable, + "-m", + "ray.util.spark.start_ray_node", f"--temp-dir={ray_temp_dir}", f"--num-cpus={num_spark_task_cpus}", "--block", @@ -503,7 +501,7 @@ def setup_sigterm_on_parent_death(): # submitted task. # Currently all workers uses the same ray_temp_dir as head side, # and one machine might run mulitple ray workers concurrently, - # so we cannot directly delele the temp dir here. + # so we cannot directly delete the temp dir here. # NB: Not reachable. yield 0 @@ -560,15 +558,15 @@ def background_job_thread_fn(): # For case 1 and 3, only ray workers are killed, but driver side ray head might still # be running and the ray context might be in connected status. In order to disconnect # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. - ray_cluster_handler.background_job_exception = e ray_cluster_handler.shutdown() + ray_cluster_handler.background_job_exception = e try: threading.Thread( target=inheritable_thread_target(background_job_thread_fn), args=() ).start() - time.sleep(_BACKGROUND_JOB_STARTUP_TIMEOUT) # wait background spark task starting. + time.sleep(_BACKGROUND_JOB_STARTUP_WAIT) # wait background spark task starting. if is_in_databricks_runtime(): try: diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py new file mode 100644 index 000000000000..bd232ccc5e85 --- /dev/null +++ b/python/ray/util/spark/start_ray_node.py @@ -0,0 +1,53 @@ +import os.path +import subprocess +import sys +import time +import shutil +import fcntl +import signal + + +_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 + + +if __name__ == '__main__': + arg_list = sys.argv[1:] + + temp_dir_arg_prefix = "--temp-dir=" + temp_dir = None + + for arg in arg_list: + if arg.startswith(temp_dir_arg_prefix): + temp_dir = arg[len(temp_dir_arg_prefix):] + + if temp_dir is None: + raise ValueError("Please explicitly set --temp-dir option.") + + temp_dir = os.path.normpath(temp_dir) + + ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") + + process = subprocess.Popen([ray_exec_path, "start", *arg_list], text=True) + + def sigterm_handler(*args): + process.terminate() + + time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) + lock_file = temp_dir + ".lock" + try: + mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC + lock_fd = os.open(lock_file, mode) + # because one spark job might start multiple ray worker node on one spark worker + # machine, and they use the same temp dir, so acquire an exclusive file lock when + # deleting the temp dir. + fcntl.flock(lock_fd, fcntl.LOCK_EX) + + if os.path.exists(temp_dir): + shutil.rmtree(temp_dir, ignore_errors=True) + finally: + fcntl.flock(lock_fd, fcntl.LOCK_UN) + os.close(lock_fd) + + signal.signal(signal.SIGTERM, sigterm_handler) + + sys.exit(process.wait()) From 66569f88b732f3a03ecb8f8fc790d3abb98ce1f9 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 5 Dec 2022 22:14:26 +0800 Subject: [PATCH 090/121] fix --- python/ray/util/spark/utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 0e6a76063fbf..650034b685a0 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -75,6 +75,7 @@ def exec_cmd( text=True, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, + **kwargs ) tail_output_deque = collections.deque(maxlen=100) From afc50470c3e32ff02165c4ff46c8a7bd330f5f60 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 5 Dec 2022 22:48:18 +0800 Subject: [PATCH 091/121] object_store_memory_per_node Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 24 +++++++----------------- python/ray/util/spark/utils.py | 20 +++++++++++--------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 513a9053a9a8..c82000f33337 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -23,7 +23,6 @@ get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, - _HEAP_TO_SHARED_RATIO, _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup, _display_databricks_driver_proxy_url, gen_cmd_exec_failure_msg, @@ -181,7 +180,7 @@ def _init_ray_cluster( total_gpus=None, total_heap_memory_bytes=None, total_object_store_memory_bytes=None, - heap_to_object_store_memory_ratio=None, + object_store_memory_per_node=None, head_options=None, worker_options=None, ray_temp_root_dir=None, @@ -228,13 +227,10 @@ def _init_ray_cluster( spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0") ) - if heap_to_object_store_memory_ratio is None: - heap_to_object_store_memory_ratio = _HEAP_TO_SHARED_RATIO - ( ray_worker_node_heap_mem_bytes, ray_worker_node_object_store_mem_bytes, - ) = get_avail_mem_per_ray_worker_node(spark, heap_to_object_store_memory_ratio) + ) = get_avail_mem_per_ray_worker_node(spark, object_store_memory_per_node) if total_gpus is not None and num_spark_task_gpus == 0: raise ValueError( @@ -497,12 +493,6 @@ def setup_sigterm_on_parent_death(): preexec_fn=setup_sigterm_on_parent_death, ) - # TODO: Delete the worker temp and log directories at the conclusion of running the - # submitted task. - # Currently all workers uses the same ray_temp_dir as head side, - # and one machine might run mulitple ray workers concurrently, - # so we cannot directly delete the temp dir here. - # NB: Not reachable. yield 0 @@ -598,7 +588,7 @@ def init_ray_cluster( total_gpus=None, total_heap_memory_bytes=None, total_object_store_memory_bytes=None, - heap_to_object_store_memory_ratio=None, + object_store_memory_per_node=None, head_options=None, worker_options=None, ray_temp_root_dir=None, @@ -631,9 +621,9 @@ def init_ray_cluster( to utilize. total_object_store_memory_bytes: The total amount of object store memory (in bytes) for the ray cluster to utilize. - heap_to_object_store_memory_ratio: The ratio of per-ray worker node available memory to the - size of the `/dev/shm` capacity per worker on the spark worker. Without modification, - this ratio is 0.4. + object_store_memory_per_node: Object store memory available to per-ray worker node, but + it cannot exceed + "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker" head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster @@ -657,7 +647,7 @@ def init_ray_cluster( total_gpus=total_gpus, total_heap_memory_bytes=total_heap_memory_bytes, total_object_store_memory_bytes=total_object_store_memory_bytes, - heap_to_object_store_memory_ratio=heap_to_object_store_memory_ratio, + object_store_memory_per_node=object_store_memory_per_node, head_options=head_options, worker_options=worker_options, ray_temp_root_dir=ray_temp_root_dir, diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 650034b685a0..05ed92d70154 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -10,7 +10,6 @@ _MEMORY_BUFFER_OFFSET = 0.8 -_HEAP_TO_SHARED_RATIO = 0.4 def is_in_databricks_runtime(): @@ -168,7 +167,7 @@ def _get_cpu_cores(): def _calc_mem_per_ray_worker_node( - num_task_slots, physical_mem_bytes, shared_mem_bytes, heap_to_object_store_ratio + num_task_slots, physical_mem_bytes, shared_mem_bytes, object_store_memory_per_node ): available_physical_mem_per_node = int( physical_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET @@ -177,12 +176,15 @@ def _calc_mem_per_ray_worker_node( shared_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET ) - object_store_bytes = int( - min( - available_physical_mem_per_node * heap_to_object_store_ratio, - available_shared_mem_per_node, + if object_store_memory_per_node is None: + object_store_bytes = available_shared_mem_per_node + else: + object_store_bytes = int( + min( + object_store_memory_per_node, + available_shared_mem_per_node, + ) ) - ) heap_mem_bytes = ( available_physical_mem_per_node - object_store_bytes ) @@ -273,7 +275,7 @@ def get_target_spark_tasks( return num_spark_tasks -def get_avail_mem_per_ray_worker_node(spark, heap_to_object_store_ratio): +def get_avail_mem_per_ray_worker_node(spark, object_store_memory_per_node): """ Return the available heap memory and object store memory for each ray worker. NB: We have one ray node per spark task. @@ -297,7 +299,7 @@ def mapper(_): num_task_slots, physical_mem_bytes, shared_mem_bytes, - heap_to_object_store_ratio, + object_store_memory_per_node, ) return ray_worker_node_heap_mem_bytes, ray_worker_node_object_store_bytes, None except Exception as e: From 1c4b34bb83e5312ff831ee8522984e41dbd3f1f3 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 6 Dec 2022 21:09:26 +0800 Subject: [PATCH 092/121] update start node code Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 5 ++++ python/ray/util/spark/start_ray_node.py | 40 ++++++++++++++++--------- 2 files changed, 31 insertions(+), 14 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index c82000f33337..18267a3f623f 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -173,6 +173,8 @@ def _convert_ray_node_options(options): _BACKGROUND_JOB_STARTUP_WAIT = 30 _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 +_RAY_HEAD_NODE_TAG_FILE = "used_by_ray_on_spark_head_node" + def _init_ray_cluster( num_worker_nodes=None, @@ -313,6 +315,9 @@ def _init_ray_cluster( ) os.makedirs(ray_temp_dir, exist_ok=True) + with open(os.path.join(ray_temp_dir, _RAY_HEAD_NODE_TAG_FILE), "w"): + pass + ray_head_node_cmd = [ sys.executable, "-m", diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index bd232ccc5e85..e5b496118246 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -6,6 +6,8 @@ import fcntl import signal +from .cluster_init import _RAY_HEAD_NODE_TAG_FILE + _WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 @@ -23,7 +25,10 @@ if temp_dir is None: raise ValueError("Please explicitly set --temp-dir option.") + is_head_node = True if "--head" in arg_list else False + temp_dir = os.path.normpath(temp_dir) + head_node_is_on_local = os.path.exists(os.path.join(temp_dir, _RAY_HEAD_NODE_TAG_FILE)) ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") @@ -32,22 +37,29 @@ def sigterm_handler(*args): process.terminate() - time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) - lock_file = temp_dir + ".lock" try: - mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC - lock_fd = os.open(lock_file, mode) - # because one spark job might start multiple ray worker node on one spark worker - # machine, and they use the same temp dir, so acquire an exclusive file lock when - # deleting the temp dir. - fcntl.flock(lock_fd, fcntl.LOCK_EX) - - if os.path.exists(temp_dir): - shutil.rmtree(temp_dir, ignore_errors=True) + # if a worker node and head node runs on the same machine, + # when the worker node exits, we cannot delete the temp dir immediately + # because head node is still using it. + if is_head_node or not head_node_is_on_local: + time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) + lock_file = temp_dir + ".lock" + try: + mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC + lock_fd = os.open(lock_file, mode) + # because one spark job might start multiple ray worker node on one spark worker + # machine, and they use the same temp dir, so acquire an exclusive file lock when + # deleting the temp dir. + fcntl.flock(lock_fd, fcntl.LOCK_EX) + + if os.path.exists(temp_dir): + shutil.rmtree(temp_dir, ignore_errors=True) + finally: + fcntl.flock(lock_fd, fcntl.LOCK_UN) + os.close(lock_fd) finally: - fcntl.flock(lock_fd, fcntl.LOCK_UN) - os.close(lock_fd) + os._exit(143) signal.signal(signal.SIGTERM, sigterm_handler) - sys.exit(process.wait()) + process.wait() From f42446179a925ca5b30e413538b9b1819be909b5 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 6 Dec 2022 21:50:58 +0800 Subject: [PATCH 093/121] fix Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 26 ++++++++++++++----------- python/ray/util/spark/start_ray_node.py | 2 +- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 18267a3f623f..ef83c1161918 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -66,10 +66,12 @@ def __init__(self, address, head_proc, spark_job_group_id, num_workers_node): self.spark_job_group_id = spark_job_group_id self.ray_context = None self.is_shutdown = False + self.spark_job_is_canceled = False self.num_worker_nodes = num_workers_node self.background_job_exception = None def _cancel_background_spark_job(self): + self.spark_job_is_canceled = True get_spark_session().sparkContext.cancelJobGroup(self.spark_job_group_id) def connect(self): @@ -133,21 +135,22 @@ def disconnect(self): else: _logger.warning("Already disconnected from this ray cluster.") - def shutdown(self): + def shutdown(self, cancel_background_job=True): """ Shutdown the ray cluster created by the `init_ray_cluster` API. """ if not self.is_shutdown: if self.ray_context is not None: self.disconnect() - try: - self._cancel_background_spark_job() - except Exception as e: - # swallow exception. - _logger.warning( - f"An error occurred while cancelling the ray cluster background spark job: " - f"{repr(e)}" - ) + if cancel_background_job: + try: + self._cancel_background_spark_job() + except Exception as e: + # swallow exception. + _logger.warning( + f"An error occurred while cancelling the ray cluster background spark job: " + f"{repr(e)}" + ) try: self.head_proc.terminate() except Exception as e: @@ -553,8 +556,9 @@ def background_job_thread_fn(): # For case 1 and 3, only ray workers are killed, but driver side ray head might still # be running and the ray context might be in connected status. In order to disconnect # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. - ray_cluster_handler.shutdown() - ray_cluster_handler.background_job_exception = e + if not ray_cluster_handler.spark_job_is_canceled: + ray_cluster_handler.shutdown(cancel_background_job=False) + ray_cluster_handler.background_job_exception = e try: threading.Thread( diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index e5b496118246..e6c5b4c975bf 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -62,4 +62,4 @@ def sigterm_handler(*args): signal.signal(signal.SIGTERM, sigterm_handler) - process.wait() + sys.exit(process.wait()) From 613a3362d72b5cf49c5b49a68d2052d438f5d0ce Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 9 Dec 2022 18:06:33 +0800 Subject: [PATCH 094/121] improve start_ray_node code Signed-off-by: Weichen Xu --- python/ray/util/spark/__init__.py | 2 +- python/ray/util/spark/cluster_init.py | 7 +-- python/ray/util/spark/start_ray_node.py | 70 ++++++++++++++----------- 3 files changed, 41 insertions(+), 38 deletions(-) diff --git a/python/ray/util/spark/__init__.py b/python/ray/util/spark/__init__.py index 8705a0475a39..8bac1d3ad290 100644 --- a/python/ray/util/spark/__init__.py +++ b/python/ray/util/spark/__init__.py @@ -1,4 +1,4 @@ -from ray.util.spark.cluster_init import RayClusterOnSpark, init_ray_cluster +from ray.util.spark.cluster_init import init_ray_cluster, shutdown_ray_cluster __all__ = [ "init_ray_cluster", diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ef83c1161918..a355cf97c7c2 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -176,8 +176,6 @@ def _convert_ray_node_options(options): _BACKGROUND_JOB_STARTUP_WAIT = 30 _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 -_RAY_HEAD_NODE_TAG_FILE = "used_by_ray_on_spark_head_node" - def _init_ray_cluster( num_worker_nodes=None, @@ -202,7 +200,7 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 012.") + _logger.warning("Test version 013.") head_options = head_options or {} worker_options = worker_options or {} @@ -318,9 +316,6 @@ def _init_ray_cluster( ) os.makedirs(ray_temp_dir, exist_ok=True) - with open(os.path.join(ray_temp_dir, _RAY_HEAD_NODE_TAG_FILE), "w"): - pass - ray_head_node_cmd = [ sys.executable, "-m", diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index e6c5b4c975bf..23bce7c14754 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -6,8 +6,6 @@ import fcntl import signal -from .cluster_init import _RAY_HEAD_NODE_TAG_FILE - _WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 @@ -25,41 +23,51 @@ if temp_dir is None: raise ValueError("Please explicitly set --temp-dir option.") - is_head_node = True if "--head" in arg_list else False - temp_dir = os.path.normpath(temp_dir) - head_node_is_on_local = os.path.exists(os.path.join(temp_dir, _RAY_HEAD_NODE_TAG_FILE)) ray_exec_path = os.path.join(os.path.dirname(sys.executable), "ray") - process = subprocess.Popen([ray_exec_path, "start", *arg_list], text=True) - - def sigterm_handler(*args): - process.terminate() + lock_file = temp_dir + ".lock" + lock_fd = os.open(lock_file, os.O_RDWR | os.O_CREAT | os.O_TRUNC) + def try_clean_temp_dir_at_exit(): try: - # if a worker node and head node runs on the same machine, - # when the worker node exits, we cannot delete the temp dir immediately - # because head node is still using it. - if is_head_node or not head_node_is_on_local: - time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) - lock_file = temp_dir + ".lock" - try: - mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC - lock_fd = os.open(lock_file, mode) - # because one spark job might start multiple ray worker node on one spark worker - # machine, and they use the same temp dir, so acquire an exclusive file lock when - # deleting the temp dir. - fcntl.flock(lock_fd, fcntl.LOCK_EX) - - if os.path.exists(temp_dir): - shutil.rmtree(temp_dir, ignore_errors=True) - finally: - fcntl.flock(lock_fd, fcntl.LOCK_UN) - os.close(lock_fd) + # Wait for a while to ensure the children processes of the ray node all exited. + time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) + + # Start clean the temp dir, + try: + # try to upgrade shared lock to exclusive lock first + # to ensure removing dir safely. + fcntl.flock(lock_fd, fcntl.LOCK_EX | fcntl.LOCK_NB) + shutil.rmtree(temp_dir, ignore_errors=True) + except BlockingIOError: + # Lock is used by other processes, representing there are other ray nodes + # running, skip cleaning temp-dir. + pass + except Exception: + # swallow any exception. + pass finally: + fcntl.flock(lock_fd, fcntl.LOCK_UN) + os.close(lock_fd) + + try: + # Mutilple ray nodes might start on the same machine, and they are using the + # same temp directory, adding a shared lock representing current ray node is + # using the temp directory. + fcntl.flock(lock_fd, fcntl.LOCK_SH) + process = subprocess.Popen([ray_exec_path, "start", *arg_list], text=True) + + def sigterm_handler(*args): + process.terminate() + try_clean_temp_dir_at_exit() os._exit(143) - signal.signal(signal.SIGTERM, sigterm_handler) - - sys.exit(process.wait()) + signal.signal(signal.SIGTERM, sigterm_handler) + ret_code = process.wait() + try_clean_temp_dir_at_exit() + sys.exit(ret_code) + except Exception: + try_clean_temp_dir_at_exit() + raise From 10b22c0848f5c89247df9319d108977cfb2f1f53 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 9 Dec 2022 19:12:22 +0800 Subject: [PATCH 095/121] update init_ray_cluster Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 7 +++++-- python/ray/util/spark/start_ray_node.py | 14 +++++++++----- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index a355cf97c7c2..ed76b3c1779c 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -645,7 +645,7 @@ def init_ray_cluster( "Current active ray cluster on spark haven't shut down. You cannot create a new ray " "cluster." ) - _active_ray_cluster = _init_ray_cluster( + cluster = _init_ray_cluster( num_worker_nodes=num_worker_nodes, total_cpus=total_cpus, total_gpus=total_gpus, @@ -657,7 +657,10 @@ def init_ray_cluster( ray_temp_root_dir=ray_temp_root_dir, safe_mode=safe_mode, ) - _active_ray_cluster.connect() + cluster.connect() # NB: this line might raise error. + + # If connect cluster successfully, set global _active_ray_cluster to be the started cluster. + _active_ray_cluster = cluster @PublicAPI(stability="alpha") diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 23bce7c14754..b6af0424be1b 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -35,15 +35,19 @@ def try_clean_temp_dir_at_exit(): # Wait for a while to ensure the children processes of the ray node all exited. time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) - # Start clean the temp dir, + # Release the shared lock, representing current ray node does not use the + # temp dir. + fcntl.flock(lock_fd, fcntl.LOCK_UN) + try: - # try to upgrade shared lock to exclusive lock first - # to ensure removing dir safely. + # Start clean the temp-dir, + # acquiring exclusive lock to ensure removing dir safely. fcntl.flock(lock_fd, fcntl.LOCK_EX | fcntl.LOCK_NB) shutil.rmtree(temp_dir, ignore_errors=True) except BlockingIOError: - # Lock is used by other processes, representing there are other ray nodes - # running, skip cleaning temp-dir. + # The file has active shared lock or exclusive lock, representing there + # are other ray nodes running, or other node running cleanup temp-dir routine. + # skip cleaning temp-dir. pass except Exception: # swallow any exception. From 6c32f4222215a292569b21e47b0f4c8cd5ea74bb Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 12 Dec 2022 15:15:41 +0800 Subject: [PATCH 096/121] head node PR_SET_PDEATHSIG Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 180 +++++++++++++++++++++----- python/ray/util/spark/utils.py | 146 +++------------------ 2 files changed, 163 insertions(+), 163 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ed76b3c1779c..98216c9e9793 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -1,7 +1,6 @@ import os -import shutil +import fcntl import socket -import subprocess import sys import time import threading @@ -23,9 +22,9 @@ get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, - _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup, _display_databricks_driver_proxy_url, gen_cmd_exec_failure_msg, + setup_sigterm_on_parent_death, ) if not sys.platform.startswith("linux"): @@ -177,6 +176,138 @@ def _convert_ray_node_options(options): _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 +def _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup(): + """ + If we start multiple ray workers on a machine concurrently, some ray worker processes + might fail due to ray port conflicts, this is because race condition on getting free + port and opening the free port. + To address the issue, this function use an exclusive file lock to delay the worker processes + to ensure that port acquisition does not create a resource contention issue due to a race + condition. + + After acquiring lock, it will allocate port range for worker ports + (for ray node config --min-worker-port and --max-worker-port). + Because on a spark cluster, multiple ray cluster might be created, so on one spark worker + machine, there might be multiple ray worker nodes running, these worker nodes might belong + to different ray cluster, and we must ensure these ray nodes on the same machine using + non-overlapping worker port range, to achieve this, in this function, it creates a file + `/tmp/ray_on_spark_worker_port_allocation.txt` file, the file format is composed of multiple + lines, each line contains 2 number: `pid` and `port_range_slot_index`, + each port range slot allocates 1000 ports, and corresponding port range is: + range_begin (inclusive): 20000 + port_range_slot_index * 1000 + range_end (exclusive): range_begin + 1000 + In this function, it first scans `/tmp/ray_on_spark_worker_port_allocation.txt` file, + removing lines that containing dead process pid, then find the first unused + port_range_slot_index, then regenerate this file, and return the allocated port range. + + Returns: Allocated port range for current worker ports + """ + import psutil + + def acquire_lock(file_path): + mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC + try: + fd = os.open(file_path, mode) + # The lock file must be readable / writable to all users. + os.chmod(file_path, 0o0777) + # Allow for retrying getting a file lock a maximum number of seconds + max_lock_iter = 600 + for _ in range(max_lock_iter): + try: + fcntl.flock(fd, fcntl.LOCK_EX | fcntl.LOCK_NB) + except BlockingIOError: + # Lock is used by other processes, continue loop to wait for lock available + pass + else: + # Acquire lock successfully. + return fd + time.sleep(10) + raise TimeoutError(f"Acquiring lock on file {file_path} timeout.") + except Exception: + os.close(fd) + + lock_file_path = "/tmp/ray_on_spark_worker_startup_barrier_lock.lock" + try: + lock_fd = acquire_lock(lock_file_path) + except TimeoutError: + # If timeout happens, the file lock might be hold by another process and that process + # does not release the lock in time by some unexpected reason. + # In this case, remove the existing lock file and create the file again, and then + # acquire file lock on the new file. + try: + os.remove(lock_file_path) + except Exception: + pass + lock_fd = acquire_lock(lock_file_path) + + def release_lock(): + fcntl.flock(lock_fd, fcntl.LOCK_UN) + os.close(lock_fd) + + try: + port_alloc_file = "/tmp/ray_on_spark_worker_port_allocation.txt" + + # NB: reading / writing `port_alloc_file` is protected by exclusive lock + # on file `lock_file_path` + if os.path.exists(port_alloc_file): + with open(port_alloc_file, mode="r") as fp: + port_alloc_data = fp.read() + port_alloc_table = [line.split(" ") for line in port_alloc_data.strip().split("\n")] + port_alloc_table = [ + (int(pid_str), int(slot_index_str)) + for pid_str, slot_index_str in port_alloc_table + ] + else: + port_alloc_table = [] + with open(port_alloc_file, mode="w"): + pass + # The port range allocation file must be readable / writable to all users. + os.chmod(port_alloc_file, 0o0777) + + port_alloc_map = { + pid: slot_index + for pid, slot_index in port_alloc_table + if psutil.pid_exists(pid) # remove slot used by dead process + } + + allocated_slot_set = set(port_alloc_map.values()) + + if len(allocated_slot_set) == 0: + new_slot_index = 0 + else: + new_slot_index = max(allocated_slot_set) + 1 + for index in range(new_slot_index): + if index not in allocated_slot_set: + new_slot_index = index + break + + port_alloc_map[os.getpid()] = new_slot_index + + with open(port_alloc_file, mode="w") as fp: + for pid, slot_index in port_alloc_map.items(): + fp.write(f"{pid} {slot_index}\n") + + worker_port_range_begin = 20000 + new_slot_index * 1000 + worker_port_range_end = worker_port_range_begin + 1000 + + if worker_port_range_end > 65536: + raise RuntimeError( + "Too many ray worker nodes are running on this machine, cannot " + "allocate worker port range for new ray worker node." + ) + except Exception: + release_lock() + raise + + def hold_lock_for_10s_and_release(): + time.sleep(10) + release_lock() + + threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() + + return worker_port_range_begin, worker_port_range_end + + def _init_ray_cluster( num_worker_nodes=None, total_cpus=None, @@ -342,9 +473,12 @@ def _init_ray_cluster( _logger.info(f"Starting Ray head, command: {' '.join(ray_head_node_cmd)}") + # `preexec_fn=setup_sigterm_on_parent_death` ensures the ray head node being killed + # if parent process died unexpectedly. ray_head_proc, tail_output_deque = exec_cmd( ray_head_node_cmd, synchronous=False, + preexec_fn=setup_sigterm_on_parent_death, ) # wait ray head node spin up. @@ -424,7 +558,6 @@ def ray_cluster_job_mapper(_): f"--temp-dir={ray_temp_dir}", f"--num-cpus={num_spark_task_cpus}", "--block", - "--disable-usage-stats", f"--address={ray_head_ip}:{ray_head_port}", f"--memory={ray_worker_node_heap_mem_bytes}", f"--object-store-memory={ray_worker_node_object_store_mem_bytes}", @@ -459,36 +592,15 @@ def ray_cluster_job_mapper(_): _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_node_cmd)}") - def setup_sigterm_on_parent_death(): - """ - Uses prctl to automatically send SIGTERM to the command process when its parent is - dead. - - This handles the case when the parent is a PySpark worker process. - If a user cancels the PySpark job, the worker process gets killed, regardless of - PySpark daemon and worker reuse settings. - We use prctl to ensure the command process receives SIGTERM after spark job - cancellation. - The command process itself should handle SIGTERM properly. - This is a no-op on macOS because prctl is not supported. - - Note: - When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", - This case neither "atexit" nor signal handler can capture SIGKILL signal. - prctl is the only way to capture SIGKILL signal. - """ - try: - import ctypes - import signal - - libc = ctypes.CDLL("libc.so.6") - # Set the parent process death signal of the command process to SIGTERM. - libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h - except OSError as e: - _worker_logger.warning( - f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." - ) - + # `preexec_fn=setup_sigterm_on_parent_death` handles the case: + # If a user cancels the PySpark job, the worker process gets killed, regardless of + # PySpark daemon and worker reuse settings. + # We use prctl to ensure the command process receives SIGTERM after spark job + # cancellation. + # Note: + # When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", + # This case neither "atexit" nor signal handler can capture SIGKILL signal. + # prctl is the only way to capture SIGKILL signal. exec_cmd( ray_worker_node_cmd, synchronous=True, diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 05ed92d70154..284326defde3 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -3,15 +3,17 @@ import sys import random import math -import time -import fcntl import threading import collections +import logging _MEMORY_BUFFER_OFFSET = 0.8 +_logger = logging.getLogger("ray.util.spark.utils") + + def is_in_databricks_runtime(): return "DATABRICKS_RUNTIME_VERSION" in os.environ @@ -328,136 +330,22 @@ def get_spark_task_assigned_physical_gpus(gpu_addr_list): return gpu_addr_list -def _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup(): +def setup_sigterm_on_parent_death(): """ - If we start multiple ray workers on a machine concurrently, some ray worker processes - might fail due to ray port conflicts, this is because race condition on getting free - port and opening the free port. - To address the issue, this function use an exclusive file lock to delay the worker processes - to ensure that port acquisition does not create a resource contention issue due to a race - condition. - - After acquiring lock, it will allocate port range for worker ports - (for ray node config --min-worker-port and --max-worker-port). - Because on a spark cluster, multiple ray cluster might be created, so on one spark worker - machine, there might be multiple ray worker nodes running, these worker nodes might belong - to different ray cluster, and we must ensure these ray nodes on the same machine using - non-overlapping worker port range, to achieve this, in this function, it creates a file - `/tmp/ray_on_spark_worker_port_allocation.txt` file, the file format is composed of multiple - lines, each line contains 2 number: `pid` and `port_range_slot_index`, - each port range slot allocates 1000 ports, and corresponding port range is: - range_begin (inclusive): 20000 + port_range_slot_index * 1000 - range_end (exclusive): range_begin + 1000 - In this function, it first scans `/tmp/ray_on_spark_worker_port_allocation.txt` file, - removing lines that containing dead process pid, then find the first unused - port_range_slot_index, then regenerate this file, and return the allocated port range. - - Returns: Allocated port range for current worker ports + Uses prctl to automatically send SIGTERM to the child process when its parent is + dead. The child process itself should handle SIGTERM properly. """ - import psutil - - def acquire_lock(file_path): - mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC - try: - fd = os.open(file_path, mode) - # The lock file must be readable / writable to all users. - os.chmod(file_path, 0o0777) - # Allow for retrying getting a file lock a maximum number of seconds - max_lock_iter = 600 - for _ in range(max_lock_iter): - try: - fcntl.flock(fd, fcntl.LOCK_EX | fcntl.LOCK_NB) - except BlockingIOError: - # Lock is used by other processes, continue loop to wait for lock available - pass - else: - # Acquire lock successfully. - return fd - time.sleep(10) - raise TimeoutError(f"Acquiring lock on file {file_path} timeout.") - except Exception: - os.close(fd) - - lock_file_path = "/tmp/ray_on_spark_worker_startup_barrier_lock.lock" - try: - lock_fd = acquire_lock(lock_file_path) - except TimeoutError: - # If timeout happens, the file lock might be hold by another process and that process - # does not release the lock in time by some unexpected reason. - # In this case, remove the existing lock file and create the file again, and then - # acquire file lock on the new file. - try: - os.remove(lock_file_path) - except Exception: - pass - lock_fd = acquire_lock(lock_file_path) - - def release_lock(): - fcntl.flock(lock_fd, fcntl.LOCK_UN) - os.close(lock_fd) - try: - port_alloc_file = "/tmp/ray_on_spark_worker_port_allocation.txt" - - # NB: reading / writing `port_alloc_file` is protected by exclusive lock - # on file `lock_file_path` - if os.path.exists(port_alloc_file): - with open(port_alloc_file, mode="r") as fp: - port_alloc_data = fp.read() - port_alloc_table = [line.split(" ") for line in port_alloc_data.strip().split("\n")] - port_alloc_table = [ - (int(pid_str), int(slot_index_str)) - for pid_str, slot_index_str in port_alloc_table - ] - else: - port_alloc_table = [] - with open(port_alloc_file, mode="w"): - pass - # The port range allocation file must be readable / writable to all users. - os.chmod(port_alloc_file, 0o0777) - - port_alloc_map = { - pid: slot_index - for pid, slot_index in port_alloc_table - if psutil.pid_exists(pid) # remove slot used by dead process - } - - allocated_slot_set = set(port_alloc_map.values()) - - if len(allocated_slot_set) == 0: - new_slot_index = 0 - else: - new_slot_index = max(allocated_slot_set) + 1 - for index in range(new_slot_index): - if index not in allocated_slot_set: - new_slot_index = index - break - - port_alloc_map[os.getpid()] = new_slot_index - - with open(port_alloc_file, mode="w") as fp: - for pid, slot_index in port_alloc_map.items(): - fp.write(f"{pid} {slot_index}\n") - - worker_port_range_begin = 20000 + new_slot_index * 1000 - worker_port_range_end = worker_port_range_begin + 1000 - - if worker_port_range_end > 65536: - raise RuntimeError( - "Too many ray worker nodes are running on this machine, cannot " - "allocate worker port range for new ray worker node." - ) - except Exception: - release_lock() - raise - - def hold_lock_for_10s_and_release(): - time.sleep(10) - release_lock() - - threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() - - return worker_port_range_begin, worker_port_range_end + import ctypes + import signal + + libc = ctypes.CDLL("libc.so.6") + # Set the parent process death signal of the command process to SIGTERM. + libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h + except OSError as e: + _logger.warning( + f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." + ) def _display_databricks_driver_proxy_url(spark_context, port, title): From 7c67d5c02cdfb63449ad5e5a0fa536166c186c19 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 12 Dec 2022 21:08:03 +0800 Subject: [PATCH 097/121] start hook Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 22 +++--- python/ray/util/spark/cluster_init.py | 87 ++++++++++++--------- python/ray/util/spark/databricks_hook.py | 65 +++++++++++++++ python/ray/util/spark/start_hook_base.py | 10 +++ python/ray/util/spark/utils.py | 37 --------- 5 files changed, 135 insertions(+), 86 deletions(-) create mode 100644 python/ray/util/spark/databricks_hook.py create mode 100644 python/ray/util/spark/start_hook_base.py diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 290c01ad32fa..e2530445057a 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -1,11 +1,10 @@ import os from abc import ABC -from unittest.mock import patch import ray -from ray.util.spark import _init_ray_cluster, RayClusterOnSpark +from ray.util.spark.cluster_init import _init_ray_cluster, RayClusterOnSpark from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time @@ -45,7 +44,7 @@ def test_cpu_allocation(self): assert worker_res['CPU'] == self.num_cpus_per_spark_task def test_basic_ray_app(self): - with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: @ray.remote def f(x): return x * x @@ -54,28 +53,30 @@ def f(x): results = ray.get(futures) assert results == [i * i for i in range(32)] + # assert temp dir is removed. + time.sleep(5) + assert not os.path.exists(cluster.temp_dir) + def test_ray_cluster_shutdown(self): with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. - def fake_shutdown(_): - pass - - with patch.object(RayClusterOnSpark, "shutdown", fake_shutdown): - cluster._cancel_background_spark_job() - time.sleep(20) + cluster._cancel_background_spark_job() + time.sleep(20) assert len(self.get_ray_worker_resources_list()) == 0 - time.sleep(3) # wait ray head node exit. + time.sleep(5) # wait ray head node exit. # assert ray head node exit by checking head port being closed. hostname, port = cluster.address.split(":") assert not check_port_open(hostname, int(port)) def test_background_spark_job_exit_trigger_ray_head_exit(self): with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + # Mimic the case the job failed unexpectedly. cluster._cancel_background_spark_job() + cluster.spark_job_is_canceled = False time.sleep(5) # assert ray head node exit by checking head port being closed. @@ -185,3 +186,4 @@ def setup_class(cls): .config("spark.task.maxFailures", "1") \ .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ .getOrCreate() + diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 98216c9e9793..372ea75bd22f 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -7,8 +7,10 @@ import logging import uuid from packaging.version import Version +from typing import Optional, Dict from ray.util.annotations import PublicAPI +from ray._private.storage import _load_class from .utils import ( exec_cmd, @@ -19,13 +21,13 @@ is_in_databricks_runtime, get_spark_task_assigned_physical_gpus, get_avail_mem_per_ray_worker_node, - get_dbutils, get_max_num_concurrent_tasks, get_target_spark_tasks, - _display_databricks_driver_proxy_url, gen_cmd_exec_failure_msg, setup_sigterm_on_parent_death, ) +from .start_hook_base import RayOnSparkStartHook +from .databricks_hook import DefaultDatabricksRayOnSparkStartHook if not sys.platform.startswith("linux"): raise RuntimeError("Ray on spark only supports running on Linux.") @@ -43,6 +45,9 @@ raise RuntimeError(_spark_dependency_error) +RAY_ON_SPARK_START_HOOK = "RAY_ON_SPARK_START_HOOK" + + @PublicAPI(stability="alpha") class RayClusterOnSpark: """ @@ -59,14 +64,23 @@ class RayClusterOnSpark: num_workers_node: The number of workers in the ray cluster. """ - def __init__(self, address, head_proc, spark_job_group_id, num_workers_node): + def __init__( + self, + address, + head_proc, + spark_job_group_id, + num_workers_node, + temp_dir, + ): self.address = address self.head_proc = head_proc self.spark_job_group_id = spark_job_group_id + self.num_worker_nodes = num_workers_node + self.temp_dir = temp_dir + self.ray_context = None self.is_shutdown = False self.spark_job_is_canceled = False - self.num_worker_nodes = num_workers_node self.background_job_exception = None def _cancel_background_spark_job(self): @@ -331,7 +345,13 @@ def _init_ray_cluster( """ from pyspark.util import inheritable_thread_target - _logger.warning("Test version 013.") + if RAY_ON_SPARK_START_HOOK in os.environ: + start_hook = _load_class(os.environ[RAY_ON_SPARK_START_HOOK])() + elif is_in_databricks_runtime(): + start_hook = DefaultDatabricksRayOnSparkStartHook() + else: + start_hook = RayOnSparkStartHook() + head_options = head_options or {} worker_options = worker_options or {} @@ -438,10 +458,7 @@ def _init_ray_cluster( # TODO: Set individual temp dir for ray worker nodes, and auto GC temp data when ray node exits # See https://github.com/ray-project/ray/issues/28876#issuecomment-1322016494 if ray_temp_root_dir is None: - if is_in_databricks_runtime(): - ray_temp_root_dir = "/local_disk0/tmp" - else: - ray_temp_root_dir = "/tmp" + ray_temp_root_dir = start_hook.get_default_temp_dir() ray_temp_dir = os.path.join( ray_temp_root_dir, f"ray-{ray_head_port}-{temp_dir_unique_suffix}" ) @@ -482,6 +499,7 @@ def _init_ray_cluster( ) # wait ray head node spin up. + time.sleep(5) for _ in range(_RAY_HEAD_STARTUP_TIMEOUT): time.sleep(1) if ray_head_proc.poll() is not None: @@ -506,12 +524,7 @@ def _init_ray_cluster( _logger.info("Ray head node started.") - if is_in_databricks_runtime(): - _display_databricks_driver_proxy_url( - spark.sparkContext, - ray_dashboard_port, - "Ray Cluster Dashboard" - ) + start_hook.on_ray_dashboard_created(ray_dashboard_port) # NB: # In order to start ray worker nodes on spark cluster worker machines, @@ -620,6 +633,7 @@ def ray_cluster_job_mapper(_): head_proc=ray_head_proc, spark_job_group_id=spark_job_group_id, num_workers_node=num_worker_nodes, + temp_dir=ray_temp_dir, ) def background_job_thread_fn(): @@ -672,19 +686,14 @@ def background_job_thread_fn(): target=inheritable_thread_target(background_job_thread_fn), args=() ).start() - time.sleep(_BACKGROUND_JOB_STARTUP_WAIT) # wait background spark task starting. + # wait background spark task starting. + for _ in range(_BACKGROUND_JOB_STARTUP_WAIT): + time.sleep(1) + if ray_cluster_handler.background_job_exception is not None: + raise RuntimeError("Ray workers failed to start.") \ + from ray_cluster_handler.background_job_exception - if is_in_databricks_runtime(): - try: - get_dbutils().entry_point.registerBackgroundSparkJobGroup( - spark_job_group_id - ) - except Exception: - _logger.warning( - "Register ray cluster spark job as background job failed. You need to manually " - "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " - "python REPL." - ) + start_hook.on_spark_background_job_created(spark_job_group_id) return ray_cluster_handler except Exception: # If driver side setup ray-cluster routine raises exception, it might result in part of ray @@ -699,17 +708,17 @@ def background_job_thread_fn(): @PublicAPI(stability="alpha") def init_ray_cluster( - num_worker_nodes=None, - total_cpus=None, - total_gpus=None, - total_heap_memory_bytes=None, - total_object_store_memory_bytes=None, - object_store_memory_per_node=None, - head_options=None, - worker_options=None, - ray_temp_root_dir=None, - safe_mode=True, -): + num_worker_nodes: Optional[int] = None, + total_cpus: Optional[int] = None, + total_gpus: Optional[int] = None, + total_heap_memory_bytes: Optional[int] = None, + total_object_store_memory_bytes: Optional[int] = None, + object_store_memory_per_node: Optional[int] = None, + head_options: Optional[Dict] = None, + worker_options: Optional[Dict] = None, + ray_temp_root_dir: Optional[str] = None, + safe_mode: Optional[bool] = True, +) -> None: """ Initialize a ray cluster on the spark cluster by starting a ray head node in the spark application's driver side node. @@ -776,7 +785,7 @@ def init_ray_cluster( @PublicAPI(stability="alpha") -def shutdown_ray_cluster(): +def shutdown_ray_cluster() -> None: """ Shut down the active ray cluster. """ diff --git a/python/ray/util/spark/databricks_hook.py b/python/ray/util/spark/databricks_hook.py new file mode 100644 index 000000000000..40fec8dede61 --- /dev/null +++ b/python/ray/util/spark/databricks_hook.py @@ -0,0 +1,65 @@ +from .start_hook_base import RayOnSparkStartHook +from .utils import get_spark_session +import logging + +_logger = logging.getLogger("ray.util.spark.databricks_hook") + + +class _NoDbutilsError(Exception): + pass + + +def get_dbutils(): + try: + import IPython + + ip_shell = IPython.get_ipython() + if ip_shell is None: + raise _NoDbutilsError + return ip_shell.ns_table["user_global"]["dbutils"] + except ImportError: + raise _NoDbutilsError + except KeyError: + raise _NoDbutilsError + + +def display_databricks_driver_proxy_url(spark_context, port, title): + from dbruntime.display import displayHTML + driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal + commandContextTags = driverLocal.commandContext().get().toStringMap().apply("tags") + orgId = commandContextTags.apply("orgId") + clusterId = commandContextTags.apply("clusterId") + + template = "/driver-proxy/o/{orgId}/{clusterId}/{port}/" + proxy_url = template.format(orgId=orgId, clusterId=clusterId, port=port) + + displayHTML(f""" + + """) + + +class DefaultDatabricksRayOnSparkStartHook(RayOnSparkStartHook): + + def get_default_temp_dir(self): + return "/local_disk0/tmp" + + def on_ray_dashboard_created(self, port): + display_databricks_driver_proxy_url( + get_spark_session().sparkContext, + port, + "Ray Cluster Dashboard" + ) + + def on_spark_background_job_created(self, job_group_id): + try: + get_dbutils().entry_point.registerBackgroundSparkJobGroup(job_group_id) + except Exception: + _logger.warning( + "Register ray cluster spark job as background job failed. You need to manually " + "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " + "python REPL." + ) diff --git a/python/ray/util/spark/start_hook_base.py b/python/ray/util/spark/start_hook_base.py new file mode 100644 index 000000000000..6c430a6c19f1 --- /dev/null +++ b/python/ray/util/spark/start_hook_base.py @@ -0,0 +1,10 @@ +class RayOnSparkStartHook: + + def get_default_temp_dir(self): + return "/tmp" + + def on_ray_dashboard_created(self, port): + pass + + def on_spark_background_job_created(self, job_group): + pass diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 284326defde3..369f048239bc 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -18,24 +18,6 @@ def is_in_databricks_runtime(): return "DATABRICKS_RUNTIME_VERSION" in os.environ -class _NoDbutilsError(Exception): - pass - - -def get_dbutils(): - try: - import IPython - - ip_shell = IPython.get_ipython() - if ip_shell is None: - raise _NoDbutilsError - return ip_shell.ns_table["user_global"]["dbutils"] - except ImportError: - raise _NoDbutilsError - except KeyError: - raise _NoDbutilsError - - def gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque): cmd_str = " ".join(cmd) tail_output = "".join(tail_output_deque) @@ -346,22 +328,3 @@ def setup_sigterm_on_parent_death(): _logger.warning( f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." ) - - -def _display_databricks_driver_proxy_url(spark_context, port, title): - from dbruntime.display import displayHTML - driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal - commandContextTags = driverLocal.commandContext().get().toStringMap().apply("tags") - orgId = commandContextTags.apply("orgId") - clusterId = commandContextTags.apply("clusterId") - - template = "/driver-proxy/o/{orgId}/{clusterId}/{port}/" - proxy_url = template.format(orgId=orgId, clusterId=clusterId, port=port) - - displayHTML(f""" - - """) From 1a4e5b74642632ecf3858bf1bc1350791b2cc5a1 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 13 Dec 2022 17:43:52 +0800 Subject: [PATCH 098/121] update test and build Signed-off-by: Weichen Xu --- python/ray/tests/BUILD | 10 +++ python/ray/tests/spark/test_ray_on_spark.py | 86 +++++++++++---------- python/ray/util/spark/cluster_init.py | 58 +++++++------- python/ray/util/spark/databricks_hook.py | 12 +-- python/ray/util/spark/start_hook_base.py | 1 - python/ray/util/spark/start_ray_node.py | 4 +- python/ray/util/spark/utils.py | 46 +++++++---- 7 files changed, 129 insertions(+), 88 deletions(-) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 2d92a26ac81e..93adb024ccbd 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -517,3 +517,13 @@ py_test_module_list( tags = ["exclusive", "asan_tests", "team:core"], deps = ["//:ray_lib", ":conftest"], ) + +py_test_module_list( + files = [ + "spark/test_ray_on_spark.py", + "spark/test_utils.py", + ], + size = "large", + tags = ["exclusive", "spark_plugin_tests", "team:serverless"], + deps = ["//:ray_lib", ":conftest"], +) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index e2530445057a..c6608ad4fd2b 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -4,7 +4,7 @@ import ray -from ray.util.spark.cluster_init import _init_ray_cluster, RayClusterOnSpark +from ray.util.spark.cluster_init import _init_ray_cluster from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time @@ -31,8 +31,8 @@ def get_ray_worker_resources_list(): wr_list = [] for node in ray.nodes(): # exclude dead node and head node (with 0 CPU resource) - if node['Alive'] and node['Resources'].get('CPU', 0) > 0: - wr_list.append(node['Resources']) + if node["Alive"] and node["Resources"].get("CPU", 0) > 0: + wr_list.append(node["Resources"]) return wr_list def test_cpu_allocation(self): @@ -41,10 +41,13 @@ def test_cpu_allocation(self): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: - assert worker_res['CPU'] == self.num_cpus_per_spark_task + assert worker_res["CPU"] == self.num_cpus_per_spark_task def test_basic_ray_app(self): - with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + with _init_ray_cluster( + num_worker_nodes=self.max_spark_tasks, safe_mode=False + ) as cluster: + @ray.remote def f(x): return x * x @@ -58,7 +61,9 @@ def f(x): assert not os.path.exists(cluster.temp_dir) def test_ray_cluster_shutdown(self): - with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + with _init_ray_cluster( + num_worker_nodes=self.max_spark_tasks, safe_mode=False + ) as cluster: assert len(self.get_ray_worker_resources_list()) == self.max_spark_tasks # Test: cancel background spark job will cause all ray worker nodes exit. @@ -73,7 +78,9 @@ def test_ray_cluster_shutdown(self): assert not check_port_open(hostname, int(port)) def test_background_spark_job_exit_trigger_ray_head_exit(self): - with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False) as cluster: + with _init_ray_cluster( + num_worker_nodes=self.max_spark_tasks, safe_mode=False + ) as cluster: # Mimic the case the job failed unexpectedly. cluster._cancel_background_spark_job() cluster.spark_job_is_canceled = False @@ -96,7 +103,7 @@ def test_gpu_allocation(self): worker_res_list = self.get_ray_worker_resources_list() assert len(worker_res_list) == num_spark_tasks for worker_res in worker_res_list: - assert worker_res['GPU'] == self.num_gpus_per_spark_task + assert worker_res["GPU"] == self.num_gpus_per_spark_task def test_basic_ray_app_using_gpu(self): @@ -109,7 +116,7 @@ def f(_): time.sleep(5) return [ int(gpu_id) - for gpu_id in os.environ['CUDA_VISIBLE_DEVICES'].split(",") + for gpu_id in os.environ["CUDA_VISIBLE_DEVICES"].split(",") ] futures = [f.remote(i) for i in range(self.num_total_gpus)] @@ -120,7 +127,6 @@ def f(_): class TestBasicSparkCluster(RayOnSparkCPUClusterTestBase): - @classmethod def setup_class(cls): cls.num_total_cpus = 2 @@ -129,15 +135,15 @@ def setup_class(cls): cls.num_gpus_per_spark_task = 0 cls.max_spark_tasks = 2 os.environ["SPARK_WORKER_CORES"] = "2" - cls.spark = SparkSession.builder \ - .master("local-cluster[1, 2, 1024]") \ - .config("spark.task.cpus", "1") \ - .config("spark.task.maxFailures", "1") \ + cls.spark = ( + SparkSession.builder.master("local-cluster[1, 2, 1024]") + .config("spark.task.cpus", "1") + .config("spark.task.maxFailures", "1") .getOrCreate() + ) class TestBasicSparkGPUCluster(RayOnSparkGPUClusterTestBase): - @classmethod def setup_class(cls): cls.num_total_cpus = 2 @@ -146,24 +152,25 @@ def setup_class(cls): cls.num_gpus_per_spark_task = 1 cls.max_spark_tasks = 2 gpu_discovery_script_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), - "discover_2_gpu.sh" + os.path.dirname(os.path.abspath(__file__)), "discover_2_gpu.sh" ) os.environ["SPARK_WORKER_CORES"] = "4" - cls.spark = SparkSession.builder \ - .master("local-cluster[1, 2, 1024]") \ - .config("spark.task.cpus", "1") \ - .config("spark.task.resource.gpu.amount", "1") \ - .config("spark.executor.cores", "2") \ - .config("spark.worker.resource.gpu.amount", "2") \ - .config("spark.executor.resource.gpu.amount", "2") \ - .config("spark.task.maxFailures", "1") \ - .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ + cls.spark = ( + SparkSession.builder.master("local-cluster[1, 2, 1024]") + .config("spark.task.cpus", "1") + .config("spark.task.resource.gpu.amount", "1") + .config("spark.executor.cores", "2") + .config("spark.worker.resource.gpu.amount", "2") + .config("spark.executor.resource.gpu.amount", "2") + .config("spark.task.maxFailures", "1") + .config( + "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path + ) .getOrCreate() + ) class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): - @classmethod def setup_class(cls): cls.num_total_cpus = 4 @@ -172,18 +179,19 @@ def setup_class(cls): cls.num_gpus_per_spark_task = 2 cls.max_spark_tasks = 2 gpu_discovery_script_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), - "discover_4_gpu.sh" + os.path.dirname(os.path.abspath(__file__)), "discover_4_gpu.sh" ) os.environ["SPARK_WORKER_CORES"] = "4" - cls.spark = SparkSession.builder \ - .master("local-cluster[1, 4, 1024]") \ - .config("spark.task.cpus", "2") \ - .config("spark.task.resource.gpu.amount", "2") \ - .config("spark.executor.cores", "4") \ - .config("spark.worker.resource.gpu.amount", "4") \ - .config("spark.executor.resource.gpu.amount", "4") \ - .config("spark.task.maxFailures", "1") \ - .config("spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path) \ + cls.spark = ( + SparkSession.builder.master("local-cluster[1, 4, 1024]") + .config("spark.task.cpus", "2") + .config("spark.task.resource.gpu.amount", "2") + .config("spark.executor.cores", "4") + .config("spark.worker.resource.gpu.amount", "4") + .config("spark.executor.resource.gpu.amount", "4") + .config("spark.task.maxFailures", "1") + .config( + "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path + ) .getOrCreate() - + ) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 372ea75bd22f..ec7c89137619 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -65,12 +65,12 @@ class RayClusterOnSpark: """ def __init__( - self, - address, - head_proc, - spark_job_group_id, - num_workers_node, - temp_dir, + self, + address, + head_proc, + spark_job_group_id, + num_workers_node, + temp_dir, ): self.address = address self.head_proc = head_proc @@ -91,7 +91,9 @@ def connect(self): import ray if self.background_job_exception is not None: - raise RuntimeError("Ray workers has exited.") from self.background_job_exception + raise RuntimeError( + "Ray workers has exited." + ) from self.background_job_exception if self.is_shutdown: raise RuntimeError( @@ -105,13 +107,9 @@ def connect(self): last_progress_move_time = time.time() while True: time.sleep(_RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL) - cur_alive_worker_count = len( - [ - node - for node in ray.nodes() - if node["Alive"] - ] - ) - 1 # Minus 1 means excluding the head node. + cur_alive_worker_count = ( + len([node for node in ray.nodes() if node["Alive"]]) - 1 + ) # Minus 1 means excluding the head node. if cur_alive_worker_count == self.num_worker_nodes: return @@ -266,7 +264,9 @@ def release_lock(): if os.path.exists(port_alloc_file): with open(port_alloc_file, mode="r") as fp: port_alloc_data = fp.read() - port_alloc_table = [line.split(" ") for line in port_alloc_data.strip().split("\n")] + port_alloc_table = [ + line.split(" ") for line in port_alloc_data.strip().split("\n") + ] port_alloc_table = [ (int(pid_str), int(slot_index_str)) for pid_str, slot_index_str in port_alloc_table @@ -448,7 +448,10 @@ def _init_ray_cluster( ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port] ) ray_dashboard_agent_port = get_random_unused_port( - ray_head_ip, min_port=9000, max_port=10000, exclude_list=[ray_head_port, ray_dashboard_port] + ray_head_ip, + min_port=9000, + max_port=10000, + exclude_list=[ray_head_port, ray_dashboard_port], ) _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") @@ -518,9 +521,7 @@ def _init_ray_cluster( cmd_exec_failure_msg = gen_cmd_exec_failure_msg( ray_head_node_cmd, ray_head_proc.returncode, tail_output_deque ) - raise RuntimeError( - "Start Ray head node failed!\n" + cmd_exec_failure_msg - ) + raise RuntimeError("Start Ray head node failed!\n" + cmd_exec_failure_msg) _logger.info("Ray head node started.") @@ -554,8 +555,12 @@ def ray_cluster_job_mapper(_): context = TaskContext.get() - worker_port_range_begin, worker_port_range_end = \ + ( + worker_port_range_begin, + worker_port_range_end, + ) = ( _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup() + ) # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -603,7 +608,9 @@ def ray_cluster_job_mapper(_): [str(gpu_id) for gpu_id in available_physical_gpus] ) - _worker_logger.info(f"Start Ray worker, command: {' '.join(ray_worker_node_cmd)}") + _worker_logger.info( + f"Start Ray worker, command: {' '.join(ray_worker_node_cmd)}" + ) # `preexec_fn=setup_sigterm_on_parent_death` handles the case: # If a user cancels the PySpark job, the worker process gets killed, regardless of @@ -624,9 +631,7 @@ def ray_cluster_job_mapper(_): # NB: Not reachable. yield 0 - spark_job_group_id = ( - f"ray-cluster-job-head-{ray_head_ip}-port-{ray_head_port}" - ) + spark_job_group_id = f"ray-cluster-job-head-{ray_head_ip}-port-{ray_head_port}" ray_cluster_handler = RayClusterOnSpark( address=f"{ray_head_ip}:{ray_head_port}", @@ -690,8 +695,9 @@ def background_job_thread_fn(): for _ in range(_BACKGROUND_JOB_STARTUP_WAIT): time.sleep(1) if ray_cluster_handler.background_job_exception is not None: - raise RuntimeError("Ray workers failed to start.") \ - from ray_cluster_handler.background_job_exception + raise RuntimeError( + "Ray workers failed to start." + ) from ray_cluster_handler.background_job_exception start_hook.on_spark_background_job_created(spark_job_group_id) return ray_cluster_handler diff --git a/python/ray/util/spark/databricks_hook.py b/python/ray/util/spark/databricks_hook.py index 40fec8dede61..50aab25266d0 100644 --- a/python/ray/util/spark/databricks_hook.py +++ b/python/ray/util/spark/databricks_hook.py @@ -25,6 +25,7 @@ def get_dbutils(): def display_databricks_driver_proxy_url(spark_context, port, title): from dbruntime.display import displayHTML + driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal commandContextTags = driverLocal.commandContext().get().toStringMap().apply("tags") orgId = commandContextTags.apply("orgId") @@ -33,25 +34,24 @@ def display_databricks_driver_proxy_url(spark_context, port, title): template = "/driver-proxy/o/{orgId}/{clusterId}/{port}/" proxy_url = template.format(orgId=orgId, clusterId=clusterId, port=port) - displayHTML(f""" + displayHTML( + f""" - """) + """ + ) class DefaultDatabricksRayOnSparkStartHook(RayOnSparkStartHook): - def get_default_temp_dir(self): return "/local_disk0/tmp" def on_ray_dashboard_created(self, port): display_databricks_driver_proxy_url( - get_spark_session().sparkContext, - port, - "Ray Cluster Dashboard" + get_spark_session().sparkContext, port, "Ray Cluster Dashboard" ) def on_spark_background_job_created(self, job_group_id): diff --git a/python/ray/util/spark/start_hook_base.py b/python/ray/util/spark/start_hook_base.py index 6c430a6c19f1..6421c0fc3c58 100644 --- a/python/ray/util/spark/start_hook_base.py +++ b/python/ray/util/spark/start_hook_base.py @@ -1,5 +1,4 @@ class RayOnSparkStartHook: - def get_default_temp_dir(self): return "/tmp" diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index b6af0424be1b..4da8fe66b40f 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -10,7 +10,7 @@ _WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 -if __name__ == '__main__': +if __name__ == "__main__": arg_list = sys.argv[1:] temp_dir_arg_prefix = "--temp-dir=" @@ -18,7 +18,7 @@ for arg in arg_list: if arg.startswith(temp_dir_arg_prefix): - temp_dir = arg[len(temp_dir_arg_prefix):] + temp_dir = arg[len(temp_dir_arg_prefix) :] if temp_dir is None: raise ValueError("Please explicitly set --temp-dir option.") diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 369f048239bc..902b1955b842 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -58,7 +58,7 @@ def exec_cmd( text=True, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, - **kwargs + **kwargs, ) tail_output_deque = collections.deque(maxlen=100) @@ -78,17 +78,22 @@ def redirect_log_thread_fn(): return_code = process.wait() if return_code != 0: - raise RuntimeError(gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque)) + raise RuntimeError( + gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque) + ) def check_port_open(host, port): import socket from contextlib import closing + with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as sock: return sock.connect_ex((host, port)) == 0 -def get_random_unused_port(host, min_port=1024, max_port=65535, max_retries=100, exclude_list=None): +def get_random_unused_port( + host, min_port=1024, max_port=65535, max_retries=100, exclude_list=None +): """ Get random unused port. """ @@ -102,7 +107,9 @@ def get_random_unused_port(host, min_port=1024, max_port=65535, max_retries=100, continue if not check_port_open(host, port): return port - raise RuntimeError(f"Get available port between range {min_port} and {max_port} failed.") + raise RuntimeError( + f"Get available port between range {min_port} and {max_port} failed." + ) def get_spark_session(): @@ -169,9 +176,7 @@ def _calc_mem_per_ray_worker_node( available_shared_mem_per_node, ) ) - heap_mem_bytes = ( - available_physical_mem_per_node - object_store_bytes - ) + heap_mem_bytes = available_physical_mem_per_node - object_store_bytes return heap_mem_bytes, object_store_bytes @@ -236,7 +241,11 @@ def get_target_spark_tasks( ) calculated_tasks.append( - int(math.ceil(total_heap_memory_bytes / ray_worker_node_heap_memory_bytes)) + int( + math.ceil( + total_heap_memory_bytes / ray_worker_node_heap_memory_bytes + ) + ) ) if total_object_store_memory_bytes is not None: @@ -285,13 +294,21 @@ def mapper(_): shared_mem_bytes, object_store_memory_per_node, ) - return ray_worker_node_heap_mem_bytes, ray_worker_node_object_store_bytes, None + return ( + ray_worker_node_heap_mem_bytes, + ray_worker_node_object_store_bytes, + None, + ) except Exception as e: return -1, -1, repr(e) # Running memory inference routine on spark executor side since the spark worker nodes may # have a different machine configuration compared to the spark driver node. - inferred_ray_worker_node_heap_mem_bytes, inferred_ray_worker_node_object_store_bytes, err = ( + ( + inferred_ray_worker_node_heap_mem_bytes, + inferred_ray_worker_node_object_store_bytes, + err, + ) = ( spark.sparkContext.parallelize([1], 1).map(mapper).collect()[0] ) @@ -299,7 +316,10 @@ def mapper(_): raise RuntimeError( f"Inferring ray worker available memory failed, error: {err}" ) - return inferred_ray_worker_node_heap_mem_bytes, inferred_ray_worker_node_object_store_bytes + return ( + inferred_ray_worker_node_heap_mem_bytes, + inferred_ray_worker_node_object_store_bytes, + ) def get_spark_task_assigned_physical_gpus(gpu_addr_list): @@ -325,6 +345,4 @@ def setup_sigterm_on_parent_death(): # Set the parent process death signal of the command process to SIGTERM. libc.prctl(1, signal.SIGTERM) # PR_SET_PDEATHSIG, see prctl.h except OSError as e: - _logger.warning( - f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}." - ) + _logger.warning(f"Setup libc.prctl PR_SET_PDEATHSIG failed, error {repr(e)}.") From 96e33e71b1b26093473abf7667e7e8a80e013748 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 13 Dec 2022 19:09:47 +0800 Subject: [PATCH 099/121] update ci pipeline Signed-off-by: Weichen Xu --- .buildkite/pipeline.build.yml | 12 ++++++++++++ python/ray/tests/BUILD | 4 ++++ python/ray/tests/spark/discover_8_gpu.sh | 4 ---- python/requirements_test.txt | 2 +- 4 files changed, 17 insertions(+), 5 deletions(-) delete mode 100755 python/ray/tests/spark/discover_8_gpu.sh diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml index c51cc2c27655..1f86f158ad91 100644 --- a/.buildkite/pipeline.build.yml +++ b/.buildkite/pipeline.build.yml @@ -484,6 +484,18 @@ --test_env=CONDA_DEFAULT_ENV python/ray/tests/... +- label: ":python: Ray on Spark Test" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + instance_size: medium + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - pip uninstall -y ray + - RAY_DEBUG_BUILD=debug ./ci/ci.sh build + - ./ci/env/env_info.sh + - bazel test --config=ci-debug $(./ci/run/bazel_export_options) + --test_tag_filters=-kubernetes,spark_plugin_tests + python/ray/tests/... + # https://github.com/ray-project/ray/issues/22460 #- label: ":python: (Privileged test)" #conditions: ["RAY_CI_PYTHON_AFFECTED"] diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 93adb024ccbd..2d97e871d019 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -526,4 +526,8 @@ py_test_module_list( size = "large", tags = ["exclusive", "spark_plugin_tests", "team:serverless"], deps = ["//:ray_lib", ":conftest"], + data = [ + "spark/discover_2_gpu.sh", + "spark/discover_4_gpu.sh" + ], ) diff --git a/python/ray/tests/spark/discover_8_gpu.sh b/python/ray/tests/spark/discover_8_gpu.sh deleted file mode 100755 index 8d9957a7d6d7..000000000000 --- a/python/ray/tests/spark/discover_8_gpu.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/bash - -# This script is used in spark GPU cluster config for discovering available GPU. -echo "{\"name\":\"gpu\",\"addresses\":[\"0\",\"1\",\"2\",\"3\",\"4\",\"5\",\"6\",\"7\"]}" diff --git a/python/requirements_test.txt b/python/requirements_test.txt index 14da27675e77..00cee27d23d4 100644 --- a/python/requirements_test.txt +++ b/python/requirements_test.txt @@ -54,7 +54,7 @@ PyOpenSSL==22.1.0 pygame==2.1.2 Pygments==2.13.0 pymongo==4.3.2 -pyspark==3.1.2 +pyspark==3.3.1 pytest==7.0.1 pytest-asyncio==0.16.0 pytest-rerunfailures==10.2 From c13641692b18d72a0160bd39e907a446fb851885 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 13 Dec 2022 19:21:31 +0800 Subject: [PATCH 100/121] remove total_xx argumens Signed-off-by: Weichen Xu --- python/ray/util/spark/__init__.py | 7 ++- python/ray/util/spark/cluster_init.py | 68 ++++----------------- python/ray/util/spark/utils.py | 88 --------------------------- 3 files changed, 18 insertions(+), 145 deletions(-) diff --git a/python/ray/util/spark/__init__.py b/python/ray/util/spark/__init__.py index 8bac1d3ad290..26851a9c72b0 100644 --- a/python/ray/util/spark/__init__.py +++ b/python/ray/util/spark/__init__.py @@ -1,6 +1,11 @@ -from ray.util.spark.cluster_init import init_ray_cluster, shutdown_ray_cluster +from ray.util.spark.cluster_init import ( + init_ray_cluster, + shutdown_ray_cluster, + MAX_NUM_WORKER_NODES +) __all__ = [ "init_ray_cluster", "shutdown_ray_cluster", + "MAX_NUM_WORKER_NODES" ] diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ec7c89137619..5a4c45526914 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -47,6 +47,8 @@ RAY_ON_SPARK_START_HOOK = "RAY_ON_SPARK_START_HOOK" +MAX_NUM_WORKER_NODES = -1 + @PublicAPI(stability="alpha") class RayClusterOnSpark: @@ -323,11 +325,7 @@ def hold_lock_for_10s_and_release(): def _init_ray_cluster( - num_worker_nodes=None, - total_cpus=None, - total_gpus=None, - total_heap_memory_bytes=None, - total_object_store_memory_bytes=None, + num_worker_nodes, object_store_memory_per_node=None, head_options=None, worker_options=None, @@ -355,23 +353,6 @@ def _init_ray_cluster( head_options = head_options or {} worker_options = worker_options or {} - num_worker_nodes_specified = num_worker_nodes is not None - total_resources_req_specified = ( - total_cpus is not None - or total_gpus is not None - or total_heap_memory_bytes is not None - or total_object_store_memory_bytes is not None - ) - - if (num_worker_nodes_specified and total_resources_req_specified) or ( - not num_worker_nodes_specified and not total_resources_req_specified - ): - raise ValueError( - "You should specify either 'num_worker_nodes' argument or an argument group of " - "'total_cpus', 'total_gpus', 'total_heap_memory_bytes' and " - "'total_object_store_memory_bytes'." - ) - spark = get_spark_session() # Environment configurations within the Spark Session that dictate how many cpus and gpus to @@ -386,27 +367,16 @@ def _init_ray_cluster( ray_worker_node_object_store_mem_bytes, ) = get_avail_mem_per_ray_worker_node(spark, object_store_memory_per_node) - if total_gpus is not None and num_spark_task_gpus == 0: + max_concurrent_tasks = get_max_num_concurrent_tasks(spark.sparkContext) + if num_worker_nodes == -1: + # num_worker_nodes=-1 represents using all available spark task slots + num_spark_tasks = max_concurrent_tasks + elif num_worker_nodes <= 0: raise ValueError( - "The spark cluster is not configured with available GPUs. Start a GPU instance cluster " - "to set the 'total_gpus' argument" + "The value of 'num_worker_nodes' argument must be either a positive integer or " + "'ray.util.spark.MAX_NUM_WORKER_NODES'." ) - max_concurrent_tasks = get_max_num_concurrent_tasks(spark.sparkContext) - - num_worker_nodes = get_target_spark_tasks( - max_concurrent_tasks, - num_spark_task_cpus, - num_spark_task_gpus, - ray_worker_node_heap_mem_bytes, - ray_worker_node_object_store_mem_bytes, - num_worker_nodes, - total_cpus, - total_gpus, - total_heap_memory_bytes, - total_object_store_memory_bytes, - ) - insufficient_resources = [] if num_spark_task_cpus < 4: @@ -714,11 +684,7 @@ def background_job_thread_fn(): @PublicAPI(stability="alpha") def init_ray_cluster( - num_worker_nodes: Optional[int] = None, - total_cpus: Optional[int] = None, - total_gpus: Optional[int] = None, - total_heap_memory_bytes: Optional[int] = None, - total_object_store_memory_bytes: Optional[int] = None, + num_worker_nodes: int, object_store_memory_per_node: Optional[int] = None, head_options: Optional[Dict] = None, worker_options: Optional[Dict] = None, @@ -745,13 +711,7 @@ def init_ray_cluster( will use all available spark tasks slots (and resources allocated to the spark application) on the spark cluster. To create a spark cluster that is intended to be used exclusively as a shared ray - cluster, it is recommended to set this argument to `-1`. - total_cpus: The total cpu core count for the ray cluster to utilize. - total_gpus: The total gpu count for the ray cluster to utilize. - total_heap_memory_bytes: The total amount of heap memory (in bytes) for the ray cluster - to utilize. - total_object_store_memory_bytes: The total amount of object store memory (in bytes) for - the ray cluster to utilize. + cluster, it is recommended to set this argument to `ray.spark.utils.MAX_NUM_WORKER_NODES`. object_store_memory_per_node: Object store memory available to per-ray worker node, but it cannot exceed "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker" @@ -774,10 +734,6 @@ def init_ray_cluster( ) cluster = _init_ray_cluster( num_worker_nodes=num_worker_nodes, - total_cpus=total_cpus, - total_gpus=total_gpus, - total_heap_memory_bytes=total_heap_memory_bytes, - total_object_store_memory_bytes=total_object_store_memory_bytes, object_store_memory_per_node=object_store_memory_per_node, head_options=head_options, worker_options=worker_options, diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 902b1955b842..ae4013f19aff 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -180,94 +180,6 @@ def _calc_mem_per_ray_worker_node( return heap_mem_bytes, object_store_bytes -def _resolve_target_spark_tasks(calculated_limits): - """ - Return the max value of a list of spark task total count calculations based on the - provided configuration arguments to `init_ray_cluster`. - Args: - calculated_limits: A list of calculated values wherein the highest value based on - spark cluster worker instance sizes and user-specified ray cluster configuration is - taken to ensure that spark cluster limits are not exceeded. - - Returns: The maximum calculated number of spark tasks for the configured ray cluster. - - """ - return max(*calculated_limits) - - -def get_target_spark_tasks( - max_concurrent_tasks, - num_spark_task_cpus, - num_spark_task_gpus, - ray_worker_node_heap_memory_bytes, - ray_worker_node_object_store_memory_bytes, - num_spark_tasks, - total_cpus, - total_gpus, - total_heap_memory_bytes, - total_object_store_memory_bytes, -): - - if num_spark_tasks is not None: - if num_spark_tasks == -1: - # num_worker_nodes=-1 represents using all available spark task slots - num_spark_tasks = max_concurrent_tasks - elif num_spark_tasks <= 0: - raise ValueError( - "The value of 'num_worker_nodes' argument must be either a positive integer or -1." - ) - else: - calculated_tasks = [1] - if total_cpus is not None: - if total_cpus <= 0: - raise ValueError( - "The value of 'total_cpus' argument must be a positive integer." - ) - - calculated_tasks.append(int(math.ceil(total_cpus / num_spark_task_cpus))) - - if total_gpus is not None: - if total_gpus <= 0: - raise ValueError( - "The value of 'total_gpus' argument must be a positive integer." - ) - - calculated_tasks.append(int(math.ceil(total_gpus / num_spark_task_gpus))) - - if total_heap_memory_bytes is not None: - if total_heap_memory_bytes <= 0: - raise ValueError( - "The value of 'total_heap_memory_bytes' argument must be a positive integer." - ) - - calculated_tasks.append( - int( - math.ceil( - total_heap_memory_bytes / ray_worker_node_heap_memory_bytes - ) - ) - ) - - if total_object_store_memory_bytes is not None: - if total_object_store_memory_bytes <= 0: - raise ValueError( - "The value of 'total_object_store_memory_bytes' argument must be a " - "positive integer." - ) - - calculated_tasks.append( - int( - math.ceil( - total_object_store_memory_bytes - / ray_worker_node_object_store_memory_bytes - ) - ) - ) - - num_spark_tasks = _resolve_target_spark_tasks(calculated_tasks) - return num_spark_tasks - - def get_avail_mem_per_ray_worker_node(spark, object_store_memory_per_node): """ Return the available heap memory and object store memory for each ray worker. From fca032584e9b035fd69631b29dc2a90e80e2d21d Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 13 Dec 2022 19:57:07 +0800 Subject: [PATCH 101/121] fix Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 5a4c45526914..e888b1f67c0a 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -22,7 +22,6 @@ get_spark_task_assigned_physical_gpus, get_avail_mem_per_ray_worker_node, get_max_num_concurrent_tasks, - get_target_spark_tasks, gen_cmd_exec_failure_msg, setup_sigterm_on_parent_death, ) From f4884020f06eb8945890f3612fdd8ac10c034aff Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 13 Dec 2022 20:37:52 +0800 Subject: [PATCH 102/121] fix Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 9 +++++++++ python/ray/tests/spark/test_utils.py | 22 ++++++++++++++------- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index c6608ad4fd2b..fe1875e81a64 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -1,4 +1,6 @@ import os +import pytest +import sys from abc import ABC @@ -195,3 +197,10 @@ def setup_class(cls): ) .getOrCreate() ) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index f2af0273b8f0..d167c7a1afc6 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -1,10 +1,11 @@ from unittest.mock import patch import os +import sys import pytest from ray.util.spark.utils import ( get_spark_task_assigned_physical_gpus, - _calc_mem_per_ray_worker, + _calc_mem_per_ray_worker_node, get_target_spark_tasks, ) @@ -18,12 +19,12 @@ def test_get_spark_task_assigned_physical_gpus(): assert get_spark_task_assigned_physical_gpus([0, 2]) == [2, 6] -def test_calc_mem_per_ray_worker(): - assert _calc_mem_per_ray_worker(4, 1000000, 400000, 0.4) == (120000, 80000) - assert _calc_mem_per_ray_worker(6, 1000000, 400000, 0.4) == (80000, 53333) - assert _calc_mem_per_ray_worker(4, 800000, 600000, 0.2) == (128000, 32000) - assert _calc_mem_per_ray_worker(4, 800000, 600000, 0.5) == (80000, 80000) - assert _calc_mem_per_ray_worker(8, 2000000, 600000, 0.3) == (140000, 60000) +def test_calc_mem_per_ray_worker_node(): + assert _calc_mem_per_ray_worker_node(4, 1000000, 400000, 0.4) == (120000, 80000) + assert _calc_mem_per_ray_worker_node(6, 1000000, 400000, 0.4) == (80000, 53333) + assert _calc_mem_per_ray_worker_node(4, 800000, 600000, 0.2) == (128000, 32000) + assert _calc_mem_per_ray_worker_node(4, 800000, 600000, 0.5) == (80000, 80000) + assert _calc_mem_per_ray_worker_node(8, 2000000, 600000, 0.3) == (140000, 60000) def test_target_spark_tasks(): @@ -89,3 +90,10 @@ def _mem_in_gbs(gb): total_heap_memory_bytes=_mem_in_gbs(400), total_object_store_memory_bytes=_mem_in_gbs(80), ) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) From 4f8773ae374d100792c94c97b13e8e9ada81bab9 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 15 Dec 2022 20:12:21 +0800 Subject: [PATCH 103/121] address comments Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 8 +-- python/ray/tests/spark/test_utils.py | 66 --------------------- python/ray/util/spark/cluster_init.py | 25 ++++---- python/ray/util/spark/databricks_hook.py | 11 +++- python/ray/util/spark/start_ray_node.py | 11 ++++ 5 files changed, 36 insertions(+), 85 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index fe1875e81a64..42a478e47855 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -59,7 +59,7 @@ def f(x): assert results == [i * i for i in range(32)] # assert temp dir is removed. - time.sleep(5) + time.sleep(4) assert not os.path.exists(cluster.temp_dir) def test_ray_cluster_shutdown(self): @@ -70,11 +70,11 @@ def test_ray_cluster_shutdown(self): # Test: cancel background spark job will cause all ray worker nodes exit. cluster._cancel_background_spark_job() - time.sleep(20) + time.sleep(3) assert len(self.get_ray_worker_resources_list()) == 0 - time.sleep(5) # wait ray head node exit. + time.sleep(2) # wait ray head node exit. # assert ray head node exit by checking head port being closed. hostname, port = cluster.address.split(":") assert not check_port_open(hostname, int(port)) @@ -86,7 +86,7 @@ def test_background_spark_job_exit_trigger_ray_head_exit(self): # Mimic the case the job failed unexpectedly. cluster._cancel_background_spark_job() cluster.spark_job_is_canceled = False - time.sleep(5) + time.sleep(3) # assert ray head node exit by checking head port being closed. hostname, port = cluster.address.split(":") diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index d167c7a1afc6..61af0ef5890c 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -6,7 +6,6 @@ from ray.util.spark.utils import ( get_spark_task_assigned_physical_gpus, _calc_mem_per_ray_worker_node, - get_target_spark_tasks, ) @@ -27,71 +26,6 @@ def test_calc_mem_per_ray_worker_node(): assert _calc_mem_per_ray_worker_node(8, 2000000, 600000, 0.3) == (140000, 60000) -def test_target_spark_tasks(): - def _mem_in_gbs(gb): - return 1024 * 1024 * 1024 * gb - - # CPU availability sets the task count - cpu_defined_task_count = get_target_spark_tasks( - max_concurrent_tasks=400, - num_spark_task_cpus=4, - num_spark_task_gpus=None, - ray_worker_heap_memory_bytes=_mem_in_gbs(10), - ray_worker_object_store_memory_bytes=_mem_in_gbs(2), - num_spark_tasks=None, - total_cpus=400, - total_gpus=None, - total_heap_memory_bytes=_mem_in_gbs(800), - total_object_store_memory_bytes=_mem_in_gbs(100), - ) - assert cpu_defined_task_count == 100 - - # Heap memory sets the task count - heap_defined_task_count = get_target_spark_tasks( - max_concurrent_tasks=1600, - num_spark_task_cpus=8, - num_spark_task_gpus=None, - ray_worker_heap_memory_bytes=_mem_in_gbs(20), - ray_worker_object_store_memory_bytes=_mem_in_gbs(4), - num_spark_tasks=None, - total_cpus=1600, - total_gpus=None, - total_heap_memory_bytes=_mem_in_gbs(8000), - total_object_store_memory_bytes=_mem_in_gbs(400), - ) - assert heap_defined_task_count == 400 - - # GPU - gpu_defined_task_count = get_target_spark_tasks( - max_concurrent_tasks=400, - num_spark_task_cpus=None, - num_spark_task_gpus=4, - ray_worker_heap_memory_bytes=_mem_in_gbs(40), - ray_worker_object_store_memory_bytes=_mem_in_gbs(8), - num_spark_tasks=None, - total_cpus=None, - total_gpus=80, - total_heap_memory_bytes=_mem_in_gbs(400), - total_object_store_memory_bytes=_mem_in_gbs(80), - ) - assert gpu_defined_task_count == 20 - - # Invalid configuration raises - with pytest.raises(ValueError, match="The value of `num_worker_nodes` argument"): - get_target_spark_tasks( - max_concurrent_tasks=400, - num_spark_task_cpus=None, - num_spark_task_gpus=4, - ray_worker_heap_memory_bytes=_mem_in_gbs(40), - ray_worker_object_store_memory_bytes=_mem_in_gbs(8), - num_spark_tasks=-2, - total_cpus=None, - total_gpus=80, - total_heap_memory_bytes=_mem_in_gbs(400), - total_object_store_memory_bytes=_mem_in_gbs(80), - ) - - if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index e888b1f67c0a..e5be73edc606 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -49,7 +49,6 @@ MAX_NUM_WORKER_NODES = -1 -@PublicAPI(stability="alpha") class RayClusterOnSpark: """ This class is the type of instance returned by the `init_ray_cluster` API. @@ -150,6 +149,9 @@ def disconnect(self): def shutdown(self, cancel_background_job=True): """ Shutdown the ray cluster created by the `init_ray_cluster` API. + NB: In the background thread that runs the background spark job, if spark job raise + unexpected error, its exception handler will also call this method, in the case, + it will set cancel_background_job=False to avoid recursive call. """ if not self.is_shutdown: if self.ray_context is not None: @@ -184,7 +186,7 @@ def _convert_ray_node_options(options): return [f"--{k.replace('_', '-')}={str(v)}" for k, v in options.items()] -_RAY_HEAD_STARTUP_TIMEOUT = 40 +_RAY_HEAD_STARTUP_TIMEOUT = 5 _BACKGROUND_JOB_STARTUP_WAIT = 30 _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 @@ -315,7 +317,7 @@ def release_lock(): raise def hold_lock_for_10s_and_release(): - time.sleep(10) + time.sleep(5) release_lock() threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() @@ -329,7 +331,7 @@ def _init_ray_cluster( head_options=None, worker_options=None, ray_temp_root_dir=None, - safe_mode=True, + safe_mode=False, ): """ This function is used in testing, it has the same arguments with @@ -471,14 +473,7 @@ def _init_ray_cluster( ) # wait ray head node spin up. - time.sleep(5) - for _ in range(_RAY_HEAD_STARTUP_TIMEOUT): - time.sleep(1) - if ray_head_proc.poll() is not None: - # ray head node process terminated. - break - if check_port_open(ray_head_ip, ray_head_port): - break + time.sleep(_RAY_HEAD_STARTUP_TIMEOUT) if not check_port_open(ray_head_ip, ray_head_port): if ray_head_proc.poll() is None: @@ -688,7 +683,7 @@ def init_ray_cluster( head_options: Optional[Dict] = None, worker_options: Optional[Dict] = None, ray_temp_root_dir: Optional[str] = None, - safe_mode: Optional[bool] = True, + safe_mode: Optional[bool] = False, ) -> None: """ Initialize a ray cluster on the spark cluster by starting a ray head node in the spark @@ -713,7 +708,9 @@ def init_ray_cluster( cluster, it is recommended to set this argument to `ray.spark.utils.MAX_NUM_WORKER_NODES`. object_store_memory_per_node: Object store memory available to per-ray worker node, but it cannot exceed - "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker" + "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". + The default value equals to + "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster diff --git a/python/ray/util/spark/databricks_hook.py b/python/ray/util/spark/databricks_hook.py index 50aab25266d0..4a2e4b10bfe4 100644 --- a/python/ray/util/spark/databricks_hook.py +++ b/python/ray/util/spark/databricks_hook.py @@ -2,7 +2,7 @@ from .utils import get_spark_session import logging -_logger = logging.getLogger("ray.util.spark.databricks_hook") +_logger = logging.getLogger(__name__) class _NoDbutilsError(Exception): @@ -10,6 +10,9 @@ class _NoDbutilsError(Exception): def get_dbutils(): + """ + Get databricks runtime dbutils module. + """ try: import IPython @@ -24,6 +27,12 @@ def get_dbutils(): def display_databricks_driver_proxy_url(spark_context, port, title): + """ + This helper function create a proxy URL for databricks driver webapp forwarding. + In databricks runtime, user does not have permission to directly access web service + binding on driver machine port, but user can visit it by a proxy URL with following + format: "/driver-proxy/o/{orgId}/{clusterId}/{port}/" + """ from dbruntime.display import displayHTML driverLocal = spark_context._jvm.com.databricks.backend.daemon.driver.DriverLocal diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 4da8fe66b40f..d11f6cf541b9 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -7,6 +7,17 @@ import signal +# Spark on ray implementation does not directly invoke `ray start ...` script to create ray node +# subprocess, instead, it creates a subprocess to run this `ray.util.spark.start_ray_node` +# module, and in this module it invokes `ray start ...` script to start ray node, +# the purpose of `start_ray_node` module is to set up a SIGTERM handler for cleaning ray temp +# directory when ray node exits. +# When spark driver python process dies, or spark python worker dies, because they registered +# the PR_SET_PDEATHSIG signal, so OS will send a SIGTERM signal to its children processes, so +# `start_ray_node` subprocess will receive a SIGTERM signal and the SIGTERM handler will do +# cleanup work. + + _WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 From 596ae9021aec21a1c0c8eb6cc80b698b0a81abf2 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 15 Dec 2022 21:16:48 +0800 Subject: [PATCH 104/121] speedup test --- python/ray/tests/spark/test_ray_on_spark.py | 6 ++++-- python/ray/util/spark/cluster_init.py | 14 ++++++++------ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index 42a478e47855..cd2f30bef940 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -59,7 +59,7 @@ def f(x): assert results == [i * i for i in range(32)] # assert temp dir is removed. - time.sleep(4) + time.sleep(5) assert not os.path.exists(cluster.temp_dir) def test_ray_cluster_shutdown(self): @@ -70,7 +70,7 @@ def test_ray_cluster_shutdown(self): # Test: cancel background spark job will cause all ray worker nodes exit. cluster._cancel_background_spark_job() - time.sleep(3) + time.sleep(6) assert len(self.get_ray_worker_resources_list()) == 0 @@ -200,6 +200,8 @@ def setup_class(cls): if __name__ == "__main__": + os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" + os.environ["RAY_ON_SPARK_RAY_WORKER_STARTUP_INTERVAL"] = "5" if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) else: diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index e5be73edc606..e92c5ac50ada 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -111,7 +111,7 @@ def connect(self): len([node for node in ray.nodes() if node["Alive"]]) - 1 ) # Minus 1 means excluding the head node. - if cur_alive_worker_count == self.num_worker_nodes: + if cur_alive_worker_count >= self.num_worker_nodes: return if cur_alive_worker_count > last_alive_worker_count: @@ -122,7 +122,7 @@ def connect(self): f"({cur_alive_worker_count} / {self.num_worker_nodes})" ) else: - if time.time() - last_progress_move_time > 120: + if time.time() - last_progress_move_time > _RAY_CONNECT_CLUSTER_POLL_PROGRESS_TIMEOUT: _logger.warning( "Timeout in waiting for all ray workers to start. Started / Total " f"requested: ({cur_alive_worker_count} / {self.num_worker_nodes}). " @@ -187,8 +187,10 @@ def _convert_ray_node_options(options): _RAY_HEAD_STARTUP_TIMEOUT = 5 -_BACKGROUND_JOB_STARTUP_WAIT = 30 +_BACKGROUND_JOB_STARTUP_WAIT = int(os.environ.get("RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT", "30")) _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 +_RAY_WORKER_NODE_STARTUP_INTERVAL = int(os.environ.get("RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL", "10")) +_RAY_CONNECT_CLUSTER_POLL_PROGRESS_TIMEOUT = 120 def _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup(): @@ -316,11 +318,11 @@ def release_lock(): release_lock() raise - def hold_lock_for_10s_and_release(): - time.sleep(5) + def hold_lock(): + time.sleep(_RAY_WORKER_NODE_STARTUP_INTERVAL) release_lock() - threading.Thread(target=hold_lock_for_10s_and_release, args=()).start() + threading.Thread(target=hold_lock, args=()).start() return worker_port_range_begin, worker_port_range_end From ad6fb6b287783112e5c0920c599fe0fbc220a9d7 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 15 Dec 2022 21:57:50 +0800 Subject: [PATCH 105/121] fix Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_ray_on_spark.py | 3 ++- python/ray/tests/spark/test_utils.py | 8 +++----- python/ray/util/spark/utils.py | 1 - 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_ray_on_spark.py index cd2f30bef940..4ad52b984bee 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_ray_on_spark.py @@ -26,6 +26,7 @@ def setup_class(cls): @classmethod def teardown_class(cls): + time.sleep(8) # Wait all background spark job canceled. cls.spark.stop() @staticmethod @@ -201,7 +202,7 @@ def setup_class(cls): if __name__ == "__main__": os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" - os.environ["RAY_ON_SPARK_RAY_WORKER_STARTUP_INTERVAL"] = "5" + os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) else: diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index 61af0ef5890c..6f102041d259 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -19,11 +19,9 @@ def test_get_spark_task_assigned_physical_gpus(): def test_calc_mem_per_ray_worker_node(): - assert _calc_mem_per_ray_worker_node(4, 1000000, 400000, 0.4) == (120000, 80000) - assert _calc_mem_per_ray_worker_node(6, 1000000, 400000, 0.4) == (80000, 53333) - assert _calc_mem_per_ray_worker_node(4, 800000, 600000, 0.2) == (128000, 32000) - assert _calc_mem_per_ray_worker_node(4, 800000, 600000, 0.5) == (80000, 80000) - assert _calc_mem_per_ray_worker_node(8, 2000000, 600000, 0.3) == (140000, 60000) + assert _calc_mem_per_ray_worker_node(4, 1000000, 400000, 100000) == (120000, 80000) + assert _calc_mem_per_ray_worker_node(4, 1000000, 400000, 70000) == (130000, 70000) + assert _calc_mem_per_ray_worker_node(4, 1000000, 400000, None) == (120000, 80000) if __name__ == "__main__": diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index ae4013f19aff..e74299c7ecee 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -166,7 +166,6 @@ def _calc_mem_per_ray_worker_node( available_shared_mem_per_node = int( shared_mem_bytes / num_task_slots * _MEMORY_BUFFER_OFFSET ) - if object_store_memory_per_node is None: object_store_bytes = available_shared_mem_per_node else: From b777d83100d56c3ff536532f2f20820e03354ff5 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 16 Dec 2022 09:43:25 +0800 Subject: [PATCH 106/121] split tests Signed-off-by: Weichen Xu --- .buildkite/pipeline.build.yml | 4 +- python/ray/tests/BUILD | 4 +- python/ray/tests/spark/test_GPU.py | 84 +++++++++++++++++ .../{test_ray_on_spark.py => test_basic.py} | 92 ------------------- .../tests/spark/test_multicores_per_task.py | 43 +++++++++ 5 files changed, 133 insertions(+), 94 deletions(-) create mode 100644 python/ray/tests/spark/test_GPU.py rename python/ray/tests/spark/{test_ray_on_spark.py => test_basic.py} (51%) create mode 100644 python/ray/tests/spark/test_multicores_per_task.py diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml index 1f86f158ad91..ccc5b527156f 100644 --- a/.buildkite/pipeline.build.yml +++ b/.buildkite/pipeline.build.yml @@ -492,7 +492,9 @@ - pip uninstall -y ray - RAY_DEBUG_BUILD=debug ./ci/ci.sh build - ./ci/env/env_info.sh - - bazel test --config=ci-debug $(./ci/run/bazel_export_options) + - RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT=1 + RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL=5 + bazel test --config=ci-debug $(./ci/run/bazel_export_options) --test_tag_filters=-kubernetes,spark_plugin_tests python/ray/tests/... diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 2d97e871d019..75ffe9f664ab 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -520,7 +520,9 @@ py_test_module_list( py_test_module_list( files = [ - "spark/test_ray_on_spark.py", + "spark/test_basic.py", + "spark/test_GPU.py", + "spark/test_multicores_per_task.py", "spark/test_utils.py", ], size = "large", diff --git a/python/ray/tests/spark/test_GPU.py b/python/ray/tests/spark/test_GPU.py new file mode 100644 index 000000000000..c3663511e62c --- /dev/null +++ b/python/ray/tests/spark/test_GPU.py @@ -0,0 +1,84 @@ +import sys +import pytest +import os +import time +import functools +from abc import ABC +from pyspark.sql import SparkSession +from ray.tests.spark.test_basic import RayOnSparkCPUClusterTestBase + +import ray +from ray.util.spark.cluster_init import _init_ray_cluster + + +os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" +os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" + + +class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): + + num_total_gpus = None + num_gpus_per_spark_task = None + + def test_gpu_allocation(self): + + for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: + with _init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): + worker_res_list = self.get_ray_worker_resources_list() + assert len(worker_res_list) == num_spark_tasks + for worker_res in worker_res_list: + assert worker_res["GPU"] == self.num_gpus_per_spark_task + + def test_basic_ray_app_using_gpu(self): + + with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): + + @ray.remote(num_cpus=1, num_gpus=1) + def f(_): + # Add a sleep to avoid the task finishing too fast, + # so that it can make all ray tasks concurrently running in all idle task slots. + time.sleep(5) + return [ + int(gpu_id) + for gpu_id in os.environ["CUDA_VISIBLE_DEVICES"].split(",") + ] + + futures = [f.remote(i) for i in range(self.num_total_gpus)] + results = ray.get(futures) + merged_results = functools.reduce(lambda x, y: x + y, results) + # Test all ray tasks are assigned with different GPUs. + assert sorted(merged_results) == list(range(self.num_total_gpus)) + + +class TestBasicSparkGPUCluster(RayOnSparkGPUClusterTestBase): + @classmethod + def setup_class(cls): + cls.num_total_cpus = 2 + cls.num_total_gpus = 2 + cls.num_cpus_per_spark_task = 1 + cls.num_gpus_per_spark_task = 1 + cls.max_spark_tasks = 2 + gpu_discovery_script_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), "discover_2_gpu.sh" + ) + os.environ["SPARK_WORKER_CORES"] = "4" + cls.spark = ( + SparkSession.builder.master("local-cluster[1, 2, 1024]") + .config("spark.task.cpus", "1") + .config("spark.task.resource.gpu.amount", "1") + .config("spark.executor.cores", "2") + .config("spark.worker.resource.gpu.amount", "2") + .config("spark.executor.resource.gpu.amount", "2") + .config("spark.task.maxFailures", "1") + .config( + "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path + ) + .getOrCreate() + ) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/spark/test_ray_on_spark.py b/python/ray/tests/spark/test_basic.py similarity index 51% rename from python/ray/tests/spark/test_ray_on_spark.py rename to python/ray/tests/spark/test_basic.py index 4ad52b984bee..6ca39650f6a6 100644 --- a/python/ray/tests/spark/test_ray_on_spark.py +++ b/python/ray/tests/spark/test_basic.py @@ -10,7 +10,6 @@ from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time -import functools class RayOnSparkCPUClusterTestBase(ABC): @@ -94,41 +93,6 @@ def test_background_spark_job_exit_trigger_ray_head_exit(self): assert not check_port_open(hostname, int(port)) -class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): - - num_total_gpus = None - num_gpus_per_spark_task = None - - def test_gpu_allocation(self): - - for num_spark_tasks in [self.max_spark_tasks // 2, self.max_spark_tasks]: - with _init_ray_cluster(num_worker_nodes=num_spark_tasks, safe_mode=False): - worker_res_list = self.get_ray_worker_resources_list() - assert len(worker_res_list) == num_spark_tasks - for worker_res in worker_res_list: - assert worker_res["GPU"] == self.num_gpus_per_spark_task - - def test_basic_ray_app_using_gpu(self): - - with _init_ray_cluster(num_worker_nodes=self.max_spark_tasks, safe_mode=False): - - @ray.remote(num_cpus=1, num_gpus=1) - def f(_): - # Add a sleep to avoid the task finishing too fast, - # so that it can make all ray tasks concurrently running in all idle task slots. - time.sleep(5) - return [ - int(gpu_id) - for gpu_id in os.environ["CUDA_VISIBLE_DEVICES"].split(",") - ] - - futures = [f.remote(i) for i in range(self.num_total_gpus)] - results = ray.get(futures) - merged_results = functools.reduce(lambda x, y: x + y, results) - # Test all ray tasks are assigned with different GPUs. - assert sorted(merged_results) == list(range(self.num_total_gpus)) - - class TestBasicSparkCluster(RayOnSparkCPUClusterTestBase): @classmethod def setup_class(cls): @@ -146,63 +110,7 @@ def setup_class(cls): ) -class TestBasicSparkGPUCluster(RayOnSparkGPUClusterTestBase): - @classmethod - def setup_class(cls): - cls.num_total_cpus = 2 - cls.num_total_gpus = 2 - cls.num_cpus_per_spark_task = 1 - cls.num_gpus_per_spark_task = 1 - cls.max_spark_tasks = 2 - gpu_discovery_script_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), "discover_2_gpu.sh" - ) - os.environ["SPARK_WORKER_CORES"] = "4" - cls.spark = ( - SparkSession.builder.master("local-cluster[1, 2, 1024]") - .config("spark.task.cpus", "1") - .config("spark.task.resource.gpu.amount", "1") - .config("spark.executor.cores", "2") - .config("spark.worker.resource.gpu.amount", "2") - .config("spark.executor.resource.gpu.amount", "2") - .config("spark.task.maxFailures", "1") - .config( - "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path - ) - .getOrCreate() - ) - - -class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): - @classmethod - def setup_class(cls): - cls.num_total_cpus = 4 - cls.num_total_gpus = 4 - cls.num_cpus_per_spark_task = 2 - cls.num_gpus_per_spark_task = 2 - cls.max_spark_tasks = 2 - gpu_discovery_script_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), "discover_4_gpu.sh" - ) - os.environ["SPARK_WORKER_CORES"] = "4" - cls.spark = ( - SparkSession.builder.master("local-cluster[1, 4, 1024]") - .config("spark.task.cpus", "2") - .config("spark.task.resource.gpu.amount", "2") - .config("spark.executor.cores", "4") - .config("spark.worker.resource.gpu.amount", "4") - .config("spark.executor.resource.gpu.amount", "4") - .config("spark.task.maxFailures", "1") - .config( - "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path - ) - .getOrCreate() - ) - - if __name__ == "__main__": - os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" - os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) else: diff --git a/python/ray/tests/spark/test_multicores_per_task.py b/python/ray/tests/spark/test_multicores_per_task.py new file mode 100644 index 000000000000..8adf5b358839 --- /dev/null +++ b/python/ray/tests/spark/test_multicores_per_task.py @@ -0,0 +1,43 @@ +import sys +import pytest +import os +from pyspark.sql import SparkSession +from ray.tests.spark.test_GPU import RayOnSparkGPUClusterTestBase + + +os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" +os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" + + +class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): + @classmethod + def setup_class(cls): + cls.num_total_cpus = 4 + cls.num_total_gpus = 4 + cls.num_cpus_per_spark_task = 2 + cls.num_gpus_per_spark_task = 2 + cls.max_spark_tasks = 2 + gpu_discovery_script_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), "discover_4_gpu.sh" + ) + os.environ["SPARK_WORKER_CORES"] = "4" + cls.spark = ( + SparkSession.builder.master("local-cluster[1, 4, 1024]") + .config("spark.task.cpus", "2") + .config("spark.task.resource.gpu.amount", "2") + .config("spark.executor.cores", "4") + .config("spark.worker.resource.gpu.amount", "4") + .config("spark.executor.resource.gpu.amount", "4") + .config("spark.task.maxFailures", "1") + .config( + "spark.worker.resource.gpu.discoveryScript", gpu_discovery_script_path + ) + .getOrCreate() + ) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) From 1d04fc05a58f886a75afcf7319b65d00e3816d36 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 16 Dec 2022 11:26:10 +0800 Subject: [PATCH 107/121] set test env Signed-off-by: Weichen Xu --- .buildkite/pipeline.build.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml index ccc5b527156f..5954bdd5fe50 100644 --- a/.buildkite/pipeline.build.yml +++ b/.buildkite/pipeline.build.yml @@ -487,13 +487,14 @@ - label: ":python: Ray on Spark Test" conditions: ["RAY_CI_PYTHON_AFFECTED"] instance_size: medium + env: + RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT: 1 + RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL: 5 commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - pip uninstall -y ray - RAY_DEBUG_BUILD=debug ./ci/ci.sh build - ./ci/env/env_info.sh - - RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT=1 - RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL=5 bazel test --config=ci-debug $(./ci/run/bazel_export_options) --test_tag_filters=-kubernetes,spark_plugin_tests python/ray/tests/... From eb87eb2772be0e0f3cef65f334a2da53a62a6b11 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 16 Dec 2022 11:54:10 +0800 Subject: [PATCH 108/121] fix ci Signed-off-by: Weichen Xu --- .buildkite/pipeline.build.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml index 5954bdd5fe50..a0f9fbfec09d 100644 --- a/.buildkite/pipeline.build.yml +++ b/.buildkite/pipeline.build.yml @@ -495,7 +495,7 @@ - pip uninstall -y ray - RAY_DEBUG_BUILD=debug ./ci/ci.sh build - ./ci/env/env_info.sh - bazel test --config=ci-debug $(./ci/run/bazel_export_options) + - bazel test --config=ci-debug $(./ci/run/bazel_export_options) --test_tag_filters=-kubernetes,spark_plugin_tests python/ray/tests/... From a2863b6face6b8ee72a1d7ec604d46ec73c45c2f Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 16 Dec 2022 20:25:58 +0800 Subject: [PATCH 109/121] add param collect_log_to_path Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_GPU.py | 1 + python/ray/tests/spark/test_basic.py | 11 ++++++++- .../tests/spark/test_multicores_per_task.py | 1 + python/ray/util/spark/cluster_init.py | 14 ++++++++++- python/ray/util/spark/start_ray_node.py | 24 +++++++++++++++++-- 5 files changed, 47 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/spark/test_GPU.py b/python/ray/tests/spark/test_GPU.py index c3663511e62c..cb4ab4226263 100644 --- a/python/ray/tests/spark/test_GPU.py +++ b/python/ray/tests/spark/test_GPU.py @@ -53,6 +53,7 @@ def f(_): class TestBasicSparkGPUCluster(RayOnSparkGPUClusterTestBase): @classmethod def setup_class(cls): + super().setup_class() cls.num_total_cpus = 2 cls.num_total_gpus = 2 cls.num_cpus_per_spark_task = 1 diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index 6ca39650f6a6..47cc8adeab5f 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -10,6 +10,9 @@ from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time +import logging + +_logger = logging.getLogger(__name__) class RayOnSparkCPUClusterTestBase(ABC): @@ -21,7 +24,12 @@ class RayOnSparkCPUClusterTestBase(ABC): @classmethod def setup_class(cls): - raise NotImplementedError() + _logger.info( + f"RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT={os.environ.get('RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT')}" + ) + _logger.info( + f"RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL={os.environ.get('RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL')}" + ) @classmethod def teardown_class(cls): @@ -96,6 +104,7 @@ def test_background_spark_job_exit_trigger_ray_head_exit(self): class TestBasicSparkCluster(RayOnSparkCPUClusterTestBase): @classmethod def setup_class(cls): + super().setup_class() cls.num_total_cpus = 2 cls.num_total_gpus = 0 cls.num_cpus_per_spark_task = 1 diff --git a/python/ray/tests/spark/test_multicores_per_task.py b/python/ray/tests/spark/test_multicores_per_task.py index 8adf5b358839..f7a3b1c54420 100644 --- a/python/ray/tests/spark/test_multicores_per_task.py +++ b/python/ray/tests/spark/test_multicores_per_task.py @@ -12,6 +12,7 @@ class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): @classmethod def setup_class(cls): + super().setup_class() cls.num_total_cpus = 4 cls.num_total_gpus = 4 cls.num_cpus_per_spark_task = 2 diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index e92c5ac50ada..ae3808b5a093 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -48,6 +48,8 @@ MAX_NUM_WORKER_NODES = -1 +RAY_ON_SPARK_COLLECT_LOG_TO_PATH = 'RAY_ON_SPARK_COLLECT_LOG_TO_PATH' + class RayClusterOnSpark: """ @@ -334,6 +336,7 @@ def _init_ray_cluster( worker_options=None, ray_temp_root_dir=None, safe_mode=False, + collect_log_to_path=None, ): """ This function is used in testing, it has the same arguments with @@ -472,6 +475,7 @@ def _init_ray_cluster( ray_head_node_cmd, synchronous=False, preexec_fn=setup_sigterm_on_parent_death, + extra_env={RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path} ) # wait ray head node spin up. @@ -551,7 +555,9 @@ def ray_cluster_job_mapper(_): *_convert_ray_node_options(worker_options), ] - ray_worker_node_extra_envs = {} + ray_worker_node_extra_envs = { + RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path + } if num_spark_task_gpus > 0: task_resources = context.resources() @@ -686,6 +692,7 @@ def init_ray_cluster( worker_options: Optional[Dict] = None, ray_temp_root_dir: Optional[str] = None, safe_mode: Optional[bool] = False, + collect_log_to_path: Optional[str] = None, ) -> None: """ Initialize a ray cluster on the spark cluster by starting a ray head node in the spark @@ -723,6 +730,11 @@ def init_ray_cluster( not available for minimum recommended functionality, an exception will be raised that details the inadequate spark cluster configuration settings. If overridden as `False`, a warning is raised. + collect_log_to_path: If specified, after ray head / worker nodes terminated, collect their + logs to the specified path. On Databricks Runtime, we recommend you to + specify a local path starts with '/dbfs/', because the path mounts + with a centralized storage device and stored data is persisted after + databricks spark cluster terminated. """ global _active_ray_cluster if _active_ray_cluster is not None: diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index d11f6cf541b9..11e68699fe51 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -5,6 +5,8 @@ import shutil import fcntl import signal +import socket +from ray.util.spark.cluster_init import RAY_ON_SPARK_COLLECT_LOG_TO_PATH # Spark on ray implementation does not directly invoke `ray start ...` script to create ray node @@ -24,12 +26,15 @@ if __name__ == "__main__": arg_list = sys.argv[1:] + collect_log_to_path = os.environ[RAY_ON_SPARK_COLLECT_LOG_TO_PATH] + is_head = ("--head" in arg_list) + temp_dir_arg_prefix = "--temp-dir=" temp_dir = None for arg in arg_list: if arg.startswith(temp_dir_arg_prefix): - temp_dir = arg[len(temp_dir_arg_prefix) :] + temp_dir = arg[len(temp_dir_arg_prefix):] if temp_dir is None: raise ValueError("Please explicitly set --temp-dir option.") @@ -55,10 +60,25 @@ def try_clean_temp_dir_at_exit(): # acquiring exclusive lock to ensure removing dir safely. fcntl.flock(lock_fd, fcntl.LOCK_EX | fcntl.LOCK_NB) shutil.rmtree(temp_dir, ignore_errors=True) + + # This is the final terminated ray node on current spark worker, + # start copy logs (including all local ray nodes logs) to destination. + if collect_log_to_path is not None: + copy_log_dest_path = os.path.join( + collect_log_to_path, + os.path.basename(temp_dir) + "-logs", + ("head" if is_head else socket.gethostname()) + ) + os.makedirs(copy_log_dest_path, exist_ok=True) + shutil.copytree( + os.path.join(temp_dir, "logs"), + copy_log_dest_path, + dirs_exist_ok=True, + ) except BlockingIOError: # The file has active shared lock or exclusive lock, representing there # are other ray nodes running, or other node running cleanup temp-dir routine. - # skip cleaning temp-dir. + # skip cleaning temp-dir, and skip copy logs to destination directory as well. pass except Exception: # swallow any exception. From 92b8d40377209b1fb1454bc70418908d3208f1c5 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 16 Dec 2022 23:33:23 +0800 Subject: [PATCH 110/121] add tests Signed-off-by: Weichen Xu --- .buildkite/pipeline.build.yml | 5 +- python/ray/tests/spark/test_GPU.py | 4 -- python/ray/tests/spark/test_basic.py | 42 +++++++++++++--- .../tests/spark/test_multicores_per_task.py | 4 -- python/ray/util/spark/cluster_init.py | 5 +- python/ray/util/spark/start_ray_node.py | 50 +++++++++++-------- 6 files changed, 70 insertions(+), 40 deletions(-) diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml index a0f9fbfec09d..924b1592aa6e 100644 --- a/.buildkite/pipeline.build.yml +++ b/.buildkite/pipeline.build.yml @@ -487,15 +487,14 @@ - label: ":python: Ray on Spark Test" conditions: ["RAY_CI_PYTHON_AFFECTED"] instance_size: medium - env: - RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT: 1 - RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL: 5 commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - pip uninstall -y ray - RAY_DEBUG_BUILD=debug ./ci/ci.sh build - ./ci/env/env_info.sh - bazel test --config=ci-debug $(./ci/run/bazel_export_options) + --test_env=RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT=1 + --test_env=RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL=5 --test_tag_filters=-kubernetes,spark_plugin_tests python/ray/tests/... diff --git a/python/ray/tests/spark/test_GPU.py b/python/ray/tests/spark/test_GPU.py index cb4ab4226263..2a43a08a62fa 100644 --- a/python/ray/tests/spark/test_GPU.py +++ b/python/ray/tests/spark/test_GPU.py @@ -11,10 +11,6 @@ from ray.util.spark.cluster_init import _init_ray_cluster -os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" -os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" - - class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): num_total_gpus = None diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index 47cc8adeab5f..c714543faaaf 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -1,4 +1,7 @@ import os +import shutil +import tempfile +import socket import pytest import sys @@ -6,6 +9,8 @@ import ray +import ray.util.spark.cluster_init +from ray.util.spark import init_ray_cluster, shutdown_ray_cluster from ray.util.spark.cluster_init import _init_ray_cluster from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession @@ -53,10 +58,16 @@ def test_cpu_allocation(self): for worker_res in worker_res_list: assert worker_res["CPU"] == self.num_cpus_per_spark_task - def test_basic_ray_app(self): - with _init_ray_cluster( - num_worker_nodes=self.max_spark_tasks, safe_mode=False - ) as cluster: + def test_public_api(self): + try: + ray_temp_root_dir = tempfile.mkdtemp() + collect_log_to_path = tempfile.mkdtemp() + init_ray_cluster( + num_worker_nodes=self.max_spark_tasks, + safe_mode=False, + collect_log_to_path=collect_log_to_path, + ray_temp_root_dir=ray_temp_root_dir, + ) @ray.remote def f(x): @@ -66,9 +77,26 @@ def f(x): results = ray.get(futures) assert results == [i * i for i in range(32)] - # assert temp dir is removed. - time.sleep(5) - assert not os.path.exists(cluster.temp_dir) + shutdown_ray_cluster() + + time.sleep(5) + # assert temp dir is removed. + assert len(os.listdir(ray_temp_root_dir)) == 1 and \ + os.listdir(ray_temp_root_dir)[0].endswith(".lock") + + # assert logs are copied to specified path + listed_items = os.listdir(collect_log_to_path) + assert len(listed_items) == 1 and listed_items[0].startswith("ray-") + log_dest_dir = os.path.join(collect_log_to_path, listed_items[0], socket.gethostname()) + assert os.path.exists(log_dest_dir) and len(os.listdir(log_dest_dir)) > 0 + finally: + if ray.util.spark.cluster_init._active_ray_cluster is not None: + # if the test raised error and does not destroy cluster, + # destroy it here. + ray.util.spark._active_ray_cluster.shutdown() + time.sleep(5) + shutil.rmtree(ray_temp_root_dir) + shutil.rmtree(collect_log_to_path) def test_ray_cluster_shutdown(self): with _init_ray_cluster( diff --git a/python/ray/tests/spark/test_multicores_per_task.py b/python/ray/tests/spark/test_multicores_per_task.py index f7a3b1c54420..44f036ecab24 100644 --- a/python/ray/tests/spark/test_multicores_per_task.py +++ b/python/ray/tests/spark/test_multicores_per_task.py @@ -5,10 +5,6 @@ from ray.tests.spark.test_GPU import RayOnSparkGPUClusterTestBase -os.environ["RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT"] = "1" -os.environ["RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL"] = "5" - - class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): @classmethod def setup_class(cls): diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index ae3808b5a093..cf97960715c9 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -475,7 +475,7 @@ def _init_ray_cluster( ray_head_node_cmd, synchronous=False, preexec_fn=setup_sigterm_on_parent_death, - extra_env={RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path} + extra_env={RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or ''} ) # wait ray head node spin up. @@ -556,7 +556,7 @@ def ray_cluster_job_mapper(_): ] ray_worker_node_extra_envs = { - RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path + RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or '' } if num_spark_task_gpus > 0: @@ -749,6 +749,7 @@ def init_ray_cluster( worker_options=worker_options, ray_temp_root_dir=ray_temp_root_dir, safe_mode=safe_mode, + collect_log_to_path=collect_log_to_path, ) cluster.connect() # NB: this line might raise error. diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 11e68699fe51..4fe53439bc31 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -6,6 +6,7 @@ import fcntl import signal import socket +import logging from ray.util.spark.cluster_init import RAY_ON_SPARK_COLLECT_LOG_TO_PATH @@ -22,12 +23,13 @@ _WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 +_logger = logging.getLogger(__name__) + if __name__ == "__main__": arg_list = sys.argv[1:] collect_log_to_path = os.environ[RAY_ON_SPARK_COLLECT_LOG_TO_PATH] - is_head = ("--head" in arg_list) temp_dir_arg_prefix = "--temp-dir=" temp_dir = None @@ -56,30 +58,38 @@ def try_clean_temp_dir_at_exit(): fcntl.flock(lock_fd, fcntl.LOCK_UN) try: - # Start clean the temp-dir, - # acquiring exclusive lock to ensure removing dir safely. + # acquiring exclusive lock to ensure copy logs and removing dir safely. fcntl.flock(lock_fd, fcntl.LOCK_EX | fcntl.LOCK_NB) - shutil.rmtree(temp_dir, ignore_errors=True) - - # This is the final terminated ray node on current spark worker, - # start copy logs (including all local ray nodes logs) to destination. - if collect_log_to_path is not None: - copy_log_dest_path = os.path.join( - collect_log_to_path, - os.path.basename(temp_dir) + "-logs", - ("head" if is_head else socket.gethostname()) - ) - os.makedirs(copy_log_dest_path, exist_ok=True) - shutil.copytree( - os.path.join(temp_dir, "logs"), - copy_log_dest_path, - dirs_exist_ok=True, - ) + lock_acquired = True except BlockingIOError: # The file has active shared lock or exclusive lock, representing there # are other ray nodes running, or other node running cleanup temp-dir routine. # skip cleaning temp-dir, and skip copy logs to destination directory as well. - pass + lock_acquired = False + + if lock_acquired: + # This is the final terminated ray node on current spark worker, + # start copy logs (including all local ray nodes logs) to destination. + if collect_log_to_path: + try: + copy_log_dest_path = os.path.join( + collect_log_to_path, + os.path.basename(temp_dir) + "-logs", + socket.gethostname(), + ) + os.makedirs(copy_log_dest_path, exist_ok=True) + shutil.copytree( + os.path.join(temp_dir, "session_latest", "logs"), + copy_log_dest_path, + dirs_exist_ok=True, + ) + except Exception as e: + _logger.warning( + f"Collect logs to destination directory failed, error: {repr(e)}." + ) + + # Start cleaning the temp-dir, + shutil.rmtree(temp_dir, ignore_errors=True) except Exception: # swallow any exception. pass From ddc0fd92ac9a8a110b40163297bdaac2a7879a20 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 17 Dec 2022 17:26:36 +0800 Subject: [PATCH 111/121] improve start script Signed-off-by: Weichen Xu --- python/ray/util/spark/start_ray_node.py | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 4fe53439bc31..4ea193051a0d 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -8,6 +8,7 @@ import socket import logging from ray.util.spark.cluster_init import RAY_ON_SPARK_COLLECT_LOG_TO_PATH +from ray._private.ray_process_reaper import SIGTERM_GRACE_PERIOD_SECONDS # Spark on ray implementation does not directly invoke `ray start ...` script to create ray node @@ -21,8 +22,6 @@ # cleanup work. -_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR = 1 - _logger = logging.getLogger(__name__) @@ -48,10 +47,19 @@ lock_file = temp_dir + ".lock" lock_fd = os.open(lock_file, os.O_RDWR | os.O_CREAT | os.O_TRUNC) + # Mutilple ray nodes might start on the same machine, and they are using the + # same temp directory, adding a shared lock representing current ray node is + # using the temp directory. + fcntl.flock(lock_fd, fcntl.LOCK_SH) + process = subprocess.Popen([ray_exec_path, "start", *arg_list], text=True) + def try_clean_temp_dir_at_exit(): try: # Wait for a while to ensure the children processes of the ray node all exited. - time.sleep(_WAIT_TIME_BEFORE_CLEAN_TEMP_DIR) + time.sleep(SIGTERM_GRACE_PERIOD_SECONDS + 0.5) + if process.poll() is None: + # "ray start ..." command process is still alive. Force to kill it. + process.kill() # Release the shared lock, representing current ray node does not use the # temp dir. @@ -98,12 +106,6 @@ def try_clean_temp_dir_at_exit(): os.close(lock_fd) try: - # Mutilple ray nodes might start on the same machine, and they are using the - # same temp directory, adding a shared lock representing current ray node is - # using the temp directory. - fcntl.flock(lock_fd, fcntl.LOCK_SH) - process = subprocess.Popen([ray_exec_path, "start", *arg_list], text=True) - def sigterm_handler(*args): process.terminate() try_clean_temp_dir_at_exit() From 49ab76e619a905c6c9944d0b0080f37b8b9fd002 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 17 Dec 2022 19:17:40 +0800 Subject: [PATCH 112/121] fix linter Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_GPU.py | 3 +- python/ray/tests/spark/test_basic.py | 24 +- python/ray/util/spark/__init__.py | 8 +- python/ray/util/spark/cluster_init.py | 361 ++++++++++++----------- python/ray/util/spark/databricks_hook.py | 12 +- python/ray/util/spark/start_ray_node.py | 32 +- python/ray/util/spark/utils.py | 31 +- 7 files changed, 252 insertions(+), 219 deletions(-) diff --git a/python/ray/tests/spark/test_GPU.py b/python/ray/tests/spark/test_GPU.py index 2a43a08a62fa..04470004923e 100644 --- a/python/ray/tests/spark/test_GPU.py +++ b/python/ray/tests/spark/test_GPU.py @@ -32,7 +32,8 @@ def test_basic_ray_app_using_gpu(self): @ray.remote(num_cpus=1, num_gpus=1) def f(_): # Add a sleep to avoid the task finishing too fast, - # so that it can make all ray tasks concurrently running in all idle task slots. + # so that it can make all ray tasks concurrently running in all idle + # task slots. time.sleep(5) return [ int(gpu_id) diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index c714543faaaf..96558656e02c 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -29,16 +29,11 @@ class RayOnSparkCPUClusterTestBase(ABC): @classmethod def setup_class(cls): - _logger.info( - f"RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT={os.environ.get('RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT')}" - ) - _logger.info( - f"RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL={os.environ.get('RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL')}" - ) + pass @classmethod def teardown_class(cls): - time.sleep(8) # Wait all background spark job canceled. + time.sleep(10) # Wait all background spark job canceled. cls.spark.stop() @staticmethod @@ -79,15 +74,18 @@ def f(x): shutdown_ray_cluster() - time.sleep(5) + time.sleep(7) # assert temp dir is removed. - assert len(os.listdir(ray_temp_root_dir)) == 1 and \ - os.listdir(ray_temp_root_dir)[0].endswith(".lock") + assert len(os.listdir(ray_temp_root_dir)) == 1 and os.listdir( + ray_temp_root_dir + )[0].endswith(".lock") # assert logs are copied to specified path listed_items = os.listdir(collect_log_to_path) assert len(listed_items) == 1 and listed_items[0].startswith("ray-") - log_dest_dir = os.path.join(collect_log_to_path, listed_items[0], socket.gethostname()) + log_dest_dir = os.path.join( + collect_log_to_path, listed_items[0], socket.gethostname() + ) assert os.path.exists(log_dest_dir) and len(os.listdir(log_dest_dir)) > 0 finally: if ray.util.spark.cluster_init._active_ray_cluster is not None: @@ -106,7 +104,7 @@ def test_ray_cluster_shutdown(self): # Test: cancel background spark job will cause all ray worker nodes exit. cluster._cancel_background_spark_job() - time.sleep(6) + time.sleep(8) assert len(self.get_ray_worker_resources_list()) == 0 @@ -122,7 +120,7 @@ def test_background_spark_job_exit_trigger_ray_head_exit(self): # Mimic the case the job failed unexpectedly. cluster._cancel_background_spark_job() cluster.spark_job_is_canceled = False - time.sleep(3) + time.sleep(5) # assert ray head node exit by checking head port being closed. hostname, port = cluster.address.split(":") diff --git a/python/ray/util/spark/__init__.py b/python/ray/util/spark/__init__.py index 26851a9c72b0..ddfd2de835bc 100644 --- a/python/ray/util/spark/__init__.py +++ b/python/ray/util/spark/__init__.py @@ -1,11 +1,7 @@ from ray.util.spark.cluster_init import ( init_ray_cluster, shutdown_ray_cluster, - MAX_NUM_WORKER_NODES + MAX_NUM_WORKER_NODES, ) -__all__ = [ - "init_ray_cluster", - "shutdown_ray_cluster", - "MAX_NUM_WORKER_NODES" -] +__all__ = ["init_ray_cluster", "shutdown_ray_cluster", "MAX_NUM_WORKER_NODES"] diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index cf97960715c9..92e40e6160e6 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -48,7 +48,7 @@ MAX_NUM_WORKER_NODES = -1 -RAY_ON_SPARK_COLLECT_LOG_TO_PATH = 'RAY_ON_SPARK_COLLECT_LOG_TO_PATH' +RAY_ON_SPARK_COLLECT_LOG_TO_PATH = "RAY_ON_SPARK_COLLECT_LOG_TO_PATH" class RayClusterOnSpark: @@ -59,8 +59,8 @@ class RayClusterOnSpark: Serve as a Python context manager for the `RayClusterOnSpark` instance. Args - address: The url for the ray head node (defined as the hostname and unused port on - Spark driver node) + address: The url for the ray head node (defined as the hostname and unused + port on Spark driver node) head_proc: Ray head process spark_job_group_id: The Spark job id for a submitted ray job num_workers_node: The number of workers in the ray cluster. @@ -124,11 +124,16 @@ def connect(self): f"({cur_alive_worker_count} / {self.num_worker_nodes})" ) else: - if time.time() - last_progress_move_time > _RAY_CONNECT_CLUSTER_POLL_PROGRESS_TIMEOUT: + if ( + time.time() - last_progress_move_time + > _RAY_CONNECT_CLUSTER_POLL_PROGRESS_TIMEOUT + ): _logger.warning( - "Timeout in waiting for all ray workers to start. Started / Total " - f"requested: ({cur_alive_worker_count} / {self.num_worker_nodes}). " - "Please check ray logs to see why some ray workers failed to start." + "Timeout in waiting for all ray workers to start. " + "Started / Total requested: " + f"({cur_alive_worker_count} / {self.num_worker_nodes}). " + "Please check ray logs to see why some ray workers " + "failed to start." ) return else: @@ -151,9 +156,9 @@ def disconnect(self): def shutdown(self, cancel_background_job=True): """ Shutdown the ray cluster created by the `init_ray_cluster` API. - NB: In the background thread that runs the background spark job, if spark job raise - unexpected error, its exception handler will also call this method, in the case, - it will set cancel_background_job=False to avoid recursive call. + NB: In the background thread that runs the background spark job, if spark job + raise unexpected error, its exception handler will also call this method, in + the case, it will set cancel_background_job=False to avoid recursive call. """ if not self.is_shutdown: if self.ray_context is not None: @@ -164,15 +169,15 @@ def shutdown(self, cancel_background_job=True): except Exception as e: # swallow exception. _logger.warning( - f"An error occurred while cancelling the ray cluster background spark job: " - f"{repr(e)}" + f"An error occurred while cancelling the ray cluster " + f"background spark job: {repr(e)}" ) try: self.head_proc.terminate() except Exception as e: # swallow exception. _logger.warning( - f"An Error occurred during shutdown of ray head node: {repr(e)}" + "An Error occurred during shutdown of ray head node: " f"{repr(e)}" ) self.is_shutdown = True @@ -189,35 +194,41 @@ def _convert_ray_node_options(options): _RAY_HEAD_STARTUP_TIMEOUT = 5 -_BACKGROUND_JOB_STARTUP_WAIT = int(os.environ.get("RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT", "30")) +_BACKGROUND_JOB_STARTUP_WAIT = int( + os.environ.get("RAY_ON_SPARK_BACKGROUND_JOB_STARTUP_WAIT", "30") +) _RAY_CLUSTER_STARTUP_PROGRESS_CHECKING_INTERVAL = 3 -_RAY_WORKER_NODE_STARTUP_INTERVAL = int(os.environ.get("RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL", "10")) +_RAY_WORKER_NODE_STARTUP_INTERVAL = int( + os.environ.get("RAY_ON_SPARK_RAY_WORKER_NODE_STARTUP_INTERVAL", "10") +) _RAY_CONNECT_CLUSTER_POLL_PROGRESS_TIMEOUT = 120 -def _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup(): +def _prepare_for_ray_worker_node_startup(): """ - If we start multiple ray workers on a machine concurrently, some ray worker processes - might fail due to ray port conflicts, this is because race condition on getting free - port and opening the free port. - To address the issue, this function use an exclusive file lock to delay the worker processes - to ensure that port acquisition does not create a resource contention issue due to a race - condition. + If we start multiple ray workers on a machine concurrently, some ray worker + processes might fail due to ray port conflicts, this is because race condition + on getting free port and opening the free port. + To address the issue, this function use an exclusive file lock to delay the + worker processes to ensure that port acquisition does not create a resource + contention issue due to a race condition. After acquiring lock, it will allocate port range for worker ports (for ray node config --min-worker-port and --max-worker-port). - Because on a spark cluster, multiple ray cluster might be created, so on one spark worker - machine, there might be multiple ray worker nodes running, these worker nodes might belong - to different ray cluster, and we must ensure these ray nodes on the same machine using - non-overlapping worker port range, to achieve this, in this function, it creates a file - `/tmp/ray_on_spark_worker_port_allocation.txt` file, the file format is composed of multiple - lines, each line contains 2 number: `pid` and `port_range_slot_index`, - each port range slot allocates 1000 ports, and corresponding port range is: - range_begin (inclusive): 20000 + port_range_slot_index * 1000 - range_end (exclusive): range_begin + 1000 - In this function, it first scans `/tmp/ray_on_spark_worker_port_allocation.txt` file, - removing lines that containing dead process pid, then find the first unused - port_range_slot_index, then regenerate this file, and return the allocated port range. + Because on a spark cluster, multiple ray cluster might be created, so on one spark + worker machine, there might be multiple ray worker nodes running, these worker + nodes might belong to different ray cluster, and we must ensure these ray nodes on + the same machine using non-overlapping worker port range, to achieve this, in this + function, it creates a file `/tmp/ray_on_spark_worker_port_allocation.txt` file, + the file format is composed of multiple lines, each line contains 2 number: `pid` + and `port_range_slot_index`, each port range slot allocates 1000 ports, and + corresponding port range is: + - range_begin (inclusive): 20000 + port_range_slot_index * 1000 + - range_end (exclusive): range_begin + 1000 + In this function, it first scans `/tmp/ray_on_spark_worker_port_allocation.txt` + file, removing lines that containing dead process pid, then find the first unused + port_range_slot_index, then regenerate this file, and return the allocated port + range. Returns: Allocated port range for current worker ports """ @@ -235,7 +246,8 @@ def acquire_lock(file_path): try: fcntl.flock(fd, fcntl.LOCK_EX | fcntl.LOCK_NB) except BlockingIOError: - # Lock is used by other processes, continue loop to wait for lock available + # Lock is used by other processes, continue loop to wait for lock + # available pass else: # Acquire lock successfully. @@ -249,10 +261,10 @@ def acquire_lock(file_path): try: lock_fd = acquire_lock(lock_file_path) except TimeoutError: - # If timeout happens, the file lock might be hold by another process and that process - # does not release the lock in time by some unexpected reason. - # In this case, remove the existing lock file and create the file again, and then - # acquire file lock on the new file. + # If timeout happens, the file lock might be hold by another process and that + # process does not release the lock in time by some unexpected reason. + # In this case, remove the existing lock file and create the file again, and + # then acquire file lock on the new file. try: os.remove(lock_file_path) except Exception: @@ -340,12 +352,14 @@ def _init_ray_cluster( ): """ This function is used in testing, it has the same arguments with - `ray.util.spark.init_ray_cluster` API, but it returns a `RayClusterOnSpark` instance instead. - - The returned instance can be used to connect to, disconnect from and shutdown the ray cluster. - This instance can also be used as a context manager (used by encapsulating operations within - `with init_ray_cluster(...):`). Upon entering the managed scope, the ray cluster is initiated - and connected to. When exiting the scope, the ray cluster is disconnected and shut down. + `ray.util.spark.init_ray_cluster` API, but it returns a `RayClusterOnSpark` + instance instead. + + The returned instance can be used to connect to, disconnect from and shutdown the + ray cluster. This instance can also be used as a context manager (used by + encapsulating operations within `with init_ray_cluster(...):`). Upon entering the + managed scope, the ray cluster is initiated and connected to. When exiting the + scope, the ray cluster is disconnected and shut down. """ from pyspark.util import inheritable_thread_target @@ -361,8 +375,8 @@ def _init_ray_cluster( spark = get_spark_session() - # Environment configurations within the Spark Session that dictate how many cpus and gpus to - # use for each submitted spark task. + # Environment configurations within the Spark Session that dictate how many cpus + # and gpus to use for each submitted spark task. num_spark_task_cpus = int(spark.sparkContext.getConf().get("spark.task.cpus", "1")) num_spark_task_gpus = int( spark.sparkContext.getConf().get("spark.task.resource.gpu.amount", "0") @@ -376,42 +390,46 @@ def _init_ray_cluster( max_concurrent_tasks = get_max_num_concurrent_tasks(spark.sparkContext) if num_worker_nodes == -1: # num_worker_nodes=-1 represents using all available spark task slots - num_spark_tasks = max_concurrent_tasks + num_worker_nodes = max_concurrent_tasks elif num_worker_nodes <= 0: raise ValueError( - "The value of 'num_worker_nodes' argument must be either a positive integer or " - "'ray.util.spark.MAX_NUM_WORKER_NODES'." + "The value of 'num_worker_nodes' argument must be either a positive " + "integer or 'ray.util.spark.MAX_NUM_WORKER_NODES'." ) insufficient_resources = [] if num_spark_task_cpus < 4: insufficient_resources.append( - f"The provided CPU resources for each ray worker are inadequate to start a ray " - f"cluster. Based on the total cpu resources available and the configured task sizing, " - f"each ray worker would start with {num_spark_task_cpus} CPU cores. This is " - "less than the recommended value of `4` CPUs per worker. Increasing the spark " - "configuration 'spark.task.cpus' to a minimum of `4` addresses it." + "The provided CPU resources for each ray worker are inadequate to start " + "a ray cluster. Based on the total cpu resources available and the " + "configured task sizing, each ray worker would start with " + f"{num_spark_task_cpus} CPU cores. This is less than the recommended " + "value of `4` CPUs per worker. Increasing the spark configuration " + "'spark.task.cpus' to a minimum of `4` addresses it." ) if ray_worker_node_heap_mem_bytes < 10 * 1024 * 1024 * 1024: insufficient_resources.append( - f"The provided memory resources for each ray worker are inadequate. Based on the total " - f"memory available on the spark cluster and the configured task sizing, each ray " - f"worker would start with {ray_worker_node_heap_mem_bytes} bytes heap " - "memory. This is less than the recommended value of 10GB. The ray worker node heap " - "memory size is calculated by (SPARK_WORKER_NODE_PHYSICAL_MEMORY - SHARED_MEMORY) / " - "num_local_spark_task_slots * 0.8. To increase the heap space available, " - "increase the memory in the spark cluster by changing instance types or worker count, " - "reduce the target `num_worker_nodes`, or apply a lower " - "`heap_to_object_store_memory_ratio`." + "The provided memory resources for each ray worker are inadequate. Based " + "on the total memory available on the spark cluster and the configured " + "task sizing, each ray worker would start with " + f"{ray_worker_node_heap_mem_bytes} bytes heap memory. This is less than " + "the recommended value of 10GB. The ray worker node heap memory size is " + "calculated by " + "(SPARK_WORKER_NODE_PHYSICAL_MEMORY / num_local_spark_task_slots * 0.8) - " + "object_store_memory_per_node. To increase the heap space available, " + "increase the memory in the spark cluster by changing instance types or " + "worker count, reduce the target `num_worker_nodes`, or apply a lower " + "`object_store_memory_per_node`." ) if insufficient_resources: if safe_mode: raise ValueError( - "You are creating ray cluster on spark with safe mode (it can be disabled by " - "setting argument 'safe_mode=False' when calling API 'init_ray_cluster'), " - "safe mode requires the spark cluster config satisfying following criterion: " + "You are creating ray cluster on spark with safe mode (it can be " + "disabled by setting argument 'safe_mode=False' when calling API " + "'init_ray_cluster'), safe mode requires the spark cluster config " + "satisfying following criterion: " "\n".join(insufficient_resources) ) else: @@ -434,8 +452,6 @@ def _init_ray_cluster( temp_dir_unique_suffix = uuid.uuid4().hex[:8] - # TODO: Set individual temp dir for ray worker nodes, and auto GC temp data when ray node exits - # See https://github.com/ray-project/ray/issues/28876#issuecomment-1322016494 if ray_temp_root_dir is None: ray_temp_root_dir = start_hook.get_default_temp_dir() ray_temp_dir = os.path.join( @@ -456,26 +472,27 @@ def _init_ray_cluster( "--dashboard-host=0.0.0.0", f"--dashboard-port={ray_dashboard_port}", f"--dashboard-agent-listen-port={ray_dashboard_agent_port}", - # disallow ray tasks with cpu requirements from being scheduled on the head node. - f"--num-cpus=0", - # limit the memory allocation to the head node (actual usage may increase beyond this - # for processing of tasks and actors). - f"--memory={128 * 1024 * 1024}", - # limit the object store memory allocation to the head node (actual usage may increase + # disallow ray tasks with cpu requirements from being scheduled on the head + # node. + "--num-cpus=0", + # limit the memory allocation to the head node (actual usage may increase # beyond this for processing of tasks and actors). + f"--memory={128 * 1024 * 1024}", + # limit the object store memory allocation to the head node (actual usage + # may increase beyond this for processing of tasks and actors). f"--object-store-memory={128 * 1024 * 1024}", *_convert_ray_node_options(head_options), ] _logger.info(f"Starting Ray head, command: {' '.join(ray_head_node_cmd)}") - # `preexec_fn=setup_sigterm_on_parent_death` ensures the ray head node being killed - # if parent process died unexpectedly. + # `preexec_fn=setup_sigterm_on_parent_death` ensures the ray head node being + # killed if parent process died unexpectedly. ray_head_proc, tail_output_deque = exec_cmd( ray_head_node_cmd, synchronous=False, preexec_fn=setup_sigterm_on_parent_death, - extra_env={RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or ''} + extra_env={RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or ""}, ) # wait ray head node spin up. @@ -500,23 +517,25 @@ def _init_ray_cluster( # NB: # In order to start ray worker nodes on spark cluster worker machines, # We launch a background spark job: - # 1. Each spark task launches one ray worker node. This design ensures all ray worker nodes - # have the same shape (same cpus / gpus / memory configuration). If ray worker nodes have a - # non-uniform shape, the Ray cluster setup will be non-deterministic and could create - # issues with node sizing. - # 2. A ray worker node is started via the `ray start` CLI. In each spark task, a child - # process is started and will execute a `ray start ...` command in blocking mode. - # 3. Each task will acquire a file lock for 10s to ensure that the ray worker init will - # acquire a port connection to the ray head node that does not contend with other - # worker processes on the same Spark worker node. + # 1. Each spark task launches one ray worker node. This design ensures all ray + # worker nodes have the same shape (same cpus / gpus / memory configuration). + # If ray worker nodes have a non-uniform shape, the Ray cluster setup will + # be non-deterministic and could create issues with node sizing. + # 2. A ray worker node is started via the `ray start` CLI. In each spark task, + # a child process is started and will execute a `ray start ...` command in + # blocking mode. + # 3. Each task will acquire a file lock for 10s to ensure that the ray worker + # init will acquire a port connection to the ray head node that does not + # contend with other worker processes on the same Spark worker node. # 4. When the ray cluster is shutdown, killing ray worker nodes is implemented by: # Installing a PR_SET_PDEATHSIG signal for the `ray start ...` child processes # so that when parent process (pyspark task) is killed, the child processes # (`ray start ...` processes) will receive a SIGTERM signal, killing it. - # Shutting down the ray cluster is performed by calling `sparkContext.cancelJobGroup` - # to cancel the background spark job, sending a SIGKILL signal to all spark tasks. - # Once the spark tasks are killed, this triggers the sending of a SIGTERM to the child - # processes spawned by the `ray_start ...` process. + # Shutting down the ray cluster is performed by calling + # `sparkContext.cancelJobGroup` to cancel the background spark job, sending a + # SIGKILL signal to all spark tasks. Once the spark tasks are killed, this + # triggers the sending of a SIGTERM to the child processes spawned by the + # `ray_start ...` process. def ray_cluster_job_mapper(_): from pyspark.taskcontext import TaskContext @@ -528,9 +547,7 @@ def ray_cluster_job_mapper(_): ( worker_port_range_begin, worker_port_range_end, - ) = ( - _allocate_port_range_and_start_lock_barrier_thread_for_ray_worker_node_startup() - ) + ) = _prepare_for_ray_worker_node_startup() # Ray worker might run on a machine different with the head node, so create the # local log dir and temp dir again. @@ -556,7 +573,7 @@ def ray_cluster_job_mapper(_): ] ray_worker_node_extra_envs = { - RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or '' + RAY_ON_SPARK_COLLECT_LOG_TO_PATH: collect_log_to_path or "" } if num_spark_task_gpus > 0: @@ -564,7 +581,8 @@ def ray_cluster_job_mapper(_): if "gpu" not in task_resources: raise RuntimeError( - "Couldn't get the gpu id, Please check the GPU resource configuration" + "Couldn't get the gpu id, Please check the GPU resource " + "configuration" ) gpu_addr_list = [ int(addr.strip()) for addr in task_resources["gpu"].addresses @@ -585,14 +603,14 @@ def ray_cluster_job_mapper(_): ) # `preexec_fn=setup_sigterm_on_parent_death` handles the case: - # If a user cancels the PySpark job, the worker process gets killed, regardless of - # PySpark daemon and worker reuse settings. + # If a user cancels the PySpark job, the worker process gets killed, regardless + # of PySpark daemon and worker reuse settings. # We use prctl to ensure the command process receives SIGTERM after spark job # cancellation. # Note: - # When a pyspark job cancelled, the UDF python process are killed by signal "SIGKILL", - # This case neither "atexit" nor signal handler can capture SIGKILL signal. - # prctl is the only way to capture SIGKILL signal. + # When a pyspark job cancelled, the UDF python process are killed by signal + # "SIGKILL", This case neither "atexit" nor signal handler can capture SIGKILL + # signal. prctl is the only way to capture SIGKILL signal. exec_cmd( ray_worker_node_cmd, synchronous=True, @@ -618,42 +636,46 @@ def background_job_thread_fn(): try: spark.sparkContext.setJobGroup( spark_job_group_id, - "This job group is for spark job which runs the Ray cluster with ray head node " - f"{ray_head_ip}:{ray_head_port}", + "This job group is for spark job which runs the Ray cluster with ray " + f"head node {ray_head_ip}:{ray_head_port}", ) - # Starting a normal spark job (not barrier spark job) to run ray workers, the design - # purpose is: - # 1. Using normal spark job, spark tasks can automatically retry individually, we don't - # need to write additional retry logic, But, in barrier mode, if one spark task fails, - # it will cause all other spark tasks killed. - # 2. Using normal spark job, we can support failover when a spark worker physical - # machine crashes. (spark will try to re-schedule the spark task to other spark worker - # nodes) + # Starting a normal spark job (not barrier spark job) to run ray worker + # nodes, the design purpose is: + # 1. Using normal spark job, spark tasks can automatically retry + # individually, we don't need to write additional retry logic, But, in + # barrier mode, if one spark task fails, it will cause all other spark + # tasks killed. + # 2. Using normal spark job, we can support failover when a spark worker + # physical machine crashes. (spark will try to re-schedule the spark task + # to other spark worker nodes) # 3. Using barrier mode job, if the cluster resources does not satisfy - # "idle spark task slots >= argument num_spark_task", then the barrier job gets stuck - # and waits until enough idle task slots available, this behavior is not user-friendly, - # on a shared spark cluster, user is hard to estimate how many idle tasks available at - # a time, But, if using normal spark job, it can launch job with less spark tasks - # (i.e. user will see a ray cluster setup with less worker number initially), and when - # more task slots become available, it continues to launch tasks on new available - # slots, and user can see the ray cluster worker number increases when more slots - # available. + # "idle spark task slots >= argument num_spark_task", then the barrier + # job gets stuck and waits until enough idle task slots available, this + # behavior is not user-friendly, on a shared spark cluster, user is hard + # to estimate how many idle tasks available at a time, But, if using normal + # spark job, it can launch job with less spark tasks (i.e. user will see a + # ray cluster setup with less worker number initially), and when more task + # slots become available, it continues to launch tasks on new available + # slots, and user can see the ray cluster worker number increases when more + # slots available. spark.sparkContext.parallelize( list(range(num_worker_nodes)), num_worker_nodes ).mapPartitions(ray_cluster_job_mapper).collect() except Exception as e: # NB: - # The background spark job is designed to running forever until it is killed, - # The exception might be raised in following cases: + # The background spark job is designed to running forever until it is + # killed, The exception might be raised in following cases: # 1. The background job raises unexpected exception (i.e. ray cluster dies # unexpectedly) # 2. User explicitly orders shutting down the ray cluster. - # 3. On Databricks runtime, when a notebook is detached, it triggers python REPL - # `onCancel` event, cancelling the background running spark job - # For case 1 and 3, only ray workers are killed, but driver side ray head might still - # be running and the ray context might be in connected status. In order to disconnect - # and kill the ray head node, a call to `ray_cluster_handler.shutdown()` is performed. + # 3. On Databricks runtime, when a notebook is detached, it triggers + # python REPL `onCancel` event, cancelling the background running spark + # job. + # For case 1 and 3, only ray workers are killed, but driver side ray head + # might still be running and the ray context might be in connected status. + # In order to disconnect and kill the ray head node, a call to + # `ray_cluster_handler.shutdown()` is performed. if not ray_cluster_handler.spark_job_is_canceled: ray_cluster_handler.shutdown(cancel_background_job=False) ray_cluster_handler.background_job_exception = e @@ -674,9 +696,10 @@ def background_job_thread_fn(): start_hook.on_spark_background_job_created(spark_job_group_id) return ray_cluster_handler except Exception: - # If driver side setup ray-cluster routine raises exception, it might result in part of ray - # processes has been launched (e.g. ray head or some ray workers have been launched), - # calling `ray_cluster_handler.shutdown()` to kill them and clean status. + # If driver side setup ray-cluster routine raises exception, it might result + # in part of ray processes has been launched (e.g. ray head or some ray workers + # have been launched), calling `ray_cluster_handler.shutdown()` to kill them + # and clean status. ray_cluster_handler.shutdown() raise @@ -695,52 +718,57 @@ def init_ray_cluster( collect_log_to_path: Optional[str] = None, ) -> None: """ - Initialize a ray cluster on the spark cluster by starting a ray head node in the spark - application's driver side node. + Initialize a ray cluster on the spark cluster by starting a ray head node in the + spark application's driver side node. After creating the head node, a background spark job is created that - generates an instance of `RayClusterOnSpark` that contains configuration for the ray cluster - that will run on the Spark cluster's worker nodes. - After a ray cluster initialized, your python process automatically connect to the ray cluster, - you can call `ray.util.spark.shutdown_ray_cluster` to shut down the ray cluster. - Note: If the active ray cluster haven't shut down, you cannot create a new ray cluster. + generates an instance of `RayClusterOnSpark` that contains configuration for the + ray cluster that will run on the Spark cluster's worker nodes. + After a ray cluster initialized, your python process automatically connect to the + ray cluster, you can call `ray.util.spark.shutdown_ray_cluster` to shut down the + ray cluster. + Note: If the active ray cluster haven't shut down, you cannot create a new ray + cluster. Args - num_worker_nodes: The number of spark worker nodes that the spark job will be submitted to. - This argument represents how many concurrent spark tasks will be available in the - creation of the ray cluster. The ray cluster's total available resources - (memory, CPU and/or GPU) - is equal to the quantity of resources allocated within these spark tasks. - Specifying the `num_worker_nodes` as `-1` represents a ray cluster configuration that - will use all available spark tasks slots (and resources allocated to the spark - application) on the spark cluster. - To create a spark cluster that is intended to be used exclusively as a shared ray - cluster, it is recommended to set this argument to `ray.spark.utils.MAX_NUM_WORKER_NODES`. - object_store_memory_per_node: Object store memory available to per-ray worker node, but - it cannot exceed - "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". - The default value equals to - "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". + num_worker_nodes: The number of spark worker nodes that the spark job will be + submitted to. This argument represents how many concurrent spark tasks will + be available in the creation of the ray cluster. The ray cluster's total + available resources (memory, CPU and/or GPU) is equal to the quantity of + resources allocated within these spark tasks. + Specifying the `num_worker_nodes` as `-1` represents a ray cluster + configuration that will use all available spark tasks slots (and resources + allocated to the spark application) on the spark cluster. + To create a spark cluster that is intended to be used exclusively as a + shared ray cluster, it is recommended to set this argument to + `ray.spark.utils.MAX_NUM_WORKER_NODES`. + object_store_memory_per_node: Object store memory available to per-ray worker + node, but it is capped by + "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". + The default value equals to + "dev_shm_available_size * 0.8 / num_tasks_per_spark_worker". head_options: A dict representing Ray head node options. worker_options: A dict representing Ray worker node options. - ray_temp_root_dir: A local disk path to store the ray temporary data. The created cluster - will create a subdirectory "ray-{head_port}_{random_suffix}" beneath this path. - safe_mode: Boolean flag to fast-fail initialization of the ray cluster if the available - spark cluster does not have sufficient resources to fulfill the resource allocation - for memory, cpu and gpu. When set to true, if the requested resources are - not available for minimum recommended functionality, an exception will be raised that - details the inadequate spark cluster configuration settings. If overridden as `False`, + ray_temp_root_dir: A local disk path to store the ray temporary data. The + created cluster will create a subdirectory + "ray-{head_port}-{random_suffix}" beneath this path. + safe_mode: Boolean flag to fast-fail initialization of the ray cluster if + the available spark cluster does not have sufficient resources to fulfill + the resource allocation for memory, cpu and gpu. When set to true, if the + requested resources are not available for minimum recommended + functionality, an exception will be raised that details the inadequate + spark cluster configuration settings. If overridden as `False`, a warning is raised. - collect_log_to_path: If specified, after ray head / worker nodes terminated, collect their - logs to the specified path. On Databricks Runtime, we recommend you to - specify a local path starts with '/dbfs/', because the path mounts - with a centralized storage device and stored data is persisted after - databricks spark cluster terminated. + collect_log_to_path: If specified, after ray head / worker nodes terminated, + collect their logs to the specified path. On Databricks Runtime, we + recommend you to specify a local path starts with '/dbfs/', because the + path mounts with a centralized storage device and stored data is persisted + after databricks spark cluster terminated. """ global _active_ray_cluster if _active_ray_cluster is not None: raise RuntimeError( - "Current active ray cluster on spark haven't shut down. You cannot create a new ray " - "cluster." + "Current active ray cluster on spark haven't shut down. You cannot create " + "a new ray cluster." ) cluster = _init_ray_cluster( num_worker_nodes=num_worker_nodes, @@ -753,7 +781,8 @@ def init_ray_cluster( ) cluster.connect() # NB: this line might raise error. - # If connect cluster successfully, set global _active_ray_cluster to be the started cluster. + # If connect cluster successfully, set global _active_ray_cluster to be the started + # cluster. _active_ray_cluster = cluster diff --git a/python/ray/util/spark/databricks_hook.py b/python/ray/util/spark/databricks_hook.py index 4a2e4b10bfe4..b55c5b1b9db6 100644 --- a/python/ray/util/spark/databricks_hook.py +++ b/python/ray/util/spark/databricks_hook.py @@ -29,9 +29,9 @@ def get_dbutils(): def display_databricks_driver_proxy_url(spark_context, port, title): """ This helper function create a proxy URL for databricks driver webapp forwarding. - In databricks runtime, user does not have permission to directly access web service - binding on driver machine port, but user can visit it by a proxy URL with following - format: "/driver-proxy/o/{orgId}/{clusterId}/{port}/" + In databricks runtime, user does not have permission to directly access web + service binding on driver machine port, but user can visit it by a proxy URL with + following format: "/driver-proxy/o/{orgId}/{clusterId}/{port}/". """ from dbruntime.display import displayHTML @@ -68,7 +68,7 @@ def on_spark_background_job_created(self, job_group_id): get_dbutils().entry_point.registerBackgroundSparkJobGroup(job_group_id) except Exception: _logger.warning( - "Register ray cluster spark job as background job failed. You need to manually " - "call `ray_cluster_on_spark.shutdown()` before detaching your databricks " - "python REPL." + "Register ray cluster spark job as background job failed. You need to " + "manually call `ray_cluster_on_spark.shutdown()` before detaching " + "your databricks python REPL." ) diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 4ea193051a0d..e3c6dc211c71 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -11,15 +11,15 @@ from ray._private.ray_process_reaper import SIGTERM_GRACE_PERIOD_SECONDS -# Spark on ray implementation does not directly invoke `ray start ...` script to create ray node -# subprocess, instead, it creates a subprocess to run this `ray.util.spark.start_ray_node` -# module, and in this module it invokes `ray start ...` script to start ray node, -# the purpose of `start_ray_node` module is to set up a SIGTERM handler for cleaning ray temp -# directory when ray node exits. -# When spark driver python process dies, or spark python worker dies, because they registered -# the PR_SET_PDEATHSIG signal, so OS will send a SIGTERM signal to its children processes, so -# `start_ray_node` subprocess will receive a SIGTERM signal and the SIGTERM handler will do -# cleanup work. +# Spark on ray implementation does not directly invoke `ray start ...` script to create +# ray node subprocess, instead, it creates a subprocess to run this +# `ray.util.spark.start_ray_node` module, and in this module it invokes `ray start ...` +# script to start ray node, the purpose of `start_ray_node` module is to set up a +# SIGTERM handler for cleaning ray temp directory when ray node exits. +# When spark driver python process dies, or spark python worker dies, because they +# registered the PR_SET_PDEATHSIG signal, so OS will send a SIGTERM signal to its +# children processes, so `start_ray_node` subprocess will receive a SIGTERM signal and +# the SIGTERM handler will do cleanup work. _logger = logging.getLogger(__name__) @@ -35,7 +35,7 @@ for arg in arg_list: if arg.startswith(temp_dir_arg_prefix): - temp_dir = arg[len(temp_dir_arg_prefix):] + temp_dir = arg[len(temp_dir_arg_prefix) :] if temp_dir is None: raise ValueError("Please explicitly set --temp-dir option.") @@ -55,7 +55,8 @@ def try_clean_temp_dir_at_exit(): try: - # Wait for a while to ensure the children processes of the ray node all exited. + # Wait for a while to ensure the children processes of the ray node all + # exited. time.sleep(SIGTERM_GRACE_PERIOD_SECONDS + 0.5) if process.poll() is None: # "ray start ..." command process is still alive. Force to kill it. @@ -71,8 +72,9 @@ def try_clean_temp_dir_at_exit(): lock_acquired = True except BlockingIOError: # The file has active shared lock or exclusive lock, representing there - # are other ray nodes running, or other node running cleanup temp-dir routine. - # skip cleaning temp-dir, and skip copy logs to destination directory as well. + # are other ray nodes running, or other node running cleanup temp-dir + # routine. skip cleaning temp-dir, and skip copy logs to destination + # directory as well. lock_acquired = False if lock_acquired: @@ -93,7 +95,8 @@ def try_clean_temp_dir_at_exit(): ) except Exception as e: _logger.warning( - f"Collect logs to destination directory failed, error: {repr(e)}." + "Collect logs to destination directory failed, " + f"error: {repr(e)}." ) # Start cleaning the temp-dir, @@ -106,6 +109,7 @@ def try_clean_temp_dir_at_exit(): os.close(lock_fd) try: + def sigterm_handler(*args): process.terminate() try_clean_temp_dir_at_exit() diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index e74299c7ecee..94ec4d13e05e 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -22,8 +22,8 @@ def gen_cmd_exec_failure_msg(cmd, return_code, tail_output_deque): cmd_str = " ".join(cmd) tail_output = "".join(tail_output_deque) return ( - f"Command {cmd_str} failed with return code {return_code}, tail output are included " - f"below.\n{tail_output}\n" + f"Command {cmd_str} failed with return code {return_code}, tail output are " + f"included below.\n{tail_output}\n" ) @@ -35,12 +35,14 @@ def exec_cmd( **kwargs, ): """ - A convenience wrapper of `subprocess.Popen` for running a command from a Python script. - If `synchronous` is True, wait until the process terminated and if subprocess return code - is not 0, raise error containing last 100 lines output. - If `synchronous` is False, return an `Popen` instance and a deque instance holding tail - outputs. - The subprocess stdout / stderr output will be streamly redirected to current process stdout. + A convenience wrapper of `subprocess.Popen` for running a command from a Python + script. + If `synchronous` is True, wait until the process terminated and if subprocess + return code is not 0, raise error containing last 100 lines output. + If `synchronous` is False, return an `Popen` instance and a deque instance holding + tail outputs. + The subprocess stdout / stderr output will be streamly redirected to current + process stdout. """ illegal_kwargs = set(kwargs.keys()).intersection({"text", "stdout", "stderr"}) if illegal_kwargs: @@ -118,8 +120,9 @@ def get_spark_session(): spark_session = SparkSession.getActiveSession() if spark_session is None: raise RuntimeError( - "Spark session haven't been initiated yet. Please use `SparkSession.builder` to " - "create a spark session and connect to a spark cluster." + "Spark session haven't been initiated yet. Please use " + "`SparkSession.builder` to create a spark session and connect to a spark " + "cluster." ) return spark_session @@ -131,7 +134,8 @@ def get_spark_application_driver_host(spark): def get_max_num_concurrent_tasks(spark_context): """Gets the current max number of concurrent tasks.""" # pylint: disable=protected-access - # spark version 3.1 and above have a different API for fetching max concurrent tasks + # spark version 3.1 and above have a different API for fetching max concurrent + # tasks if spark_context._jsc.sc().version() >= "3.1": return spark_context._jsc.sc().maxNumConcurrentTasks( spark_context._jsc.sc().resourceProfileManager().resourceProfileFromId(0) @@ -213,8 +217,9 @@ def mapper(_): except Exception as e: return -1, -1, repr(e) - # Running memory inference routine on spark executor side since the spark worker nodes may - # have a different machine configuration compared to the spark driver node. + # Running memory inference routine on spark executor side since the spark worker + # nodes may have a different machine configuration compared to the spark driver + # node. ( inferred_ray_worker_node_heap_mem_bytes, inferred_ray_worker_node_object_store_bytes, From d2febe2d0404dfe3c9ba644edf1aa5551a3c1585 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 17 Dec 2022 19:48:36 +0800 Subject: [PATCH 113/121] update tests Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_basic.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index 96558656e02c..d11531694a74 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -12,7 +12,7 @@ import ray.util.spark.cluster_init from ray.util.spark import init_ray_cluster, shutdown_ray_cluster from ray.util.spark.cluster_init import _init_ray_cluster -from ray.util.spark.utils import check_port_open +from ray.util.spark.utils import check_port_open, exec_cmd from pyspark.sql import SparkSession import time import logging @@ -72,6 +72,9 @@ def f(x): results = ray.get(futures) assert results == [i * i for i in range(32)] + # print files generated in temp directory. + exec_cmd(["find", ray_temp_root_dir, "-print"], synchronous=True) + shutdown_ray_cluster() time.sleep(7) From ecc0139cd4ddc27ddc79aa66e6ee26aadf4c537c Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sat, 17 Dec 2022 20:19:59 +0800 Subject: [PATCH 114/121] update tests Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_basic.py | 7 ++----- python/ray/util/spark/start_ray_node.py | 5 ++++- python/ray/util/spark/utils.py | 1 - 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index d11531694a74..1b10a0a86d62 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -72,9 +72,6 @@ def f(x): results = ray.get(futures) assert results == [i * i for i in range(32)] - # print files generated in temp directory. - exec_cmd(["find", ray_temp_root_dir, "-print"], synchronous=True) - shutdown_ray_cluster() time.sleep(7) @@ -96,8 +93,8 @@ def f(x): # destroy it here. ray.util.spark._active_ray_cluster.shutdown() time.sleep(5) - shutil.rmtree(ray_temp_root_dir) - shutil.rmtree(collect_log_to_path) + shutil.rmtree(ray_temp_root_dir, ignore_errors=True) + shutil.rmtree(collect_log_to_path, ignore_errors=True) def test_ray_cluster_shutdown(self): with _init_ray_cluster( diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index e3c6dc211c71..505028daf418 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -88,8 +88,11 @@ def try_clean_temp_dir_at_exit(): socket.gethostname(), ) os.makedirs(copy_log_dest_path, exist_ok=True) + ray_session_dir = os.readlink( + os.path.join(temp_dir, "session_latest") + ) shutil.copytree( - os.path.join(temp_dir, "session_latest", "logs"), + os.path.join(ray_session_dir, "logs"), copy_log_dest_path, dirs_exist_ok=True, ) diff --git a/python/ray/util/spark/utils.py b/python/ray/util/spark/utils.py index 94ec4d13e05e..56413abfa50b 100644 --- a/python/ray/util/spark/utils.py +++ b/python/ray/util/spark/utils.py @@ -2,7 +2,6 @@ import os import sys import random -import math import threading import collections import logging From f2d0919264141c1ecc42bc75e3e036076a504260 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 18 Dec 2022 10:55:47 +0800 Subject: [PATCH 115/121] fix copy logs Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_basic.py | 2 +- python/ray/util/spark/start_ray_node.py | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index 1b10a0a86d62..9a8883e89f8e 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -12,7 +12,7 @@ import ray.util.spark.cluster_init from ray.util.spark import init_ray_cluster, shutdown_ray_cluster from ray.util.spark.cluster_init import _init_ray_cluster -from ray.util.spark.utils import check_port_open, exec_cmd +from ray.util.spark.utils import check_port_open from pyspark.sql import SparkSession import time import logging diff --git a/python/ray/util/spark/start_ray_node.py b/python/ray/util/spark/start_ray_node.py index 505028daf418..cedd2c786550 100644 --- a/python/ray/util/spark/start_ray_node.py +++ b/python/ray/util/spark/start_ray_node.py @@ -87,14 +87,12 @@ def try_clean_temp_dir_at_exit(): os.path.basename(temp_dir) + "-logs", socket.gethostname(), ) - os.makedirs(copy_log_dest_path, exist_ok=True) ray_session_dir = os.readlink( os.path.join(temp_dir, "session_latest") ) shutil.copytree( os.path.join(ray_session_dir, "logs"), copy_log_dest_path, - dirs_exist_ok=True, ) except Exception as e: _logger.warning( From 105721a2fe5a92086acba40f1f52b843c18a02f2 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 18 Dec 2022 14:34:23 +0800 Subject: [PATCH 116/121] skip tests on windows/macos Signed-off-by: Weichen Xu --- python/ray/tests/spark/test_GPU.py | 5 +++++ python/ray/tests/spark/test_basic.py | 5 +++++ python/ray/tests/spark/test_multicores_per_task.py | 5 +++++ python/ray/tests/spark/test_utils.py | 5 +++++ python/ray/util/spark/cluster_init.py | 2 +- 5 files changed, 21 insertions(+), 1 deletion(-) diff --git a/python/ray/tests/spark/test_GPU.py b/python/ray/tests/spark/test_GPU.py index 04470004923e..8de58bc2e2ec 100644 --- a/python/ray/tests/spark/test_GPU.py +++ b/python/ray/tests/spark/test_GPU.py @@ -10,6 +10,11 @@ import ray from ray.util.spark.cluster_init import _init_ray_cluster +pytestmark = pytest.mark.skipif( + not sys.platform.startswith("linux"), + reason="Ray on spark only supports running on Linux.", +) + class RayOnSparkGPUClusterTestBase(RayOnSparkCPUClusterTestBase, ABC): diff --git a/python/ray/tests/spark/test_basic.py b/python/ray/tests/spark/test_basic.py index 9a8883e89f8e..9b3d9c6a6e13 100644 --- a/python/ray/tests/spark/test_basic.py +++ b/python/ray/tests/spark/test_basic.py @@ -17,6 +17,11 @@ import time import logging +pytestmark = pytest.mark.skipif( + not sys.platform.startswith("linux"), + reason="Ray on spark only supports running on Linux.", +) + _logger = logging.getLogger(__name__) diff --git a/python/ray/tests/spark/test_multicores_per_task.py b/python/ray/tests/spark/test_multicores_per_task.py index 44f036ecab24..95bdee432c89 100644 --- a/python/ray/tests/spark/test_multicores_per_task.py +++ b/python/ray/tests/spark/test_multicores_per_task.py @@ -4,6 +4,11 @@ from pyspark.sql import SparkSession from ray.tests.spark.test_GPU import RayOnSparkGPUClusterTestBase +pytestmark = pytest.mark.skipif( + not sys.platform.startswith("linux"), + reason="Ray on spark only supports running on Linux.", +) + class TestMultiCoresPerTaskCluster(RayOnSparkGPUClusterTestBase): @classmethod diff --git a/python/ray/tests/spark/test_utils.py b/python/ray/tests/spark/test_utils.py index 6f102041d259..fa7fe1b93942 100644 --- a/python/ray/tests/spark/test_utils.py +++ b/python/ray/tests/spark/test_utils.py @@ -8,6 +8,11 @@ _calc_mem_per_ray_worker_node, ) +pytestmark = pytest.mark.skipif( + not sys.platform.startswith("linux"), + reason="Ray on spark only supports running on Linux.", +) + def test_get_spark_task_assigned_physical_gpus(): with patch.dict(os.environ, {}, clear=True): diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 92e40e6160e6..1535fe94b3a6 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -1,5 +1,4 @@ import os -import fcntl import socket import sys import time @@ -233,6 +232,7 @@ def _prepare_for_ray_worker_node_startup(): Returns: Allocated port range for current worker ports """ import psutil + import fcntl def acquire_lock(file_path): mode = os.O_RDWR | os.O_CREAT | os.O_TRUNC From 9fbf3de8f1303a3675f146b18cdc1cc3e5bdec5a Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Sun, 18 Dec 2022 15:49:07 +0800 Subject: [PATCH 117/121] minor updates Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 55 +++++++++++++++++++-------- 1 file changed, 39 insertions(+), 16 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index 1535fe94b3a6..aafccb0e7c48 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -8,6 +8,7 @@ from packaging.version import Version from typing import Optional, Dict +import ray from ray.util.annotations import PublicAPI from ray._private.storage import _load_class @@ -27,21 +28,10 @@ from .start_hook_base import RayOnSparkStartHook from .databricks_hook import DefaultDatabricksRayOnSparkStartHook -if not sys.platform.startswith("linux"): - raise RuntimeError("Ray on spark only supports running on Linux.") _logger = logging.getLogger("ray.util.spark") _logger.setLevel(logging.INFO) -_spark_dependency_error = "ray.util.spark module requires pyspark >= 3.3" -try: - import pyspark - - if Version(pyspark.__version__) < Version("3.3"): - raise RuntimeError(_spark_dependency_error) -except ImportError: - raise RuntimeError(_spark_dependency_error) - RAY_ON_SPARK_START_HOOK = "RAY_ON_SPARK_START_HOOK" @@ -50,6 +40,20 @@ RAY_ON_SPARK_COLLECT_LOG_TO_PATH = "RAY_ON_SPARK_COLLECT_LOG_TO_PATH" +def _check_system_environment(): + if not sys.platform.startswith("linux"): + raise RuntimeError("Ray on spark only supports running on Linux.") + + spark_dependency_error = "ray.util.spark module requires pyspark >= 3.3" + try: + import pyspark + + if Version(pyspark.__version__) < Version("3.3"): + raise RuntimeError(spark_dependency_error) + except ImportError: + raise RuntimeError(spark_dependency_error) + + class RayClusterOnSpark: """ This class is the type of instance returned by the `init_ray_cluster` API. @@ -101,8 +105,12 @@ def connect(self): "The ray cluster has been shut down or it failed to start." ) if self.ray_context is None: - # connect to the ray cluster. - self.ray_context = ray.init(address=self.address) + try: + # connect to the ray cluster. + self.ray_context = ray.init(address=self.address) + except Exception: + self.shutdown() + raise last_alive_worker_count = 0 last_progress_move_time = time.time() @@ -716,7 +724,7 @@ def init_ray_cluster( ray_temp_root_dir: Optional[str] = None, safe_mode: Optional[bool] = False, collect_log_to_path: Optional[str] = None, -) -> None: +) -> str: """ Initialize a ray cluster on the spark cluster by starting a ray head node in the spark application's driver side node. @@ -763,13 +771,27 @@ def init_ray_cluster( recommend you to specify a local path starts with '/dbfs/', because the path mounts with a centralized storage device and stored data is persisted after databricks spark cluster terminated. + + Returns: + The address of the initiated Ray cluster on spark. """ global _active_ray_cluster + + _check_system_environment() + if _active_ray_cluster is not None: raise RuntimeError( - "Current active ray cluster on spark haven't shut down. You cannot create " - "a new ray cluster." + "Current active ray cluster on spark haven't shut down. Please call " + "`ray.util.spark.shutdown_ray_cluster()` before initiating a new Ray " + "cluster on spark." ) + + if ray.is_initialized(): + raise RuntimeError( + "Current python process already initialized Ray, Please shut down it " + "by `ray.shutdown()` before initiating a Ray cluster on spark." + ) + cluster = _init_ray_cluster( num_worker_nodes=num_worker_nodes, object_store_memory_per_node=object_store_memory_per_node, @@ -784,6 +806,7 @@ def init_ray_cluster( # If connect cluster successfully, set global _active_ray_cluster to be the started # cluster. _active_ray_cluster = cluster + return cluster.address @PublicAPI(stability="alpha") From adbe42d9a88cc82363926afda795465d030fb927 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 19 Dec 2022 20:40:37 +0800 Subject: [PATCH 118/121] add user guide Signed-off-by: Weichen Xu --- .../vms/user-guides/community/index.rst | 1 + .../vms/user-guides/community/spark.rst | 92 +++++++++++++++++++ 2 files changed, 93 insertions(+) create mode 100644 doc/source/cluster/vms/user-guides/community/spark.rst diff --git a/doc/source/cluster/vms/user-guides/community/index.rst b/doc/source/cluster/vms/user-guides/community/index.rst index d3f354b39b59..5f72eedd6db9 100644 --- a/doc/source/cluster/vms/user-guides/community/index.rst +++ b/doc/source/cluster/vms/user-guides/community/index.rst @@ -15,6 +15,7 @@ The following is a list of community supported cluster managers. yarn.rst slurm.rst lsf.rst + spark.rst .. _ref-additional-cloud-providers: diff --git a/doc/source/cluster/vms/user-guides/community/spark.rst b/doc/source/cluster/vms/user-guides/community/spark.rst new file mode 100644 index 000000000000..1654b7a73b96 --- /dev/null +++ b/doc/source/cluster/vms/user-guides/community/spark.rst @@ -0,0 +1,92 @@ +.. _ray-Spark-deploy: + +Deploying on Spark Standalone cluster +===================================== + +This document describes a couple high-level steps to run Ray clusters on `Spark Standalone cluster `_. + +Running a basic example +======================= + +This is a spark application example code that starts Ray cluster on spark, +and then execute ray application code, then shut down initiated ray cluster. + +1) Create a python file that contains a spark application code, +Assuming the python file name is 'ray-on-spark-example1.py'. + +.. code-block:: python + + from pyspark.sql import SparkSession + from ray.util.spark import init_ray_cluster, shutdown_ray_cluster, MAX_NUM_WORKER_NODES + if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("Ray on spark example 1") \ + .config("spark.task.cpus", "4") \ + .getOrCreate() + + # initiate a ray cluster on this spark application, it creates a background + # spark job that each spark task launches one ray worker node. + # ray head node is launched in spark application driver side. + # Resources (CPU / GPU / memory) allocated to each ray worker node is equal + # to resources allocated to the corresponding spark task. + init_ray_cluster(num_worker_nodes=MAX_NUM_WORKER_NODES) + + # You can any ray application code here, the ray application will be executed + # on the ray cluster setup above. + # Note that you don't need to call `ray.init`. + ... + + # Terminate ray cluster explicitly. + # If you don't call it, when spark application is terminated, the ray cluster will + # also be terminated. + shutdown_ray_cluster() + +2) Submit the spark application above to spark standalone cluster. + +.. code-block:: bash + + #!/bin/bash + spark-submit \ + --master spark://{spark_master_IP}:{spark_master_port} \ + path/to/ray-on-spark-example1.py + +Creating a long running ray cluster on spark cluster +==================================================== + +This is a spark application example code that starts a long running Ray cluster on spark. +The created ray cluster can be accessed by remote python processes. + +1) Create a python file that contains a spark application code, +Assuming the python file name is 'long-running-ray-cluster-on-spark.py'. + +.. code-block:: python + + from pyspark.sql import SparkSession + import time + from ray.util.spark import init_ray_cluster, shutdown_ray_cluster, MAX_NUM_WORKER_NODES + + if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("Ray on spark example 1") \ + .config("spark.task.cpus", "4") \ + .getOrCreate() + + cluster_address = init_ray_cluster(num_worker_nodes=MAX_NUM_WORKER_NODES) + print("Ray cluster is initiated, you can connect to this ray cluster via address " + f"ray://{cluster_address}") + + # Sleep forever until the spark application being terminated, + # at that time, the ray cluster will also be terminated. + while True: + time.sleep(10) + +2) Submit the spark application above to spark standalone cluster. + +.. code-block:: bash + + #!/bin/bash + spark-submit \ + --master spark://{spark_master_IP}:{spark_master_port} \ + path/to/long-running-ray-cluster-on-spark.py From bce13d75083540c2fdeef3be3a1930889f41f664 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 19 Dec 2022 20:43:29 +0800 Subject: [PATCH 119/121] minor update Signed-off-by: Weichen Xu --- python/ray/util/spark/cluster_init.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/util/spark/cluster_init.py b/python/ray/util/spark/cluster_init.py index aafccb0e7c48..08fdb5c58243 100644 --- a/python/ray/util/spark/cluster_init.py +++ b/python/ray/util/spark/cluster_init.py @@ -458,12 +458,12 @@ def _init_ray_cluster( _logger.info(f"Ray head hostname {ray_head_ip}, port {ray_head_port}") - temp_dir_unique_suffix = uuid.uuid4().hex[:8] + cluster_unique_suffix = uuid.uuid4().hex[:8] if ray_temp_root_dir is None: ray_temp_root_dir = start_hook.get_default_temp_dir() ray_temp_dir = os.path.join( - ray_temp_root_dir, f"ray-{ray_head_port}-{temp_dir_unique_suffix}" + ray_temp_root_dir, f"ray-{ray_head_port}-{cluster_unique_suffix}" ) os.makedirs(ray_temp_dir, exist_ok=True) @@ -629,7 +629,7 @@ def ray_cluster_job_mapper(_): # NB: Not reachable. yield 0 - spark_job_group_id = f"ray-cluster-job-head-{ray_head_ip}-port-{ray_head_port}" + spark_job_group_id = f"ray-cluster-{ray_head_port}-{cluster_unique_suffix}" ray_cluster_handler = RayClusterOnSpark( address=f"{ray_head_ip}:{ray_head_port}", From 303cf1824705151da7a53645d1be562b5607c31a Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Mon, 19 Dec 2022 20:48:29 +0800 Subject: [PATCH 120/121] minor updates Signed-off-by: Weichen Xu --- doc/source/cluster/vms/user-guides/community/spark.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/cluster/vms/user-guides/community/spark.rst b/doc/source/cluster/vms/user-guides/community/spark.rst index 1654b7a73b96..8cfcdb4b4bf7 100644 --- a/doc/source/cluster/vms/user-guides/community/spark.rst +++ b/doc/source/cluster/vms/user-guides/community/spark.rst @@ -69,7 +69,7 @@ Assuming the python file name is 'long-running-ray-cluster-on-spark.py'. if __name__ == "__main__": spark = SparkSession \ .builder \ - .appName("Ray on spark example 1") \ + .appName("long running ray cluster on spark") \ .config("spark.task.cpus", "4") \ .getOrCreate() From 19e0d16551c859e410bca089d134e475d1c04de0 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Tue, 20 Dec 2022 10:43:18 +0800 Subject: [PATCH 121/121] update user-guide Signed-off-by: Weichen Xu --- doc/source/cluster/vms/user-guides/community/spark.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doc/source/cluster/vms/user-guides/community/spark.rst b/doc/source/cluster/vms/user-guides/community/spark.rst index 8cfcdb4b4bf7..9192afed1ace 100644 --- a/doc/source/cluster/vms/user-guides/community/spark.rst +++ b/doc/source/cluster/vms/user-guides/community/spark.rst @@ -6,7 +6,7 @@ Deploying on Spark Standalone cluster This document describes a couple high-level steps to run Ray clusters on `Spark Standalone cluster `_. Running a basic example -======================= +----------------------- This is a spark application example code that starts Ray cluster on spark, and then execute ray application code, then shut down initiated ray cluster. @@ -52,7 +52,7 @@ Assuming the python file name is 'ray-on-spark-example1.py'. path/to/ray-on-spark-example1.py Creating a long running ray cluster on spark cluster -==================================================== +---------------------------------------------------- This is a spark application example code that starts a long running Ray cluster on spark. The created ray cluster can be accessed by remote python processes. @@ -64,7 +64,7 @@ Assuming the python file name is 'long-running-ray-cluster-on-spark.py'. from pyspark.sql import SparkSession import time - from ray.util.spark import init_ray_cluster, shutdown_ray_cluster, MAX_NUM_WORKER_NODES + from ray.util.spark import init_ray_cluster, MAX_NUM_WORKER_NODES if __name__ == "__main__": spark = SparkSession \