diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 442a114e940f3..30698184846c3 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -18,7 +18,7 @@ ) from ray.serve.schema import DeploymentDetails import time -from ray.exceptions import RayTaskError, RuntimeEnvSetupError +from ray.exceptions import RuntimeEnvSetupError from ray.serve._private.constants import SERVE_LOGGER_NAME from ray.serve._private.deploy_utils import deploy_args_to_deployment_info from ray.serve._private.utils import check_obj_ref_ready_nowait @@ -335,17 +335,17 @@ def _reconcile_deploy_obj_ref(self) -> Tuple[BuildAppStatus, str]: if check_obj_ref_ready_nowait(self._deploy_obj_ref): deploy_obj_ref, self._deploy_obj_ref = self._deploy_obj_ref, None try: - ray.get(deploy_obj_ref) - logger.info(f"Deploy task for app '{self._name}' ran successfully.") - return BuildAppStatus.SUCCEEDED, "" - except RayTaskError as e: - # NOTE(zcin): we should use str(e) instead of traceback.format_exc() - # here because the full details of the error is not displayed - # properly with traceback.format_exc(). RayTaskError has its own - # custom __str__ function. - error_msg = f"Deploying app '{self._name}' failed:\n{str(e)}" - logger.warning(error_msg) - return BuildAppStatus.FAILED, error_msg + err = ray.get(deploy_obj_ref) + if err is None: + logger.info(f"Deploy task for app '{self._name}' ran successfully.") + return BuildAppStatus.SUCCEEDED, "" + else: + error_msg = ( + f"Deploying app '{self._name}' failed with " + f"exception:\n{err}" + ) + logger.warning(error_msg) + return BuildAppStatus.FAILED, error_msg except RuntimeEnvSetupError: error_msg = ( f"Runtime env setup for app '{self._name}' failed:\n" diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index df4f033e8a6bc..726635131747c 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -834,7 +834,7 @@ def deploy_serve_application( route_prefix: str, name: str, args: Dict, -): +) -> Optional[str]: """Deploy Serve application from a user-provided config. Args: @@ -848,6 +848,8 @@ def deploy_serve_application( name: application name. If specified, application will be deployed without removing existing applications. route_prefix: route_prefix. Define the route path for the application. + Returns: + Returns None if no error is raised. Otherwise, returns error message. """ try: from ray import serve @@ -905,6 +907,8 @@ def deploy_serve_application( # Error is raised when this task is canceled with ray.cancel(), which # happens when deploy_apps() is called. logger.debug("Existing config deployment request terminated.") + except Exception as e: + return repr(e) @ray.remote(num_cpus=0) diff --git a/python/ray/serve/tests/test_config_files/broken_app.py b/python/ray/serve/tests/test_config_files/broken_app.py new file mode 100644 index 0000000000000..ec0a1cd1bc80f --- /dev/null +++ b/python/ray/serve/tests/test_config_files/broken_app.py @@ -0,0 +1,26 @@ +from ray import serve +import ray.cloudpickle as pickle + + +class NonserializableException(Exception): + """This exception cannot be serialized.""" + + def __reduce__(self): + raise RuntimeError("This exception cannot be serialized!") + + +# Confirm that NonserializableException cannot be serialized. +try: + pickle.dumps(NonserializableException()) +except RuntimeError as e: + assert "This exception cannot be serialized!" in repr(e) + +raise NonserializableException("custom exception info") + + +@serve.deployment +def f(): + pass + + +app = f.bind() diff --git a/python/ray/serve/tests/test_controller.py b/python/ray/serve/tests/test_controller.py index f5a3250e8c6ba..676f7d378f7fe 100644 --- a/python/ray/serve/tests/test_controller.py +++ b/python/ray/serve/tests/test_controller.py @@ -1,7 +1,8 @@ -import pytest import time +import pytest import ray +from ray._private.test_utils import wait_for_condition from ray import serve from ray.serve._private.common import DeploymentInfo @@ -10,6 +11,8 @@ SERVE_DEFAULT_APP_NAME, DEPLOYMENT_NAME_PREFIX_SEPARATOR, ) +from ray.serve.schema import ServeDeploySchema +from ray.serve._private.common import ApplicationStatus def get_deployment_name(name: str): @@ -49,6 +52,33 @@ def test(_): assert start_time_ms_1 == start_time_ms_2 +def test_deploy_app_custom_exception(serve_instance): + """Check that controller doesn't deserialize an exception from deploy_app.""" + + client = serve_instance + controller = serve.context.get_global_client()._controller + + config = { + "applications": [ + { + "name": "broken_app", + "route_prefix": "/broken", + "import_path": "ray.serve.tests.test_config_files.broken_app:app", + } + ] + } + + ray.get(controller.deploy_apps.remote(config=ServeDeploySchema.parse_obj(config))) + + def check_custom_exception() -> bool: + status = client.get_serve_status(name="broken_app") + assert status.app_status.status == ApplicationStatus.DEPLOY_FAILED + assert "custom exception info" in status.app_status.message + return True + + wait_for_condition(check_custom_exception, timeout=10) + + if __name__ == "__main__": import sys