Skip to content

Commit 427100d

Browse files
sampan-s-nayaksampan
andauthored
[core] Improve docs for custom serialization for exceptions + add test (#56156)
Signed-off-by: sampan <sampan@anyscale.com> Signed-off-by: Sampan S Nayak <sampansnayak2@gmail.com> Co-authored-by: sampan <sampan@anyscale.com>
1 parent 119aa6c commit 427100d

File tree

3 files changed

+97
-3
lines changed

3 files changed

+97
-3
lines changed

doc/source/ray-core/objects/serialization.rst

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -202,6 +202,63 @@ There are at least 3 ways to define your custom serialization process:
202202
except TypeError:
203203
pass
204204

205+
.. _custom-exception-serializer:
206+
207+
Custom Serializers for Exceptions
208+
----------------------------------
209+
210+
When Ray tasks raise exceptions that cannot be serialized with the default pickle mechanism, you can register custom serializers to handle them (Note: the serializer must be registered in the driver and all workers).
211+
212+
.. testcode::
213+
214+
import ray
215+
import threading
216+
217+
class CustomError(Exception):
218+
def __init__(self, message, data):
219+
self.message = message
220+
self.data = data
221+
self.lock = threading.Lock() # Cannot be serialized
222+
223+
def custom_serializer(exc):
224+
return {"message": exc.message, "data": str(exc.data)}
225+
226+
def custom_deserializer(state):
227+
return CustomError(state["message"], state["data"])
228+
229+
# Register in the driver
230+
ray.util.register_serializer(
231+
CustomError,
232+
serializer=custom_serializer,
233+
deserializer=custom_deserializer
234+
)
235+
236+
@ray.remote
237+
def task_that_registers_serializer_and_raises():
238+
# Register the custom serializer in the worker
239+
ray.util.register_serializer(
240+
CustomError,
241+
serializer=custom_serializer,
242+
deserializer=custom_deserializer
243+
)
244+
245+
# Now raise the custom exception
246+
raise CustomError("Something went wrong", {"complex": "data"})
247+
248+
# The custom exception will be properly serialized across worker boundaries
249+
try:
250+
ray.get(task_that_registers_serializer_and_raises.remote())
251+
except ray.exceptions.RayTaskError as e:
252+
print(f"Caught exception: {e.cause}") # This will be our CustomError
253+
254+
When a custom exception is raised in a remote task, Ray will:
255+
256+
1. Serialize the exception using your custom serializer
257+
2. Wrap it in a :class:`RayTaskError <ray.exceptions.RayTaskError>`
258+
3. The deserialized exception will be available as ``ray_task_error.cause``
259+
260+
Whenever serialization fails, Ray throws an :class:`UnserializableException <ray.exceptions.UnserializableException>` containing the string representation of the original stack trace.
261+
205262

206263
Troubleshooting
207264
---------------

python/ray/exceptions.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -915,7 +915,7 @@ class UnserializableException(RayError):
915915
the original exception along with its stack trace that was captured at the
916916
time of serialization.
917917
918-
reference for more details: https://docs.ray.io/en/latest/ray-core/objects/serialization.html
918+
For more details and how to handle this with custom serializers, :ref:`configuring custom exeception serializers <custom-exception-serializer>`
919919
920920
Args:
921921
original_stack_trace: The string representation and stack trace of the
@@ -927,7 +927,7 @@ def __init__(self, original_stack_trace: str):
927927

928928
def __str__(self):
929929
return (
930-
"Failed to deserialize exception. Refer to https://docs.ray.io/en/latest/ray-core/objects/serialization.html#troubleshooting to troubleshoot.\n"
930+
"Failed to deserialize exception. Refer to https://docs.ray.io/en/latest/ray-core/objects/serialization.html#custom-serializers-for-exceptions for more information.\n"
931931
"Original exception:\n"
932932
f"{self._original_stack_trace}"
933933
)

python/ray/tests/test_traceback.py

Lines changed: 38 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -301,7 +301,7 @@ def __repr__(self):
301301

302302

303303
def test_unpickleable_stacktrace(shutdown_only):
304-
expected_output = """Failed to deserialize exception. Refer to https://docs.ray.io/en/latest/ray-core/objects/serialization.html#troubleshooting to troubleshoot.
304+
expected_output = """Failed to deserialize exception. Refer to https://docs.ray.io/en/latest/ray-core/objects/serialization.html#custom-serializers-for-exceptions for more information.
305305
Original exception:
306306
ray.exceptions.RayTaskError: ray::f() (pid=XXX, ip=YYY)
307307
File "FILE", line ZZ, in f
@@ -330,6 +330,43 @@ def f():
330330
assert clean_noqa(expected_output) == scrub_traceback(str(excinfo.value))
331331

332332

333+
def test_exception_with_registered_serializer(shutdown_only):
334+
class NoPickleError(OSError):
335+
def __init__(self, msg):
336+
self.msg = msg
337+
338+
def __str__(self):
339+
return f"message: {self.msg}"
340+
341+
def _serializer(e: NoPickleError):
342+
return {"msg": e.msg}
343+
344+
def _deserializer(state):
345+
return NoPickleError(state["msg"] + " deserialized")
346+
347+
@ray.remote
348+
def raise_custom_exception():
349+
ray.util.register_serializer(
350+
NoPickleError, serializer=_serializer, deserializer=_deserializer
351+
)
352+
raise NoPickleError("message")
353+
354+
try:
355+
with pytest.raises(NoPickleError) as exc_info:
356+
ray.get(raise_custom_exception.remote())
357+
358+
# Ensure dual-typed exception and message propagation
359+
assert isinstance(exc_info.value, RayTaskError)
360+
# if custom serializer was not registered, this would be an instance of UnserializableException()
361+
assert isinstance(exc_info.value, NoPickleError)
362+
assert "message" in str(exc_info.value)
363+
# modified message should not be in the exception string, only in the cause
364+
assert "deserialized" not in str(exc_info.value)
365+
assert "message deserialized" in str(exc_info.value.cause)
366+
finally:
367+
ray.util.deregister_serializer(NoPickleError)
368+
369+
333370
def test_serialization_error_message(shutdown_only):
334371
expected_output_ray_put = """Could not serialize the put value <unlocked _thread.lock object at ADDRESS>:\nINSPECT_SERIALIZABILITY""" # noqa
335372
expected_output_task = """Could not serialize the argument <unlocked _thread.lock object at ADDRESS> for a task or actor test_traceback.test_serialization_error_message.<locals>.task_with_unserializable_arg:\nINSPECT_SERIALIZABILITY""" # noqa

0 commit comments

Comments
 (0)