Skip to content

Commit

Permalink
Make it possible to use Ray distributed debugger without setting RAY_…
Browse files Browse the repository at this point in the history
…DEBUG (ray-project#48301)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

Currently in order to use the distributed debugger, the user has to set
`RAY_DEBUG=1`. This has two disadvantages:

1. It is disruptive to the workflow and much more overhead than just
adding the `breakpoint()` instruction and re-running the program (since
the runtime environment has to be updated and the user needs to make
sure that the driver uses the flag too e.g. by restarting the python
kernel or in the worst case the container).
2. It is very easy to forget this step and then get the impression that
the debugger is not working.

There is no reason to require `RAY_DEBUG=1` to be set (the CLI debugger
works without the flag too and in particular the flag has no impact on
performance unless the debugger is actually entered). The reason this
flag was originally introduced was as a feature flag to switch between
the CLI debugger and the UI debugger. Now that the UI debugger is
getting more mature, it is better to make it the default and let people
who want to use the CLI debugger use a `RAY_DEBUG=legacy` flag.

This PR also renames the `RAY_PDB` flag to `RAY_DEBUG_POST_MORTEM` and
unifies the usage of the flag between the old and new debugger (in
particular, with the new debugger, post mortem debugging is now off
unless the user activates it).

## Related issue number

<!-- For example: "Closes ray-project#1234" -->

## Checks

- [ ] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [ ] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [ ] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [ ] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Philipp Moritz <pcmoritz@gmail.com>
Signed-off-by: mohitjain2504 <mohit.jain@dream11.com>
  • Loading branch information
pcmoritz authored and mohitjain2504 committed Nov 15, 2024
1 parent 85b4605 commit 2994d93
Show file tree
Hide file tree
Showing 13 changed files with 113 additions and 137 deletions.
Original file line number Diff line number Diff line change
@@ -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"},
}
)

Expand Down
4 changes: 3 additions & 1 deletion doc/source/ray-observability/ray-distributed-debugger.rst
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
.. _ray-distributed-debugger:

Ray Distributed Debugger
========================

Expand Down Expand Up @@ -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
Expand Down
4 changes: 3 additions & 1 deletion doc/source/ray-observability/user-guides/debug-apps/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ debug-memory
debug-hangs
debug-failures
optimize-performance
../../ray-distributed-debugger
ray-debugging
```

Expand All @@ -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)
103 changes: 27 additions & 76 deletions doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst
Original file line number Diff line number Diff line change
Expand Up @@ -14,22 +14,21 @@ 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::
:skipif: True

import ray

ray.init(runtime_env={"env_vars": {"RAY_DEBUG": "legacy"}})

@ray.remote
def f(x):
breakpoint()
Expand Down Expand Up @@ -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:
Expand Down Expand Up @@ -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!
Expand Down
2 changes: 1 addition & 1 deletion python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
4 changes: 2 additions & 2 deletions python/ray/data/_internal/planner/plan_udf_map_op.py
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions python/ray/data/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -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__)

Expand Down Expand Up @@ -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

Expand Down
2 changes: 1 addition & 1 deletion python/ray/data/tests/test_exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion python/ray/serve/_private/replica.py
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
6 changes: 3 additions & 3 deletions python/ray/tests/test_output.py
Original file line number Diff line number Diff line change
Expand Up @@ -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():
Expand All @@ -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)
Expand Down
Loading

0 comments on commit 2994d93

Please sign in to comment.