From aeba0717ab8e4cbd61d98b91b8359f6b75587649 Mon Sep 17 00:00:00 2001 From: bhuang Date: Wed, 10 Jan 2024 22:25:14 +0000 Subject: [PATCH 1/3] adding debupgy as the ray debugger --- .../install-core-prerelease-dependencies.sh | 2 +- python/ray/util/__init__.py | 2 + python/ray/util/debugpy.py | 110 ++++++++++++++++++ python/ray/util/rpdb.py | 11 +- python/requirements.txt | 1 + python/setup.py | 1 + 6 files changed, 125 insertions(+), 2 deletions(-) create mode 100644 python/ray/util/debugpy.py diff --git a/ci/env/install-core-prerelease-dependencies.sh b/ci/env/install-core-prerelease-dependencies.sh index 498ecd0245817..2ca91b0baf304 100755 --- a/ci/env/install-core-prerelease-dependencies.sh +++ b/ci/env/install-core-prerelease-dependencies.sh @@ -5,7 +5,7 @@ set -e # install all unbounded dependencies in setup.py for ray core # TOOD(scv119) reenable grpcio once https://github.com/grpc/grpc/issues/31885 is fixed. # TOOD(scv119) reenable jsonschema once https://github.com/ray-project/ray/issues/33411 is fixed. -for dependency in aiosignal frozenlist requests protobuf +for dependency in aiosignal frozenlist requests protobuf debugpy do python -m pip install -U --pre --upgrade-strategy=eager $dependency done diff --git a/python/ray/util/__init__.py b/python/ray/util/__init__.py index 2d409bc2dc0fb..257cfd3494d0f 100644 --- a/python/ray/util/__init__.py +++ b/python/ray/util/__init__.py @@ -6,6 +6,7 @@ from ray._private.services import get_node_ip_address from ray.util import iter from ray.util import rpdb as pdb +from ray.util import debugpy as ray_debugpy from ray.util.actor_pool import ActorPool from ray.util import accelerators from ray.util.annotations import PublicAPI @@ -62,6 +63,7 @@ def list_named_actors(all_namespaces: bool = False) -> List[str]: "get_current_placement_group", "get_node_ip_address", "remove_placement_group", + "ray_debugpy", "inspect_serializability", "collective", "connect", diff --git a/python/ray/util/debugpy.py b/python/ray/util/debugpy.py new file mode 100644 index 0000000000000..3ba5e6b2737fb --- /dev/null +++ b/python/ray/util/debugpy.py @@ -0,0 +1,110 @@ +import logging +import os +import sys +import threading + +import ray +from ray.util.annotations import DeveloperAPI + +log = logging.getLogger(__name__) + +POST_MORTEM_ERROR_UUID = "post_mortem_error_uuid" + +# A lock to ensure that only one thread can open the debugger port. +debugger_port_lock = threading.Lock() + + +def _override_breakpoint_hooks(): + """ + This method overrides the breakpoint() function to set_trace() + so that other threads can reuse the same setup logic. + This is based on: https://github.com/microsoft/debugpy/blob/ef9a67fe150179ee4df9997f9273723c26687fab/src/debugpy/_vendored/pydevd/pydev_sitecustomize/sitecustomize.py#L87 # noqa: E501 + """ + sys.__breakpointhook__ = set_trace + sys.breakpointhook = set_trace + import builtins as __builtin__ + + __builtin__.breakpoint = set_trace + + +def _ensure_debugger_port_open_thread_safe(): + """ + This is a thread safe method that ensure that the debugger port + is open, and if not, open it. + """ + + # The lock is acquired before checking the debugger port so only + # one thread can open the debugger port. + with debugger_port_lock: + import debugpy + + debugger_port = ray._private.worker.global_worker.debugger_port + if not debugger_port: + (host, port) = debugpy.listen( + (ray._private.worker.global_worker.node_ip_address, 0) + ) + ray._private.worker.global_worker.set_debugger_port(port) + log.info(f"Ray debugger is listening on {host}:{port}") + else: + log.info(f"Ray debugger is already open on {debugger_port}") + + +@DeveloperAPI +def set_trace(breakpoint_uuid=None): + """Interrupt the flow of the program and drop into the Ray debugger. + Can be used within a Ray task or actor. + """ + import debugpy + + _ensure_debugger_port_open_thread_safe() + + # debugpy overrides the breakpoint() function, so we need to set it back + # so other threads can reuse it. + _override_breakpoint_hooks() + + with ray._private.worker.global_worker.worker_paused_by_debugger(): + log.info("Waiting for debugger to attach...") + debugpy.wait_for_client() + + log.info("Debugger client is connected") + if breakpoint_uuid == POST_MORTEM_ERROR_UUID: + _debugpy_excepthook() + else: + _debugpy_breakpoint() + + +def _debugpy_breakpoint(): + """ + Drop the user into the debugger on a breakpoint. + """ + import pydevd + + pydevd.settrace(stop_at_frame=sys._getframe().f_back) + + +def _debugpy_excepthook(): + """ + Drop the user into the debugger on an unhandled exception. + """ + import threading + + import pydevd + + py_db = pydevd.get_global_debugger() + thread = threading.current_thread() + additional_info = py_db.set_additional_thread_info(thread) + additional_info.is_tracing += 1 + try: + error = sys.exc_info() + py_db.stop_on_unhandled_exception(py_db, thread, additional_info, error) + sys.excepthook(error[0], error[1], error[2]) + finally: + additional_info.is_tracing -= 1 + + +def _is_ray_debugger_enabled(): + return "RAY_DEBUG" in os.environ + + +def _post_mortem(): + return set_trace(POST_MORTEM_ERROR_UUID) diff --git a/python/ray/util/rpdb.py b/python/ray/util/rpdb.py index 63144706032a3..e6a976996690a 100644 --- a/python/ray/util/rpdb.py +++ b/python/ray/util/rpdb.py @@ -280,6 +280,9 @@ def set_trace(breakpoint_uuid=None): Can be used within a Ray task or actor. """ + if ray.util.ray_debugpy._is_ray_debugger_enabled(): + return ray.util.ray_debugpy.set_trace(breakpoint_uuid) + # If there is an active debugger already, we do not want to # start another one, so "set_trace" is just a no-op in that case. if ray._private.worker.global_worker.debugger_breakpoint == b"": @@ -301,6 +304,9 @@ def _driver_set_trace(): This disables Ray driver logs temporarily so that the PDB console is not spammed: https://github.com/ray-project/ray/issues/18172 """ + if ray.util.ray_debugpy._is_ray_debugger_enabled(): + return ray.util.ray_debugpy.set_trace() + print("*** Temporarily disabling Ray worker logs ***") ray._private.worker._worker_logs_enabled = False @@ -314,10 +320,13 @@ def enable_logging(): def _is_ray_debugger_enabled(): - return "RAY_PDB" in os.environ + return "RAY_PDB" in os.environ or ray.util.ray_debugpy._is_ray_debugger_enabled() def _post_mortem(): + if ray.util.ray_debugpy._is_ray_debugger_enabled(): + return ray.util.ray_debugpy._post_mortem() + rdb = _connect_ray_pdb( host=None, port=None, diff --git a/python/requirements.txt b/python/requirements.txt index a6b8c2eefc299..a43eb70d72071 100644 --- a/python/requirements.txt +++ b/python/requirements.txt @@ -60,3 +60,4 @@ pandas>=1.3 pydantic!=2.0.*,!=2.1.*,!=2.2.*,!=2.3.*,!=2.4.*,<3 # Serve users can use pydantic<2 py-spy>=0.2.0 watchfiles +debugpy diff --git a/python/setup.py b/python/setup.py index b4a67058b872f..4ebc3d700f201 100644 --- a/python/setup.py +++ b/python/setup.py @@ -342,6 +342,7 @@ def get_packages(self): "aiosignal", "frozenlist", "requests", + "debugpy", ] From cecd021ffbe2486dfd48deacf9a4625a1523fd13 Mon Sep 17 00:00:00 2001 From: bhuang Date: Fri, 12 Jan 2024 00:16:57 +0000 Subject: [PATCH 2/3] soft import debugpy --- .../install-core-prerelease-dependencies.sh | 2 +- python/ray/util/debugpy.py | 22 +++++++++++++++++-- python/requirements.txt | 1 - python/setup.py | 1 - 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/ci/env/install-core-prerelease-dependencies.sh b/ci/env/install-core-prerelease-dependencies.sh index 2ca91b0baf304..498ecd0245817 100755 --- a/ci/env/install-core-prerelease-dependencies.sh +++ b/ci/env/install-core-prerelease-dependencies.sh @@ -5,7 +5,7 @@ set -e # install all unbounded dependencies in setup.py for ray core # TOOD(scv119) reenable grpcio once https://github.com/grpc/grpc/issues/31885 is fixed. # TOOD(scv119) reenable jsonschema once https://github.com/ray-project/ray/issues/33411 is fixed. -for dependency in aiosignal frozenlist requests protobuf debugpy +for dependency in aiosignal frozenlist requests protobuf do python -m pip install -U --pre --upgrade-strategy=eager $dependency done diff --git a/python/ray/util/debugpy.py b/python/ray/util/debugpy.py index 3ba5e6b2737fb..d01b62cdb815c 100644 --- a/python/ray/util/debugpy.py +++ b/python/ray/util/debugpy.py @@ -2,6 +2,7 @@ import os import sys import threading +import importlib import ray from ray.util.annotations import DeveloperAPI @@ -10,6 +11,19 @@ POST_MORTEM_ERROR_UUID = "post_mortem_error_uuid" + +def _try_import(module): + try: + return importlib.import_module(module) + except ModuleNotFoundError: + log.error( + f"Module '{module}' cannot be loaded. " + f"Ray Debugger will not work without '{module}'. " + f"Install this module using 'pip install {module}' " + ) + return None + + # A lock to ensure that only one thread can open the debugger port. debugger_port_lock = threading.Lock() @@ -36,7 +50,9 @@ def _ensure_debugger_port_open_thread_safe(): # The lock is acquired before checking the debugger port so only # one thread can open the debugger port. with debugger_port_lock: - import debugpy + debugpy = _try_import("debugpy") + if not debugpy: + return debugger_port = ray._private.worker.global_worker.debugger_port if not debugger_port: @@ -54,7 +70,9 @@ def set_trace(breakpoint_uuid=None): """Interrupt the flow of the program and drop into the Ray debugger. Can be used within a Ray task or actor. """ - import debugpy + debugpy = _try_import("debugpy") + if not debugpy: + return _ensure_debugger_port_open_thread_safe() diff --git a/python/requirements.txt b/python/requirements.txt index a43eb70d72071..a6b8c2eefc299 100644 --- a/python/requirements.txt +++ b/python/requirements.txt @@ -60,4 +60,3 @@ pandas>=1.3 pydantic!=2.0.*,!=2.1.*,!=2.2.*,!=2.3.*,!=2.4.*,<3 # Serve users can use pydantic<2 py-spy>=0.2.0 watchfiles -debugpy diff --git a/python/setup.py b/python/setup.py index 4ebc3d700f201..b4a67058b872f 100644 --- a/python/setup.py +++ b/python/setup.py @@ -342,7 +342,6 @@ def get_packages(self): "aiosignal", "frozenlist", "requests", - "debugpy", ] From 44aed5b841e34d87850b6584c4d7269f11e974d1 Mon Sep 17 00:00:00 2001 From: bhuang Date: Fri, 12 Jan 2024 01:45:13 +0000 Subject: [PATCH 3/3] be specific on debugpy version --- python/ray/util/debugpy.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/python/ray/util/debugpy.py b/python/ray/util/debugpy.py index d01b62cdb815c..ff1291b3210ea 100644 --- a/python/ray/util/debugpy.py +++ b/python/ray/util/debugpy.py @@ -12,14 +12,17 @@ POST_MORTEM_ERROR_UUID = "post_mortem_error_uuid" -def _try_import(module): +def _try_import_debugpy(): try: - return importlib.import_module(module) - except ModuleNotFoundError: + debugpy = importlib.import_module("debugpy") + if not hasattr(debugpy, "__version__") or debugpy.__version__ < "1.8.0": + raise ImportError() + return debugpy + except (ModuleNotFoundError, ImportError): log.error( - f"Module '{module}' cannot be loaded. " - f"Ray Debugger will not work without '{module}'. " - f"Install this module using 'pip install {module}' " + "Module 'debugpy>=1.8.0' cannot be loaded. " + "Ray Debugpy Debugger will not work without 'debugpy>=1.8.0' installed. " + "Install this module using 'pip install debugpy==1.8.0' " ) return None @@ -50,7 +53,7 @@ def _ensure_debugger_port_open_thread_safe(): # The lock is acquired before checking the debugger port so only # one thread can open the debugger port. with debugger_port_lock: - debugpy = _try_import("debugpy") + debugpy = _try_import_debugpy() if not debugpy: return @@ -70,7 +73,7 @@ def set_trace(breakpoint_uuid=None): """Interrupt the flow of the program and drop into the Ray debugger. Can be used within a Ray task or actor. """ - debugpy = _try_import("debugpy") + debugpy = _try_import_debugpy() if not debugpy: return