diff --git a/doc/source/ray-observability/doc_code/ray-distributed-debugger.py b/doc/source/ray-observability/doc_code/ray-distributed-debugger.py index 5f8eff38211c..28c365865910 100644 --- a/doc/source/ray-observability/doc_code/ray-distributed-debugger.py +++ b/doc/source/ray-observability/doc_code/ray-distributed-debugger.py @@ -1,10 +1,11 @@ import ray import sys -# Add RAY_DEBUG environment variable to enable Ray Debugger. +# Add the RAY_DEBUG_POST_MORTEM=1 environment variable +# if you want to activate post-mortem debugging ray.init( runtime_env={ - "env_vars": {"RAY_DEBUG": "1"}, + "env_vars": {"RAY_DEBUG_POST_MORTEM": "1"}, } ) diff --git a/doc/source/ray-observability/ray-distributed-debugger.rst b/doc/source/ray-observability/ray-distributed-debugger.rst index 82f6b40556a2..3a3682eb474c 100644 --- a/doc/source/ray-observability/ray-distributed-debugger.rst +++ b/doc/source/ray-observability/ray-distributed-debugger.rst @@ -1,3 +1,5 @@ +.. _ray-distributed-debugger: + Ray Distributed Debugger ======================== @@ -48,7 +50,7 @@ Find and click the Ray extension in the VS Code left side nav. Add the Ray clust Create a Ray task ~~~~~~~~~~~~~~~~~ -Create a file `job.py` with the following snippet. Add the `RAY_DEBUG` environment variable to enable Ray Debugger and add `breakpoint()` in the Ray task. +Create a file `job.py` with the following snippet. Add `breakpoint()` in the Ray task. If you want to use the post-mortem debugging below, also add the `RAY_DEBUG_POST_MORTEM=1` environment variable. .. literalinclude:: ./doc_code/ray-distributed-debugger.py :language: python diff --git a/doc/source/ray-observability/user-guides/debug-apps/index.md b/doc/source/ray-observability/user-guides/debug-apps/index.md index 8599421cda07..6c574de14b68 100644 --- a/doc/source/ray-observability/user-guides/debug-apps/index.md +++ b/doc/source/ray-observability/user-guides/debug-apps/index.md @@ -10,6 +10,7 @@ debug-memory debug-hangs debug-failures optimize-performance +../../ray-distributed-debugger ray-debugging ``` @@ -19,4 +20,5 @@ These guides help you perform common debugging or optimization tasks for your di * {ref}`observability-debug-hangs` * {ref}`observability-debug-failures` * {ref}`observability-optimize-performance` -* {ref}`ray-debugger` +* {ref}`ray-distributed-debugger` +* {ref}`ray-debugger` (deprecated) diff --git a/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst b/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst index ec48e1907f23..0b4513514c79 100644 --- a/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst +++ b/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst @@ -14,15 +14,12 @@ drop into a PDB session that you can then use to: .. warning:: The Ray Debugger is deprecated. Use the :doc:`Ray Distributed Debugger <../../ray-distributed-debugger>` instead. + Starting with Ray 2.39, the new debugger is the default and you need to set the environment variable `RAY_DEBUG=legacy` to + use the old debugger (e.g. by using a runtime environment). Getting Started --------------- -.. note:: - - On Python 3.6, the ``breakpoint()`` function is not supported and you need to use - ``ray.util.pdb.set_trace()`` instead. - Take the following example: .. testcode:: @@ -30,6 +27,8 @@ Take the following example: import ray + ray.init(runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) + @ray.remote def f(x): breakpoint() @@ -118,6 +117,8 @@ following recursive function as an example: import ray + ray.init(runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) + @ray.remote def fact(n): if n == 1: @@ -222,104 +223,54 @@ Post Mortem Debugging Often we do not know in advance where an error happens, so we cannot set a breakpoint. In these cases, we can automatically drop into the debugger when an error occurs or an exception is thrown. This is called *post-mortem debugging*. -We will show how this works using a Ray serve application. To get started, install the required dependencies: - -.. code-block:: bash - - pip install "ray[serve]" scikit-learn - -Next, copy the following code into a file called ``serve_debugging.py``: +Copy the following code into a file called ``post_mortem_debugging.py``. The flag ``RAY_DEBUG_POST_MORTEM=1`` will have the effect +that if an exception happens, Ray will drop into the debugger instead of propagating it further. .. testcode:: :skipif: True - import time - - from sklearn.datasets import load_iris - from sklearn.ensemble import GradientBoostingClassifier - import ray - from ray import serve - - serve.start() - # Train model - iris_dataset = load_iris() - model = GradientBoostingClassifier() - model.fit(iris_dataset["data"], iris_dataset["target"]) + ray.init(runtime_env={"env_vars": {"RAY_DEBUG": "legacy", "RAY_DEBUG_POST_MORTEM": "1"}}) - # Define Ray Serve model, - @serve.deployment - class BoostingModel: - def __init__(self): - self.model = model - self.label_list = iris_dataset["target_names"].tolist() - - async def __call__(self, starlette_request): - payload = (await starlette_request.json())["vector"] - print(f"Worker: received request with data: {payload}") - - prediction = self.model.predict([payload])[0] - human_name = self.label_list[prediction] - return {"result": human_name} - - # Deploy model - serve.run(BoostingModel.bind(), route_prefix="/iris") - - time.sleep(3600.0) - -Let's start the program with the post-mortem debugging activated (``RAY_PDB=1``): - -.. code-block:: bash + @ray.remote + def post_mortem(x): + x += 1 + raise Exception("An exception is raised.") + return x - RAY_PDB=1 python serve_debugging.py + ray.get(post_mortem.remote(10)) -The flag ``RAY_PDB=1`` will have the effect that if an exception happens, Ray will -drop into the debugger instead of propagating it further. Let's see how this works! -First query the model with an invalid request using +Let's start the program: .. code-block:: bash - python -c 'import requests; response = requests.get("http://localhost:8000/iris", json={"vector": [1.2, 1.0, 1.1, "a"]})' + python post_mortem_debugging.py -When the ``serve_debugging.py`` driver hits the breakpoint, it will tell you to run -``ray debug``. After we do that, we see an output like the following: +Now run ``ray debug``. After we do that, we see an output like the following: .. code-block:: text Active breakpoints: - index | timestamp | Ray task | filename:lineno - 0 | 2021-07-13 23:49:14 | ray::RayServeWrappedReplica.handle_request() | /home/ubuntu/ray/python/ray/serve/backend_worker.py:249 + index | timestamp | Ray task | filename:lineno + 0 | 2024-11-01 20:14:00 | /Users/pcmoritz/ray/python/ray/_private/workers/default_worker.py --node-ip-address=127.0.0.1 --node-manager-port=49606 --object-store-name=/tmp/ray/session_2024-11-01_13-13-51_279910_8596/sockets/plasma_store --raylet-name=/tmp/ray/session_2024-11-01_13-13-51_279910_8596/sockets/raylet --redis-address=None --metrics-agent-port=58655 --runtime-env-agent-port=56999 --logging-rotate-bytes=536870912 --logging-rotate-backup-count=5 --runtime-env-agent-port=56999 --gcs-address=127.0.0.1:6379 --session-name=session_2024-11-01_13-13-51_279910_8596 --temp-dir=/tmp/ray --webui=127.0.0.1:8265 --cluster-id=6d341469ae0f85b6c3819168dde27cceda12e95c8efdfc256e0fd8ce --startup-token=12 --worker-launch-time-ms=1730492039955 --node-id=0d43573a606286125da39767a52ce45ad101324c8af02cc25a9fbac7 --runtime-env-hash=-1746935720 | /Users/pcmoritz/ray/python/ray/_private/worker.py:920 Traceback (most recent call last): - File "/home/ubuntu/ray/python/ray/serve/backend_worker.py", line 242, in invoke_single - result = await method_to_call(*args, **kwargs) - - File "serve_debugging.py", line 24, in __call__ - prediction = self.model.predict([payload])[0] - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1188, in predict - raw_predictions = self.decision_function(X) - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1143, in decision_function - X = check_array(X, dtype=DTYPE, order="C", accept_sparse='csr') + File "python/ray/_raylet.pyx", line 1856, in ray._raylet.execute_task - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 63, in inner_f - return f(*args, **kwargs) + File "python/ray/_raylet.pyx", line 1957, in ray._raylet.execute_task - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 673, in check_array - array = np.asarray(array, order=order, dtype=dtype) + File "python/ray/_raylet.pyx", line 1862, in ray._raylet.execute_task - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/numpy/core/_asarray.py", line 83, in asarray - return array(a, dtype, copy=False, order=order) + File "/Users/pcmoritz/ray-debugger-test/post_mortem_debugging.py", line 8, in post_mortem + raise Exception("An exception is raised.") - ValueError: could not convert string to float: 'a' + Exception: An exception is raised. Enter breakpoint index or press enter to refresh: We now press ``0`` and then Enter to enter the debugger. With ``ll`` we can see the context and with -``print(a)`` we an print the array that causes the problem. As we see, it contains a string (``'a'``) -instead of a number as the last element. +``print(x)`` we an print the value of ``x``. In a similar manner as above, you can also debug Ray actors. Happy debugging! diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 5bb3e4603b77..fa49a88c691f 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -1026,7 +1026,7 @@ cdef store_task_errors( CoreWorker core_worker = worker.core_worker # If the debugger is enabled, drop into the remote pdb here. - if ray.util.pdb._is_ray_debugger_enabled(): + if ray.util.pdb._is_ray_debugger_post_mortem_enabled(): ray.util.pdb._post_mortem() backtrace = ray._private.utils.format_error_message( diff --git a/python/ray/data/_internal/planner/plan_udf_map_op.py b/python/ray/data/_internal/planner/plan_udf_map_op.py index c7b05a680d47..a5b1ccd46a75 100644 --- a/python/ray/data/_internal/planner/plan_udf_map_op.py +++ b/python/ray/data/_internal/planner/plan_udf_map_op.py @@ -45,7 +45,7 @@ ) from ray.data.context import DataContext from ray.data.exceptions import UserCodeException -from ray.util.rpdb import _is_ray_debugger_enabled +from ray.util.rpdb import _is_ray_debugger_post_mortem_enabled class _MapActorContext: @@ -205,7 +205,7 @@ def _handle_debugger_exception(e: Exception): so that the debugger can stop at the initial unhandled exception. Otherwise, clear the stack trace to omit noisy internal code path.""" ctx = ray.data.DataContext.get_current() - if _is_ray_debugger_enabled() or ctx.raise_original_map_exception: + if _is_ray_debugger_post_mortem_enabled() or ctx.raise_original_map_exception: raise e else: raise UserCodeException() from e diff --git a/python/ray/data/exceptions.py b/python/ray/data/exceptions.py index f8ce88a984df..894d0e1504fc 100644 --- a/python/ray/data/exceptions.py +++ b/python/ray/data/exceptions.py @@ -6,7 +6,7 @@ from ray.exceptions import UserCodeException from ray.util import log_once from ray.util.annotations import DeveloperAPI -from ray.util.rpdb import _is_ray_debugger_enabled +from ray.util.rpdb import _is_ray_debugger_post_mortem_enabled logger = logging.getLogger(__name__) @@ -52,7 +52,7 @@ def handle_trace(*args, **kwargs): # via DataContext, or when the Ray Debugger is enabled. # The full stack trace will always be emitted to the Ray Data log file. log_to_stdout = DataContext.get_current().log_internal_stack_trace_to_stdout - if _is_ray_debugger_enabled(): + if _is_ray_debugger_post_mortem_enabled(): logger.exception("Full stack trace:") raise e diff --git a/python/ray/data/tests/test_exceptions.py b/python/ray/data/tests/test_exceptions.py index 917eb7433704..2b0de96dad04 100644 --- a/python/ray/data/tests/test_exceptions.py +++ b/python/ray/data/tests/test_exceptions.py @@ -77,7 +77,7 @@ class FakeException(Exception): def test_full_traceback_logged_with_ray_debugger( caplog, propagate_logs, ray_start_regular_shared, monkeypatch ): - monkeypatch.setenv("RAY_PDB", 1) + monkeypatch.setenv("RAY_DEBUG_POST_MORTEM", 1) def f(row): 1 / 0 diff --git a/python/ray/serve/_private/replica.py b/python/ray/serve/_private/replica.py index d2fb2446dbd8..c390c05bc055 100644 --- a/python/ray/serve/_private/replica.py +++ b/python/ray/serve/_private/replica.py @@ -415,7 +415,7 @@ def _wrap_user_method_call( except Exception as e: user_exception = e logger.exception("Request failed.") - if ray.util.pdb._is_ray_debugger_enabled(): + if ray.util.pdb._is_ray_debugger_post_mortem_enabled(): ray.util.pdb._post_mortem() finally: self._metrics_manager.dec_num_ongoing_requests() diff --git a/python/ray/tests/test_output.py b/python/ray/tests/test_output.py index 6d8c3a3be421..54a5e9d33fa8 100644 --- a/python/ray/tests/test_output.py +++ b/python/ray/tests/test_output.py @@ -569,7 +569,8 @@ def test_disable_driver_logs_breakpoint(): import sys import threading -ray.init(num_cpus=2) +os.environ["RAY_DEBUG"] = "legacy" +ray.init(num_cpus=2, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def f(): @@ -588,8 +589,7 @@ def kill(): t.start() x = f.remote() time.sleep(2) # Enough time to print one hello. -ray.util.rpdb._driver_set_trace() # This should disable worker logs. -# breakpoint() # Only works in Py3.7+ +breakpoint() # This should disable worker logs. """ proc = run_string_as_driver_nonblocking(script) diff --git a/python/ray/tests/test_ray_debugger.py b/python/ray/tests/test_ray_debugger.py index f3aec80e8fe5..ea3639f10c6f 100644 --- a/python/ray/tests/test_ray_debugger.py +++ b/python/ray/tests/test_ray_debugger.py @@ -16,7 +16,7 @@ def test_ray_debugger_breakpoint(shutdown_only): - ray.init(num_cpus=1) + ray.init(num_cpus=1, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def f(): @@ -56,7 +56,7 @@ def f(): @pytest.mark.skipif(platform.system() == "Windows", reason="Failing on Windows.") def test_ray_debugger_commands(shutdown_only): - ray.init(num_cpus=2) + ray.init(num_cpus=2, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def f(): @@ -96,7 +96,8 @@ def f(): @pytest.mark.skipif(platform.system() == "Windows", reason="Failing on Windows.") def test_ray_debugger_stepping(shutdown_only): - ray.init(num_cpus=1) + os.environ["RAY_DEBUG"] = "legacy" + ray.init(num_cpus=1, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def g(): @@ -110,6 +111,15 @@ def f(): result = f.remote() + wait_for_condition( + lambda: len( + ray.experimental.internal_kv._internal_kv_list( + "RAY_PDB_", namespace=ray_constants.KV_NAMESPACE_PDB + ) + ) + > 0 + ) + p = pexpect.spawn("ray debug") p.expect("Enter breakpoint index or press enter to refresh: ") p.sendline("0") @@ -126,7 +136,8 @@ def f(): @pytest.mark.skipif(platform.system() == "Windows", reason="Failing on Windows.") def test_ray_debugger_recursive(shutdown_only): - ray.init(num_cpus=1) + os.environ["RAY_DEBUG"] = "legacy" + ray.init(num_cpus=1, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def fact(n): @@ -165,7 +176,7 @@ def test_job_exit_cleanup(ray_start_regular): import time import ray -ray.init(address="{}") +ray.init(address="{}", runtime_env={{"env_vars": {{"RAY_DEBUG": "legacy"}}}}) @ray.remote def f(): @@ -225,7 +236,7 @@ def test_ray_debugger_public(shutdown_only, call_ray_stop_only, ray_debugger_ext address = out[address_location:] address = address.split("'")[0] - ray.init(address=address) + ray.init(address=address, runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) @ray.remote def f(): @@ -272,7 +283,7 @@ def f(): def test_ray_debugger_public_multi_node(shutdown_only, ray_debugger_external): c = Cluster( initialize_head=True, - connect=True, + connect=False, head_node_args={ "num_cpus": 0, "num_gpus": 1, @@ -281,6 +292,8 @@ def test_ray_debugger_public_multi_node(shutdown_only, ray_debugger_external): ) c.add_node(num_cpus=1, ray_debugger_external=ray_debugger_external) + ray.init(runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}}) + @ray.remote def f(): ray.util.pdb.set_trace() @@ -341,18 +354,20 @@ def f(): def test_env_var_enables_ray_debugger(): with unittest.mock.patch.dict(os.environ): - os.environ["RAY_PDB"] = "1" - assert ( - ray.util.pdb._is_ray_debugger_enabled() - ), "Expected Ray Debugger to be enabled when RAY_PDB env var is present." + os.environ["RAY_DEBUG_POST_MORTEM"] = "1" + assert ray.util.pdb._is_ray_debugger_post_mortem_enabled(), ( + "Expected post-mortem Debugger to be enabled when " + "RAY_DEBUG_POST_MORTEM env var is present." + ) with unittest.mock.patch.dict(os.environ): - if "RAY_PDB" in os.environ: - del os.environ["RAY_PDB"] + if "RAY_DEBUG_POST_MORTEM" in os.environ: + del os.environ["RAY_DEBUG_POST_MORTEM"] - assert ( - not ray.util.pdb._is_ray_debugger_enabled() - ), "Expected Ray Debugger to be disabled when RAY_PDB env var is absent." + assert not ray.util.pdb._is_ray_debugger_post_mortem_enabled(), ( + "Expected post-mortem Debugger to be disabled when " + "RAY_DEBUG_POST_MORTEM env var is absent." + ) if __name__ == "__main__": diff --git a/python/ray/util/debugpy.py b/python/ray/util/debugpy.py index ff1291b3210e..32b265d1d451 100644 --- a/python/ray/util/debugpy.py +++ b/python/ray/util/debugpy.py @@ -84,7 +84,12 @@ def set_trace(breakpoint_uuid=None): _override_breakpoint_hooks() with ray._private.worker.global_worker.worker_paused_by_debugger(): - log.info("Waiting for debugger to attach...") + msg = ( + "Waiting for debugger to attach (see " + "https://docs.ray.io/en/latest/ray-observability/" + "ray-distributed-debugger.html)..." + ) + log.info(msg) debugpy.wait_for_client() log.info("Debugger client is connected") @@ -123,8 +128,8 @@ def _debugpy_excepthook(): additional_info.is_tracing -= 1 -def _is_ray_debugger_enabled(): - return "RAY_DEBUG" in os.environ +def _is_ray_debugger_post_mortem_enabled(): + return os.environ.get("RAY_DEBUG_POST_MORTEM", "0") == "1" def _post_mortem(): diff --git a/python/ray/util/rpdb.py b/python/ray/util/rpdb.py index be314a2484e8..dbabaf82b9b5 100644 --- a/python/ray/util/rpdb.py +++ b/python/ray/util/rpdb.py @@ -284,22 +284,22 @@ def set_trace(breakpoint_uuid=None): Can be used within a Ray task or actor. """ - if ray.util.ray_debugpy._is_ray_debugger_enabled(): + if os.environ.get("RAY_DEBUG", "1") == "1": 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"": - frame = sys._getframe().f_back - rdb = _connect_ray_pdb( - host=None, - port=None, - patch_stdstreams=False, - quiet=None, - breakpoint_uuid=breakpoint_uuid.decode() if breakpoint_uuid else None, - debugger_external=ray._private.worker.global_worker.ray_debugger_external, - ) - rdb.set_trace(frame=frame) + if os.environ.get("RAY_DEBUG", "1") == "legacy": + # 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"": + frame = sys._getframe().f_back + rdb = _connect_ray_pdb( + host=None, + port=None, + patch_stdstreams=False, + quiet=None, + breakpoint_uuid=breakpoint_uuid.decode() if breakpoint_uuid else None, + debugger_external=ray._private.worker.global_worker.ray_debugger_external, # noqa: E501 + ) + rdb.set_trace(frame=frame) def _driver_set_trace(): @@ -308,27 +308,27 @@ 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(): + if os.environ.get("RAY_DEBUG", "1") == "1": return ray.util.ray_debugpy.set_trace() + if os.environ.get("RAY_DEBUG", "1") == "legacy": + print("*** Temporarily disabling Ray worker logs ***") + ray._private.worker._worker_logs_enabled = False - print("*** Temporarily disabling Ray worker logs ***") - ray._private.worker._worker_logs_enabled = False - - def enable_logging(): - print("*** Re-enabling Ray worker logs ***") - ray._private.worker._worker_logs_enabled = True + def enable_logging(): + print("*** Re-enabling Ray worker logs ***") + ray._private.worker._worker_logs_enabled = True - pdb = _PdbWrap(enable_logging) - frame = sys._getframe().f_back - pdb.set_trace(frame) + pdb = _PdbWrap(enable_logging) + frame = sys._getframe().f_back + pdb.set_trace(frame) -def _is_ray_debugger_enabled(): - return "RAY_PDB" in os.environ or ray.util.ray_debugpy._is_ray_debugger_enabled() +def _is_ray_debugger_post_mortem_enabled(): + return os.environ.get("RAY_DEBUG_POST_MORTEM", "0") == "1" def _post_mortem(): - if ray.util.ray_debugpy._is_ray_debugger_enabled(): + if os.environ.get("RAY_DEBUG", "1") == "1": return ray.util.ray_debugpy._post_mortem() rdb = _connect_ray_pdb(