diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 0a6a099a83823..55eb4693af3ee 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -134,6 +134,7 @@ from ray.util.scheduling_strategies import ( ) import ray._private.ray_constants as ray_constants import ray.cloudpickle as ray_pickle +from ray.core.generated.common_pb2 import ActorDiedErrorContext from ray._private.async_compat import sync_to_async, get_new_event_loop from ray._private.client_mode_hook import disable_client_hook import ray._private.gcs_utils as gcs_utils @@ -774,10 +775,17 @@ cdef void execute_task( if len(inspect.getmembers( actor.__class__, predicate=inspect.iscoroutinefunction)) == 0: + error_message = ( + "Failed to create actor. The failure reason " + "is that you set the async flag, but the actor does not " + "have any coroutine functions.") raise RayActorError( - f"Failed to create the actor {core_worker.get_actor_id()}. " - "The failure reason is that you set the async flag, " - "but the actor has no any coroutine function.") + ActorDiedErrorContext( + error_message=error_message, + actor_id=core_worker.get_actor_id(), + class_name=class_name + ) + ) # Increase recursion limit if necessary. In asyncio mode, # we have many parallel callstacks (represented in fibers) # that's suspended for execution. Python interpreter will