exceptions.py 36 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108
  1. import logging
  2. import os
  3. import sys
  4. from traceback import format_exception
  5. from typing import Optional, Union
  6. import colorama
  7. import ray._private.ray_constants as ray_constants
  8. import ray.cloudpickle as pickle
  9. from ray._raylet import ActorID, TaskID, WorkerID
  10. from ray.core.generated.common_pb2 import (
  11. PYTHON,
  12. ActorDiedErrorContext,
  13. Address,
  14. ErrorType,
  15. Language,
  16. NodeDeathInfo,
  17. RayException,
  18. )
  19. from ray.util.annotations import DeveloperAPI, PublicAPI
  20. logger = logging.getLogger(__name__)
  21. @PublicAPI
  22. class RayError(Exception):
  23. """Super class of all ray exception types."""
  24. def to_bytes(self):
  25. # Extract exc_info from exception object.
  26. exc_info = (type(self), self, self.__traceback__)
  27. formatted_exception_string = "\n".join(format_exception(*exc_info))
  28. return RayException(
  29. language=PYTHON,
  30. serialized_exception=pickle.dumps(self),
  31. formatted_exception_string=formatted_exception_string,
  32. ).SerializeToString()
  33. @staticmethod
  34. def from_bytes(b):
  35. ray_exception = RayException()
  36. ray_exception.ParseFromString(b)
  37. return RayError.from_ray_exception(ray_exception)
  38. @staticmethod
  39. def from_ray_exception(ray_exception):
  40. if ray_exception.language == PYTHON:
  41. try:
  42. return pickle.loads(ray_exception.serialized_exception)
  43. except Exception:
  44. # formatted_exception_string is set in to_bytes() above by calling
  45. # traceback.format_exception() on the original exception. It contains
  46. # the string representation and stack trace of the original error.
  47. original_stacktrace = getattr(
  48. ray_exception,
  49. "formatted_exception_string",
  50. "No formatted exception string available.",
  51. )
  52. return UnserializableException(original_stacktrace)
  53. else:
  54. return CrossLanguageError(ray_exception)
  55. @PublicAPI
  56. class CrossLanguageError(RayError):
  57. """Raised from another language."""
  58. def __init__(self, ray_exception):
  59. super().__init__(
  60. "An exception raised from {}:\n{}".format(
  61. Language.Name(ray_exception.language),
  62. ray_exception.formatted_exception_string,
  63. )
  64. )
  65. @PublicAPI
  66. class TaskCancelledError(RayError):
  67. """Raised when this task is cancelled.
  68. Args:
  69. task_id: The TaskID of the function that was directly
  70. cancelled.
  71. """
  72. def __init__(
  73. self, task_id: Optional[TaskID] = None, error_message: Optional[str] = None
  74. ):
  75. self.task_id = task_id
  76. self.error_message = error_message
  77. def __str__(self):
  78. msg = ""
  79. if self.task_id:
  80. msg = "Task: " + str(self.task_id) + " was cancelled. "
  81. if self.error_message:
  82. msg += self.error_message
  83. return msg
  84. @PublicAPI
  85. class RayTaskError(RayError):
  86. """Indicates that a task threw an exception during execution.
  87. If a task throws an exception during execution, a RayTaskError is stored in
  88. the object store for each of the task's outputs. When an object is
  89. retrieved from the object store, the Python method that retrieved it checks
  90. to see if the object is a RayTaskError and if it is then an exception is
  91. thrown propagating the error message.
  92. """
  93. def __init__(
  94. self,
  95. function_name,
  96. traceback_str,
  97. cause,
  98. proctitle=None,
  99. pid=None,
  100. ip=None,
  101. actor_repr=None,
  102. actor_id=None,
  103. ):
  104. """Initialize a RayTaskError."""
  105. import ray
  106. if proctitle:
  107. self.proctitle = proctitle
  108. else:
  109. self.proctitle = ray._raylet.getproctitle()
  110. self.pid = pid or os.getpid()
  111. self.ip = ip or ray.util.get_node_ip_address()
  112. self.function_name = function_name
  113. self.traceback_str = traceback_str
  114. self.actor_repr = actor_repr
  115. self._actor_id = actor_id
  116. self.cause = cause
  117. try:
  118. pickle.dumps(cause)
  119. except (pickle.PicklingError, TypeError) as e:
  120. err_type = f"{cause.__class__.__module__}.{cause.__class__.__name__}"
  121. err_msg = (
  122. f"Exception {err_type} isn't serializable: {e}.\n"
  123. f"Original exception details:\n{traceback_str}"
  124. )
  125. logger.exception(
  126. f"The original cause of the RayTaskError ({err_type}) isn't serializable."
  127. )
  128. self.cause = RayError(err_msg)
  129. # BaseException implements a __reduce__ method that returns
  130. # a tuple with the type and the value of self.args.
  131. # https://stackoverflow.com/a/49715949/2213289
  132. self.args = (function_name, traceback_str, self.cause, proctitle, pid, ip)
  133. assert traceback_str is not None
  134. def make_dual_exception_instance(self) -> "RayTaskError":
  135. """Makes a object instance that inherits from both RayTaskError and the type of
  136. `self.cause`. Raises TypeError if the cause class can't be subclassed"""
  137. # For normal user Exceptions, we subclass from both
  138. # RayTaskError and the user exception. For ExceptionGroup,
  139. # we special handle it because it has a different __new__()
  140. # signature from Exception.
  141. # Ref: https://docs.python.org/3/library/exceptions.html#exception-groups
  142. if sys.version_info >= (3, 11) and isinstance(
  143. self.cause, ExceptionGroup # noqa: F821
  144. ):
  145. return self._make_exceptiongroup_dual_exception_instance()
  146. return self._make_normal_dual_exception_instance()
  147. def _make_normal_dual_exception_instance(self) -> "RayTaskError":
  148. cause_cls = self.cause.__class__
  149. error_msg = str(self)
  150. class cls(RayTaskError, cause_cls):
  151. def __init__(self, cause):
  152. self.cause = cause
  153. # Store args separately to avoid writing to user-defined
  154. # read-only or property-based `args`.
  155. self._ray_task_error_args = (cause,)
  156. @property
  157. def args(self):
  158. return self._ray_task_error_args
  159. @args.setter
  160. def args(self, value):
  161. self._ray_task_error_args = value
  162. def __reduce__(self):
  163. return (cls, self._ray_task_error_args)
  164. def __getattr__(self, name):
  165. return getattr(self.cause, name)
  166. def __str__(self):
  167. return error_msg
  168. name = f"RayTaskError({cause_cls.__name__})"
  169. cls.__name__ = name
  170. cls.__qualname__ = name
  171. return cls(self.cause)
  172. def _make_exceptiongroup_dual_exception_instance(self) -> "RayTaskError":
  173. cause_cls = self.cause.__class__
  174. error_msg = str(self)
  175. class cls(RayTaskError, cause_cls):
  176. def __new__(cls, cause):
  177. self = super().__new__(cls, cause.message, cause.exceptions)
  178. return self
  179. def __init__(self, cause):
  180. self.cause = cause
  181. self._ray_task_error_args = (cause,)
  182. @property
  183. def args(self):
  184. return self._ray_task_error_args
  185. @args.setter
  186. def args(self, value):
  187. self._ray_task_error_args = value
  188. def __reduce__(self):
  189. return (cls, self._ray_task_error_args)
  190. def __getattr__(self, name):
  191. return getattr(self.cause, name)
  192. def __str__(self):
  193. return error_msg
  194. name = f"RayTaskError({cause_cls.__name__})"
  195. cls.__name__ = name
  196. cls.__qualname__ = name
  197. return cls(self.cause)
  198. def as_instanceof_cause(self):
  199. """Returns an exception that's an instance of the cause's class.
  200. The returned exception inherits from both RayTaskError and the
  201. cause class and contains all of the attributes of the cause
  202. exception.
  203. If the cause class can't be subclassed, issues a warning and returns `self`.
  204. """
  205. cause_cls = self.cause.__class__
  206. if issubclass(RayTaskError, cause_cls):
  207. return self # already satisfied
  208. try:
  209. return self.make_dual_exception_instance()
  210. except TypeError as e:
  211. logger.warning(
  212. f"User exception type {type(self.cause)} in RayTaskError can't"
  213. " be subclassed! This exception is raised as"
  214. " RayTaskError only. You can use `ray_task_error.cause` to"
  215. f" access the user exception. Failure in subclassing: {e}"
  216. )
  217. return self
  218. except Exception as e:
  219. logger.warning(
  220. "Failed to combine RayTaskError with user exception type "
  221. f"{type(self.cause)}; raising RayTaskError only. This can "
  222. "happen when the user exception overrides attributes like "
  223. "`args` or otherwise blocks subclass construction. "
  224. f"Failure in subclassing: {e}"
  225. )
  226. return self
  227. def __str__(self):
  228. """Format a RayTaskError as a string."""
  229. lines = self.traceback_str.strip().split("\n")
  230. out = []
  231. code_from_internal_file = False
  232. # Format tracebacks.
  233. # Python stacktrace consists of
  234. # Traceback...: Indicate the next line will be a traceback.
  235. # File [file_name + line number]
  236. # code
  237. # XError: [message]
  238. # NOTE: For _raylet.pyx (Cython), the code is not always included.
  239. for i, line in enumerate(lines):
  240. # Convert traceback to the readable information.
  241. if line.startswith("Traceback "):
  242. traceback_line = (
  243. f"{colorama.Fore.CYAN}"
  244. f"{self.proctitle}()"
  245. f"{colorama.Fore.RESET} "
  246. f"(pid={self.pid}, ip={self.ip}"
  247. )
  248. if self.actor_repr:
  249. traceback_line += (
  250. f", actor_id={self._actor_id}, repr={self.actor_repr})"
  251. )
  252. else:
  253. traceback_line += ")"
  254. code_from_internal_file = False
  255. out.append(traceback_line)
  256. elif line.startswith(" File ") and (
  257. "ray/worker.py" in line
  258. or "ray/_private/" in line
  259. or "ray/util/tracing/" in line
  260. or "ray/_raylet.pyx" in line
  261. ):
  262. # TODO(windows)
  263. # Process the internal file line.
  264. # The file line always starts with 2 space and File.
  265. # https://github.com/python/cpython/blob/0a0a135bae2692d069b18d2d590397fbe0a0d39a/Lib/traceback.py#L421 # noqa
  266. if "ray._raylet.raise_if_dependency_failed" in line:
  267. # It means the current task is failed
  268. # due to the dependency failure.
  269. # Print out an user-friendly
  270. # message to explain that..
  271. out.append(
  272. " At least one of the input arguments for "
  273. "this task could not be computed:"
  274. )
  275. if i + 1 < len(lines) and lines[i + 1].startswith(" "):
  276. # If the next line is indented with 2 space,
  277. # that means it contains internal code information.
  278. # For example,
  279. # File [file_name] [line]
  280. # [code] # if the next line is indented, it is code.
  281. # Note there there are 4 spaces in the code line.
  282. code_from_internal_file = True
  283. elif code_from_internal_file:
  284. # If the current line is internal file's code,
  285. # the next line is not code anymore.
  286. code_from_internal_file = False
  287. else:
  288. out.append(line)
  289. return "\n".join(out)
  290. @PublicAPI
  291. class LocalRayletDiedError(RayError):
  292. """Indicates that the task's local raylet died."""
  293. def __str__(self):
  294. return "The task's local raylet died. Check raylet.out for more information."
  295. @PublicAPI
  296. class WorkerCrashedError(RayError):
  297. """Indicates that the worker died unexpectedly while executing a task."""
  298. def __str__(self):
  299. return (
  300. "The worker died unexpectedly while executing this task. "
  301. "Check python-core-worker-*.log files for more information."
  302. )
  303. @PublicAPI
  304. class RayActorError(RayError):
  305. """Indicates that the actor has outages unexpectedly before finishing a task.
  306. This exception could happen because the actor process is dead, or is unavailable for
  307. the moment. Ray raises subclasses `ActorDiedError` and `ActorUnavailableError`
  308. respectively.
  309. """
  310. BASE_ERROR_MSG = "The actor experienced an error before finishing this task."
  311. def __init__(
  312. self,
  313. actor_id: str = None,
  314. error_msg: str = BASE_ERROR_MSG,
  315. actor_init_failed: bool = False,
  316. preempted: bool = False,
  317. ):
  318. #: The actor ID in hex string.
  319. self.actor_id = actor_id
  320. #: Whether the actor failed in the middle of __init__.
  321. self.error_msg = error_msg
  322. #: The full error message.
  323. self._actor_init_failed = actor_init_failed
  324. #: Whether the actor died because the node was preempted.
  325. self._preempted = preempted
  326. def __str__(self) -> str:
  327. return self.error_msg
  328. @property
  329. def preempted(self) -> bool:
  330. return self._preempted
  331. @property
  332. def actor_init_failed(self) -> bool:
  333. return self._actor_init_failed
  334. @DeveloperAPI
  335. class ActorDiedError(RayActorError):
  336. """Indicates that the actor died unexpectedly before finishing a task.
  337. This exception could happen either because the actor process dies while
  338. executing a task, or because a task is submitted to a dead actor.
  339. Args:
  340. cause: The cause of the actor error. `RayTaskError` type means
  341. the actor has died because of an exception within `__init__`.
  342. `ActorDiedErrorContext` means the actor has died because of
  343. an unexpected system error. None means the cause isn't known.
  344. Theoretically, this shouldn't happen,
  345. but it's there as a safety check.
  346. """
  347. BASE_ERROR_MSG = "The actor died unexpectedly before finishing this task."
  348. def __init__(
  349. self, cause: Optional[Union[RayTaskError, ActorDiedErrorContext]] = None
  350. ):
  351. """
  352. Construct a RayActorError by building the arguments.
  353. """
  354. actor_id = None
  355. error_msg = ActorDiedError.BASE_ERROR_MSG
  356. actor_init_failed = False
  357. preempted = False
  358. if not cause:
  359. # Use the defaults above.
  360. pass
  361. elif isinstance(cause, RayTaskError):
  362. actor_init_failed = True
  363. actor_id = cause._actor_id
  364. error_msg = (
  365. "The actor died because of an error"
  366. " raised in its creation task, "
  367. f"{cause.__str__()}"
  368. )
  369. else:
  370. # Indicating system-level actor failures.
  371. assert isinstance(cause, ActorDiedErrorContext)
  372. error_msg_lines = [ActorDiedError.BASE_ERROR_MSG]
  373. error_msg_lines.append(f"\tclass_name: {cause.class_name}")
  374. error_msg_lines.append(f"\tactor_id: {ActorID(cause.actor_id).hex()}")
  375. # Below items are optional fields.
  376. if cause.pid != 0:
  377. error_msg_lines.append(f"\tpid: {cause.pid}")
  378. if cause.name != "":
  379. error_msg_lines.append(f"\tname: {cause.name}")
  380. if cause.ray_namespace != "":
  381. error_msg_lines.append(f"\tnamespace: {cause.ray_namespace}")
  382. if cause.node_ip_address != "":
  383. error_msg_lines.append(f"\tip: {cause.node_ip_address}")
  384. error_msg_lines.append(cause.error_message)
  385. if cause.never_started:
  386. error_msg_lines.append(
  387. "The actor never ran - it was cancelled before it started running."
  388. )
  389. if (
  390. cause.node_death_info
  391. and cause.node_death_info.reason
  392. == NodeDeathInfo.AUTOSCALER_DRAIN_PREEMPTED
  393. ):
  394. preempted = True
  395. error_msg = "\n".join(error_msg_lines)
  396. actor_id = ActorID(cause.actor_id).hex()
  397. super().__init__(actor_id, error_msg, actor_init_failed, preempted)
  398. @staticmethod
  399. def from_task_error(task_error: RayTaskError):
  400. return ActorDiedError(task_error)
  401. @DeveloperAPI
  402. class ActorUnavailableError(RayActorError):
  403. """Raised when the actor is temporarily unavailable but may be available later."""
  404. def __init__(self, error_message: str, actor_id: Optional[bytes]):
  405. actor_id = ActorID(actor_id).hex() if actor_id is not None else None
  406. error_msg = (
  407. f"The actor {actor_id} is unavailable: {error_message}. The task may or "
  408. "may not have been executed on the actor."
  409. )
  410. actor_init_failed = False
  411. preempted = False
  412. super().__init__(actor_id, error_msg, actor_init_failed, preempted)
  413. @PublicAPI
  414. class RaySystemError(RayError):
  415. """Indicates that Ray encountered a system error.
  416. This exception can be thrown when the raylet is killed.
  417. """
  418. def __init__(self, client_exc, traceback_str=None):
  419. self.client_exc = client_exc
  420. self.traceback_str = traceback_str
  421. def __str__(self):
  422. error_msg = f"System error: {self.client_exc}"
  423. if self.traceback_str:
  424. error_msg += f"\ntraceback: {self.traceback_str}"
  425. return error_msg
  426. @PublicAPI
  427. class AuthenticationError(RayError):
  428. """Indicates that an authentication error occurred.
  429. Most commonly, this is caused by a missing or mismatching token set on the client
  430. (e.g., a Ray CLI command interacting with a remote cluster).
  431. Only applicable when `RAY_AUTH_MODE` is not set to `disabled`.
  432. """
  433. def __init__(self, message: str):
  434. self.message = message
  435. # Always hide traceback for cleaner output
  436. self.__suppress_context__ = True
  437. super().__init__(message)
  438. def __str__(self) -> str:
  439. # Check if RAY_AUTH_MODE is set to token and add a heads-up if not
  440. auth_mode_note = ""
  441. from ray._private.authentication.authentication_utils import (
  442. get_authentication_mode_name,
  443. )
  444. from ray._raylet import AuthenticationMode, get_authentication_mode
  445. current_mode = get_authentication_mode()
  446. if current_mode != AuthenticationMode.TOKEN:
  447. mode_name = get_authentication_mode_name(current_mode)
  448. auth_mode_note = (
  449. f" Note: RAY_AUTH_MODE is currently '{mode_name}' (not 'token')."
  450. )
  451. help_text = (
  452. " Ensure that the token for the cluster is available in a local file (e.g., ~/.ray/auth_token or via "
  453. "RAY_AUTH_TOKEN_PATH) or as the `RAY_AUTH_TOKEN` environment variable. "
  454. "To generate a token for local development, use `ray get-auth-token --generate` "
  455. "For remote clusters, ensure that the token is propagated to all nodes of the cluster when token authentication is enabled. "
  456. "For more information, see: https://docs.ray.io/en/latest/ray-security/token-auth.html"
  457. )
  458. return self.message + "." + auth_mode_note + help_text
  459. @DeveloperAPI
  460. class UserCodeException(RayError):
  461. """Indicates that an exception occurred while executing user code.
  462. For example, this exception can be used to wrap user code exceptions
  463. from a remote task or actor. The `retry_exceptions` parameter will
  464. still respect the underlying cause of this exception."""
  465. pass
  466. @PublicAPI
  467. class ObjectStoreFullError(RayError):
  468. """Indicates that the object store is full.
  469. This is raised if the attempt to store the object fails
  470. because the object store is full even after multiple retries.
  471. """
  472. def __str__(self):
  473. return super(ObjectStoreFullError, self).__str__() + (
  474. "\n"
  475. "The local object store is full of objects that are still in "
  476. "scope and cannot be evicted. Tip: Use the `ray memory` command "
  477. "to list active objects in the cluster."
  478. )
  479. @PublicAPI
  480. class OutOfDiskError(RayError):
  481. """Indicates that the local disk is full.
  482. This is raised if the attempt to store the object fails
  483. because both the object store and disk are full.
  484. """
  485. def __str__(self):
  486. # TODO(scv119): expose more disk usage information and link to a doc.
  487. return super(OutOfDiskError, self).__str__() + (
  488. "\n"
  489. "The object cannot be created because the local object store"
  490. " is full and the local disk's utilization is over capacity"
  491. " (95% by default)."
  492. "Tip: Use `df` on this node to check disk usage and "
  493. "`ray memory` to check object store memory usage."
  494. )
  495. @PublicAPI
  496. class OutOfMemoryError(RayError):
  497. """Indicates that the node is running out of memory and is close to full.
  498. This is raised if the node is low on memory and tasks or actors are being
  499. evicted to free up memory.
  500. """
  501. # TODO: (clarng) expose the error message string here and format it with proto
  502. def __init__(self, message):
  503. self.message = message
  504. def __str__(self):
  505. return self.message
  506. @PublicAPI
  507. class NodeDiedError(RayError):
  508. """Indicates that the node is either dead or unreachable."""
  509. # TODO: (clarng) expose the error message string here and format it with proto
  510. def __init__(self, message):
  511. self.message = message
  512. def __str__(self):
  513. return self.message
  514. @PublicAPI
  515. class ObjectLostError(RayError):
  516. """Indicates that the object is lost from distributed memory, due to
  517. node failure or system error.
  518. Args:
  519. object_ref_hex: Hex ID of the object.
  520. """
  521. def __init__(self, object_ref_hex, owner_address, call_site):
  522. self.object_ref_hex = object_ref_hex
  523. self.owner_address = owner_address
  524. self.call_site = call_site.replace(
  525. ray_constants.CALL_STACK_LINE_DELIMITER, "\n "
  526. )
  527. def _base_str(self):
  528. msg = f"Failed to retrieve object {self.object_ref_hex}. "
  529. if self.call_site:
  530. msg += f"The ObjectRef was created at: {self.call_site}"
  531. else:
  532. msg += (
  533. "To see information about where this ObjectRef was created "
  534. "in Python, set the environment variable "
  535. "RAY_record_ref_creation_sites=1 during `ray start` and "
  536. "`ray.init()`."
  537. )
  538. return msg
  539. def __str__(self):
  540. return (
  541. self._base_str()
  542. + "\n\n"
  543. + (
  544. f"All copies of {self.object_ref_hex} have been lost due to node "
  545. "failure. Check cluster logs (`/tmp/ray/session_latest/logs`) for "
  546. "more information about the failure."
  547. )
  548. )
  549. @PublicAPI
  550. class ObjectFetchTimedOutError(ObjectLostError):
  551. """Indicates that an object fetch timed out.
  552. Args:
  553. object_ref_hex: Hex ID of the object.
  554. """
  555. def __str__(self):
  556. return (
  557. self._base_str()
  558. + "\n\n"
  559. + (
  560. f"Fetch for object {self.object_ref_hex} timed out because no "
  561. "locations were found for the object. This may indicate a "
  562. "system-level bug."
  563. )
  564. )
  565. @DeveloperAPI
  566. class RpcError(RayError):
  567. """Indicates an error in the underlying RPC system."""
  568. def __init__(self, message, rpc_code=None):
  569. self.message = message
  570. self.rpc_code = rpc_code
  571. def __str__(self):
  572. return self.message
  573. @DeveloperAPI
  574. class ReferenceCountingAssertionError(ObjectLostError, AssertionError):
  575. """Indicates that an object has been deleted while there was still a
  576. reference to it.
  577. Args:
  578. object_ref_hex: Hex ID of the object.
  579. """
  580. def __str__(self):
  581. return (
  582. self._base_str()
  583. + "\n\n"
  584. + (
  585. "The object has already been deleted by the reference counting "
  586. "protocol. This should not happen."
  587. )
  588. )
  589. @DeveloperAPI
  590. class ObjectFreedError(ObjectLostError):
  591. """Indicates that an object was manually freed by the application.
  592. Attributes:
  593. object_ref_hex: Hex ID of the object.
  594. """
  595. def __str__(self):
  596. return (
  597. self._base_str()
  598. + "\n\n"
  599. + (
  600. "The object was manually freed using the internal `free` call. "
  601. "Please ensure that `free` is only called once the object is no "
  602. "longer needed."
  603. )
  604. )
  605. @PublicAPI
  606. class OwnerDiedError(ObjectLostError):
  607. """Indicates that the owner of the object has died while there is still a
  608. reference to the object.
  609. Args:
  610. object_ref_hex: Hex ID of the object.
  611. """
  612. def __str__(self):
  613. log_loc = "`/tmp/ray/session_latest/logs`"
  614. if self.owner_address:
  615. try:
  616. addr = Address()
  617. addr.ParseFromString(self.owner_address)
  618. ip_addr = addr.ip_address
  619. worker_id = WorkerID(addr.worker_id)
  620. log_loc = (
  621. f"`/tmp/ray/session_latest/logs/*{worker_id.hex()}*`"
  622. f" at IP address {ip_addr}"
  623. )
  624. except Exception:
  625. # Catch all to make sure we always at least print the default
  626. # message.
  627. pass
  628. return (
  629. self._base_str()
  630. + "\n\n"
  631. + (
  632. "The object's owner has exited. This is the Python "
  633. "worker that first created the ObjectRef via `.remote()` or "
  634. "`ray.put()`. "
  635. f"Check cluster logs ({log_loc}) for more "
  636. "information about the Python worker failure."
  637. )
  638. )
  639. @PublicAPI
  640. class ObjectReconstructionFailedError(ObjectLostError):
  641. """Indicates that the object cannot be reconstructed."""
  642. REASON_MESSAGES = {
  643. ErrorType.OBJECT_UNRECONSTRUCTABLE_MAX_ATTEMPTS_EXCEEDED: (
  644. "The object cannot be reconstructed because the maximum number of "
  645. "task retries has been exceeded. "
  646. "Consider increasing the number of retries using `@ray.remote(max_retries=N)`."
  647. ),
  648. ErrorType.OBJECT_UNRECONSTRUCTABLE_LINEAGE_EVICTED: (
  649. "The object cannot be reconstructed because its lineage has been "
  650. "evicted to reduce memory pressure. "
  651. "To prevent this error, set the environment variable "
  652. "RAY_max_lineage_bytes=<bytes> (default 1GB) during `ray start`."
  653. ),
  654. ErrorType.OBJECT_UNRECONSTRUCTABLE_PUT: (
  655. "The object cannot be reconstructed because it was created by "
  656. "ray.put(), which has no task lineage. "
  657. "To prevent this error, return the value from a task instead."
  658. ),
  659. ErrorType.OBJECT_UNRECONSTRUCTABLE_RETRIES_DISABLED: (
  660. "The object cannot be reconstructed because the task was created "
  661. "with max_retries=0. "
  662. "Consider enabling retries using `@ray.remote(max_retries=N)`."
  663. ),
  664. ErrorType.OBJECT_UNRECONSTRUCTABLE_BORROWED: (
  665. "The object cannot be reconstructed because it crossed an ownership "
  666. "boundary. Only the owner of an object can trigger reconstruction, "
  667. "but this worker borrowed the object from another worker."
  668. ),
  669. ErrorType.OBJECT_UNRECONSTRUCTABLE_LOCAL_MODE: (
  670. "The object cannot be reconstructed because Ray is running in "
  671. "local mode. Local mode does not support object reconstruction."
  672. ),
  673. ErrorType.OBJECT_UNRECONSTRUCTABLE_REF_NOT_FOUND: (
  674. "The object cannot be reconstructed because its reference was "
  675. "not found in the reference counter. "
  676. "Please file an issue at https://github.com/ray-project/ray/issues."
  677. ),
  678. ErrorType.OBJECT_UNRECONSTRUCTABLE_TASK_CANCELLED: (
  679. "The object cannot be reconstructed because the task that would "
  680. "produce it was cancelled."
  681. ),
  682. ErrorType.OBJECT_UNRECONSTRUCTABLE_LINEAGE_DISABLED: (
  683. "The object cannot be reconstructed because lineage reconstruction "
  684. "is disabled system-wide (object_reconstruction_enabled=False)."
  685. ),
  686. }
  687. def __init__(
  688. self,
  689. object_ref_hex: str,
  690. reason: "ErrorType" = None,
  691. reason_message: str = None,
  692. owner_address: Optional[Address] = None,
  693. call_site: str = "",
  694. ):
  695. """Initialize ObjectReconstructionFailedError.
  696. Args:
  697. object_ref_hex: Hex string of the object reference.
  698. reason: ErrorType enum value indicating why reconstruction failed.
  699. reason_message: Human-readable explanation of the failure.
  700. owner_address: Address of the object's owner.
  701. call_site: Call site where the object was created.
  702. """
  703. super().__init__(object_ref_hex, owner_address, call_site)
  704. self.reason = reason
  705. self.reason_message = reason_message or self.REASON_MESSAGES.get(
  706. self.reason,
  707. "Unknown error reason. This should not happen, please file an issue "
  708. "at https://github.com/ray-project/ray/issues.",
  709. )
  710. def __str__(self):
  711. base = self._base_str()
  712. if self.reason_message:
  713. reason_name = ErrorType.Name(self.reason) if self.reason else "UNKNOWN"
  714. return base + f"\n\n[{reason_name}] {self.reason_message}"
  715. return base
  716. @PublicAPI
  717. class GetTimeoutError(RayError, TimeoutError):
  718. """Indicates that a call to the worker timed out."""
  719. pass
  720. @PublicAPI
  721. class PlasmaObjectNotAvailable(RayError):
  722. """Called when an object was not available within the given timeout."""
  723. pass
  724. @PublicAPI
  725. class AsyncioActorExit(RayError):
  726. """Raised when an asyncio actor intentionally exits via exit_actor()."""
  727. pass
  728. @PublicAPI
  729. class RuntimeEnvSetupError(RayError):
  730. """Raised when a runtime environment fails to be set up.
  731. Args:
  732. error_message: The error message that explains
  733. why runtime env setup has failed.
  734. """
  735. def __init__(self, error_message: str = None):
  736. self.error_message = error_message
  737. def __str__(self):
  738. msgs = ["Failed to set up runtime environment."]
  739. if self.error_message:
  740. msgs.append(self.error_message)
  741. return "\n".join(msgs)
  742. @PublicAPI
  743. class TaskPlacementGroupRemoved(RayError):
  744. """Raised when the corresponding placement group was removed."""
  745. def __str__(self):
  746. return "The placement group corresponding to this task has been removed."
  747. @PublicAPI
  748. class ActorPlacementGroupRemoved(RayError):
  749. """Raised when the corresponding placement group was removed."""
  750. def __str__(self):
  751. return "The placement group corresponding to this Actor has been removed."
  752. @PublicAPI
  753. class PendingCallsLimitExceeded(RayError):
  754. """Raised when the pending actor calls exceeds `max_pending_calls` option.
  755. This exception could happen probably because the caller calls the callee
  756. too frequently.
  757. """
  758. pass
  759. @PublicAPI
  760. class TaskUnschedulableError(RayError):
  761. """Raised when the task cannot be scheduled.
  762. One example is that the node specified through
  763. NodeAffinitySchedulingStrategy is dead.
  764. """
  765. def __init__(self, error_message: str):
  766. self.error_message = error_message
  767. def __str__(self):
  768. return f"The task is not schedulable: {self.error_message}"
  769. @PublicAPI
  770. class ActorUnschedulableError(RayError):
  771. """Raised when the actor cannot be scheduled.
  772. One example is that the node specified through
  773. NodeAffinitySchedulingStrategy is dead.
  774. """
  775. def __init__(self, error_message: str):
  776. self.error_message = error_message
  777. def __str__(self):
  778. return f"The actor is not schedulable: {self.error_message}"
  779. @DeveloperAPI
  780. class ObjectRefStreamEndOfStreamError(RayError):
  781. """Raised by streaming generator tasks when there are no more ObjectRefs to
  782. read.
  783. """
  784. pass
  785. @DeveloperAPI
  786. class OufOfBandObjectRefSerializationException(RayError):
  787. """Raised when an `ray.ObjectRef` is out of band serialized by
  788. `ray.cloudpickle`. It is an anti pattern.
  789. """
  790. pass
  791. @PublicAPI(stability="alpha")
  792. class RayChannelError(RaySystemError):
  793. """Indicates that Ray encountered a system error related
  794. to ray.experimental.channel.
  795. """
  796. pass
  797. @PublicAPI(stability="alpha")
  798. class RayChannelTimeoutError(RayChannelError, TimeoutError):
  799. """Raised when the Compiled Graph channel operation times out."""
  800. pass
  801. @PublicAPI(stability="alpha")
  802. class RayCgraphCapacityExceeded(RaySystemError):
  803. """Raised when the Compiled Graph channel's buffer is at max capacity"""
  804. pass
  805. @PublicAPI(stability="alpha")
  806. class RayDirectTransportError(RaySystemError):
  807. """Raised when there is an error during a Ray direct transport transfer."""
  808. pass
  809. @PublicAPI(stability="alpha")
  810. class UnserializableException(RayError):
  811. """Raised when there is an error deserializing a serialized exception.
  812. This occurs when deserializing (unpickling) a previously serialized exception
  813. fails. In this case, we fall back to raising the string representation of
  814. the original exception along with its stack trace that was captured at the
  815. time of serialization.
  816. For more details and how to handle this with custom serializers, :ref:`configuring custom exception serializers <custom-exception-serializer>`
  817. Args:
  818. original_stack_trace: The string representation and stack trace of the
  819. original exception that was captured during serialization.
  820. """
  821. def __init__(self, original_stack_trace: str):
  822. self._original_stack_trace = original_stack_trace
  823. def __str__(self):
  824. return (
  825. "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"
  826. "Original exception:\n"
  827. f"{self._original_stack_trace}"
  828. )
  829. @DeveloperAPI
  830. class ActorAlreadyExistsError(ValueError, RayError):
  831. """Raised when a named actor already exists.
  832. Note that this error is not only a subclass of RayError, but also a subclass of ValueError, to maintain backward compatibility.
  833. Args:
  834. error_message: The error message that contains information about the actor name and namespace.
  835. """
  836. def __init__(self, error_message: str):
  837. super().__init__(error_message)
  838. self.error_message = error_message
  839. def __str__(self):
  840. return self.error_message
  841. @DeveloperAPI
  842. class ActorHandleNotFoundError(ValueError, RayError):
  843. """Raised when trying to kill an actor handle that doesn't exist.
  844. This typically happens when using an actor handle from a previous Ray session
  845. after calling ray.shutdown() and ray.init().
  846. Note that this error is not only a subclass of RayError, but also a subclass of ValueError,
  847. to maintain backward compatibility.
  848. Args:
  849. error_message: The error message that contains information about the actor handle.
  850. """
  851. def __init__(self, error_message: str):
  852. super().__init__(error_message)
  853. self.error_message = error_message
  854. def __str__(self):
  855. return self.error_message
  856. RAY_EXCEPTION_TYPES = [
  857. PlasmaObjectNotAvailable,
  858. RayError,
  859. RayTaskError,
  860. WorkerCrashedError,
  861. RayActorError,
  862. ObjectStoreFullError,
  863. ObjectLostError,
  864. ObjectFetchTimedOutError,
  865. ReferenceCountingAssertionError,
  866. ObjectReconstructionFailedError,
  867. OwnerDiedError,
  868. GetTimeoutError,
  869. AsyncioActorExit,
  870. RuntimeEnvSetupError,
  871. TaskPlacementGroupRemoved,
  872. ActorPlacementGroupRemoved,
  873. PendingCallsLimitExceeded,
  874. LocalRayletDiedError,
  875. TaskUnschedulableError,
  876. ActorDiedError,
  877. ActorUnschedulableError,
  878. ActorUnavailableError,
  879. RayChannelError,
  880. RayChannelTimeoutError,
  881. OufOfBandObjectRefSerializationException,
  882. RayCgraphCapacityExceeded,
  883. UnserializableException,
  884. ActorAlreadyExistsError,
  885. ActorHandleNotFoundError,
  886. AuthenticationError,
  887. ]