Skip to content

Commit

Permalink
[Serve] Return error string from deploy_serve_application task (#36744
Browse files Browse the repository at this point in the history
)

The controller runs the deploy_serve_application task to build and run the user's Serve app. If the task raises an error, the controller will try to deserialize it when it calls ray.get() on the task's reference. If the error contains a custom dependency, the deserialization will fail, and the controller will log an error about the deserialization failing instead of the actual error itself.

This change catches any error in the deploy_serve_application task itself and returns it as a string to the controller. The controller then simply logs the string.

Related issue number
Closes #35677 and #35678.

---------

Signed-off-by: Shreyas Krishnaswamy <shrekris@anyscale.com>
  • Loading branch information
shrekris-anyscale authored Jun 28, 2023
1 parent 684e28b commit 0fe1149
Show file tree
Hide file tree
Showing 4 changed files with 74 additions and 14 deletions.
24 changes: 12 additions & 12 deletions python/ray/serve/_private/application_state.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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"
Expand Down
6 changes: 5 additions & 1 deletion python/ray/serve/controller.py
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand All @@ -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
Expand Down Expand Up @@ -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)
Expand Down
26 changes: 26 additions & 0 deletions python/ray/serve/tests/test_config_files/broken_app.py
Original file line number Diff line number Diff line change
@@ -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()
32 changes: 31 additions & 1 deletion python/ray/serve/tests/test_controller.py
Original file line number Diff line number Diff line change
@@ -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
Expand All @@ -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):
Expand Down Expand Up @@ -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

Expand Down

0 comments on commit 0fe1149

Please sign in to comment.