From 0c02ff948a5a25ea7d08f37f4084b136d89bc4a8 Mon Sep 17 00:00:00 2001 From: Edward Oakes Date: Wed, 17 Aug 2022 23:04:00 -0500 Subject: [PATCH] Cherry-pick all Serve doc changes for Ray 2.0 (#27960) Cherry-picks all docs changes for Serve in Ray 2.0. I did this by overwriting the entire `doc/source/serve/` directory in addition to `doc/source/_toc.yml`. The changes should be isolated to Serve (manually verified). --- doc/source/_toc.yml | 38 +- doc/source/ray-core/ray-dag.rst | 2 - doc/source/ray-references/faq.rst | 1 - doc/source/serve/architecture-2.0.svg | 1 + doc/source/serve/architecture.md | 109 ++-- doc/source/serve/architecture.svg | 1 - doc/source/serve/autoscaling.md | 48 -- doc/source/serve/deploying-serve.md | 271 --------- doc/source/serve/deployment-graph.md | 17 - .../chain_nodes_same_class_different_args.md | 25 - ...e_two_nodes_with_passing_input_parallel.md | 27 - .../control_flow_based_on_user_inputs.md | 54 -- .../deployment-graph-e2e-tutorial.md | 575 ------------------ .../http_endpoint_for_dag_graph.md | 29 - .../visualize_dag_during_development.md | 33 - doc/source/serve/dev-workflow.md | 111 ++++ doc/source/serve/doc_code/batching_guide.py | 35 ++ .../serve/doc_code/create_deployment.py | 123 ---- .../serve/doc_code/deploying_serve_example.py | 54 ++ ...e_two_nodes_with_passing_input_parallel.py | 31 - ...graph_control_flow_based_on_user_inputs.py | 35 -- .../doc_code/deployment_graph_dag_http.py | 54 -- .../branching_input.py | 41 ++ .../deployment_graph_patterns/conditional.py | 49 ++ .../linear_pipeline.py | 37 ++ ...loyment_graph_same_class_different_args.py | 30 - doc/source/serve/doc_code/fastapi_example.py | 3 +- .../getting_started/model_deployment.py | 59 ++ .../getting_started/model_deployment_full.py | 45 ++ .../doc_code/getting_started/model_graph.py | 77 +++ .../serve/doc_code/getting_started/models.py | 63 ++ doc/source/serve/doc_code/handle_guide.py | 121 ++++ doc/source/serve/doc_code/http_guide.py | 112 ++++ .../doc_code/key-concepts-deployment-graph.py | 4 +- doc/source/serve/doc_code/local_dev.py | 34 ++ .../serve/doc_code/managing_deployments.py | 79 +++ .../serve/doc_code/migration_example.py | 14 +- .../serve/doc_code/ml_models_examples.py | 26 + .../doc_code/model_composition/arithmetic.py | 99 +++ .../doc_code/model_composition/class_nodes.py | 96 +++ .../model_composition/deployment_graph_viz.py | 38 ++ .../monitoring/custom_metric_snippet.py | 37 ++ .../doc_code/monitoring/deployment_logger.py | 29 + .../doc_code/monitoring/metrics_snippet.py | 23 + .../serve/doc_code/monitoring/monitoring.py | 27 + doc/source/serve/doc_code/quickstart.py | 8 +- doc/source/serve/doc_code/quickstart_graph.py | 35 ++ .../serve/doc_code/sklearn_quickstart.py | 7 +- .../serve/doc_code/transformers_example.py | 9 +- doc/source/serve/doc_code/tutorial_batch.py | 34 ++ doc/source/serve/doc_code/tutorial_pytorch.py | 54 ++ doc/source/serve/doc_code/tutorial_sklearn.py | 80 +++ .../serve/doc_code/tutorial_tensorflow.py | 75 +++ .../visualize_dag_during_deployment.py | 37 -- doc/source/serve/faq.md | 77 --- doc/source/serve/getting_started.md | 427 +++++-------- doc/source/serve/handle-guide.md | 81 --- doc/source/serve/http-adapters.md | 146 ----- doc/source/serve/http-guide.md | 274 +++++---- doc/source/serve/index.md | 131 +++- doc/source/serve/key-concepts.md | 136 +++-- doc/source/serve/managing-deployments.md | 197 ------ doc/source/serve/managing-java-deployments.md | 139 +++++ doc/source/serve/migration.md | 104 ++++ doc/source/serve/ml-models.md | 301 --------- doc/source/serve/model_composition.md | 398 ++++++++++++ doc/source/serve/monitoring.md | 266 -------- doc/source/serve/package-ref.md | 186 +----- doc/source/serve/performance.md | 177 ++++-- doc/source/serve/production-guide/config.md | 201 ++++++ doc/source/serve/production-guide/failures.md | 40 ++ doc/source/serve/production-guide/index.md | 119 ++++ .../serve/production-guide/kubernetes.md | 252 ++++++++ .../serve/production-guide/monitoring.md | 300 +++++++++ doc/source/serve/production-guide/rest-api.md | 321 ++++++++++ doc/source/serve/production.md | 554 ----------------- doc/source/serve/rest_api.md | 154 +++++ .../serve/scaling-and-resource-allocation.md | 150 +++++ doc/source/serve/serve_cli.md | 9 + doc/source/serve/tutorials/batch.md | 134 +++- .../tutorials/deployment-graph-patterns.md | 9 + .../branching_input.md | 36 ++ .../deployment-graph-patterns/conditional.md | 72 +++ .../linear_pipeline.md | 25 + .../serve/tutorials/gradio-integration.md | 45 +- doc/source/serve/tutorials/index.md | 10 +- doc/source/serve/tutorials/java.md | 113 ++++ doc/source/serve/tutorials/pytorch.md | 48 -- doc/source/serve/tutorials/rllib.md | 13 +- doc/source/serve/tutorials/serve-ml-models.md | 256 ++++++++ doc/source/serve/tutorials/sklearn.md | 54 -- doc/source/serve/tutorials/tensorflow.md | 56 -- .../serve/tutorials/web-server-integration.md | 33 - doc/source/serve/user-guide.md | 19 +- .../serve/docdemo/HttpStrategyOnRayServe.java | 20 + .../ray/serve/docdemo/ManageDeployment.java | 78 +++ .../serve/docdemo/ManagePythonDeployment.java | 30 + .../java/io/ray/serve/docdemo/Strategy.java | 35 ++ .../io/ray/serve/docdemo/StrategyCalc.java | 28 + .../serve/docdemo/StrategyCalcOnRayServe.java | 92 +++ .../ray/serve/docdemo/StrategyOnRayServe.java | 11 + python/ray/serve/api.py | 9 +- 102 files changed, 5437 insertions(+), 4085 deletions(-) create mode 100644 doc/source/serve/architecture-2.0.svg delete mode 100644 doc/source/serve/architecture.svg delete mode 100644 doc/source/serve/autoscaling.md delete mode 100644 doc/source/serve/deploying-serve.md delete mode 100644 doc/source/serve/deployment-graph.md delete mode 100644 doc/source/serve/deployment-graph/chain_nodes_same_class_different_args.md delete mode 100644 doc/source/serve/deployment-graph/combine_two_nodes_with_passing_input_parallel.md delete mode 100644 doc/source/serve/deployment-graph/control_flow_based_on_user_inputs.md delete mode 100644 doc/source/serve/deployment-graph/deployment-graph-e2e-tutorial.md delete mode 100644 doc/source/serve/deployment-graph/http_endpoint_for_dag_graph.md delete mode 100644 doc/source/serve/deployment-graph/visualize_dag_during_development.md create mode 100644 doc/source/serve/dev-workflow.md create mode 100644 doc/source/serve/doc_code/batching_guide.py delete mode 100644 doc/source/serve/doc_code/create_deployment.py create mode 100644 doc/source/serve/doc_code/deploying_serve_example.py delete mode 100644 doc/source/serve/doc_code/deployment_graph_combine_two_nodes_with_passing_input_parallel.py delete mode 100644 doc/source/serve/doc_code/deployment_graph_control_flow_based_on_user_inputs.py delete mode 100644 doc/source/serve/doc_code/deployment_graph_dag_http.py create mode 100644 doc/source/serve/doc_code/deployment_graph_patterns/branching_input.py create mode 100644 doc/source/serve/doc_code/deployment_graph_patterns/conditional.py create mode 100644 doc/source/serve/doc_code/deployment_graph_patterns/linear_pipeline.py delete mode 100644 doc/source/serve/doc_code/deployment_graph_same_class_different_args.py create mode 100644 doc/source/serve/doc_code/getting_started/model_deployment.py create mode 100644 doc/source/serve/doc_code/getting_started/model_deployment_full.py create mode 100644 doc/source/serve/doc_code/getting_started/model_graph.py create mode 100644 doc/source/serve/doc_code/getting_started/models.py create mode 100644 doc/source/serve/doc_code/handle_guide.py create mode 100644 doc/source/serve/doc_code/http_guide.py create mode 100644 doc/source/serve/doc_code/local_dev.py create mode 100644 doc/source/serve/doc_code/managing_deployments.py create mode 100644 doc/source/serve/doc_code/ml_models_examples.py create mode 100644 doc/source/serve/doc_code/model_composition/arithmetic.py create mode 100644 doc/source/serve/doc_code/model_composition/class_nodes.py create mode 100644 doc/source/serve/doc_code/model_composition/deployment_graph_viz.py create mode 100644 doc/source/serve/doc_code/monitoring/custom_metric_snippet.py create mode 100644 doc/source/serve/doc_code/monitoring/deployment_logger.py create mode 100644 doc/source/serve/doc_code/monitoring/metrics_snippet.py create mode 100644 doc/source/serve/doc_code/monitoring/monitoring.py create mode 100644 doc/source/serve/doc_code/quickstart_graph.py create mode 100644 doc/source/serve/doc_code/tutorial_batch.py create mode 100644 doc/source/serve/doc_code/tutorial_pytorch.py create mode 100644 doc/source/serve/doc_code/tutorial_sklearn.py create mode 100644 doc/source/serve/doc_code/tutorial_tensorflow.py delete mode 100644 doc/source/serve/doc_code/visualize_dag_during_deployment.py delete mode 100644 doc/source/serve/faq.md delete mode 100644 doc/source/serve/handle-guide.md delete mode 100644 doc/source/serve/http-adapters.md delete mode 100644 doc/source/serve/managing-deployments.md create mode 100644 doc/source/serve/managing-java-deployments.md create mode 100644 doc/source/serve/migration.md delete mode 100644 doc/source/serve/ml-models.md create mode 100644 doc/source/serve/model_composition.md delete mode 100644 doc/source/serve/monitoring.md create mode 100644 doc/source/serve/production-guide/config.md create mode 100644 doc/source/serve/production-guide/failures.md create mode 100644 doc/source/serve/production-guide/index.md create mode 100644 doc/source/serve/production-guide/kubernetes.md create mode 100644 doc/source/serve/production-guide/monitoring.md create mode 100644 doc/source/serve/production-guide/rest-api.md delete mode 100644 doc/source/serve/production.md create mode 100644 doc/source/serve/rest_api.md create mode 100644 doc/source/serve/scaling-and-resource-allocation.md create mode 100644 doc/source/serve/serve_cli.md create mode 100644 doc/source/serve/tutorials/deployment-graph-patterns.md create mode 100644 doc/source/serve/tutorials/deployment-graph-patterns/branching_input.md create mode 100644 doc/source/serve/tutorials/deployment-graph-patterns/conditional.md create mode 100644 doc/source/serve/tutorials/deployment-graph-patterns/linear_pipeline.md create mode 100644 doc/source/serve/tutorials/java.md delete mode 100644 doc/source/serve/tutorials/pytorch.md create mode 100644 doc/source/serve/tutorials/serve-ml-models.md delete mode 100644 doc/source/serve/tutorials/sklearn.md delete mode 100644 doc/source/serve/tutorials/tensorflow.md delete mode 100644 doc/source/serve/tutorials/web-server-integration.md create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/HttpStrategyOnRayServe.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/ManagePythonDeployment.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/Strategy.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalc.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java create mode 100644 java/serve/src/test/java/io/ray/serve/docdemo/StrategyOnRayServe.java diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 7657186f6ea57..3396fc461ada6 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -168,28 +168,29 @@ parts: - file: serve/key-concepts - file: serve/user-guide sections: - - file: serve/managing-deployments - - file: serve/handling-dependencies - file: serve/http-guide - - file: serve/http-adapters - - file: serve/handle-guide - - file: serve/ml-models - - file: serve/deploying-serve - - file: serve/monitoring - - file: serve/performance - - file: serve/autoscaling - - file: serve/deployment-graph + - file: serve/scaling-and-resource-allocation + - file: serve/model_composition + - file: serve/dev-workflow + - file: serve/production-guide/index sections: - - file: serve/deployment-graph/deployment-graph-e2e-tutorial - - file: serve/deployment-graph/chain_nodes_same_class_different_args - - file: serve/deployment-graph/combine_two_nodes_with_passing_input_parallel - - file: serve/deployment-graph/control_flow_based_on_user_inputs - - file: serve/deployment-graph/visualize_dag_during_development - - file: serve/deployment-graph/http_endpoint_for_dag_graph - - file: serve/production + - file: serve/production-guide/config + - file: serve/production-guide/rest-api + - file: serve/production-guide/kubernetes + - file: serve/production-guide/monitoring + - file: serve/production-guide/failures + - file: serve/performance + - file: serve/handling-dependencies + - file: serve/managing-java-deployments + - file: serve/migration - file: serve/architecture - file: serve/tutorials/index - - file: serve/faq + sections: + - file: serve/tutorials/deployment-graph-patterns + sections: + - file: serve/tutorials/deployment-graph-patterns/linear_pipeline + - file: serve/tutorials/deployment-graph-patterns/branching_input + - file: serve/tutorials/deployment-graph-patterns/conditional - file: serve/package-ref - file: rllib/index @@ -295,6 +296,7 @@ parts: - file: cluster/running-applications/index title: Applications Guide + - caption: References chapters: - file: ray-references/api diff --git a/doc/source/ray-core/ray-dag.rst b/doc/source/ray-core/ray-dag.rst index 105ecb5334962..7bdda594065c3 100644 --- a/doc/source/ray-core/ray-dag.rst +++ b/doc/source/ray-core/ray-dag.rst @@ -154,6 +154,4 @@ More Resources You can find more application patterns and examples in the following resources from other Ray libraries built on top of Ray DAG API with same mechanism. -| `Visualization of DAGs `_ -| `DAG Cookbook and patterns `_ | `Serve Deployment Graph's original REP `_ diff --git a/doc/source/ray-references/faq.rst b/doc/source/ray-references/faq.rst index 39bc1d60cff9c..9fbf54fc9c60f 100644 --- a/doc/source/ray-references/faq.rst +++ b/doc/source/ray-references/faq.rst @@ -8,7 +8,6 @@ FAQ :caption: Frequently Asked Questions ./../tune/faq.rst - ./../serve/faq.rst Further Questions or Issues? diff --git a/doc/source/serve/architecture-2.0.svg b/doc/source/serve/architecture-2.0.svg new file mode 100644 index 0000000000000..5dc7061d18f2c --- /dev/null +++ b/doc/source/serve/architecture-2.0.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/doc/source/serve/architecture.md b/doc/source/serve/architecture.md index 4995682e5d6fb..b2791b71283e3 100644 --- a/doc/source/serve/architecture.md +++ b/doc/source/serve/architecture.md @@ -1,85 +1,115 @@ (serve-architecture)= -# Serve Architecture +# Architecture -This section should help you: - -- understand an overview of how each component in Serve works -- understand the different types of actors that make up a Serve instance +In this section, we explore Serve's key architectural concepts and components. It will offer insight and overview into: +- the role of each component in Serve and how they work +- the different types of actors that make up a Serve application % Figure source: https://docs.google.com/drawings/d/1jSuBN5dkSj2s9-0eGzlU_ldsRa3TsswQUZM-cMQ29a0/edit?usp=sharing -```{image} architecture.svg +```{image} architecture-2.0.svg :align: center :width: 600px ``` -## High Level View +(serve-architecture-high-level-view)= +## High-Level View Serve runs on Ray and utilizes [Ray actors](actor-guide). There are three kinds of actors that are created to make up a Serve instance: -- Controller: A global actor unique to each Serve instance that manages +- **Controller**: A global actor unique to each Serve instance that manages the control plane. The Controller is responsible for creating, updating, and destroying other actors. Serve API calls like creating or getting a deployment make remote calls to the Controller. -- Router: There is one router per node. Each router is a [Uvicorn](https://www.uvicorn.org/) HTTP +- **HTTP Proxy**: By default there is one HTTP proxy actor on the head node. This actor runs a [Uvicorn](https://www.uvicorn.org/) HTTP server that accepts incoming requests, forwards them to replicas, and - responds once they are completed. -- Worker Replica: Worker replicas actually execute the code in response to a + responds once they are completed. For scalability and high availability, + you can also run a proxy on each node in the cluster via the `location` field of [`http_options`](core-apis). +- **Replicas**: Actors that actually execute the code in response to a request. For example, they may contain an instantiation of an ML model. Each - replica processes individual requests from the routers (they may be batched - by the replica using `@serve.batch`, see the [batching](serve-batching) docs). + replica processes individual requests from the HTTP proxy (these may be batched + by the replica using `@serve.batch`, see the [batching](serve-performance-batching-requests) docs). ## Lifetime of a Request -When an HTTP request is sent to the router, the follow things happen: +When an HTTP request is sent to the HTTP proxy, the following happens: -- The HTTP request is received and parsed. -- The correct deployment associated with the HTTP url path is looked up. The +1. The HTTP request is received and parsed. +2. The correct deployment associated with the HTTP URL path is looked up. The request is placed on a queue. -- For each request in a deployment queue, an available replica is looked up - and the request is sent to it. If there are no available replicas (there - are more than `max_concurrent_queries` requests outstanding), the request - is left in the queue until an outstanding request is finished. +3. For each request in a deployment's queue, an available replica is looked up in round-robin fashion + and the request is sent to it. If there are no available replicas (i.e. there + are more than `max_concurrent_queries` requests outstanding at each replica), the request + is left in the queue until a replica becomes available. -Each replica maintains a queue of requests and executes one at a time, possibly -using asyncio to process them concurrently. If the handler (the function for the -deployment or `__call__`) is `async`, the replica will not wait for the -handler to run; otherwise, the replica will block until the handler returns. +Each replica maintains a queue of requests and executes requests one at a time, possibly +using `asyncio` to process them concurrently. If the handler (the deployment function or the `__call__` method of the deployment class) is declared with `async def`, the replica will not wait for the +handler to run. Otherwise, the replica will block until the handler returns. -## FAQ +When making a request via [ServeHandle](serve-handle-explainer) instead of HTTP, the request is placed on a queue in the ServeHandle, and we skip to step 3 above. (serve-ft-detail)= -### How does Serve handle fault tolerance? +## Fault tolerance Application errors like exceptions in your model evaluation code are caught and wrapped. A 500 status code will be returned with the traceback information. The replica will be able to continue to handle requests. -Machine errors and faults will be handled by Ray. Serve utilizes the [actor -reconstruction](actor-fault-tolerance) capability. For example, when a machine hosting any of the -actors crashes, those actors will be automatically restarted on another +Machine errors and faults will be handled by Ray Serve as follows: + +- When replica actors fail, the Controller actor will replace them with new ones. +- When the HTTP proxy actor fails, the Controller actor will restart it. +- When the Controller actor fails, Ray will restart it. +- When using the [KubeRay RayService](https://ray-project.github.io/kuberay/guidance/rayservice/), KubeRay will recover crashed nodes or a crashed cluster. Cluster crashes can be avoided using the [GCS FT feature](https://ray-project.github.io/kuberay/guidance/gcs-ft/). +- If not using KubeRay, when the Ray cluster fails, Ray Serve cannot recover. + +When a machine hosting any of the actors crashes, those actors will be automatically restarted on another available machine. All data in the Controller (routing policies, deployment -configurations, etc) is checkpointed to the Ray. Transient data in the -router and the replica (like network connections and internal request -queues) will be lost upon failure. +configurations, etc) is checkpointed to the Ray Global Control Store (GCS) on the head node. Transient data in the +router and the replica (like network connections and internal request queues) will be lost for this kind of failure. +See [Serve Health Checking](serve-health-checking) for how actor crashes are detected. + +(serve-autoscaling-architecture)= + +## Ray Serve Autoscaling + +Ray Serve's autoscaling feature automatically increases or decreases a deployment's number of replicas based on its load. + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/autoscaling.svg) + +- The Serve Autoscaler runs in the Serve Controller actor. +- Each ServeHandle and each replica periodically pushes its metrics to the autoscaler. +- For each deployment, the autoscaler periodically checks ServeHandle queues and in-flight queries on replicas to decide whether or not to scale the number of replicas. +- Each ServeHandle continuously polls the controller to check for new deployment replicas. Whenever new replicas are discovered, it will send any buffered or new queries to the replica until `max_concurrent_queries` is reached. Queries are sent to replicas in round-robin fashion, subject to the constraint that no replica is handling more than `max_concurrent_queries` requests at a time. + +:::{note} +When the controller dies, requests can still be sent via HTTP and ServeHandles, but autoscaling will be paused. When the controller recovers, the autoscaling will resume, but all previous metrics collected will be lost. +::: + +## Ray Serve API Server + +Ray Serve provides a [CLI](serve-cli) for managing your Ray Serve instance, as well as a [REST API](serve-rest-api). +Each node in your Ray cluster provides a Serve REST API server that can connect to Serve and respond to Serve REST requests. + +## FAQ ### How does Serve ensure horizontal scalability and availability? -Serve starts one router per node. Each router will bind the same port. You +Serve can be configured to start one HTTP proxy actor per node via the `location` field of [`http_options`](core-apis). Each one will bind the same port. You should be able to reach Serve and send requests to any models via any of the -servers. +servers. You can use your own load balancer on top of Ray Serve. -This architecture ensures horizontal scalability for Serve. You can scale the -router by adding more nodes and scale the model by increasing the number -of replicas. +This architecture ensures horizontal scalability for Serve. You can scale your HTTP ingress by adding more nodes and scale your model inference by increasing the number +of replicas via the `num_replicas` option of your deployment. ### How do ServeHandles work? -{mod}`ServeHandles ` wrap a handle to the router actor on the same node. When a +{mod}`ServeHandles ` wrap a handle to a "router" on the +same node which routes requests to replicas for a deployment. When a request is sent from one replica to another via the handle, the requests go through the same data path as incoming HTTP requests. This enables the same deployment selection and batching procedures to happen. ServeHandles are @@ -89,5 +119,4 @@ often used to implement [model composition](serve-model-composition). Serve utilizes Ray’s [shared memory object store](plasma-store) and in process memory store. Small request objects are directly sent between actors via network -call. Larger request objects (100KiB+) are written to a distributed shared -memory store and the replica can read them via zero-copy read. +call. Larger request objects (100KiB+) are written to the object store and the replica can read them via zero-copy read. diff --git a/doc/source/serve/architecture.svg b/doc/source/serve/architecture.svg deleted file mode 100644 index cbad70ea51725..0000000000000 --- a/doc/source/serve/architecture.svg +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/doc/source/serve/autoscaling.md b/doc/source/serve/autoscaling.md deleted file mode 100644 index ad1fad9ff04e0..0000000000000 --- a/doc/source/serve/autoscaling.md +++ /dev/null @@ -1,48 +0,0 @@ -(serve-autoscaling)= - -# Serve Autoscaling - -This section should help you: - -- Understand how Ray Serve autoscaling works. -- Learn how to configure the parameters for your workload. - - -## Autoscaling architecture -Ray Serve's autoscaling feature automatically increases or decreases a deployment's number of replicas based on its load. - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/autoscaling.svg) - -- Each ServeHandle and each worker replica periodically push the stats to the autoscaler. -- The autoscaler requires ServeHandle queue metrics and replicas queries metrics to make decision whether to scale (up or down) the replicas. -- ServeHandle continues to poll the updated group of replicas from the controller. Upon discovery of the new replicas, it will send any buffered or new queries to the replica until `max_concurrent_queries` is reached - -:::{note} -When the controller dies, the client will still be able to send queries, but autoscaling will be paused. When the controller recovers, the autoscaling will resume, but all previous metrics collected will be lost. -::: - -## Autoscaling parameters -There are several parameters the autoscaling algorithm takes into consideration when deciding the target replicas for your deployment - -**min_replicas[default_value=1]**: The minimal number of replicas for the deployment. ``min_replicas`` will also be the initial number of replicas when the deployment is deployed. -:::{note} -Ray Serve Autoscaling allows the `min_replicas` to be 0 when starting your deployment; the scale up will be started when you start sending traffic. There will be a cold start time as the Ray ServeHandle waits (blocks) for available replicas to assign the request. -::: -**max_replicas[default_value=1]**: Max replicas is the maximum number of replicas for the deployment. Ray Serve Autoscaling will rely on the Ray Autoscaler to scale up more nodes when the currently available cluster resources (CPUs, GPUs, etc) are not enough to bring up more replicas. - -**target_num_ongoing_requests_per_replica[default_value=1]**: The config is to maintain how many ongoing requests are expected to run concurrently per replica at most. If the number is lower, the scale up will be done more aggressively. -:::{note} -- It is always recommended to load test your workloads. For example, if the use case is latency sensitive, you can lower the `target_num_ongoing_requests_per_replica` number to maintain high performance. -- Internally, the autoscaler will compare RUNNING + PENNING tasks of each replicas and `target_num_ongoing_requests_per_replica` to decide scale up/down. -- `target_num_ongoing_requests_per_replica` is only a target value used for autoscaling (not a hard limit), the real ongoing requests number can be higher than the config. -::: - -**downscale_delay_s[default_value=600.0]**: The config is to control how long the cluster needs to wait before scaling down replicas. - -**upscale_delay_s[default_value=30.0]**: The config is to control how long the cluster need to wait before scaling up replicas. -:::{note} -`downscale_delay_s` and `upscale_delay_s` are to control the frequency of doing autoscaling work. E.g. if your use case takes a long time to do initialization work, you can increase `downscale_delay_s` to make the down scaling happen slowly. -::: -**smoothing_factor[default_value=1]**: The multiplicative factor to speedup/slowdown each autoscaling step. E.g. When the use case has high large traffic volume in short period of time, you can increase `smoothing_factor` to scale up the resource quickly. - -**metrics_interval_s[default_value=10]**: This is control how often each replica sends metrics to the autoscaler. (Normally you don't need to change this config.) diff --git a/doc/source/serve/deploying-serve.md b/doc/source/serve/deploying-serve.md deleted file mode 100644 index 58c38ea8560af..0000000000000 --- a/doc/source/serve/deploying-serve.md +++ /dev/null @@ -1,271 +0,0 @@ -(serve-deploy-tutorial)= - -# Deploying Ray Serve - -This section should help you: - -- understand how Ray Serve runs on a Ray cluster beyond the basics -- deploy and update your Serve application over time -- monitor your Serve application using the Ray Dashboard and logging - -```{contents} Deploying Ray Serve -``` - -(ray-serve-instance-lifetime)= - -## Lifetime of a Ray Serve Instance - -Ray Serve instances run on top of Ray clusters and are started using {mod}`serve.start `. -Once {mod}`serve.start ` has been called, further API calls can be used to create and update the deployments that will be used to serve your Python code (including ML models). -The Serve instance will be torn down when the script exits. - -When running on a long-lived Ray cluster (e.g., one started using `ray start`), -you can also deploy a Ray Serve instance as a long-running -service using `serve.start(detached=True)`. In this case, the Serve instance will continue to -run on the Ray cluster even after the script that calls it exits. If you want to run another script -to update the Serve instance, you can run another script that connects to the same Ray cluster and makes further API calls (e.g., to create, update, or delete a deployment). Note that there can only be one detached Serve instance on each Ray cluster. - -:::{note} -All Serve actors– including the Serve controller, the HTTP proxies, and the deployment replicas– run in the `"serve"` namespace, even if the Ray driver namespace is different. -::: - -If `serve.start()` is called again in a process in which there is already a running Serve instance, Serve will re-connect to the existing instance (regardless of whether the original instance was detached or not). To reconnect to a Serve instance that exists in the Ray cluster but not in the current process, connect to the cluster and run `serve.start()`. - -## Deploying on a Single Node - -While Ray Serve makes it easy to scale out on a multi-node Ray cluster, in some scenarios a single node may suit your needs. -There are two ways you can run Ray Serve on a single node, shown below. -In general, **Option 2 is recommended for most users** because it allows you to fully make use of Serve's ability to dynamically update running deployments. - -1. Start Ray and deploy with Ray Serve all in a single Python file. - -```python -import ray -from ray import serve -import time - -# This will start Ray locally and start Serve on top of it. -serve.start() - -@serve.deployment -def my_func(request): - return "hello" - -my_func.deploy() - -# Serve will be shut down once the script exits, so keep it alive manually. -while True: - time.sleep(5) - print(serve.list_deployments()) -``` - -2. First running `ray start --head` on the machine, then connecting to the running local Ray cluster using `ray.init(address="auto")` in your Serve script(s). You can run multiple scripts to update your deployments over time. - -```bash -ray start --head # Start local Ray cluster. -serve start # Start Serve on the local Ray cluster. -``` - -```python -import ray -from ray import serve - -# This will connect to the running Ray cluster. -ray.init(address="auto", namespace="serve") - -@serve.deployment -def my_func(request): - return "hello" - -my_func.deploy() -``` - -## Deploying on Kubernetes - -In order to deploy Ray Serve on Kubernetes, we need to do the following: - -1. Start a Ray cluster on Kubernetes. -2. Expose the head node of the cluster as a [Service]. -3. Start Ray Serve on the cluster. - -There are multiple ways to start a Ray cluster on Kubernetes, see {ref}`kuberay-index` for more information. -Here, we will be using the [Ray Cluster Launcher](cluster-index) tool, which has support for Kubernetes as a backend. - -The cluster launcher takes in a yaml config file that describes the cluster. -Here, we'll be using the [Kubernetes default config] with a few small modifications. -First, we need to make sure that the head node of the cluster, where Ray Serve will run its HTTP server, is exposed as a Kubernetes [Service]. -There is already a default head node service defined in the `services` field of the config, so we just need to make sure that it's exposing the right port: 8000, which Ray Serve binds on by default. - -```yaml -# Service that maps to the head node of the Ray cluster. -- apiVersion: v1 - kind: Service - metadata: - name: ray-head - spec: - # Must match the label in the head pod spec below. - selector: - component: ray-head - ports: - - protocol: TCP - # Port that this service will listen on. - port: 8000 - # Port that requests will be sent to in pods backing the service. - targetPort: 8000 -``` - -Then, we also need to make sure that the head node pod spec matches the selector defined here and exposes the same port: - -```yaml -head_node: - apiVersion: v1 - kind: Pod - metadata: - # Automatically generates a name for the pod with this prefix. - generateName: ray-head- - - # Matches the selector in the service definition above. - labels: - component: ray-head - - spec: - # ... - containers: - - name: ray-node - # ... - ports: - - containerPort: 8000 # Ray Serve default port. - # ... -``` - -The rest of the config remains unchanged for this example, though you may want to change the container image or the number of worker pods started by default when running your own deployment. -Now, we just need to start the cluster: - -```shell -# Start the cluster. -$ ray up ray/python/ray/autoscaler/kubernetes/example-full.yaml - -# Check the status of the service pointing to the head node. If configured -# properly, you should see the 'Endpoints' field populated with an IP -# address like below. If not, make sure the head node pod started -# successfully and the selector/labels match. -$ kubectl -n ray describe service ray-head - Name: ray-head - Namespace: ray - Labels: - Annotations: - Selector: component=ray-head - Type: ClusterIP - IP: 10.100.188.203 - Port: 8000/TCP - TargetPort: 8000/TCP - Endpoints: 192.168.73.98:8000 - Session Affinity: None - Events: -``` - -With the cluster now running, we can run a simple script to start Ray Serve and deploy a "hello world" deployment: - -> ```python -> import ray -> from ray import serve -> -> # Connect to the running Ray cluster. -> ray.init(address="auto") -> # Bind on 0.0.0.0 to expose the HTTP server on external IPs. -> serve.start(detached=True, http_options={"host": "0.0.0.0"}) -> -> -> @serve.deployment(route_prefix="/hello") -> def hello(request): -> return "hello world" -> -> hello.deploy() -> ``` - -Save this script locally as `deploy.py` and run it on the head node using `ray submit`: - -> ```shell -> $ ray submit ray/python/ray/autoscaler/kubernetes/example-full.yaml deploy.py -> ``` - -Now we can try querying the service by sending an HTTP request to the service from within the Kubernetes cluster. - -> ```shell -> # Get a shell inside of the head node. -> $ ray attach ray/python/ray/autoscaler/kubernetes/example-full.yaml -> -> # Query the Ray Serve deployment. This can be run from anywhere in the -> # Kubernetes cluster. -> $ curl -X GET http://$RAY_HEAD_SERVICE_HOST:8000/hello -> hello world -> ``` - -In order to expose the Ray Serve deployment externally, we would need to deploy the Service we created here behind an [Ingress] or a [NodePort]. -Please refer to the Kubernetes documentation for more information. - -## Health Checking - -By default, each actor making up a Serve deployment is health checked and restarted on failure. - -:::{note} -User-defined health checks are experimental and may be subject to change before the interface is stabilized. If you have any feedback or run into any issues or unexpected behaviors, please file an issue on GitHub. -::: - -You can customize this behavior to perform an application-level health check or to adjust the frequency/timeout. -To define a custom healthcheck, define a `check_health` method on your deployment class. -This method should take no arguments and return no result, raising an exception if the replica should be considered unhealthy. -You can also customize how frequently the health check is run and the timeout when a replica will be deemed unhealthy if it hasn't responded in the deployment options. - -> ```python -> @serve.deployment(health_check_period_s=10, health_check_timeout_s=30) -> class MyDeployment: -> def __init__(self, db_addr: str): -> self._my_db_connection = connect_to_db(db_addr) -> -> def __call__(self, request): -> return self._do_something_cool() -> -> # Will be called by Serve to check the health of the replica. -> def check_health(self): -> if not self._my_db_connection.is_connected(): -> # The specific type of exception is not important. -> raise RuntimeError("uh-oh, DB connection is broken.") -> ``` - -:::{tip} -You can use the Serve CLI command `serve status` to get status info -about your live deployments. The CLI was included with Serve when you did -`pip install "ray[serve]"`. If you're checking your deployments on a -remote Ray cluster, make sure to include the Ray cluster's dashboard address -in the command: `serve status --address [dashboard_address]`. -::: - -## Failure Recovery - -Ray Serve is resilient to any component failures within the Ray cluster out of the box. -You can checkout the detail of how process and worker node failure handled at {ref}`serve-ft-detail`. -However, when the Ray head node goes down, you would need to recover the state by creating a new -Ray cluster and re-deploys all Serve deployments into that cluster. - -:::{note} -Ray currently cannot survive head node failure and we recommend using application specific -failure recovery solutions. Although Ray is not currently highly available (HA), it is on -the long term roadmap and being actively worked on. -::: - -Ray Serve provides the feature to help recovering the state. -This feature enables Serve to write all your deployment configuration and code into Global Control Store -(GCS). -Upon Ray cluster failure and restarts, you can simply call Serve to reconstruct the state. - - -In Kubernetes environment, we recommend using KubeRay (a Kubernetes operator for Ray Serve) to help deploy your Serve applications with Kubernetes, and help you recover the node crash from Customized Resource. - -Feel free to open new github issues if you hit any problems from Failure Recovery. - -[ingress]: https://kubernetes.io/docs/concepts/services-networking/ingress/ -[kubernetes default config]: https://github.com/ray-project/ray/blob/master/python/ray/autoscaler/kubernetes/example-full.yaml -[nodeport]: https://kubernetes.io/docs/concepts/services-networking/service/#publishing-services-service-types -[persistent volumes]: https://kubernetes.io/docs/concepts/storage/persistent-volumes/ -[service]: https://kubernetes.io/docs/concepts/services-networking/service/ diff --git a/doc/source/serve/deployment-graph.md b/doc/source/serve/deployment-graph.md deleted file mode 100644 index 803aa31006802..0000000000000 --- a/doc/source/serve/deployment-graph.md +++ /dev/null @@ -1,17 +0,0 @@ -(serve-deployment-graph)= - -# Deployment Graph - -To learn more about deployment graph in an end-to-end walkthrough: - -- [E2E Tutorials](./deployment-graph/deployment-graph-e2e-tutorial.md) - -## Patterns - -Jump striaght into a common design patterns using deployment graph: - -- [Chain nodes with same class and different args](deployment-graph/chain_nodes_same_class_different_args.md) -- [Combine two nodes with passing same input in parallel](deployment-graph/combine_two_nodes_with_passing_input_parallel.md) -- [Control flow based on user inputs](deployment-graph/control_flow_based_on_user_inputs.md) -- [Visualize DAG during development](deployment-graph/visualize_dag_during_development.md) -- [Http endpoint for dag graph](deployment-graph/http_endpoint_for_dag_graph.md) diff --git a/doc/source/serve/deployment-graph/chain_nodes_same_class_different_args.md b/doc/source/serve/deployment-graph/chain_nodes_same_class_different_args.md deleted file mode 100644 index 7038a89ad15b4..0000000000000 --- a/doc/source/serve/deployment-graph/chain_nodes_same_class_different_args.md +++ /dev/null @@ -1,25 +0,0 @@ -# Pattern: Chain nodes with same class and different args - -The example shows how to chain nodes using the same class and having different args passed in - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/chain_nodes_same_class_different_args.svg) - -## Code - -+++ - -```{eval-rst} -.. literalinclude:: ../doc_code/deployment_graph_same_class_different_args.py - :language: python -``` - -## Outputs - -The graph will add all nodes weights plus the input (which is 0 in this case). -0(input) + 0(weight) + 1(weight) + 2(weight) = 3 - -``` -3 -``` - -+++ \ No newline at end of file diff --git a/doc/source/serve/deployment-graph/combine_two_nodes_with_passing_input_parallel.md b/doc/source/serve/deployment-graph/combine_two_nodes_with_passing_input_parallel.md deleted file mode 100644 index 00221ccc39050..0000000000000 --- a/doc/source/serve/deployment-graph/combine_two_nodes_with_passing_input_parallel.md +++ /dev/null @@ -1,27 +0,0 @@ -# Pattern: Combine two nodes with passing same input in parallel - -The example shows how to pass same input to two nodes in parallel and combine the outputs - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/deployment_graph_combine_two_nodes_with_passing_same_input_parallel.svg) - -## Code - -+++ - -```{eval-rst} -.. literalinclude:: ../doc_code/deployment_graph_combine_two_nodes_with_passing_input_parallel.py - :language: python -``` - -## Outputs - -The graph will pass input into two nodes and sum the outputs of the two model.\ -Model output1: 1(input) + 0(weight) = 1 \ -Model output2: 1(input) + 1(weight) = 2 \ -Combine sum: 1 (output1) + 2 (output2) = 3 - -``` -3 -``` - -+++ \ No newline at end of file diff --git a/doc/source/serve/deployment-graph/control_flow_based_on_user_inputs.md b/doc/source/serve/deployment-graph/control_flow_based_on_user_inputs.md deleted file mode 100644 index dc8dc2d613ec3..0000000000000 --- a/doc/source/serve/deployment-graph/control_flow_based_on_user_inputs.md +++ /dev/null @@ -1,54 +0,0 @@ -# Pattern: Control flow based on the user inputs - -The example shows how to use inputs to control the graph flow - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/control_flow_based_on_user_inputs.svg) - -## Code - -+++ - -```{eval-rst} -.. literalinclude:: ../doc_code/deployment_graph_control_flow_based_on_user_inputs.py - :language: python -``` - -````{note} -1. The dag.execute() take arbitrary number of arguments, and internally we implemented data objects to facilitate accessing by index or key. - - - code example: - ```python - dag = combine.bind(output1, output2, user_input[1]) - ``` - - -2. value1 and value2 are ObjectRef passed into the combine, the value of ObjectRef will be resolved at the runtime. - -3. we can pass value1 and value2 as a list. In this case, we are passing the ObjectRef as reference, the value of ObjectRef will not be addressed automatically. We need to explicitly use ray.get() to address value before we do sum() or max() function. ([passing objects by reference](https://docs.ray.io/en/latest/ray-core/objects.html?#passing-objects-by-reference)) - - code example: - ```python - dag = combine.bind([output1, output2], user_input[1]) - ... - @serve.deployment - def combine(value_refs, combine_type): - values = ray.get(value_refs) - ... - ``` -```` - -## Outputs - -The code uses 'max' to do combine from the output of the two models. - -Model output1: 1(input) + 0(weight) = 1 \ -Model output2: 1(input) + 1(weight) = 2 \ -So the combine max is 2, the combine sum is 3 - -``` -2 -3 -``` - -+++ \ No newline at end of file diff --git a/doc/source/serve/deployment-graph/deployment-graph-e2e-tutorial.md b/doc/source/serve/deployment-graph/deployment-graph-e2e-tutorial.md deleted file mode 100644 index faa94cfcafddb..0000000000000 --- a/doc/source/serve/deployment-graph/deployment-graph-e2e-tutorial.md +++ /dev/null @@ -1,575 +0,0 @@ ---- -jupytext: - formats: ipynb,md:myst - text_representation: - extension: .md - format_name: myst - format_version: 0.13 - jupytext_version: 1.13.6 -kernelspec: - display_name: Python 3 - language: python - name: python3 ---- - -(deployment-graph-e2e-tutorial)= - -# Deployment Graph E2E Tutorial - -```{note} -Note: This feature is in Alpha, so APIs are subject to change. -``` - -## Motivation - -Machine learning serving systems are getting longer and wider. They often consist of many models to make a single prediction. This is common in use cases like image / video content classification and tagging, fraud detection pipeline with multiple policies, multi-stage ranking and recommendation, etc. - -Meanwhile, the size of a model is also growing beyond the memory limit of a single machine due to the exponentially growing number of parameters. GPT-3 and sparse feature embeddings in large recommendation models are two prime examples. - -Ray has unique strengths suited to distributed inference pipelines: flexible scheduling, efficient communication, and shared memory. Ray Serve leverages these strengths to build inference graphs, enabling users to develop complex ML applications locally and then deploy them to production with dynamic scaling and lightweight updates (e.g., for model weights). - -## Features -- Provide the ability to build, test, and deploy a complex inference graph of deployments both locally and on remote cluster. The authoring experience is fully Python-programmable and support dynamic control flow and custom business logic, all without writing YAML. -- In production, the deployments making up the graph can be reconfigured and scaled dynamically. This should enable DevOps/MLOps teams to operate deployment graphs without modifying the underlying code. - - -__[Full Ray Enhancement Proposal, REP-001: Serve Pipeline](https://github.com/ray-project/enhancements/blob/main/reps/2022-03-08-serve_pipeline.md)__ - -+++ - -## Concepts - -- **Deployment**: Scalable, upgradeable group of actors managed by Ray Serve. __[See docs for detail](https://docs.ray.io/en/master/serve/package-ref.html#deployment-api)__ - -- **DeploymentNode**: Smallest unit in a graph, created by calling `.bind()` on a serve decorated class or function, backed by a Deployment. - -- **InputNode**: A special node that represents the input passed to a graph at runtime. - -- **Deployment Graph**: Collection of deployment nodes bound together to define an inference graph. The graph can be deployed behind an HTTP endpoint and reconfigured/scaled dynamically. - -+++ - -## Full End to End Example Walkthrough - -Let's put the concepts together and incrementally build a runnable DAG example highlighting the following features: - -```{tip} -At the end of this document we have the full and end to end executable implementation. -``` - -- Building a graph: - - A deployment node is created from `@serve.deployment` decorated function or class. - - You can construct different deployment nodes from same class or function. - - Deployment nodes can be used as input args in .bind() of other nodes in the DAG. - - Multiple nodes in the deployment graph naturally forms a DAG structure. -- Accessing input: - - Same input or output can be used in multiple nodes in the DAG. - - Deployment nodes can access partial user input. -- Dynamic control flow: - - A deployment node can call into other nodes in the deployment graph. - - You can use the dynamic calling to perform control flow operation that's hard to expressive in traditional DAG. -- Running a DAG: - - Nodes in the graph, such as functions or class methods, can be either sync or async. - -+++ - -![deployment graph](https://github.com/ray-project/images/blob/master/docs/serve/deployment_graph.png?raw=true) - -+++ - -### Step 1: User InputNode and preprocessor - -Let's start with the first layer of DAG: Building user input to two preprocessor functions, where each function receives parts of the input data. For simplicity, we use the same existing `@serve.deployment` decorator on an async function body. - -+++ - -```python -import asyncio -from ray import serve -# We will later move Ray DAG related components -# out of experimental in later stable release -from ray.serve.dag import InputNode - -@serve.deployment -async def preprocessor(input_data: str): - """Simple feature processing that converts str to int""" - await asyncio.sleep(0.1) # Manual delay for blocking computation - return int(input_data) - -@serve.deployment -async def avg_preprocessor(input_data): - """Simple feature processing that returns average of input list as float.""" - await asyncio.sleep(0.15) # Manual delay for blocking computation - return sum(input_data) / len(input_data) - -# DAG building -with InputNode() as dag_input: - # Partial access of user input by index - preprocessed_1 = preprocessor.bind(dag_input[0]) - preprocessed_2 = avg_preprocessor.bind(dag_input[1]) -``` - -+++ - -There are two new APIs used in the DAG building stage: `InputNode()` and `bind()`. - -### **```InputNode()```** : User input of the graph - -```InputNode``` is a special node in the graph that represents the user input for the graph at runtime. There can only be one for each graph, takes no arguments, and should always be created in a context manager. - -It's possible to access partial inputs by index or key if every node in the graph doesn't require the full input. Example: the input consists of `[Tensor_0, Tensor_1]` but a single model might only need to receive `Tensor_1` at rather than the full list. - - -### **`bind(*args, **kwargs)`** : The graph building API - -Once called on supported Ray-decorated function or class (`@serve.deployment` is fully supported, `@ray.remote` will be soon), generates a `DeploymentNode` of type `DAGNode` that acts as the building block of graph building. - - -In the example above, we can see we're using a context manager to build and bind user input: -```python -with InputNode() as dag_input: -``` - -Which can be used and accessed by index or key in downstream calls of .bind(), such as: -```python -preprocessed_1 = preprocessor.bind(dag_input[0]) -``` - -This means we're creating a DeploymentNode called `preprocessed_1` in graph building by calling `.bind()` on a serve decorated function, where it executes the decorated deployment function `preprocessor` that takes the user input at index 0 at runtime. - -#### bind() on function - -```bind()``` on function produces a DeploymentNode that can be executed with user input. - -```{tip} -Each deployment node used in graph is individually scalable and configurable by default. This means in real production workload where we can expect difference in compute resource and latency, we can fine tune the nodes to optimal `num_replicas` and `num_cpus` to avoid a single node being the bottleneck of your deployment graph's latency or throughput. -``` - -#### bind() on class constructor - -**`Class.bind(*args, **kwargs)`** constructs and returns a DeploymentNode that acts as the instantiated instance of Class, where `*args` and `**kwargs` are used as init args. In our implementation, we have - -```python -m1 = Model.bind(1) -m2 = Model.bind(2) -``` - -This means we're creating two more `DeploymentNode` of an instance of `Model` that is constructed with init arg of `1` and `2`, and refereced with variable name `m1`, `m2` respectively. - -#### bind() on class method - -Once a class is bound with its init args, its class methods can be directly accessed, called or bound with other args. It has the same semantics as `bind()` on a function, except it acts on an instantiated `DeploymentNode` class instead. - -+++ - -### Step 2: Model and combiner class instantiation - -After we got the preprocessed inputs, we're ready to combine them to construct request object we want to sent to two models instantiated with different initial weights. This means we need: - -(1) Two `Model` instances in the graph instantiated with different initial weights -
-(2) A `Combiner` that refereces `Model` nodes for its runtime implementation by passing them as init args in `.bind()` -
-(3) The ability of `Combiner` to receive and merge preprocessed inputs for the same user input, even they might be produced async and received out of order. - -+++ - -```python -# ... previous nodes implementation skipped - -@serve.deployment -class Model: - def __init__(self, weight: int): - self.weight = weight - - async def forward(self, input: int): - await asyncio.sleep(0.3) # Manual delay for blocking computation - return f"({self.weight} * {input})" - - -@serve.deployment -class Combiner: - def __init__(self, m1: Model, m2: Model): - self.m1 = m1 - self.m2 = m2 - - async def run(self, req_part_1, req_part_2, operation): - # Merge model input from two preprocessors - req = f"({req_part_1} + {req_part_2})" - - # Submit to both m1 and m2 with same req data in parallel - r1_ref = self.m1.forward.remote(req) - r2_ref = self.m2.forward.remote(req) - - # Async gathering of model forward results for same request data - rst = await asyncio.gather(*[r1_ref, r2_ref]) - - -# DAG building -with InputNode() as dag_input: - # Partial access of user input by index - preprocessed_1 = preprocessor.bind(dag_input[0]) - preprocessed_2 = avg_preprocessor.bind(dag_input[1]) - m1 = Model.bind(1) - m2 = Model.bind(2) - combiner = Combiner.bind(m1, m2) - dag = combiner.run.bind(preprocessed_1, preprocessed_2, dag_input[2]) -``` - -+++ - -We are adding a few more pieces to our dag builder: `bind()` on class and class method, as well as passing the output of `Model.bind()` as init args into another class `Combiner.bind()` - - -### DeploymentNode as arguments in other node's bind() - -DeploymentNode can also be passed into other `DeploymentNode` in dag binding. In the full example below, ```Combiner``` calls into two instantiations of ```Model``` class, which can be bound and passed into ```Combiner```'s constructor as if we're passing in two regular python class instances. - -```python -m1 = Model.bind(1) -m2 = Model.bind(2) -combiner = Combiner.bind(m1, m2) -``` - -Similarly, we can also pass and bind upstream `DeploymentNode` results that will be resolved upon runtime to downstream DeploymentNodes, in our example, a function `run()` that access class method of ```Combiner``` class takes two preprocessing `DeploymentNode`s' output as well as part of user input that will be resolved when upstream `DeploymentNode` are executed. - -```python -preprocessed_1 = preprocessor.bind(dag_input[0]) -preprocessed_2 = avg_preprocessor.bind(dag_input[1]) -... -dag = combiner.run.bind(preprocessed_1, preprocessed_2, dag_input[2]) -``` - -```{tip} -At runtime, calls of deployment node like ` self.m1.forward.remote()` will be automatically replaced with a `.remote()` call to the deployment handle to `self.m1`. -``` - -+++ - -### Step 3: Dynamic aggregation based on user input - -Now we have the backbone of our DAG setup: splitting and preprocessing user inputs, aggregate into new request data and send to multiple models downstream. Let's add a bit more dynamic flavor in it to demostrate deployment graph is fully python programmable by introducing control flow based on user input. - -It's as simple as adding a plain `if / else` on `combiner.run()`, and from our previous binding, we get `operation` field at runtime from user provided input data. - -```python -dag = combiner.run.bind(preprocessed_1, preprocessed_2, dag_input[2]) -``` - -+++ - -```python -@serve.deployment -class Combiner: - ... - - async def run(self, req_part_1, req_part_2, operation): - # Merge model input from two preprocessors - req = f"({req_part_1} + {req_part_2})" - - # Submit to both m1 and m2 with same req data in parallel - r1_ref = self.m1.forward.remote(req) - r2_ref = self.m2.forward.remote(req) - - # Async gathering of model forward results for same request data - rst = await asyncio.gather(*[r1_ref, r2_ref]) - - # Control flow that determines runtime behavior based on user input - if operation == "sum": - return f"sum({rst})" - else: - return f"max({rst})" -``` - -+++ - -```{tip} -Support control flow in plain python code can be very useful to build dynamic dispatcher, such as routing user request to a smaller subset of running models based on request attribute, where each model can be sharded and scaled independently. -``` - -+++ - -### Step 4: Driver deployment to handle http ingress - -Now we've built the entire serve DAG with the topology, args binding and user input. It's time to add the last piece for serve -- a Driver deployment to expose and configure http. We can configure it to start with two replicas in case the ingress of deployment becomes bottleneck of the DAG. - -Serve provides a default DAGDriver implementation that accepts HTTP request and orchestrate the deployment graph execution. You can import it from `from ray.serve.drivers import DAGDriver`. - -You can configure how does the DAGDriver convert HTTP request types. By default, we directly send in a [```starlette.requests.Request```](https://www.starlette.io/requests/) object to represent the whole request. You can also specifies built-in adapters. In this example, we will use a `json_request` adapter that parses HTTP body with JSON parser. - -```{tip} -There are several useful adapters like ndarray JSON, image object, etc. You can checkout [the list of adapters here](serve-http-adapters). You can also easily plug in your own adapter by passing in in the ```http_adapter``` field. -``` - -+++ - -```python -from ray.serve.drivers import DAGDriver -from ray.serve.http_adapters import json_request - -# DAG building -with InputNode() as dag_input: - ... - dag = combiner.run.bind( - preprocessed_1, preprocessed_2, dag_input[2] # Partial access of user input by index - ) - - # Each serve dag has a driver deployment as ingress that can be user provided. - serve_dag = DAGDriver.options(route_prefix="/my-dag", num_replicas=2).bind( - dag, http_adapter=json_request - ) - -``` - -+++ - - - -````{note} -Even though we provide a default `DAGDriver`, you don't have to use it in your Serve DAG. You can easily bring your own Serve deployment to act as the root node. You can find more information about configuring HTTP deployment at {ref}`serve-http`. For example, a simple DAG Driver that parse JSON input can be implemented as follows: - -```python -@serve.deployment(route_prefix="/my-dag") -class MyDAGDriver: - def __init__(self, dag_handle): - self.dag_handle = dag_handle - - async def predict(self, inp): - """Perform inference directly without HTTP.""" - return await self.dag_handle.remote(inp) - - async def __call__(self, request: starlette.requests.Request): - """HTTP endpoint of the DAG.""" - input_data = await request.json() - return await self.predict(input_data) - -# DAG building -with InputNode() as dag_input: - ... - dag = combiner.run.bind(...) - - serve_dag = MyDAGDriver.bind(dag) -``` - -```` - -### Step 5: Test the full DAG in both python and http - -We can now test and deploy the graph using `serve.run`: - -+++ - -### **```serve.run()```** : running the deployment graph - -The deployment graph can be deployed with ```serve.run()```. It -takes in a target `DeploymentNode`, and it deploys the node's deployments, as -well as all its child nodes' deployments. To deploy your graph, pass in the -root DeploymentNode into ```serve.run()```: - -```python -with InputNode() as dag_input: - serve_dag = ... - -dag_handle = serve.run(serve_dag) -``` - -```serve.run()``` returns the passed-in node's deployment's handle. You can use -this handle to issue requests to the deployment: - -```python -ray.get(dag_handle.remote(user_input)) -``` - -During development, you can also use the Serve CLI to run your deployment -graph. The CLI was included with Serve when you did ``pip install "ray[serve]"``. -The command ```serve run [node import path]``` will deploy the node and its -childrens' deployments. For example, we can remove the ```serve.run()``` calls -inside the Python script and save our example pipeline to a file called -example.py. Then we can run the driver DeploymentNode using its import path, -```example.serve_dag```: - -```bash -$ serve run example.serve_dag -``` - -+++ - -```{tip} -The CLI expects the import path to either be a Python module on your system -or a relative import from the command line's current working directory. You can -change the directory that the CLI searches using the ```--app-dir``` flag. -The command will block on the terminal window and periodically print all the -deployments' health statuses. You can open a separate terminal window and -issue HTTP requests to your deployments -``` - -+++ - -```bash -$ python ->>> import requests ->>> requests.post("http://127.0.0.1:8000/my-dag", json=["1", [0, 2], "max"]).text -``` -The CLI's ```serve run``` tool has useful flags to configure which Ray cluster -to run on, which runtime_env to use, and more. Use ```serve run --help``` to get -more info on these options. - -+++ - -## Full End to End Example Code - -Now we're done! The full example below covers the full example for you to try out. - - -```{code-cell} ipython3 -import time -import asyncio -import requests -import starlette - -import ray -from ray import serve -from ray.serve.dag import InputNode -from ray.serve.drivers import DAGDriver -from ray.serve.http_adapters import json_request - -ray.init(num_cpus=16) -serve.start() - -@serve.deployment -async def preprocessor(input_data: str): - """Simple feature processing that converts str to int""" - await asyncio.sleep(0.1) # Manual delay for blocking computation - return int(input_data) - -@serve.deployment -async def avg_preprocessor(input_data): - """Simple feature processing that returns average of input list as float.""" - await asyncio.sleep(0.15) # Manual delay for blocking computation - return sum(input_data) / len(input_data) - -@serve.deployment -class Model: - def __init__(self, weight: int): - self.weight = weight - - async def forward(self, input: int): - await asyncio.sleep(0.3) # Manual delay for blocking computation - return f"({self.weight} * {input})" - - -@serve.deployment -class Combiner: - def __init__(self, m1: Model, m2: Model): - self.m1 = m1 - self.m2 = m2 - - async def run(self, req_part_1, req_part_2, operation): - # Merge model input from two preprocessors - req = f"({req_part_1} + {req_part_2})" - - # Submit to both m1 and m2 with same req data in parallel - r1_ref = self.m1.forward.remote(req) - r2_ref = self.m2.forward.remote(req) - - # Async gathering of model forward results for same request data - rst = await asyncio.gather(r1_ref, r2_ref) - - # Control flow that determines runtime behavior based on user input - if operation == "sum": - return f"sum({rst})" - else: - return f"max({rst})" - -# DAG building -with InputNode() as dag_input: - # Partial access of user input by index - preprocessed_1 = preprocessor.bind(dag_input[0]) - preprocessed_2 = avg_preprocessor.bind(dag_input[1]) - # Multiple instantiation of the same class with different args - m1 = Model.bind(1) - m2 = Model.bind(2) - # Use other DeploymentNode in bind() - combiner = Combiner.bind(m1, m2) - # Use output of function DeploymentNode in bind() - dag = combiner.run.bind( - preprocessed_1, preprocessed_2, dag_input[2] - ) - - # Each serve dag has a driver deployment as ingress that can be user provided. - serve_dag = DAGDriver.options(route_prefix="/my-dag", num_replicas=2).bind( - dag, http_adapter=json_request - ) - - -dag_handle = serve.run(serve_dag) - -# Warm up -ray.get(dag_handle.predict.remote(["0", [0, 0], "sum"])) - -# Python handle -cur = time.time() -print(ray.get(dag_handle.predict.remote(["5", [1, 2], "sum"]))) -print(f"Time spent: {round(time.time() - cur, 2)} secs.") -# Http endpoint -cur = time.time() -print(requests.post("http://127.0.0.1:8000/my-dag", json=["5", [1, 2], "sum"]).text) -print(f"Time spent: {round(time.time() - cur, 2)} secs.") - -# Python handle -cur = time.time() -print(ray.get(dag_handle.predict.remote(["1", [0, 2], "max"]))) -print(f"Time spent: {round(time.time() - cur, 2)} secs.") - -# Http endpoint -cur = time.time() -print(requests.post("http://127.0.0.1:8000/my-dag", json=["1", [0, 2], "max"]).text) -print(f"Time spent: {round(time.time() - cur, 2)} secs.") -``` - -## Outputs - -``` -sum(['(1 * (5 + 1.5))', '(2 * (5 + 1.5))']) -Time spent: 0.49 secs. -sum(['(1 * (5 + 1.5))', '(2 * (5 + 1.5))']) -Time spent: 0.49 secs. - - -max(['(1 * (1 + 1.0))', '(2 * (1 + 1.0))']) -Time spent: 0.48 secs. -max(['(1 * (1 + 1.0))', '(2 * (1 + 1.0))']) -Time spent: 0.48 secs. -``` - - -Critical path for each request in the DAG is - -preprocessing: ```max(preprocessor, avg_preprocessor) = 0.15 secs``` -
-model forward: ```max(m1.forward, m2.forward) = 0.3 secs``` -
-
-Total of `0.45` secs. - -+++ - -## More Examples using deployment graph api - -We provide more examples in using the deployment graph api in [here](../deployment-graph.md) - -## Conclusion - -We've walked through key concepts and a simple representative example that covers many important details we support in deployment graph building. There're still some rough edges in user experience that we're dedicated to polish in the next a few months, so please reach out to us if you have any feedback or suggestions: - -- __[Ray Serve forum](https://discuss.ray.io/c/ray-serve/6)__ -- __[Github issues / feature request](https://github.com/ray-project/ray/issues)__ (tag `serve`) - -Potential Future improvements: - - `serve.build()` to fulfill the Ops API so user's deployment graph can generate a YAML file for deployment, scaling and reconfiguration. - - Performance optimizations: - - Tuning guide for deployment graph to avoid single node being bottleneck - - Better use of async deployment handle - - Leverage ray shared memory to reduce or eliminate intermediate data transfer - - Static compute graph transformation, fusion and placement based on profiling - - Better UX, such as visualization diff --git a/doc/source/serve/deployment-graph/http_endpoint_for_dag_graph.md b/doc/source/serve/deployment-graph/http_endpoint_for_dag_graph.md deleted file mode 100644 index 7c07a82f397d4..0000000000000 --- a/doc/source/serve/deployment-graph/http_endpoint_for_dag_graph.md +++ /dev/null @@ -1,29 +0,0 @@ -# Pattern: Http endpoint for dag graph - -This example shows how to configure ingress component of the deployment graph, such as HTTP endpoint prefix, HTTP to python object input adapter. - -## Code - -+++ - -```{eval-rst} -.. literalinclude:: ../doc_code/deployment_graph_dag_http.py - :language: python -``` - -````{note} -1. Serve provide a special driver ([DAGDriver](deployment-graph-e2e-tutorial.md)) to accept the http request and drive the dag graph execution -2. User can specify the customized http adapter to adopt the cusomized input format -```` - -## Outputs - -Model output1: 1(input) + 0(weight) + 4(len("test")) = 5 \ -Model output2: 1(input) + 1(weight) + 4(len("test")) = 6 \ -So the combine sum is 11 -``` -11 -11 -``` - -+++ \ No newline at end of file diff --git a/doc/source/serve/deployment-graph/visualize_dag_during_development.md b/doc/source/serve/deployment-graph/visualize_dag_during_development.md deleted file mode 100644 index ebc0369dfc1cc..0000000000000 --- a/doc/source/serve/deployment-graph/visualize_dag_during_development.md +++ /dev/null @@ -1,33 +0,0 @@ -# Pattern: Visualize DAG during development - -The example shows how to iteratively develop and visualize your deployment graph. For a runnable DAG, we will show both full and partial DAG depending on your choice of root node. - -Please ensure do install dependencies in order to generate visualizations `sudo apt-get install -y graphviz` and `pip install -U pydot`. - -## Code - -+++ - -```{eval-rst} -.. literalinclude:: ../doc_code/visualize_dag_during_deployment.py - :language: python -``` - -## Outputs - -```{note} -The node of user choice will become the root of the graph for both execution as well as visualization, where non-reachable nodes from root will be ignored regardless if they appeared in user code. -``` -In the development phase, when we picked `m1_output` as the root, we can see a visualization of the underlying execution path that's partial of the entire graph. - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/visualize_partial.svg) - -Similarly, when we choose the final dag output, we will capture all nodes used in execution as they're reachable from the root. - -![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/visualize_full.svg) - -```{tip} -If you run the code above within Jupyter notebook, we will automatically display it within cell. Otherwise you can either print the dot file as string and render it in graphviz tools such as https://dreampuf.github.io/GraphvizOnline, or save it as .dot file on disk with your choice of path. -``` - -+++ \ No newline at end of file diff --git a/doc/source/serve/dev-workflow.md b/doc/source/serve/dev-workflow.md new file mode 100644 index 0000000000000..f434eeff7dd64 --- /dev/null +++ b/doc/source/serve/dev-workflow.md @@ -0,0 +1,111 @@ +(serve-dev-workflow)= + +# Development Workflow + +This page describes the recommended workflow for developing Ray Serve applications. If you're ready to go to production, jump to the [Production Guide](serve-in-production) section. + +## Local Development using `serve.run` + +You can use `serve.run` in a Python script to run and test your application locally, using a handle to send requests programmatically rather than over HTTP. + +Benefits: + +- Self-contained Python is convenient for writing local integration tests. +- No need to deploy to a cloud provider or manage infrastructure. + +Drawbacks: + +- Doesn't test HTTP endpoints. +- Can't use GPUs if your local machine doesn't have them. + +Let's see a simple example. + +```{literalinclude} ../serve/doc_code/local_dev.py +:start-after: __local_dev_start__ +:end-before: __local_dev_end__ +:language: python +``` + +We can add the code below to deploy and test Serve locally. + +```{literalinclude} ../serve/doc_code/local_dev.py +:start-after: __local_dev_handle_start__ +:end-before: __local_dev_handle_end__ +:language: python +``` + +## Local Development with HTTP requests + +You can use the `serve run` CLI command to run and test your application locally using HTTP to send requests (similar to how you might use the `uvicorn` command if you're familiar with [Uvicorn](https://www.uvicorn.org/)). + +Recall our example above: + +```{literalinclude} ../serve/doc_code/local_dev.py +:start-after: __local_dev_start__ +:end-before: __local_dev_end__ +:language: python +``` + +Now run the following command in your terminal: + +```bash +serve run local_dev:HelloDeployment +# 2022-08-11 11:31:47,692 INFO scripts.py:294 -- Deploying from import path: "local_dev:graph". +# 2022-08-11 11:31:50,372 INFO worker.py:1481 -- Started a local Ray instance. View the dashboard at http://127.0.0.1:8265. +# (ServeController pid=9865) INFO 2022-08-11 11:31:54,039 controller 9865 http_state.py:129 - Starting HTTP proxy with name 'SERVE_CONTROLLER_ACTOR:SERVE_PROXY_ACTOR-dff7dc5b97b4a11facaed746f02448224aa0c1fb651988ba7197e949' on node 'dff7dc5b97b4a11facaed746f02448224aa0c1fb651988ba7197e949' listening on '127.0.0.1:8000' +# (ServeController pid=9865) INFO 2022-08-11 11:31:55,373 controller 9865 deployment_state.py:1232 - Adding 1 replicas to deployment 'Doubler'. +# (ServeController pid=9865) INFO 2022-08-11 11:31:55,389 controller 9865 deployment_state.py:1232 - Adding 1 replicas to deployment 'HelloDeployment'. +# (HTTPProxyActor pid=9872) INFO: Started server process [9872] +# 2022-08-11 11:31:57,383 SUCC scripts.py:315 -- Deployed successfully. +``` + +The `serve run` command blocks the terminal and can be canceled with Ctrl-C. + +Now that Serve is running, we can send HTTP requests to the application. +For simplicity, we'll just use the `curl` command to send requests from another terminal. + +```bash +curl -X PUT "http://localhost:8000/?name=Ray" +# Hello, Ray! Hello, Ray! +``` + +After you're done testing, you can shut down Ray Serve by interrupting the `serve run` command (e.g., with Ctrl-C): + +```console +^C2022-08-11 11:47:19,829 INFO scripts.py:323 -- Got KeyboardInterrupt, shutting down... +(ServeController pid=9865) INFO 2022-08-11 11:47:19,926 controller 9865 deployment_state.py:1257 - Removing 1 replicas from deployment 'Doubler'. +(ServeController pid=9865) INFO 2022-08-11 11:47:19,929 controller 9865 deployment_state.py:1257 - Removing 1 replicas from deployment 'HelloDeployment'. +``` + +## Testing on a remote cluster + +To test on a remote cluster, you'll use `serve run` again, but this time you'll pass in an `--address` argument to specify the address of the Ray cluster to connect to. For remote clusters, this address has the form `ray://:10001`; see [Ray Client](ray-client-ref) for more information. + +When making the transition from your local machine to a remote cluster, you'll need to make sure your cluster has a similar environment to your local machine--files, environment variables, and Python packages, for example. + +Let's see a simple example that just packages the code. Run the following command on your local machine, with your remote cluster head node IP address substituted for `` in the command: + +```bash +serve run --address=ray://:10001 --working_dir="./project/src" local_dev:HelloDeployment +``` + +This will connect to the remote cluster via Ray Client, upload the `working_dir` directory, and run your serve application. Here, the local directory specified by `working_dir` must contain `local_dev.py` so that it can be uploaded to the cluster and imported by Ray Serve. + +Once this is up and running, we can send requests to the application: + +```bash +curl -X PUT http://:8000/?name=Ray +# Hello, Ray! Hello, Ray! +``` + +For more complex dependencies, including files outside the working directory, environment variables, and Python packages, you can use {ref}`Runtime Environments`. Here is an example using the --runtime-env-json argument: + +```bash +serve run --address=ray://:10001 --runtime-env-json='{"env_vars": {"MY_ENV_VAR": "my-value"}, "working_dir": "./project/src", "pip": ["requests", "chess"]}' local_dev:HelloDeployment +``` + +You can also specify the `runtime_env` via a YAML file; see [serve run](serve_cli.md#serve-run) for details. + +## What's Next? + +Once you are ready to deploy to production, see the [Production Guide](serve-in-production). diff --git a/doc/source/serve/doc_code/batching_guide.py b/doc/source/serve/doc_code/batching_guide.py new file mode 100644 index 0000000000000..5eca72e9ba429 --- /dev/null +++ b/doc/source/serve/doc_code/batching_guide.py @@ -0,0 +1,35 @@ +# flake8: noqa +# __single_sample_begin__ +from ray import serve +import ray + + +@serve.deployment +class Model: + def __call__(self, single_sample: int) -> int: + return single_sample * 2 + + +handle = serve.run(Model.bind()) +assert ray.get(handle.remote(1)) == 2 +# __single_sample_end__ + + +# __batch_begin__ +from typing import List +import numpy as np +from ray import serve +import ray + + +@serve.deployment +class Model: + @serve.batch(max_batch_size=8, batch_wait_timeout_s=0.1) + async def __call__(self, multiple_samples: List[int]) -> List[int]: + # Use numpy's vectorized computation to efficiently process a batch. + return np.array(multiple_samples) * 2 + + +handle = serve.run(Model.bind()) +assert ray.get([handle.remote(i) for i in range(8)]) == [i * 2 for i in range(8)] +# __batch_end__ diff --git a/doc/source/serve/doc_code/create_deployment.py b/doc/source/serve/doc_code/create_deployment.py deleted file mode 100644 index d24e9eca55b5c..0000000000000 --- a/doc/source/serve/doc_code/create_deployment.py +++ /dev/null @@ -1,123 +0,0 @@ -# flake8: noqa -# fmt: off -# -# __serve_example_begin__ -# -# This brief example shows how to create, deploy, and expose access to -# deployment models, using the simple Ray Serve deployment APIs. -# Once deployed, you can access deployment via two methods: -# ServerHandle API and HTTP -# -import os -from random import random - -import requests -import starlette -from starlette.requests import Request -import ray -from ray import serve - -# -# A simple example model stored in a pickled format at an accessible path -# that can be reloaded and deserialized into a model instance. Once deployed -# in Ray Serve, we can use it for prediction. The prediction is a fake condition, -# based on threshold of weight greater than 0.5. -# - - -class Model: - def __init__(self, path): - self.path = path - - def predict(self, data): - return random() + data if data > 0.5 else data - - -@serve.deployment -class Deployment: - # Take in a path to load your desired model - def __init__(self, path: str) -> None: - self.path = path - self.model = Model(path) - # Get the pid on which this deployment is running on - self.pid = os.getpid() - - # Deployments are callable. Here we simply return a prediction from - # our request - def __call__(self, starlette_request) -> str: - # Request came via an HTTP - if isinstance(starlette_request, starlette.requests.Request): - data = starlette_request.query_params['data'] - else: - # Request came via a ServerHandle API method call. - data = starlette_request - pred = self.model.predict(float(data)) - return f"(pid: {self.pid}); path: {self.path}; data: {float(data):.3f}; prediction: {pred:.3f}" - - -if __name__ == '__main__': - - # Start a Ray Serve instance. This will automatically start - # or connect to an existing Ray cluster. - serve.start() - - # Create two distinct deployments of the same class as - # two replicas. Associate each deployment with a unique 'name'. - # This name can be used as to fetch its respective serve handle. - # See code below for method 1. - Deployment.options(name="rep-1", num_replicas=2).deploy("/model/rep-1.pkl") - Deployment.options(name="rep-2", num_replicas=2).deploy("/model/rep-2.pkl") - - # Get the current list of deployments - print(serve.list_deployments()) - - print("ServerHandle API responses: " + "--" * 5) - - # Method 1) Access each deployment using the ServerHandle API - for _ in range(2): - for d_name in ["rep-1", "rep-2"]: - # Get handle to the each deployment and invoke its method. - # Which replica the request is dispatched to is determined - # by the Router actor. - handle = serve.get_deployment(d_name).get_handle() - print(f"handle name : {d_name}") - print(f"prediction : {ray.get(handle.remote(random()))}") - print("-" * 2) - - print("HTTP responses: " + "--" * 5) - - # Method 2) Access deployment via HTTP Request - for _ in range(2): - for d_name in ["rep-1", "rep-2"]: - # Send HTTP request along with data payload - url = f"http://127.0.0.1:8000/{d_name}" - print(f"handle name : {d_name}") - print(f"prediction : {requests.get(url, params= {'data': random()}).text}") - -# Output: -# {'rep-1': Deployment(name=rep-1,version=None,route_prefix=/rep-1), -# 'rep-2': Deployment(name=rep-2,version=None,route_prefix=/rep-2)} -# -# ServerHandle API responses: ---------- -# handle name : rep-1 -# prediction : (pid: 62636); path: /model/rep-1.pkl; data: 0.600; prediction: 1.292 -# -- -# handle name : rep-2 -# prediction : (pid: 62635); path: /model/rep-2.pkl; data: 0.075; prediction: 0.075 -# -- -# handle name : rep-1 -# prediction : (pid: 62634); path: /model/rep-1.pkl; data: 0.186; prediction: 0.186 -# -- -# handle name : rep-2 -# prediction : (pid: 62637); path: /model/rep-2.pkl; data: 0.751; prediction: 1.444 -# -- -# HTTP responses: ---------- -# handle name : rep-1 -# prediction : (pid: 62636); path: /model/rep-1.pkl; data: 0.582; prediction: 1.481 -# handle name : rep-2 -# prediction : (pid: 62637); path: /model/rep-2.pkl; data: 0.778; prediction: 1.678 -# handle name : rep-1 -# prediction : (pid: 62634); path: /model/rep-1.pkl; data: 0.139; prediction: 0.139 -# handle name : rep-2 -# prediction : (pid: 62635); path: /model/rep-2.pkl; data: 0.569; prediction: 1.262 -# __serve_example_end__ diff --git a/doc/source/serve/doc_code/deploying_serve_example.py b/doc/source/serve/doc_code/deploying_serve_example.py new file mode 100644 index 0000000000000..17b0fd5aedab2 --- /dev/null +++ b/doc/source/serve/doc_code/deploying_serve_example.py @@ -0,0 +1,54 @@ +import subprocess + +# __deploy_in_single_file_1_start__ +from starlette.requests import Request + +import ray +from ray import serve + + +@serve.deployment +def my_func(request: Request) -> str: + return "hello" + + +serve.run(my_func.bind()) +# __deploy_in_single_file_1_end__ + +serve.shutdown() +ray.shutdown() +subprocess.check_output(["ray", "stop", "--force"]) +subprocess.check_output(["ray", "start", "--head"]) + +# __deploy_in_single_file_2_start__ +# This will connect to the running Ray cluster. +ray.init(address="auto", namespace="serve") + + +@serve.deployment +def my_func(request: Request) -> str: + return "hello" + + +serve.run(my_func.bind()) +# __deploy_in_single_file_2_end__ + +serve.shutdown() +ray.shutdown() +subprocess.check_output(["ray", "stop", "--force"]) +subprocess.check_output(["ray", "start", "--head"]) + +# __deploy_in_k8s_start__ +# Connect to the running Ray cluster. +ray.init(address="auto") + + +@serve.deployment(route_prefix="/hello") +def hello(request): + return "hello world" + + +serve.run(hello.bind()) +# __deploy_in_k8s_end__ + +subprocess.check_output(["ray", "stop", "--force"]) diff --git a/doc/source/serve/doc_code/deployment_graph_combine_two_nodes_with_passing_input_parallel.py b/doc/source/serve/doc_code/deployment_graph_combine_two_nodes_with_passing_input_parallel.py deleted file mode 100644 index eec8d1c258c32..0000000000000 --- a/doc/source/serve/doc_code/deployment_graph_combine_two_nodes_with_passing_input_parallel.py +++ /dev/null @@ -1,31 +0,0 @@ -import ray -from ray import serve -from ray.serve.deployment_graph import InputNode - - -ray.init() -serve.start() - - -@serve.deployment -class Model: - def __init__(self, weight): - self.weight = weight - - def forward(self, input): - return input + self.weight - - -@serve.deployment -def combine(value_refs): - return sum(ray.get(value_refs)) - - -with InputNode() as user_input: - model1 = Model.bind(0) - model2 = Model.bind(1) - output1 = model1.forward.bind(user_input) - output2 = model2.forward.bind(user_input) - dag = combine.bind([output1, output2]) - -print(ray.get(dag.execute(1))) diff --git a/doc/source/serve/doc_code/deployment_graph_control_flow_based_on_user_inputs.py b/doc/source/serve/doc_code/deployment_graph_control_flow_based_on_user_inputs.py deleted file mode 100644 index c96792edb63fe..0000000000000 --- a/doc/source/serve/doc_code/deployment_graph_control_flow_based_on_user_inputs.py +++ /dev/null @@ -1,35 +0,0 @@ -import ray -from ray import serve -from ray.dag.input_node import InputNode - - -ray.init() - - -@serve.deployment -class Model: - def __init__(self, weight): - self.weight = weight - - def forward(self, input): - return input + self.weight - - -@serve.deployment -def combine(value1, value2, combine_type): - if combine_type == "sum": - return sum([value1, value2]) - else: - return max([value1, value2]) - - -with InputNode() as user_input: - model1 = Model.bind(0) - model2 = Model.bind(1) - output1 = model1.forward.bind(user_input[0]) - output2 = model2.forward.bind(user_input[0]) - dag = combine.bind(output1, output2, user_input[1]) - - -print(ray.get(dag.execute(1, "max"))) -print(ray.get(dag.execute(1, "sum"))) diff --git a/doc/source/serve/doc_code/deployment_graph_dag_http.py b/doc/source/serve/doc_code/deployment_graph_dag_http.py deleted file mode 100644 index 4e52e494afacb..0000000000000 --- a/doc/source/serve/doc_code/deployment_graph_dag_http.py +++ /dev/null @@ -1,54 +0,0 @@ -import requests -from pydantic import BaseModel - -import ray -from ray import serve -from ray.serve.drivers import DAGDriver -from ray.dag.input_node import InputNode - - -ray.init() -serve.start() - - -class ModelInputData(BaseModel): - model_input1: int - model_input2: str - - -@serve.deployment -class Model: - def __init__(self, weight): - self.weight = weight - - def forward(self, input: ModelInputData): - return input.model_input1 + len(input.model_input2) + self.weight - - -@serve.deployment -def combine(value_refs): - return sum(ray.get(value_refs)) - - -with InputNode() as user_input: - model1 = Model.bind(0) - model2 = Model.bind(1) - output1 = model1.forward.bind(user_input) - output2 = model2.forward.bind(user_input) - dag = combine.bind([output1, output2]) - serve_dag = DAGDriver.options(route_prefix="/my-dag").bind( - dag, http_adapter=ModelInputData - ) - -dag_handle = serve.run(serve_dag) - -print( - ray.get( - dag_handle.predict.remote(ModelInputData(model_input1=1, model_input2="test")) - ) -) -print( - requests.post( - "http://127.0.0.1:8000/my-dag", json={"model_input1": 1, "model_input2": "test"} - ).text -) diff --git a/doc/source/serve/doc_code/deployment_graph_patterns/branching_input.py b/doc/source/serve/doc_code/deployment_graph_patterns/branching_input.py new file mode 100644 index 0000000000000..90f544377beb5 --- /dev/null +++ b/doc/source/serve/doc_code/deployment_graph_patterns/branching_input.py @@ -0,0 +1,41 @@ +# __graph_start__ +# File name: branching_input.py + +import ray +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.http_adapters import json_request +from ray.serve.deployment_graph import InputNode + + +@serve.deployment +class Model: + def __init__(self, weight: int): + self.weight = weight + + def forward(self, input: int) -> int: + return input + self.weight + + +@serve.deployment +def combine(value_refs): + return sum(ray.get(value_refs)) + + +model1 = Model.bind(0) +model2 = Model.bind(1) + +with InputNode() as user_input: + output1 = model1.forward.bind(user_input) + output2 = model2.forward.bind(user_input) + combine_output = combine.bind([output1, output2]) + +graph = DAGDriver.bind(combine_output, http_adapter=json_request) + +handle = serve.run(graph) + +sum = ray.get(handle.predict.remote(1)) +print(sum) +# __graph_end__ + +assert sum == 3 diff --git a/doc/source/serve/doc_code/deployment_graph_patterns/conditional.py b/doc/source/serve/doc_code/deployment_graph_patterns/conditional.py new file mode 100644 index 0000000000000..b87b00c3b56ec --- /dev/null +++ b/doc/source/serve/doc_code/deployment_graph_patterns/conditional.py @@ -0,0 +1,49 @@ +# __graph_start__ +# File name: conditional.py + +import ray +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.http_adapters import json_request +from ray.serve.deployment_graph import InputNode + + +@serve.deployment +class Model: + def __init__(self, weight: int): + self.weight = weight + + def forward(self, input: int) -> int: + return input + self.weight + + +@serve.deployment +def combine(value1: int, value2: int, operation: str) -> int: + if operation == "sum": + return sum([value1, value2]) + else: + return max([value1, value2]) + + +model1 = Model.bind(0) +model2 = Model.bind(1) + +with InputNode() as user_input: + input_number, input_operation = user_input[0], user_input[1] + output1 = model1.forward.bind(input_number) + output2 = model2.forward.bind(input_number) + combine_output = combine.bind(output1, output2, input_operation) + +graph = DAGDriver.bind(combine_output, http_adapter=json_request) + +handle = serve.run(graph) + +max_output = ray.get(handle.predict.remote(1, "max")) +print(max_output) + +sum_output = ray.get(handle.predict.remote(1, "sum")) +print(sum_output) +# __graph_end__ + +assert max_output == 2 +assert sum_output == 3 diff --git a/doc/source/serve/doc_code/deployment_graph_patterns/linear_pipeline.py b/doc/source/serve/doc_code/deployment_graph_patterns/linear_pipeline.py new file mode 100644 index 0000000000000..cccb236837ac1 --- /dev/null +++ b/doc/source/serve/doc_code/deployment_graph_patterns/linear_pipeline.py @@ -0,0 +1,37 @@ +# __graph_start__ +# File name: linear_pipeline.py + +import ray +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.http_adapters import json_request +from ray.serve.deployment_graph import InputNode + + +@serve.deployment +class Model: + def __init__(self, weight: float): + self.weight = weight + + def forward(self, input: float) -> float: + return input + self.weight + + +nodes = [Model.bind(0), Model.bind(1), Model.bind(2)] +outputs = [None, None, None] + +with InputNode() as graph_input: + outputs[0] = nodes[0].forward.bind(graph_input) + + for i in range(1, len(nodes)): + outputs[i] = nodes[i].forward.bind(outputs[i - 1]) + +graph = DAGDriver.bind(outputs[-1], http_adapter=json_request) + +handle = serve.run(graph) + +sum = ray.get(handle.predict.remote(1)) +print(sum) +# __graph_end__ + +assert sum == 3 diff --git a/doc/source/serve/doc_code/deployment_graph_same_class_different_args.py b/doc/source/serve/doc_code/deployment_graph_same_class_different_args.py deleted file mode 100644 index 5e90728ad77b4..0000000000000 --- a/doc/source/serve/doc_code/deployment_graph_same_class_different_args.py +++ /dev/null @@ -1,30 +0,0 @@ -import ray -from ray import serve -from ray.serve.deployment_graph import InputNode - - -ray.init() - - -@serve.deployment -class Model: - def __init__(self, weight): - self.weight = weight - - def forward(self, input): - return input + self.weight - - -# 3 nodes chain in a line -num_nodes = 3 -nodes = [Model.bind(w) for w in range(num_nodes)] -outputs = [None] * num_nodes -with InputNode() as dag_input: - for i in range(num_nodes): - if i == 0: - # first node - outputs[i] = nodes[i].forward.bind(dag_input) - else: - outputs[i] = nodes[i].forward.bind(outputs[i - 1]) - -print(ray.get(outputs[-1].execute(0))) diff --git a/doc/source/serve/doc_code/fastapi_example.py b/doc/source/serve/doc_code/fastapi_example.py index 9a10adc29cc45..325335abc6ca3 100644 --- a/doc/source/serve/doc_code/fastapi_example.py +++ b/doc/source/serve/doc_code/fastapi_example.py @@ -16,8 +16,7 @@ def say_hello(self, name: str) -> str: # 2: Deploy the deployment. -serve.start() -FastAPIDeployment.deploy() +serve.run(FastAPIDeployment.bind()) # 3: Query the deployment and print the result. print(requests.get("http://localhost:8000/hello", params={"name": "Theodore"}).json()) diff --git a/doc/source/serve/doc_code/getting_started/model_deployment.py b/doc/source/serve/doc_code/getting_started/model_deployment.py new file mode 100644 index 0000000000000..571de9dda515c --- /dev/null +++ b/doc/source/serve/doc_code/getting_started/model_deployment.py @@ -0,0 +1,59 @@ +# flake8: noqa + +# __import_start__ +from starlette.requests import Request + +import ray +from ray import serve + +# __import_end__ + +# __model_start__ +from transformers import pipeline + + +@serve.deployment(num_replicas=2, ray_actor_options={"num_cpus": 0.2, "num_gpus": 0}) +class Translator: + def __init__(self): + # Load model + self.model = pipeline("translation_en_to_fr", model="t5-small") + + def translate(self, text: str) -> str: + # Run inference + model_output = self.model(text) + + # Post-process output to return only the translation text + translation = model_output[0]["translation_text"] + + return translation + + async def __call__(self, http_request: Request) -> str: + english_text: str = await http_request.json() + return self.translate(english_text) + + +# __model_end__ + +# __model_deploy_start__ +translator = Translator.bind() +# __model_deploy_end__ + +translator = Translator.options(ray_actor_options={}).bind() +serve.run(translator) + +# __client_function_start__ +# File name: model_client.py +import requests + +english_text = "Hello world!" + +response = requests.post("http://127.0.0.1:8000/", json=english_text) +french_text = response.text + +print(french_text) +# __client_function_end__ + +assert french_text == "Bonjour monde!" + +serve.shutdown() +ray.shutdown() diff --git a/doc/source/serve/doc_code/getting_started/model_deployment_full.py b/doc/source/serve/doc_code/getting_started/model_deployment_full.py new file mode 100644 index 0000000000000..f6166779749e0 --- /dev/null +++ b/doc/source/serve/doc_code/getting_started/model_deployment_full.py @@ -0,0 +1,45 @@ +# flake8: noqa + +# __deployment_full_start__ +# File name: serve_deployment.py +from starlette.requests import Request + +import ray +from ray import serve + +from transformers import pipeline + + +@serve.deployment(num_replicas=2, ray_actor_options={"num_cpus": 0.2, "num_gpus": 0}) +class Translator: + def __init__(self): + # Load model + self.model = pipeline("translation_en_to_fr", model="t5-small") + + def translate(self, text: str) -> str: + # Run inference + model_output = self.model(text) + + # Post-process output to return only the translation text + translation = model_output[0]["translation_text"] + + return translation + + async def __call__(self, http_request: Request) -> str: + english_text: str = await http_request.json() + return self.translate(english_text) + + +translator = Translator.bind() +# __deployment_full_end__ + +translator = Translator.options(ray_actor_options={}).bind() +serve.run(translator) + +import requests + +response = requests.post("http://127.0.0.1:8000/", json="Hello world!").text +assert response == "Bonjour monde!" + +serve.shutdown() +ray.shutdown() diff --git a/doc/source/serve/doc_code/getting_started/model_graph.py b/doc/source/serve/doc_code/getting_started/model_graph.py new file mode 100644 index 0000000000000..b7edc0818f4ac --- /dev/null +++ b/doc/source/serve/doc_code/getting_started/model_graph.py @@ -0,0 +1,77 @@ +# flake8: noqa + +# __start_graph__ +# File name: graph.py +from starlette.requests import Request + +import ray +from ray import serve + +from transformers import pipeline + + +@serve.deployment +class Translator: + def __init__(self): + # Load model + self.model = pipeline("translation_en_to_fr", model="t5-small") + + def translate(self, text: str) -> str: + # Run inference + model_output = self.model(text) + + # Post-process output to return only the translation text + translation = model_output[0]["translation_text"] + + return translation + + +@serve.deployment +class Summarizer: + def __init__(self, translator): + # Load model + self.model = pipeline("summarization", model="t5-small") + self.translator = translator + + def summarize(self, text: str) -> str: + # Run inference + model_output = self.model(text, min_length=5, max_length=15) + + # Post-process output to return only the summary text + summary = model_output[0]["summary_text"] + + return summary + + async def __call__(self, http_request: Request) -> str: + english_text: str = await http_request.json() + summary = self.summarize(english_text) + + translation_ref = self.translator.translate.remote(summary) + translation = ray.get(translation_ref) + + return translation + + +deployment_graph = Summarizer.bind(Translator.bind()) +# __end_graph__ + +serve.run(deployment_graph) + +# __start_client__ +# File name: graph_client.py +import requests + +english_text = ( + "It was the best of times, it was the worst of times, it was the age " + "of wisdom, it was the age of foolishness, it was the epoch of belief" +) +response = requests.post("http://127.0.0.1:8000/", json=english_text) +french_text = response.text + +print(french_text) +# __end_client__ + +assert french_text == "c'était le meilleur des temps, c'était le pire des temps ." + +serve.shutdown() +ray.shutdown() diff --git a/doc/source/serve/doc_code/getting_started/models.py b/doc/source/serve/doc_code/getting_started/models.py new file mode 100644 index 0000000000000..c30782c2d2704 --- /dev/null +++ b/doc/source/serve/doc_code/getting_started/models.py @@ -0,0 +1,63 @@ +# flake8: noqa + +# __start_translation_model__ +# File name: model.py +from transformers import pipeline + + +class Translator: + def __init__(self): + # Load model + self.model = pipeline("translation_en_to_fr", model="t5-small") + + def translate(self, text: str) -> str: + # Run inference + model_output = self.model(text) + + # Post-process output to return only the translation text + translation = model_output[0]["translation_text"] + + return translation + + +translator = Translator() + +translation = translator.translate("Hello world!") +print(translation) +# __end_translation_model__ + +# Test model behavior +assert translation == "Bonjour monde!" + + +# __start_summarization_model__ +# File name: summary_model.py +from transformers import pipeline + + +class Summarizer: + def __init__(self): + # Load model + self.model = pipeline("summarization", model="t5-small") + + def summarize(self, text: str) -> str: + # Run inference + model_output = self.model(text, min_length=5, max_length=15) + + # Post-process output to return only the summary text + summary = model_output[0]["summary_text"] + + return summary + + +summarizer = Summarizer() + +summary = summarizer.summarize( + "It was the best of times, it was the worst of times, it was the age " + "of wisdom, it was the age of foolishness, it was the epoch of belief" +) +print(summary) +# __end_summarization_model__ + +# Test model behavior +assert summary == "it was the best of times, it was worst of times ." diff --git a/doc/source/serve/doc_code/handle_guide.py b/doc/source/serve/doc_code/handle_guide.py new file mode 100644 index 0000000000000..fa2e104d88e49 --- /dev/null +++ b/doc/source/serve/doc_code/handle_guide.py @@ -0,0 +1,121 @@ +# flake8: noqa + +# __begin_sync_handle__ +from starlette.requests import Request + +import ray +from ray import serve +from ray.serve.handle import RayServeSyncHandle + + +@serve.deployment +class Model: + def __call__(self) -> str: + return "hello" + + +handle: RayServeSyncHandle = serve.run(Model.bind()) +ref: ray.ObjectRef = handle.remote() # blocks until request is assigned to replica +assert ray.get(ref) == "hello" +# __end_sync_handle__ + +# __begin_async_handle__ +import asyncio +import random +import ray +from ray import serve +from ray.serve.handle import RayServeDeploymentHandle, RayServeSyncHandle + + +@serve.deployment +class Model: + def __call__(self) -> str: + return "hello" + + +@serve.deployment +class DynamicDispatcher: + def __init__( + self, handle_a: RayServeDeploymentHandle, handle_b: RayServeDeploymentHandle + ): + self.handle_a = handle_a + self.handle_b = handle_b + + async def __call__(self): + handle_chosen = self.handle_a if random.random() < 0.5 else self.handle_b + + # The request is enqueued. + submission_task: asyncio.Task = handle_chosen.remote() + # The request is assigned to a replica. + ref: ray.ObjectRef = await submission_task + # The request has been processed by the replica. + result = await ref + + return result + + +handle: RayServeSyncHandle = serve.run( + DynamicDispatcher.bind(Model.bind(), Model.bind()) +) +ref: ray.ObjectRef = handle.remote() +assert ray.get(ref) == "hello" + +# __end_async_handle__ + +# __begin_async_handle_chain__ +import asyncio +import ray +from ray import serve +from ray.serve.handle import RayServeDeploymentHandle, RayServeSyncHandle + + +@serve.deployment +class Model: + def __call__(self, inp): + return "hello " + inp + + +@serve.deployment +class Chain: + def __init__( + self, handle_a: RayServeDeploymentHandle, handle_b: RayServeDeploymentHandle + ): + self.handle_a = handle_a + self.handle_b = handle_b + + async def __call__(self, inp): + ref: asyncio.Task = await self.handle_b.remote( + # Serve can handle enqueued-task as dependencies. + self.handle_a.remote(inp) + ) + return await ref + + +handle: RayServeSyncHandle = serve.run(Chain.bind(Model.bind(), Model.bind())) +ref: ray.ObjectRef = handle.remote("Serve") +assert ray.get(ref) == "hello hello Serve" + +# __end_async_handle_chain__ + + +# __begin_handle_method__ +import ray +from ray import serve +from ray.serve.handle import RayServeSyncHandle + + +@serve.deployment +class Deployment: + def method1(self, arg: str) -> str: + return f"Method1: {arg}" + + def __call__(self, arg: str) -> str: + return f"__call__: {arg}" + + +handle: RayServeSyncHandle = serve.run(Deployment.bind()) + +ray.get(handle.remote("hi")) # Defaults to calling the __call__ method. +ray.get(handle.method1.remote("hi")) # Call a different method. + +# __end_handle_method__ diff --git a/doc/source/serve/doc_code/http_guide.py b/doc/source/serve/doc_code/http_guide.py new file mode 100644 index 0000000000000..27883ef1f8892 --- /dev/null +++ b/doc/source/serve/doc_code/http_guide.py @@ -0,0 +1,112 @@ +# flake8: noqa + +# __begin_starlette__ +import starlette.requests +import requests +from ray import serve + + +@serve.deployment +class Counter: + def __call__(self, request: starlette.requests.Request): + return request.query_params + + +serve.run(Counter.bind()) +resp = requests.get("http://localhost:8000?a=b&c=d") +assert resp.json() == {"a": "b", "c": "d"} +# __end_starlette__ + +# __begin_dagdriver__ +import numpy as np +import requests +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.http_adapters import json_to_ndarray + + +@serve.deployment +class Model: + def __call__(self, arr: np.ndarray): + return arr.sum() + + +serve.run(DAGDriver.bind(Model.bind(), http_adapter=json_to_ndarray)) +resp = requests.post("http://localhost:8000", json={"array": [[1, 2], [2, 3]]}) +assert resp.json() == 8 + +# __end_dagdriver__ + +# __begin_fastapi__ +import ray +import requests +from fastapi import FastAPI +from ray import serve + +app = FastAPI() + + +@serve.deployment(route_prefix="/hello") +@serve.ingress(app) +class MyFastAPIDeployment: + @app.get("/") + def root(self): + return "Hello, world!" + + +serve.run(MyFastAPIDeployment.bind()) +resp = requests.get("http://localhost:8000/hello") +assert resp.json() == "Hello, world!" +# __end_fastapi__ + + +# __begin_fastapi_multi_routes__ +import ray +import requests +from fastapi import FastAPI +from ray import serve + +app = FastAPI() + + +@serve.deployment(route_prefix="/hello") +@serve.ingress(app) +class MyFastAPIDeployment: + @app.get("/") + def root(self): + return "Hello, world!" + + @app.post("/{subpath}") + def root(self, subpath: str): + return f"Hello from {subpath}!" + + +serve.run(MyFastAPIDeployment.bind()) +resp = requests.post("http://localhost:8000/hello/Serve") +assert resp.json() == "Hello from Serve!" +# __end_fastapi_multi_routes__ + +# __begin_byo_fastapi__ +import ray +import requests +from fastapi import FastAPI +from ray import serve + +app = FastAPI() + + +@app.get("/") +def f(): + return "Hello from the root!" + + +@serve.deployment(route_prefix="/") +@serve.ingress(app) +class FastAPIWrapper: + pass + + +serve.run(FastAPIWrapper.bind()) +resp = requests.get("http://localhost:8000/") +assert resp.json() == "Hello from the root!" +# __end_byo_fastapi__ diff --git a/doc/source/serve/doc_code/key-concepts-deployment-graph.py b/doc/source/serve/doc_code/key-concepts-deployment-graph.py index e673882649f16..9dd190478b37a 100644 --- a/doc/source/serve/doc_code/key-concepts-deployment-graph.py +++ b/doc/source/serve/doc_code/key-concepts-deployment-graph.py @@ -5,7 +5,7 @@ @serve.deployment -def preprocess(inp: int): +def preprocess(inp: int) -> int: return inp + 1 @@ -14,7 +14,7 @@ class Model: def __init__(self, increment: int): self.increment = increment - def predict(self, inp: int): + def predict(self, inp: int) -> int: return inp + self.increment diff --git a/doc/source/serve/doc_code/local_dev.py b/doc/source/serve/doc_code/local_dev.py new file mode 100644 index 0000000000000..aaf1d934c5337 --- /dev/null +++ b/doc/source/serve/doc_code/local_dev.py @@ -0,0 +1,34 @@ +# __local_dev_start__ +# Filename: local_dev.py +import ray +from ray import serve +import starlette.requests + + +@serve.deployment +class Doubler: + def double(self, s: str): + return s + " " + s + + +@serve.deployment +class HelloDeployment: + def __init__(self, doubler): + self.doubler = doubler + + async def say_hello_twice(self, name: str): + ref = await self.doubler.double.remote(f"Hello, {name}!") + return await ref + + async def __call__(self, request: starlette.requests.Request): + return await self.say_hello_twice(request.query_params["name"]) + + +graph = HelloDeployment.bind(Doubler.bind()) +# __local_dev_end__ + +# __local_dev_handle_start__ +handle = serve.run(graph) +result = ray.get(handle.say_hello_twice.remote(name="Ray")) +assert result == "Hello, Ray! Hello, Ray!" +# __local_dev_handle_end__ diff --git a/doc/source/serve/doc_code/managing_deployments.py b/doc/source/serve/doc_code/managing_deployments.py new file mode 100644 index 0000000000000..a0dcd457008fb --- /dev/null +++ b/doc/source/serve/doc_code/managing_deployments.py @@ -0,0 +1,79 @@ +from ray import serve +import time +import os + + +# __updating_a_deployment_start__ +@serve.deployment(name="my_deployment", num_replicas=1) +class SimpleDeployment: + pass + + +# Creates one initial replica. +serve.run(SimpleDeployment.bind()) + + +# Re-deploys, creating an additional replica. +# This could be the SAME Python script, modified and re-run. +@serve.deployment(name="my_deployment", num_replicas=2) +class SimpleDeployment: + pass + + +serve.run(SimpleDeployment.bind()) + +# You can also use Deployment.options() to change options without redefining +# the class. This is useful for programmatically updating deployments. +serve.run(SimpleDeployment.options(num_replicas=2).bind()) +# __updating_a_deployment_end__ + + +# __scaling_out_start__ +# Create with a single replica. +@serve.deployment(num_replicas=1) +def func(*args): + pass + + +serve.run(func.bind()) + +# Scale up to 3 replicas. +serve.run(func.options(num_replicas=3).bind()) + +# Scale back down to 1 replica. +serve.run(func.options(num_replicas=1).bind()) +# __scaling_out_end__ + + +# __autoscaling_start__ +@serve.deployment( + autoscaling_config={ + "min_replicas": 1, + "max_replicas": 5, + "target_num_ongoing_requests_per_replica": 10, + } +) +def func(_): + time.sleep(1) + return "" + + +serve.run( + func.bind() +) # The func deployment will now autoscale based on requests demand. +# __autoscaling_end__ + + +# __configure_parallism_start__ +@serve.deployment +class MyDeployment: + def __init__(self, parallelism: str): + os.environ["OMP_NUM_THREADS"] = parallelism + # Download model weights, initialize model, etc. + + def __call__(self): + pass + + +serve.run(MyDeployment.bind("12")) +# __configure_parallism_end__ diff --git a/doc/source/serve/doc_code/migration_example.py b/doc/source/serve/doc_code/migration_example.py index 94006c72e5236..85d75204cba43 100644 --- a/doc/source/serve/doc_code/migration_example.py +++ b/doc/source/serve/doc_code/migration_example.py @@ -6,7 +6,7 @@ from ray.serve.handle import RayServeSyncHandle import requests -import starlette +from starlette.requests import Request serve.start() @@ -16,7 +16,7 @@ @serve.deployment class Model: - def forward(self, input): + def forward(self, input) -> str: # do some inference work return "done" @@ -92,7 +92,7 @@ def forward(self, input: int): # __customized_route_old_api_start__ @serve.deployment(route_prefix="/my_model1") class Model: - def __call__(self, req: starlette.requests.Request): + def __call__(self, req: Request) -> str: # some inference work return "done" @@ -157,7 +157,7 @@ def forward(self, input: int): # __customized_route_old_api_1_start__ @serve.deployment class Model: - def __call__(self, req: starlette.requests.Request): + def __call__(self, req: Request) -> str: # some inference work return "done" @@ -173,14 +173,14 @@ def __call__(self, req: starlette.requests.Request): # __customized_route_old_api_2_start__ @serve.deployment class Model: - def __call__(self, req: starlette.requests.Request): + def __call__(self, req: Request) -> str: # some inference work return "done" @serve.deployment class Model2: - def __call__(self, req: starlette.requests.Request): + def __call__(self, req: Request) -> str: # some inference work return "done" @@ -197,7 +197,7 @@ def __call__(self, req: starlette.requests.Request): # __graph_with_new_api_start__ @serve.deployment class Model: - def forward(self, input): + def forward(self, input) -> str: # do some inference work return "done" diff --git a/doc/source/serve/doc_code/ml_models_examples.py b/doc/source/serve/doc_code/ml_models_examples.py new file mode 100644 index 0000000000000..3b7bdb635090a --- /dev/null +++ b/doc/source/serve/doc_code/ml_models_examples.py @@ -0,0 +1,26 @@ +from ray import serve +from typing import List, Dict + +from starlette.requests import Request + + +# __batch_example_start__ +@serve.deployment(route_prefix="/increment") +class BatchingExample: + def __init__(self): + self.count = 0 + + @serve.batch + async def handle_batch(self, requests: List[Request]) -> List[Dict]: + responses = [] + for request in requests: + responses.append(request.json()) + + return responses + + async def __call__(self, request: Request) -> List[Dict]: + return await self.handle_batch(request) + + +serve.run(BatchingExample.bind()) +# __batch_example_end__ diff --git a/doc/source/serve/doc_code/model_composition/arithmetic.py b/doc/source/serve/doc_code/model_composition/arithmetic.py new file mode 100644 index 0000000000000..b65b11d08ace0 --- /dev/null +++ b/doc/source/serve/doc_code/model_composition/arithmetic.py @@ -0,0 +1,99 @@ +# flake8: noqa + +# __graph_start__ +# File name: arithmetic.py +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.deployment_graph import InputNode + +from starlette.requests import Request + + +@serve.deployment +class AddCls: + def __init__(self, addend: float): + self.addend = addend + + def add(self, number: float) -> float: + return number + self.addend + + async def unpack_request(self, http_request: Request) -> float: + return await http_request.json() + + +@serve.deployment +def subtract_one_fn(number: float) -> float: + return number - 1 + + +@serve.deployment +async def unpack_request(http_request: Request) -> float: + return await http_request.json() + + +add_2 = AddCls.bind(2) +add_3 = AddCls.bind(3) + +with InputNode() as http_request: + request_number = unpack_request.bind(http_request) + add_2_output = add_2.add.bind(request_number) + subtract_1_output = subtract_one_fn.bind(add_2_output) + add_3_output = add_3.add.bind(subtract_1_output) + +graph = DAGDriver.bind(add_3_output) +# __graph_end__ + +serve.run(graph) + +# __graph_client_start__ +# File name: arithmetic_client.py +import requests + +response = requests.post("http://localhost:8000/", json=5) +output = response.json() +print(output) +# __graph_client_end__ + +assert output == 9 + +# __adapter_graph_start__ +# This import can go to the top of the file. +from ray.serve.http_adapters import json_request + +add_2 = AddCls.bind(2) +add_3 = AddCls.bind(3) + +with InputNode() as request_number: + add_2_output = add_2.add.bind(request_number) + subtract_1_output = subtract_one_fn.bind(add_2_output) + add_3_output = add_3.add.bind(subtract_1_output) + +graph = DAGDriver.bind(add_3_output, http_adapter=json_request) +# __adapter_graph_end__ + +serve.run(graph) +assert requests.post("http://localhost:8000/", json=5).json() == 9 + +# __test_graph_start__ +# These imports can go to the top of the file. +import ray +from ray.serve.http_adapters import json_request + +add_2 = AddCls.bind(2) +add_3 = AddCls.bind(3) + +with InputNode() as request_number: + add_2_output = add_2.add.bind(request_number) + subtract_1_output = subtract_one_fn.bind(add_2_output) + add_3_output = add_3.add.bind(subtract_1_output) + +graph = DAGDriver.bind(add_3_output, http_adapter=json_request) + +handle = serve.run(graph) + +ref = handle.predict.remote(5) +result = ray.get(ref) +print(result) +# __test_graph_end__ + +assert result == 9 diff --git a/doc/source/serve/doc_code/model_composition/class_nodes.py b/doc/source/serve/doc_code/model_composition/class_nodes.py new file mode 100644 index 0000000000000..35c9d693bc162 --- /dev/null +++ b/doc/source/serve/doc_code/model_composition/class_nodes.py @@ -0,0 +1,96 @@ +# flake8: noqa + +import requests + +# __echo_class_start__ +# File name: echo.py +from starlette.requests import Request + +from ray import serve + + +@serve.deployment +class EchoClass: + def __init__(self, echo_str: str): + self.echo_str = echo_str + + def __call__(self, request: Request) -> str: + return self.echo_str + + +# You can create ClassNodes from the EchoClass deployment +foo_node = EchoClass.bind("foo") +bar_node = EchoClass.bind("bar") +baz_node = EchoClass.bind("baz") +# __echo_class_end__ + +for node, echo in [(foo_node, "foo"), (bar_node, "bar"), (baz_node, "baz")]: + serve.run(node) + assert requests.get("http://localhost:8000/").text == echo + +# __echo_client_start__ +# File name: echo_client.py +import requests + +response = requests.get("http://localhost:8000/") +echo = response.text +print(echo) +# __echo_client_end__ + +# __hello_start__ +# File name: hello.py +import ray +from ray import serve + + +@serve.deployment +class LanguageClassifer: + def __init__(self, spanish_responder, french_responder): + self.spanish_responder = spanish_responder + self.french_responder = french_responder + + async def __call__(self, http_request): + request = await http_request.json() + language, name = request["language"], request["name"] + + if language == "spanish": + ref = await self.spanish_responder.say_hello.remote(name) + elif language == "french": + ref = await self.french_responder.say_hello.remote(name) + else: + return "Please try again." + + return await ref + + +@serve.deployment +class SpanishResponder: + def say_hello(self, name: str): + return f"Hola {name}" + + +@serve.deployment +class FrenchResponder: + def say_hello(self, name: str): + return f"Bonjour {name}" + + +spanish_responder = SpanishResponder.bind() +french_responder = FrenchResponder.bind() +language_classifier = LanguageClassifer.bind(spanish_responder, french_responder) +# __hello_end__ + +serve.run(language_classifier) + +# __hello_client_start__ +# File name: hello_client.py +import requests + +response = requests.post( + "http://localhost:8000", json={"language": "spanish", "name": "Dora"} +) +greeting = response.text +print(greeting) +# __hello_client_end__ + +assert greeting == "Hola Dora" diff --git a/doc/source/serve/doc_code/model_composition/deployment_graph_viz.py b/doc/source/serve/doc_code/model_composition/deployment_graph_viz.py new file mode 100644 index 0000000000000..67a96c7a4879f --- /dev/null +++ b/doc/source/serve/doc_code/model_composition/deployment_graph_viz.py @@ -0,0 +1,38 @@ +# File name: deployment_graph_viz.py + +from ray import serve +from ray.serve.deployment_graph import InputNode +from ray.dag.vis_utils import _dag_to_dot + + +@serve.deployment +class Model: + def __init__(self, weight: int): + self.weight = weight + + def forward(self, input: int) -> int: + return input + self.weight + + +@serve.deployment +def combine(output_1: int, output_2: int, kwargs_output: int = 0) -> int: + return output_1 + output_2 + kwargs_output + + +m1 = Model.bind(1) +m2 = Model.bind(2) + +with InputNode() as user_input: + m1_output = m1.forward.bind(user_input[0]) + m2_output = m2.forward.bind(user_input[1]) + combine_output = combine.bind(m1_output, m2_output, kwargs_output=user_input[2]) + +# m1_output visualization +graph = _dag_to_dot(m1_output) +to_string = graph.to_string() +print(to_string) + +# Full graph visualization +graph = _dag_to_dot(combine_output) +to_string = graph.to_string() +print(to_string) diff --git a/doc/source/serve/doc_code/monitoring/custom_metric_snippet.py b/doc/source/serve/doc_code/monitoring/custom_metric_snippet.py new file mode 100644 index 0000000000000..dd1fa27341ff5 --- /dev/null +++ b/doc/source/serve/doc_code/monitoring/custom_metric_snippet.py @@ -0,0 +1,37 @@ +# __start__ +from ray import serve +from ray.util import metrics + +import time +import requests + + +@serve.deployment +class MyDeployment: + def __init__(self): + self.num_requests = 0 + self.my_counter = metrics.Counter( + "my_counter", + description=("The number of odd-numbered requests to this deployment."), + tag_keys=("deployment",), + ) + self.my_counter.set_default_tags({"deployment": "MyDeployment"}) + + def __call__(self): + self.num_requests += 1 + if self.num_requests % 2 == 1: + self.my_counter.inc() + + +my_deployment = MyDeployment.bind() +serve.run(my_deployment) + +while True: + requests.get("http://localhost:8000/") + time.sleep(1) + + # __end__ + break + +response = requests.get("http://localhost:8000/") +assert response.status_code == 200 diff --git a/doc/source/serve/doc_code/monitoring/deployment_logger.py b/doc/source/serve/doc_code/monitoring/deployment_logger.py new file mode 100644 index 0000000000000..02001369572f5 --- /dev/null +++ b/doc/source/serve/doc_code/monitoring/deployment_logger.py @@ -0,0 +1,29 @@ +# __start__ +from ray import serve + +import logging +import requests + +logger = logging.getLogger("ray.serve") + + +@serve.deployment +class Counter: + def __init__(self): + self.count = 0 + + def __call__(self, request): + self.count += 1 + logger.info(f"count: {self.count}") + return {"count": self.count} + + +counter = Counter.bind() +serve.run(counter) + +for i in range(10): + requests.get("http://127.0.0.1:8000/") +# __end__ + +response = requests.get("http://127.0.0.1:8000/") +assert response.json() == {"count": 11} diff --git a/doc/source/serve/doc_code/monitoring/metrics_snippet.py b/doc/source/serve/doc_code/monitoring/metrics_snippet.py new file mode 100644 index 0000000000000..0891b8ecfb015 --- /dev/null +++ b/doc/source/serve/doc_code/monitoring/metrics_snippet.py @@ -0,0 +1,23 @@ +# __start__ +from ray import serve + +import time +import requests + + +@serve.deployment +def sleeper(): + time.sleep(1) + + +s = sleeper.bind() + +serve.run(s) + +while True: + requests.get("http://localhost:8000/") + # __end__ + break + +response = requests.get("http://localhost:8000/") +assert response.status_code == 200 diff --git a/doc/source/serve/doc_code/monitoring/monitoring.py b/doc/source/serve/doc_code/monitoring/monitoring.py new file mode 100644 index 0000000000000..b1f3759773cb1 --- /dev/null +++ b/doc/source/serve/doc_code/monitoring/monitoring.py @@ -0,0 +1,27 @@ +# flake8: noqa + +# __start__ +# File name: monitoring.py + +from ray import serve +import logging +from starlette.requests import Request + +logger = logging.getLogger("ray.serve") + + +@serve.deployment +class SayHello: + async def __call__(self, request: Request) -> str: + logger.info("Hello world!") + return "hi" + + +say_hello = SayHello.bind() +# __end__ + +# serve.run(say_hello) + +# import requests +# response = requests.get("http://localhost:8000/") +# assert response.text == "hi" diff --git a/doc/source/serve/doc_code/quickstart.py b/doc/source/serve/doc_code/quickstart.py index 00c5b1fd68614..adaf042e3b767 100644 --- a/doc/source/serve/doc_code/quickstart.py +++ b/doc/source/serve/doc_code/quickstart.py @@ -1,4 +1,7 @@ import requests +from starlette.requests import Request +from typing import Dict + from ray import serve @@ -9,13 +12,12 @@ def __init__(self, msg: str): # Initialize model state: could be very large neural net weights. self._msg = msg - def __call__(self, request): + def __call__(self, request: Request) -> Dict: return {"result": self._msg} # 2: Deploy the model. -serve.start() -MyModelDeployment.deploy(msg="Hello world!") +serve.run(MyModelDeployment.bind(msg="Hello world!")) # 3: Query the deployment and print the result. print(requests.get("http://localhost:8000/").json()) diff --git a/doc/source/serve/doc_code/quickstart_graph.py b/doc/source/serve/doc_code/quickstart_graph.py new file mode 100644 index 0000000000000..a98e6aaf8d4a1 --- /dev/null +++ b/doc/source/serve/doc_code/quickstart_graph.py @@ -0,0 +1,35 @@ +import requests +from ray import serve +from ray.serve.drivers import DAGDriver +from ray.serve.dag import InputNode +from ray.serve.http_adapters import json_request + + +# 1. Define the models in our composition graph +@serve.deployment +class Adder: + def __init__(self, increment: int): + self.increment = increment + + def predict(self, inp: int): + return self.increment + inp + + +@serve.deployment +def combine_average(*input_values) -> float: + return {"result": sum(input_values) / len(input_values)} + + +# 2: Define the model composition graph and call it. +with InputNode() as input_node: + adder_1 = Adder.bind(increment=1) + adder_2 = Adder.bind(increment=2) + dag = combine_average.bind( + adder_1.predict.bind(input_node), adder_2.predict.bind(input_node) + ) + +serve.run(DAGDriver.bind(dag, http_adapter=json_request)) + +# 3: Query the deployment and print the result. +print(requests.post("http://localhost:8000/", json=100).json()) +# {"result": 101.5} diff --git a/doc/source/serve/doc_code/sklearn_quickstart.py b/doc/source/serve/doc_code/sklearn_quickstart.py index bba109b5fe021..3d591f3d050e7 100644 --- a/doc/source/serve/doc_code/sklearn_quickstart.py +++ b/doc/source/serve/doc_code/sklearn_quickstart.py @@ -3,13 +3,14 @@ # __serve_example_begin__ import requests +from starlette.requests import Request +from typing import Dict from sklearn.datasets import load_iris from sklearn.ensemble import GradientBoostingClassifier from ray import serve -serve.start() # Train model. iris_dataset = load_iris() @@ -23,7 +24,7 @@ def __init__(self, model): self.model = model self.label_list = iris_dataset["target_names"].tolist() - async def __call__(self, request): + async def __call__(self, request: Request) -> Dict: payload = (await request.json())["vector"] print(f"Received http request with data {payload}") @@ -33,7 +34,7 @@ async def __call__(self, request): # Deploy model. -BoostingModel.deploy(model) +serve.run(BoostingModel.bind(model)) # Query it! sample_request_input = {"vector": [1.2, 1.0, 1.1, 0.9]} diff --git a/doc/source/serve/doc_code/transformers_example.py b/doc/source/serve/doc_code/transformers_example.py index baf70927b4bb1..c5bc266c9265a 100644 --- a/doc/source/serve/doc_code/transformers_example.py +++ b/doc/source/serve/doc_code/transformers_example.py @@ -1,5 +1,9 @@ import requests +from starlette.requests import Request +from typing import Dict + from transformers import pipeline + from ray import serve @@ -9,13 +13,12 @@ class SentimentAnalysisDeployment: def __init__(self): self._model = pipeline("sentiment-analysis") - def __call__(self, request): + def __call__(self, request: Request) -> Dict: return self._model(request.query_params["text"])[0] # 2: Deploy the deployment. -serve.start() -SentimentAnalysisDeployment.deploy() +serve.run(SentimentAnalysisDeployment.bind()) # 3: Query the deployment and print the result. print( diff --git a/doc/source/serve/doc_code/tutorial_batch.py b/doc/source/serve/doc_code/tutorial_batch.py new file mode 100644 index 0000000000000..c0befdbd64925 --- /dev/null +++ b/doc/source/serve/doc_code/tutorial_batch.py @@ -0,0 +1,34 @@ +# fmt: off +# __doc_import_begin__ +from typing import List + +from starlette.requests import Request +from transformers import pipeline, Pipeline + +from ray import serve +# __doc_import_end__ +# fmt: on + + +# __doc_define_servable_begin__ +@serve.deployment +class BatchTextGenerator: + def __init__(self, model: Pipeline): + self.model = model + + @serve.batch(max_batch_size=4) + async def handle_batch(self, inputs: List[str]) -> List[str]: + print("Our input array has length:", len(inputs)) + + results = self.model(inputs) + return [result[0]["generated_text"] for result in results] + + async def __call__(self, request: Request) -> List[str]: + return await self.handle_batch(request.query_params["text"]) + # __doc_define_servable_end__ + + +# __doc_deploy_begin__ +model = pipeline("text-generation", "gpt2") +generator = BatchTextGenerator.bind(model) +# __doc_deploy_end__ diff --git a/doc/source/serve/doc_code/tutorial_pytorch.py b/doc/source/serve/doc_code/tutorial_pytorch.py new file mode 100644 index 0000000000000..66bd2fb6314f8 --- /dev/null +++ b/doc/source/serve/doc_code/tutorial_pytorch.py @@ -0,0 +1,54 @@ +# fmt: off +# __doc_import_begin__ +from ray import serve + +from io import BytesIO +from PIL import Image +from starlette.requests import Request +from typing import Dict + +import torch +from torchvision import transforms +from torchvision.models import resnet18 +# __doc_import_end__ +# fmt: on + + +# __doc_define_servable_begin__ +@serve.deployment +class ImageModel: + def __init__(self): + self.model = resnet18(pretrained=True).eval() + self.preprocessor = transforms.Compose( + [ + transforms.Resize(224), + transforms.CenterCrop(224), + transforms.ToTensor(), + transforms.Lambda(lambda t: t[:3, ...]), # remove alpha channel + transforms.Normalize( + mean=[0.485, 0.456, 0.406], std=[0.229, 0.224, 0.225] + ), + ] + ) + + async def __call__(self, starlette_request: Request) -> Dict: + image_payload_bytes = await starlette_request.body() + pil_image = Image.open(BytesIO(image_payload_bytes)) + print("[1/3] Parsed image data: {}".format(pil_image)) + + pil_images = [pil_image] # Our current batch size is one + input_tensor = torch.cat( + [self.preprocessor(i).unsqueeze(0) for i in pil_images] + ) + print("[2/3] Images transformed, tensor shape {}".format(input_tensor.shape)) + + with torch.no_grad(): + output_tensor = self.model(input_tensor) + print("[3/3] Inference done!") + return {"class_index": int(torch.argmax(output_tensor[0]))} + # __doc_define_servable_end__ + + +# __doc_deploy_begin__ +image_model = ImageModel.bind() +# __doc_deploy_end__ diff --git a/doc/source/serve/doc_code/tutorial_sklearn.py b/doc/source/serve/doc_code/tutorial_sklearn.py new file mode 100644 index 0000000000000..923e50c591dfd --- /dev/null +++ b/doc/source/serve/doc_code/tutorial_sklearn.py @@ -0,0 +1,80 @@ +# fmt: off +# __doc_import_begin__ +from ray import serve + +import pickle +import json +import numpy as np +import os +import tempfile +from starlette.requests import Request +from typing import Dict + +from sklearn.datasets import load_iris +from sklearn.ensemble import GradientBoostingClassifier +from sklearn.metrics import mean_squared_error +# __doc_import_end__ +# fmt: on + +# __doc_instantiate_model_begin__ +model = GradientBoostingClassifier() +# __doc_instantiate_model_end__ + +# __doc_data_begin__ +iris_dataset = load_iris() +data, target, target_names = ( + iris_dataset["data"], + iris_dataset["target"], + iris_dataset["target_names"], +) + +np.random.shuffle(data), np.random.shuffle(target) +train_x, train_y = data[:100], target[:100] +val_x, val_y = data[100:], target[100:] +# __doc_data_end__ + +# __doc_train_model_begin__ +model.fit(train_x, train_y) +print("MSE:", mean_squared_error(model.predict(val_x), val_y)) + +# Save the model and label to file +MODEL_PATH = os.path.join( + tempfile.gettempdir(), "iris_model_gradient_boosting_classifier.pkl" +) +LABEL_PATH = os.path.join(tempfile.gettempdir(), "iris_labels.json") + +with open(MODEL_PATH, "wb") as f: + pickle.dump(model, f) +with open(LABEL_PATH, "w") as f: + json.dump(target_names.tolist(), f) +# __doc_train_model_end__ + + +# __doc_define_servable_begin__ +@serve.deployment +class BoostingModel: + def __init__(self, model_path: str, label_path: str): + with open(model_path, "rb") as f: + self.model = pickle.load(f) + with open(label_path) as f: + self.label_list = json.load(f) + + async def __call__(self, starlette_request: Request) -> Dict: + payload = await starlette_request.json() + print("Worker: received starlette request with data", payload) + + input_vector = [ + payload["sepal length"], + payload["sepal width"], + payload["petal length"], + payload["petal width"], + ] + prediction = self.model.predict([input_vector])[0] + human_name = self.label_list[prediction] + return {"result": human_name} + # __doc_define_servable_end__ + + +# __doc_deploy_begin__ +boosting_model = BoostingModel.bind(MODEL_PATH, LABEL_PATH) +# __doc_deploy_end__ diff --git a/doc/source/serve/doc_code/tutorial_tensorflow.py b/doc/source/serve/doc_code/tutorial_tensorflow.py new file mode 100644 index 0000000000000..11dd4e2b3a3d1 --- /dev/null +++ b/doc/source/serve/doc_code/tutorial_tensorflow.py @@ -0,0 +1,75 @@ +# fmt: off +# __doc_import_begin__ +from ray import serve + +import os +import tempfile +import numpy as np +from starlette.requests import Request +from typing import Dict + +import tensorflow as tf +# __doc_import_end__ +# fmt: on + +# __doc_train_model_begin__ +TRAINED_MODEL_PATH = os.path.join(tempfile.gettempdir(), "mnist_model.h5") + + +def train_and_save_model(): + # Load mnist dataset + mnist = tf.keras.datasets.mnist + (x_train, y_train), (x_test, y_test) = mnist.load_data() + x_train, x_test = x_train / 255.0, x_test / 255.0 + + # Train a simple neural net model + model = tf.keras.models.Sequential( + [ + tf.keras.layers.Flatten(input_shape=(28, 28)), + tf.keras.layers.Dense(128, activation="relu"), + tf.keras.layers.Dropout(0.2), + tf.keras.layers.Dense(10), + ] + ) + loss_fn = tf.keras.losses.SparseCategoricalCrossentropy(from_logits=True) + model.compile(optimizer="adam", loss=loss_fn, metrics=["accuracy"]) + model.fit(x_train, y_train, epochs=1) + + model.evaluate(x_test, y_test, verbose=2) + model.summary() + + # Save the model in h5 format in local file system + model.save(TRAINED_MODEL_PATH) + + +if not os.path.exists(TRAINED_MODEL_PATH): + train_and_save_model() +# __doc_train_model_end__ + + +# __doc_define_servable_begin__ +@serve.deployment +class TFMnistModel: + def __init__(self, model_path: str): + import tensorflow as tf + + self.model_path = model_path + self.model = tf.keras.models.load_model(model_path) + + async def __call__(self, starlette_request: Request) -> Dict: + # Step 1: transform HTTP request -> tensorflow input + # Here we define the request schema to be a json array. + input_array = np.array((await starlette_request.json())["array"]) + reshaped_array = input_array.reshape((1, 28, 28)) + + # Step 2: tensorflow input -> tensorflow output + prediction = self.model(reshaped_array) + + # Step 3: tensorflow output -> web output + return {"prediction": prediction.numpy().tolist(), "file": self.model_path} + # __doc_define_servable_end__ + + +# __doc_deploy_begin__ +mnist_model = TFMnistModel.bind(TRAINED_MODEL_PATH) +# __doc_deploy_end__ diff --git a/doc/source/serve/doc_code/visualize_dag_during_deployment.py b/doc/source/serve/doc_code/visualize_dag_during_deployment.py deleted file mode 100644 index 469b05bf290dc..0000000000000 --- a/doc/source/serve/doc_code/visualize_dag_during_deployment.py +++ /dev/null @@ -1,37 +0,0 @@ -import ray -from ray import serve -from ray.serve.deployment_graph import InputNode - -ray.init() - - -@serve.deployment -class Model: - def __init__(self, weight): - self.weight = weight - - def forward(self, input): - return input + self.weight - - -@serve.deployment -def combine(output_1, output_2, kwargs_output=0): - return output_1 + output_2 + kwargs_output - - -with InputNode() as user_input: - m1 = Model.bind(1) - m2 = Model.bind(2) - m1_output = m1.forward.bind(user_input[0]) - m2_output = m2.forward.bind(user_input[1]) - dag = combine.bind(m1_output, m2_output, kwargs_output=user_input[2]) - -# Partial DAG visualization -graph = ray.dag.vis_utils._dag_to_dot(m1_output) -to_string = graph.to_string() -print(to_string) - -# Entire DAG visualization -graph = ray.dag.vis_utils._dag_to_dot(dag) -to_string = graph.to_string() -print(to_string) diff --git a/doc/source/serve/faq.md b/doc/source/serve/faq.md deleted file mode 100644 index fd1e7cc3c1659..0000000000000 --- a/doc/source/serve/faq.md +++ /dev/null @@ -1,77 +0,0 @@ -(serve-faq)= - -# Ray Serve FAQ - -This page answers some common questions about Ray Serve. If you have more -questions, feel free to ask them in the [Discussion Board](https://discuss.ray.io/). - -```{contents} -``` - -## How do I deploy Ray Serve? - -See {doc}`deploying-serve` for information about how to deploy Serve. - -## How fast is Ray Serve? - -We are continuously benchmarking Ray Serve. We can confidently say: - -- Ray Serve's **latency** overhead is single digit milliseconds, often times just 1-2 milliseconds. -- For **throughput**, Serve achieves about 3-4k qps on a single machine. -- It is **horizontally scalable** so you can add more machines to increase the overall throughput. - -You can checkout our [microbenchmark instruction](https://github.com/ray-project/ray/tree/master/python/ray/serve/benchmarks) -to benchmark on your hardware. - -## Can I use `asyncio` along with Ray Serve? - -Yes! You can make your servable methods `async def` and Serve will run them -concurrently inside a Python asyncio event loop. - -## Are there any other similar frameworks? - -Yes and no. We truly believe Serve is unique as it gives you end to end control -over the API while delivering scalability and high performance. To achieve -something like what Serve offers, you often need to glue together multiple -frameworks like Tensorflow Serving, SageMaker, or even roll your own -batching server. - -## How does Serve compare to TFServing, TorchServe, ONNXRuntime, and others? - -Ray Serve is *framework agnostic*, you can use any Python framework and libraries. -We believe data scientists are not bounded a particular machine learning framework. -They use the best tool available for the job. - -Compared to these framework specific solution, Ray Serve doesn't perform any optimizations -to make your ML model run faster. However, you can still optimize the models yourself -and run them in Ray Serve: for example, you can run a model compiled by -[PyTorch JIT](https://pytorch.org/docs/stable/jit.html). - -## How does Serve compare to AWS SageMaker, Azure ML, Google AI Platform? - -Ray Serve brings the scalability and parallelism of these hosted offering to -your own infrastructure. You can use our [cluster launcher](cluster-index) -to deploy Ray Serve to all major public clouds, K8s, as well as on bare-metal, on-premise machines. - -Compared to these offerings, Ray Serve lacks a unified user interface and functionality -let you manage the lifecycle of the models, visualize it's performance, etc. Ray -Serve focuses on just model serving and provides the primitives for you to -build your own ML platform on top. - -## How does Serve compare to Seldon, KFServing, Cortex? - -You can develop Ray Serve on your laptop, deploy it on a dev box, and scale it out -to multiple machines or K8s cluster without changing one lines of code. It's a lot -easier to get started with when you don't need to provision and manage K8s cluster. -When it's time to deploy, you can use Ray [cluster launcher](cluster-index) -to transparently put your Ray Serve application in K8s. - -Compare to these frameworks letting you deploy ML models on K8s, Ray Serve lacks -the ability to declaratively configure your ML application via YAML files. In -Ray Serve, you configure everything by Python code. - -## Is Ray Serve only for ML models? - -Nope! Ray Serve can be used to build any type of Python microservices -application. You can also use the full power of Ray within your Ray Serve -programs, so it's easy to run parallel computations within your deployments. diff --git a/doc/source/serve/getting_started.md b/doc/source/serve/getting_started.md index e1f16acaa96a1..3a0c39d0b3d47 100644 --- a/doc/source/serve/getting_started.md +++ b/doc/source/serve/getting_started.md @@ -2,9 +2,20 @@ # Getting Started -This tutorial will walk you through the process of using Ray Serve to deploy a single model behind HTTP locally. +This tutorial will walk you through the process of deploying models with Ray Serve. It will show you how to -We'll be using [HuggingFace's SummarizationPipeline](https://huggingface.co/docs/transformers/main_classes/pipelines#transformers.SummarizationPipeline) to deploy a model that summarizes text. +* expose your models over HTTP using deployments +* test your deployments over HTTP +* compose multiple-model machine learning models together into a single application + +We'll use two models in this tutorial: + +* [HuggingFace's TranslationPipeline](https://huggingface.co/docs/transformers/main_classes/pipelines#transformers.TranslationPipeline) as a text-translation model +* [HuggingFace's SummarizationPipeline](https://huggingface.co/docs/transformers/v4.21.0/en/main_classes/pipelines#transformers.SummarizationPipeline) as a text-summarizer model + +You can also follow along using your own models from any Python framework. + +After deploying those two models, we'll test them with HTTP requests. :::{tip} If you have suggestions on how to improve this tutorial, @@ -14,390 +25,244 @@ If you have suggestions on how to improve this tutorial, To run this example, you will need to install the following: ```bash -$ pip install "ray[serve]" transformers +pip install "ray[serve]" transformers requests ``` -## Example Model +## Model Example: Before Ray Serve -Let's first take a look at how the model works without using Ray Serve. -This is the code for the model: +First, let's take a look at our text-translation model. Here's its code: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_local.py -:end-before: __local_model_end__ +```{literalinclude} ../serve/doc_code/getting_started/models.py +:start-after: __start_translation_model__ +:end-before: __end_translation_model__ :language: python :linenos: true -:start-after: __local_model_start__ ``` -The Python file, called `local_model.py` uses the `summarize` function to -generate summaries of text. +The Python file, called `model.py`, uses the `Translator` class to translate English text to French. -- The `summarizer` variable on line 7 inside `summarize` points to a - function that uses the [t5-small](https://huggingface.co/t5-small) - model to summarize text. -- When `summarizer` is called on a Python String, it returns summarized text - inside a dictionary formatted as `[{"summary_text": "...", ...}, ...]`. -- `summarize` then extracts the summarized text on line 13 by indexing into - the dictionary. +- The `self.model` variable on line 8 inside `Translator`'s `__init__` method + stores a function that uses the [t5-small](https://huggingface.co/t5-small) + model to translate text. +- When `self.model` is called on English text, it returns translated French text + inside a dictionary formatted as `[{"translation_text": "..."}]`. +- The `Translator`'s `translate` method extracts the translated text on + line 15 by indexing into the dictionary. -The file can be run locally by executing the Python script, which uses the -model to summarize an article about the Apollo 11 moon landing [^f1]. +You can copy-paste this script and run it locally. It translates `"Hello world!"` +into `"Bonjour Monde!"`. -```bash -$ python local_model.py +```console +$ python model.py -"two astronauts steered their fragile lunar module safely and smoothly to the -historic landing . the first men to reach the moon -- Armstrong and his -co-pilot, col. Edwin E. Aldrin Jr. of the air force -- brought their ship to -rest on a level, rock-strewn plain ." +Bonjour Monde! ``` -Keep in mind that the `SummarizationPipeline` is an example machine learning -model for this tutorial. You can follow along using arbitrary models in any -framework that has a Python API. Check out our tutorials on sckit-learn, +Keep in mind that the `TranslationPipeline` is an example ML model for this +tutorial. You can follow along using arbitrary models from any +Python framework. Check out our tutorials on scikit-learn, PyTorch, and Tensorflow for more info and examples: -- {ref}`serve-sklearn-tutorial` -- {ref}`serve-pytorch-tutorial` -- {ref}`serve-tensorflow-tutorial` +- {ref}`serve-ml-models-tutorial` (converting-to-ray-serve-deployment)= ## Converting to a Ray Serve Deployment -This tutorial's goal is to deploy this model using Ray Serve, so it can be -scaled up and queried over HTTP. We'll start by converting the above Python -function into a Ray Serve deployment that can be launched locally on a laptop. +In this section, we'll deploy the text translation model using Ray Serve, so +it can be scaled up and queried over HTTP. We'll start by converting +`Translator` into a Ray Serve deployment that runs locally on your computer. -We start by opening a new Python file. First, we need to import `ray` and -`ray serve`, to use features in Ray Serve such as `deployments`, which -provide HTTP access to our model. +First, we open a new Python file and import `ray` and `ray serve`: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py +```{literalinclude} ../serve/doc_code/getting_started/model_deployment.py +:start-after: __import_start__ :end-before: __import_end__ :language: python -:start-after: __import_start__ ``` -After these imports, we can include our model code from above. -We won't call our `summarize` function just yet though! -We will soon add logic to handle HTTP requests, so the `summarize` function -can operate on article text sent via HTTP request. +After these imports, we can include our model code from above: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py -:end-before: __local_model_end__ +```{literalinclude} ../serve/doc_code/getting_started/model_deployment.py +:start-after: __model_start__ +:end-before: __model_end__ :language: python -:start-after: __local_model_start__ ``` -Ray Serve needs to run on top of a Ray cluster, so we connect to a local one. -See {ref}`serve-deploy-tutorial` to learn more about starting a Ray Serve -instance and deploying to a Ray cluster. +The `Translator` class has two modifications: +1. It has a decorator, `@serve.deployment`. +2. It has a new method, `__call__`. -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py -:end-before: __start_ray_cluster_end__ -:language: python -:start-after: __start_ray_cluster_start__ -``` +The decorator converts `Translator` from a Python class into a Ray Serve +`Deployment` object. -The `address` parameter in `ray.init()` connects your Serve script to a -running local Ray cluster. Later, we'll discuss how to start a local Ray -cluster. +Each deployment stores a single Python function or class that you write and uses +it to serve requests. You can scale and configure each of your deployments independently using +parameters in the `@serve.deployment` decorator. The example configures a few common parameters: -:::{note} -`ray.init()` connects to or starts a single-node Ray cluster on your -local machine, which allows you to use all your CPU cores to serve -requests in parallel. To start a multi-node cluster, see -{ref}`serve-deploy-tutorial`. -::: +* `num_replicas`: an integer that determines how many copies of our deployment process run in Ray. Requests are load balanced across these replicas, allowing you to scale your deployments horizontally. +* `ray_actor_options`: a dictionary containing configuration options for each replica. + * `num_cpus`: a float representing the logical number of CPUs each replica should reserve. You can make this a fraction to pack multiple replicas together on a machine with fewer CPUs than replicas. + * `num_gpus`: a float representing the logical number of GPUs each replica should reserve. You can make this a fraction to pack multiple replicas together on a machine with fewer GPUs than replicas. -Next, we start the Ray Serve runtime: +All these parameters are optional, so feel free to omit them: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py -:end-before: __start_serve_end__ -:language: python -:start-after: __start_serve_start__ +```python +... +@serve.deployment +class Translator: + ... ``` -:::{note} -`detached=True` means Ray Serve will continue running even when the Python -script exits. If you would rather stop Ray Serve after the script exits, use -`serve.start()` instead (see {ref}`ray-serve-instance-lifetime` for -details). -::: +Deployments receive Starlette HTTP `request` objects [^f1]. If your deployment stores a Python function, the function is called on this `request` object. If your deployment stores a class, the class's `__call__` method is called on this `request` object. The return value is sent back in the HTTP response body. -Now that we have defined our `summarize` function, connected to a Ray -Cluster, and started the Ray Serve runtime, we can define a function that -accepts HTTP requests and routes them to the `summarize` function. We -define a function called `router` that takes in a Starlette `request` -object [^f2]: +This is why `Translator` needs a new `__call__` method. The method processes the incoming HTTP request by reading its JSON data and forwarding it to the `translate` method. The translated text is returned and sent back through the HTTP response. You can also use Ray Serve's FastAPI integration to avoid working with raw HTTP requests. Check out {ref}`serve-fastapi-http` for more info about FastAPI with Serve. -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py -:end-before: __router_end__ -:language: python -:linenos: true -:start-after: __router_start__ -``` +Next, we need to `bind` our `Translator` deployment to arguments that Ray Serve can pass into its constructor. This will let Ray Serve initialize a `Translator` object that can serve requests. Since `Translator`'s constructor doesn't take in any arguments, we can call the deployment's `bind` method without passing anything in: -- In line 1, we add the decorator `@serve.deployment` - to the `router` function to turn the function into a Serve `Deployment` - object. -- In line 3, `router` uses the `"txt"` query parameter in the `request` - to get the article text to summarize. -- In line 4, it then passes this article text into the `summarize` function - and returns the value. - -:::{note} -Lines 3 and 4 define our HTTP request schema. The HTTP requests sent to this -endpoint must have a `"txt"` query parameter that contains a string. -In general, you can accept HTTP data using query parameters or the -request body. Additionally, you can add other Serve deployments with -different names to create more endpoints that can accept different schemas. -For more complex validation, you can also use FastAPI (see -{ref}`serve-fastapi-http` for more info). -::: - -:::{tip} -This routing function's name doesn't have to be `router`. -It can be any function name as long as the corresponding name is present in -the HTTP request. If you want the function name to be different than the name -in the HTTP request, you can add the `name` keyword parameter to the -`@serve.deployment` decorator to specify the name sent in the HTTP request. - -For example, if the decorator is `@serve.deployment(name="responder")` and -the function signature is `def request_manager(request)`, the HTTP request -should use `responder`, not `request_manager`. If no `name` is passed -into `@serve.deployment`, the `request` uses the function's name by -default. For example, if the decorator is `@serve.deployment` and the -function's signature is `def manager(request)`, the HTTP request should use -`manager`. -::: - -Since `@serve.deployment` makes `router` a `Deployment` object, it can be -deployed using `router.deploy()`: - -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment.py -:end-before: __router_deploy_end__ +```{literalinclude} ../serve/doc_code/getting_started/model_deployment.py +:start-after: __model_deploy_start__ +:end-before: __model_deploy_end__ :language: python -:start-after: __router_deploy_start__ ``` -Once we deploy `router`, we can query the model over HTTP. With that, we can run our model on Ray Serve! -Here's the full Ray Serve deployment script that we built for our model: +Here's the full Ray Serve script that we built: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_deployment_full.py +```{literalinclude} ../serve/doc_code/getting_started/model_deployment_full.py +:start-after: __deployment_full_start__ :end-before: __deployment_full_end__ :language: python :linenos: true -:start-after: __deployment_full_start__ ``` -To deploy `router`, we first start a local Ray cluster: +We can run our script with the `serve run` CLI command. This command takes in an import path +to our deployment formatted as `module:bound_deployment`. Make sure to run the command from a directory containing a local copy of this script, so it can find the bound deployment: -```bash -$ ray start --head +```console +$ serve run serve_deployment:translator ``` -The Ray cluster that this command launches is the same Ray cluster that the -Python code connects to using `ray.init(address="auto")`. -It is also the same Ray cluster that keeps Ray Serve (and any deployments on -it, such as `router`) alive even after the Python script exits as long as -`detached=True` inside `serve.start()`. +This command will start running `Translator` and then block. It can be killed with `ctrl-C` in the terminal. -:::{tip} -To stop the Ray cluster, run the command `ray stop`. -::: +## Testing Ray Serve Deployments -After starting the Ray cluster, we can run the Python file to deploy `router` -and begin accepting HTTP requests: +We can now test our model over HTTP. It can be reached at the following URL: -```bash -$ python model_on_ray_serve.py +``` +http://127.0.0.1:8000/ ``` -## Testing the Ray Serve Deployment - -We can now test our model over HTTP. The structure of our HTTP query is: +Since the cluster is deployed locally in this tutorial, the `127.0.0.1:8000` +refers to a localhost with port 8000 (the default port where you can reach +Serve deployments). -`http://127.0.0.1:8000/[Deployment Name]?[Parameter Name-1]=[Parameter Value-1]&[Parameter Name-2]=[Parameter Value-2]&...&[Parameter Name-n]=[Parameter Value-n]` +We'll send a POST request with JSON data containing our English text. +`Translator`'s `__call__` method will unpack this text and forward it to the +`translate` method. Here's a client script that requests a translation for "Hello world!": -Since the cluster is deployed locally in this tutorial, the `127.0.0.1:8000` -refers to a localhost with port 8000. The `[Deployment Name]` refers to -either the name of the function that we called `.deploy()` on (in our case, -this is `router`), or the `name` keyword parameter's value in -`@serve.deployment` (see the Tip under the `router` function definition -above for more info). - -Each `[Parameter Name]` refers to a field's name in the -request's `query_params` dictionary for our deployed function. In our -example, the only parameter we need to pass in is `txt`. This parameter is -referenced in the `txt = request.query_params["txt"]` line in the `router` -function. Each \[Parameter Name\] object has a corresponding \[Parameter Value\] -object. The `txt`'s \[Parameter Value\] is a string containing the article -text to summarize. We can chain together any number of the name-value pairs -using the `&` symbol in the request URL. - -Now that the `summarize` function is deployed on Ray Serve, we can make HTTP -requests to it. Here's a client script that requests a summary from the same -article as the original Python script: - -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_router_client.py +```{literalinclude} ../serve/doc_code/getting_started/model_deployment.py +:start-after: __client_function_start__ :end-before: __client_function_end__ :language: python -:start-after: __client_function_start__ ``` -We can run this script while the model is deployed to get a response over HTTP: +To test our deployment, first make sure `Translator` is running: -```bash -$ python router_client.py - -"two astronauts steered their fragile lunar module safely and smoothly to the -historic landing . the first men to reach the moon -- Armstrong and his -co-pilot, col. Edwin E. Aldrin Jr. of the air force -- brought their ship to -rest on a level, rock-strewn plain ." +``` +$ serve run serve_deployment:translator ``` -## Using Classes in the Ray Serve Deployment - -Our application is still a bit inefficient though. In particular, the -`summarize` function loads the model on each call when it sets the -`summarizer` variable. However, the model never changes, so it would be more -efficient to define `summarizer` only once and keep its value in memory -instead of reloading it for each HTTP query. +While `Translator` is running, we can open a separate terminal window and run the client script. This will get a response over HTTP: -We can achieve this by converting our `summarize` function into a class: +```console +$ python model_client.py -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_class_deployment.py -:end-before: __deployment_class_end__ -:language: python -:linenos: true -:start-after: __deployment_class_start__ +Bonjour monde! ``` -In this configuration, we can query the `Summarizer` class directly. -The `Summarizer` is initialized once (after calling `Summarizer.deploy()`). -In line 13, its `__init__` function loads and stores the model in -`self.summarize`. HTTP queries for the `Summarizer` class are routed to its -`__call__` method by default, which takes in the Starlette `request` -object. The `Summarizer` class can then take the request's `txt` data and -call the `self.summarize` function on it without loading the model on each -query. +## Composing Machine Learning Models with Deployment Graphs -:::{tip} -Instance variables can also store state. For example, to -count the number of requests served, a `@serve.deployment` class can define -a `self.counter` instance variable in its `__init__` function and set it -to 0. When the class is queried, it can increment the `self.counter` -variable inside of the function responding to the query. The `self.counter` -will keep track of the number of requests served across requests. -::: +Ray Serve's Deployment Graph API allows us to compose multiple machine learning models together into a single Ray Serve application. We can use parameters like `num_replicas`, `num_cpus`, and `num_gpus` to independently configure and scale each deployment in the graph. -HTTP queries for the Ray Serve class deployments follow a similar format to Ray -Serve function deployments. Here's an example client script for the -`Summarizer` class. Notice that the only difference from the `router`'s -client script is that the URL uses the `Summarizer` path instead of -`router`. +For example, let's deploy a machine learning pipeline with two steps: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_summarizer_client.py -:end-before: __client_class_end__ -:language: python -:start-after: __client_class_start__ -``` +1. Summarize English text +2. Translate the summary into French -We can deploy the class-based model on Serve without stopping the Ray cluster. -However, for the purposes of this tutorial, let's restart the cluster, deploy -the model, and query it over HTTP: +`Translator` already performs step 2. We can use [HuggingFace's SummarizationPipeline](https://huggingface.co/docs/transformers/v4.21.0/en/main_classes/pipelines#transformers.SummarizationPipeline) to accomplish step 1. Here's an example of the `SummarizationPipeline` that runs locally: -```bash -$ ray stop -$ ray start --head -$ python summarizer_on_ray_serve.py -$ python summarizer_client.py - -"two astronauts steered their fragile lunar module safely and smoothly to the -historic landing . the first men to reach the moon -- Armstrong and his -co-pilot, col. Edwin E. Aldrin Jr. of the air force -- brought their ship to -rest on a level, rock-strewn plain ." +```{literalinclude} ../serve/doc_code/getting_started/models.py +:start-after: __start_summarization_model__ +:end-before: __end_summarization_model__ +:language: python ``` -## Advanced HTTP Functionality with FastAPI +You can copy-paste this script and run it locally. It summarizes the snippet from _A Tale of Two Cities_ to `it was the best of times, it was worst of times .` -Now suppose we want to expose additional functionality in our model. In -particular, the `summarize` function also has `min_length` and -`max_length` parameters. Although we could expose these options as additional -parameters in URL, Ray Serve also allows us to add more route options to the -URL itself and handle each route separately. +```console +$ python model.py -Because this logic can get complex, Serve integrates with -[FastAPI](https://fastapi.tiangolo.com/). This allows us to define a Serve -deployment by adding the `@serve.ingress` decorator to a FastAPI app. For -more info about FastAPI with Serve, please see {ref}`serve-fastapi-http`. +it was the best of times, it was worst of times . +``` -As an example of FastAPI, here's a modified version of our `Summarizer` class -with route options to request a minimum or maximum length of ten words in the -summaries: +Here's a Ray Serve deployment graph that chains the two models together. The graph takes English text, summarizes it, and then translates it: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_fastapi_deployment.py -:end-before: __fastapi_end__ +```{literalinclude} ../serve/doc_code/getting_started/model_graph.py +:start-after: __start_graph__ +:end-before: __end_graph__ :language: python :linenos: true -:start-after: __fastapi_start__ ``` -The class now exposes three routes: +This script contains our `Summarizer` class converted to a deployment and our `Translator` class with some modifications. In this script, the `Summarizer` class contains the `__call__` method since requests are sent to it first. It also takes in the `Translator` as one of its constructor arguments, so it can forward summarized texts to the `Translator` deployment. The `__call__` method also contains some new code on lines 44 and 45: -- `/Summarizer`: As before, this route takes in article text and returns - a summary. -- `/Summarizer/min10`: This route takes in article text and returns a - summary with at least 10 words. -- `/Summarizer/max10`: This route takes in article text and returns a - summary with at most 10 words. +```python +translation_ref = self.translator.translate.remote(summary) +translation = ray.get(translation_ref) +``` -Notice that `Summarizer`'s methods no longer take in a Starlette `request` -object. Instead, they take in the URL's `txt` parameter directly with FastAPI's -[query parameter](https://fastapi.tiangolo.com/tutorial/query-params/) -feature. +`self.translator.translate.remote(summary)` issues an asynchronous call to the `Translator`'s `translate` method. Essentially, this line tells Ray to schedule a request to the `Translator` deployment's `translate` method, which can be fulfilled asynchronously. The line immediately returns a reference to the method's output. The next line `ray.get(translation_ref)` waits for `translate` to execute and returns the value of that execution. -Since we still deploy our model locally, the full URL still uses the -localhost IP. This means each of our three routes comes after the -`http://127.0.0.1:8000` IP and port address. As an example, we can make -requests to the `max10` route using this client script: +We compose our graph in line 50: -```{literalinclude} ../../../python/ray/serve/examples/doc/e2e_fastapi_client.py -:end-before: __client_fastapi_end__ -:language: python -:start-after: __client_fastapi_start__ +```python +deployment_graph = Summarizer.bind(Translator.bind()) ``` -```bash -$ ray stop -$ ray start --head -$ python serve_with_fastapi.py -$ python fastapi_client.py +Here, we bind `Translator` to its (empty) constructor arguments, and then we pass in the bound `Translator` as the constructor argument for the `Summarizer`. We can run this deployment graph using the `serve run` CLI command. Make sure to run this command from a directory containing a local copy of the `graph.py` code: + +```console +$ serve run graph:deployment_graph +``` + +We can use this client script to make requests to the graph: -"two astronauts steered their fragile lunar" +```{literalinclude} ../serve/doc_code/getting_started/model_graph.py +:start-after: __start_client__ +:end-before: __end_client__ +:language: python ``` -Congratulations! You just built and deployed a machine learning model on Ray -Serve! You should now have enough context to dive into the {doc}`key-concepts` to -get a deeper understanding of Ray Serve. +While the graph is running, we can open a separate terminal window and run the client script: +```console +$ python graph_client.py + +c'était le meilleur des temps, c'était le pire des temps . +``` + +Deployment graphs are useful since they let you deploy each part of your machine learning pipeline, such as inference and business logic steps, in separate deployments. Each of these deployments can be individually configured and scaled, ensuring you get maximal performance from your resources. See the guide on [model composition](serve-model-composition) to learn more. ## Next Steps - Dive into the {doc}`key-concepts` to get a deeper understanding of Ray Serve. -- Learn more about how to deploy your Ray Serve application to a multi-node cluster: {ref}`serve-deploy-tutorial`. +- Learn more about how to deploy your Ray Serve application to production: {ref}`serve-in-production`. - Check more in-depth tutorials for popular machine learning frameworks: {doc}`tutorials/index`. ```{rubric} Footnotes ``` -[^f1]: The article text comes from the New York Times article "Astronauts - Land on Plain; Collect Rocks, Plant Flag" archived - [here](https://archive.nytimes.com/www.nytimes.com/library/national/science/nasa/072169sci-nasa.html). - -[^f2]: [Starlette](https://www.starlette.io/) is a web server framework +[^f1]: [Starlette](https://www.starlette.io/) is a web server framework used by Ray Serve. Its [Request](https://www.starlette.io/requests/) class provides a nice interface for incoming HTTP requests. diff --git a/doc/source/serve/handle-guide.md b/doc/source/serve/handle-guide.md deleted file mode 100644 index ad74be708a62a..0000000000000 --- a/doc/source/serve/handle-guide.md +++ /dev/null @@ -1,81 +0,0 @@ -(serve-handle-explainer)= - -# ServeHandle: Calling Deployments from Python - -Ray Serve enables you to query models both from HTTP and Python. This feature -enables seamless [model composition](serve-model-composition). You can -get a `ServeHandle` corresponding to deployment, similar how you can -reach a deployment through HTTP via a specific route. When you issue a request -to a deployment through `ServeHandle`, the request is load balanced across -available replicas in the same way an HTTP request is. - -To call a Ray Serve deployment from python, use {mod}`Deployment.get_handle ` -to get a handle to the deployment, then use -{mod}`handle.remote ` to send requests -to that deployment. These requests can pass ordinary args and kwargs that are -passed directly to the method. This returns a Ray `ObjectRef` whose result -can be waited for or retrieved using `ray.wait` or `ray.get`. - -```python -@serve.deployment -class Deployment: - def method1(self, arg): - return f"Method1: {arg}" - - def __call__(self, arg): - return f"__call__: {arg}" - -Deployment.deploy() - -handle = Deployment.get_handle() -ray.get(handle.remote("hi")) # Defaults to calling the __call__ method. -ray.get(handle.method1.remote("hi")) # Call a different method. -``` - -If you want to use the same deployment to serve both HTTP and ServeHandle traffic, the recommended best practice is to define an internal method that the HTTP handling logic will call: - -```python -@serve.deployment(route_prefix="/api") -class Deployment: - def say_hello(self, name: str): - return f"Hello {name}!" - - def __call__(self, request): - return self.say_hello(request.query_params["name"]) - -Deployment.deploy() -``` - -Now we can invoke the same logic from both HTTP or Python: - -```python -print(requests.get("http://localhost:8000/api?name=Alice")) -# Hello Alice! - -handle = Deployment.get_handle() -print(ray.get(handle.say_hello.remote("Alice"))) -# Hello Alice! -``` - -(serve-sync-async-handles)= - -## Sync and Async Handles - -Ray Serve offers two types of `ServeHandle`. You can use the `Deployment.get_handle(..., sync=True|False)` -flag to toggle between them. - -- When you set `sync=True` (the default), a synchronous handle is returned. - Calling `handle.remote()` should return a Ray `ObjectRef`. -- When you set `sync=False`, an asyncio based handle is returned. You need to - Call it with `await handle.remote()` to return a Ray ObjectRef. To use `await`, - you have to run `Deployment.get_handle` and `handle.remote` in Python asyncio event loop. - -The async handle has performance advantage because it uses asyncio directly; as compared -to the sync handle, which talks to an asyncio event loop in a thread. To learn more about -the reasoning behind these, checkout our [architecture documentation](serve-architecture). - -## Integrating with existing web servers - -Ray Serve comes with its own HTTP server out of the box, but if you have an existing -web application, you can still plug in Ray Serve to scale up your compute using the `ServeHandle`. -For a tutorial with sample code, see {ref}`serve-web-server-integration-tutorial`. diff --git a/doc/source/serve/http-adapters.md b/doc/source/serve/http-adapters.md deleted file mode 100644 index b9f034e1b48e4..0000000000000 --- a/doc/source/serve/http-adapters.md +++ /dev/null @@ -1,146 +0,0 @@ - -(serve-http-adapters)= - -# HTTP Adapters - -HTTP adapters are functions that convert raw HTTP request to Python types that you know and recognize. -Its input arguments should be type annotated. At minimal, it should accept a `starlette.requests.Request` type. -But it can also accept any type that's recognized by the FastAPI's dependency injection framework. - -For example, here is an adapter that extra the json content from request. - -```python -async def json_resolver(request: starlette.requests.Request): - return await request.json() -``` - -Here is an adapter that accept two HTTP query parameters. - -```python -def parse_query_args(field_a: int, field_b: str): - return YourDataClass(field_a, field_b) -``` - -You can specify different type signatures to facilitate HTTP fields extraction -include -[query parameters](https://fastapi.tiangolo.com/tutorial/query-params/), -[body parameters](https://fastapi.tiangolo.com/tutorial/body/), -and [many other data types](https://fastapi.tiangolo.com/tutorial/extra-data-types/). -For more detail, you can take a look at [FastAPI documentation](https://fastapi.tiangolo.com/). - -You can use adapters in different scenarios within Serve: - -- Ray AIR `Predictor` -- Serve Deployment Graph `DAGDriver` -- Embedded in Bring Your Own `FastAPI` Application - -Let's go over them one by one. - -## Ray AIR `Predictor` - -Ray Serve provides a suite of adapters to convert HTTP requests to ML inputs like `numpy` arrays. -You can use it with [Ray AI Runtime (AIR) model wrapper](air-serving-guide) feature -to one click deploy pre-trained models. - -For example, we provide a simple adapter for n-dimensional array. - -With [model wrappers](air-serving-guide), you can specify it via the `http_adapter` field. - -```python -from ray import serve -from ray.serve.http_adapters import json_to_ndarray -from ray.serve import PredictorDeployment - -PredictorDeployment.options(name="my_model").deploy( - my_ray_air_predictor, - my_ray_air_checkpoint, - http_adapter=json_to_ndarray -) -``` - -:::{note} -Serve also supports pydantic models as a short-hand for HTTP adapters in model wrappers. Instead of functions, -you can directly pass in a pydantic model class to mean "validate the HTTP body with this schema". -Once validated, the model instance will passed to the predictor. - -```python -from pydantic import BaseModel - -class User(BaseModel): - user_id: int - user_name: str - -... -PredictorDeployment.deploy(..., http_adapter=User) -``` -::: - -## Serve Deployment Graph `DAGDriver` - -In [Serve Deployment Graph](serve-deployment-graph), you can configure -`ray.serve.drivers.DAGDriver` to accept an http adapter via it's `http_adapter` field. - -For example, the json request adapters parse JSON in HTTP body: - -```python -from ray.serve.drivers import DAGDriver -from ray.serve.http_adapters import json_request -from ray.dag.input_node import InputNode - -with InputNode() as input_node: - ... - dag = DAGDriver.bind(other_node, http_adapter=json_request) -``` - -:::{note} -Serve also supports pydantic models as a short-hand for HTTP adapters in model wrappers. Instead of functions, -you can directly pass in a pydantic model class to mean "validate the HTTP body with this schema". -Once validated, the model instance will passed as `input_node` variable. - -```python -from pydantic import BaseModel - -class User(BaseModel): - user_id: int - user_name: str - -... -DAGDriver.bind(other_node, http_adapter=User) -``` -::: - -## Embedded in Bring Your Own `FastAPI` Application - -You can also bring the adapter to your own FastAPI app using -[Depends](https://fastapi.tiangolo.com/tutorial/dependencies/#import-depends). -The input schema will automatically be part of the generated OpenAPI schema with FastAPI. - -```python -from fastapi import FastAPI, Depends -from ray.serve.http_adapters import json_to_ndarray - -app = FastAPI() - -@app.post("/endpoint") -async def endpoint(np_array = Depends(json_to_ndarray)): - ... -``` - -It has the following schema for input: - -(serve-ndarray-schema)= - -```{eval-rst} -.. autopydantic_model:: ray.serve.http_adapters.NdArray - -``` - -## List of Built-in Adapters - -Here is a list of adapters and please feel free to [contribute more](https://github.com/ray-project/ray/issues/new/choose)! - -```{eval-rst} -.. automodule:: ray.serve.http_adapters - :members: json_to_ndarray, image_to_ndarray, starlette_request, json_request, pandas_read_json, json_to_multi_ndarray - -``` \ No newline at end of file diff --git a/doc/source/serve/http-guide.md b/doc/source/serve/http-guide.md index d650de3f2067d..3c9afffee7b31 100644 --- a/doc/source/serve/http-guide.md +++ b/doc/source/serve/http-guide.md @@ -1,182 +1,204 @@ -# Calling Deployments via HTTP +# HTTP Handling +This section helps you understand how to: +- send HTTP requests to Serve deployments +- use Ray Serve to integrate with FastAPI +- use customized HTTP Adapters +- choose which feature to use for your use case -(serve-http)= +## Choosing the right HTTP feature -## Calling Deployments via HTTP +Serve offers a layered approach to expose your model with the right HTTP API. -### Basic Example +Considering your use case, you can choose the right level of abstraction: +- If you are comfortable working with the raw request object, use [`starlette.request.Requests` API](serve-http). +- If you want a fully fledged API server with validation and doc generation, use the [FastAPI integration](serve-fastapi-http). +- If you just want a pre-defined HTTP schema, use the [`DAGDriver` with `http_adapter`](serve-http-adapters). -When you create a deployment, it is exposed over HTTP by default at `/{deployment_name}`. You can change the route by specifying the `route_prefix` argument to the {mod}`@serve.deployment ` decorator. -```python -@serve.deployment(route_prefix="/counter") -class Counter: - def __call__(self, request): - pass +(serve-http)= +## Calling Deployments via HTTP +When you deploy a Serve application, the [ingress deployment](serve-key-concepts-ingress-deployment) (the one passed to `serve.run`) will be exposed over HTTP. + +```{literalinclude} ../serve/doc_code/http_guide.py +:start-after: __begin_starlette__ +:end-before: __end_starlette__ +:language: python ``` -When you make a request to the Serve HTTP server at `/counter`, it will forward the request to the deployment's `__call__` method and provide a [Starlette Request object](https://www.starlette.io/requests/) as the sole argument. The `__call__` method can return any JSON-serializable object or a [Starlette Response object](https://www.starlette.io/responses/) (e.g., to return a custom status code). +Requests to the Serve HTTP server at `/` are routed to the deployment's `__call__` method with a [Starlette Request object](https://www.starlette.io/requests/) as the sole argument. The `__call__` method can return any JSON-serializable object or a [Starlette Response object](https://www.starlette.io/responses/) (e.g., to return a custom status code or custom headers). -Below, we discuss some advanced features for customizing Ray Serve's HTTP functionality. +Often for ML models, you just need the API to accept a `numpy` array. You can use Serve's `DAGDriver` to simply the request parsing. -(serve-fastapi-http)= +```{literalinclude} ../serve/doc_code/http_guide.py +:start-after: __begin_dagdriver__ +:end-before: __end_dagdriver__ +:language: python +``` + +```{note} +Serve provides a library of HTTP adapters to help you avoid boilerplate code. The [later section](serve-http-adapters) dives deeper into how these works. +``` -### FastAPI HTTP Deployments +(serve-fastapi-http)= +## FastAPI HTTP Deployments If you want to define more complex HTTP handling logic, Serve integrates with [FastAPI](https://fastapi.tiangolo.com/). This allows you to define a Serve deployment using the {mod}`@serve.ingress ` decorator that wraps a FastAPI app with its full range of features. The most basic example of this is shown below, but for more details on all that FastAPI has to offer such as variable routes, automatic type validation, dependency injection (e.g., for database connections), and more, please check out [their documentation](https://fastapi.tiangolo.com/). -```python -import ray +```{literalinclude} ../serve/doc_code/http_guide.py +:start-after: __begin_fastapi__ +:end-before: __end_fastapi__ +:language: python +``` -from fastapi import FastAPI -from ray import serve +Now if you send a request to `/hello`, this will be routed to the `root` method of our deployment. We can also easily leverage FastAPI to define multiple routes with different HTTP methods: -app = FastAPI() -ray.init(address="auto", namespace="summarizer") -serve.start(detached=True) +```{literalinclude} ../serve/doc_code/http_guide.py +:start-after: __begin_fastapi_multi_routes__ +:end-before: __end_fastapi_multi_routes__ +:language: python +``` -@serve.deployment(route_prefix="/hello") -@serve.ingress(app) -class MyFastAPIDeployment: - @app.get("/") - def root(self): - return "Hello, world!" +You can also pass in an existing FastAPI app to a deployment to serve it as-is: -MyFastAPIDeployment.deploy() +```{literalinclude} ../serve/doc_code/http_guide.py +:start-after: __begin_byo_fastapi__ +:end-before: __end_byo_fastapi__ +:language: python ``` -Now if you send a request to `/hello`, this will be routed to the `root` method of our deployment. We can also easily leverage FastAPI to define multiple routes with different HTTP methods: - -```python -import ray +This is useful for scaling out an existing FastAPI app with no modifications necessary. +Existing middlewares, **automatic OpenAPI documentation generation**, and other advanced FastAPI features should work as-is. -from fastapi import FastAPI -from ray import serve +```{note} +Serve currently does not support WebSockets. If you have a use case that requires it, please [let us know](https://github.com/ray-project/ray/issues/new/choose)! +``` -app = FastAPI() -ray.init(address="auto", namespace="summarizer") -serve.start(detached=True) +(serve-http-adapters)= -@serve.deployment(route_prefix="/hello") -@serve.ingress(app) -class MyFastAPIDeployment: - @app.get("/") - def root(self): - return "Hello, world!" +## HTTP Adapters +HTTP adapters are functions that convert raw HTTP requests to basic Python types that you know and recognize. - @app.post("/{subpath}") - def root(self, subpath: str): - return f"Hello from {subpath}!" +For example, here is an adapter that extracts the JSON content from a request: -MyFastAPIDeployment.deploy() +```python +async def json_resolver(request: starlette.requests.Request): + return await request.json() ``` -You can also pass in an existing FastAPI app to a deployment to serve it as-is: +The input arguments to an HTTP adapter should be type-annotated. At a minimum, the adapter should accept a `starlette.requests.Request` type (https://www.starlette.io/requests/#request), +but it can also accept any type that's recognized by [FastAPI's dependency injection framework](https://fastapi.tiangolo.com/tutorial/dependencies/). + +Here is an HTTP adapter that accepts two HTTP query parameters: ```python -import ray +def parse_query_args(field_a: int, field_b: str): + return YourDataClass(field_a, field_b) +``` -from fastapi import FastAPI -from ray import serve +You can specify different type signatures to facilitate the extraction of HTTP fields, including +- [query parameters](https://fastapi.tiangolo.com/tutorial/query-params/), +- [body parameters](https://fastapi.tiangolo.com/tutorial/body/), +- [many other data types](https://fastapi.tiangolo.com/tutorial/extra-data-types/). -app = FastAPI() -ray.init(address="auto", namespace="summarizer") -serve.start(detached=True) +For more details, you can take a look at the [FastAPI documentation](https://fastapi.tiangolo.com/). + +In addition to above adapters, you also use other adapters. Below we examine at least three: -@app.get("/") -def f(): - return "Hello from the root!" +- Ray AIR `Predictor` +- Serve Deployment Graph `DAGDriver` +- Embedded in Bring Your Own `FastAPI` Application -# ... add more routes, routers, etc. to `app` ... +### Ray AIR `Predictor` -@serve.deployment(route_prefix="/") -@serve.ingress(app) -class FastAPIWrapper: - pass +Ray Serve provides a suite of adapters to convert HTTP requests to ML inputs like `numpy` arrays. +You can use them together with the [Ray AI Runtime (AIR) model wrapper](air-serving-guide) feature +to one-click deploy pre-trained models. -FastAPIWrapper.deploy() +As an example, we provide a simple adapter for an *n*-dimensional array. + +When using [model wrappers](air-serving-guide), you can specify your HTTP adapter via the `http_adapter` field: + +```python +from ray import serve +from ray.serve.http_adapters import json_to_ndarray +from ray.serve import PredictorDeployment + +serve.run(PredictorDeployment.options(name="my_model").bind( + my_ray_air_predictor, + my_ray_air_checkpoint, + http_adapter=json_to_ndarray +)) ``` -This is useful for scaling out an existing FastAPI app with no modifications necessary. -Existing middlewares, automatic OpenAPI documentation generation, and other advanced FastAPI features should work as-is. -You can also combine routes defined this way with routes defined on the deployment: +:::{note} +`my_ray_air_predictor` and `my_ray_air_checkpoint` are two arguments int `PredictorDeployment` constructor. For detailed usage, please checkout [Ray AI Runtime (AIR) model wrapper](air-serving-guide) +::: + +### Serve Deployment Graph `DAGDriver` + +When using a [Serve deployment graph](serve-model-composition-deployment-graph), you can configure +`ray.serve.drivers.DAGDriver` to accept an HTTP adapter via its `http_adapter` field. + +For example, the `json_request` adapter parses JSON in the HTTP body: ```python -import ray +from ray.serve.drivers import DAGDriver +from ray.serve.http_adapters import json_request +from ray.dag.input_node import InputNode -from fastapi import FastAPI -from ray import serve +with InputNode() as input_node: + # ... + dag = DAGDriver.bind(other_node, http_adapter=json_request) +``` + +### Embedded in your existing `FastAPI` Application + +You can also bring the adapter to your own FastAPI app using +[Depends](https://fastapi.tiangolo.com/tutorial/dependencies/#import-depends). +The input schema automatically become part of the generated OpenAPI schema with FastAPI. + +```python +from fastapi import FastAPI, Depends +from ray.serve.http_adapters import json_to_ndarray app = FastAPI() -ray.init(address="auto", namespace="summarizer") -serve.start(detached=True) - -@app.get("/") -def f(): - return "Hello from the root!" - -@serve.deployment(route_prefix="/api1") -@serve.ingress(app) -class FastAPIWrapper1: - @app.get("/subpath") - def method(self): - return "Hello 1!" - -@serve.deployment(route_prefix="/api2") -@serve.ingress(app) -class FastAPIWrapper2: - @app.get("/subpath") - def method(self): - return "Hello 2!" - -FastAPIWrapper1.deploy() -FastAPIWrapper2.deploy() + +@app.post("/endpoint") +async def endpoint(np_array = Depends(json_to_ndarray)): + ... ``` -In this example, requests to both `/api1` and `/api2` would return `Hello from the root!` while a request to `/api1/subpath` would return `Hello 1!` and a request to `/api2/subpath` would return `Hello 2!`. -To try it out, save a code snippet in a local python file (i.e. main.py) and in the same directory, run the following commands to start a local Ray cluster on your machine. +### Pydantic models as adapters -```bash -ray start --head -python main.py -``` +Serve also supports [pydantic models](https://pydantic-docs.helpmanual.io/usage/models/) as a shorthand for HTTP adapters in model wrappers. Instead of using a function to define your HTTP adapter as in the examples above, +you can directly pass in a pydantic model class to effectively tell Ray Serve to validate the HTTP body with this schema. +Once validated, the model instance will passed to the predictor. +```python +from pydantic import BaseModel -### Configuring HTTP Server Locations +class User(BaseModel): + user_id: int + user_name: str -By default, Ray Serve starts a single HTTP server on the head node of the Ray cluster. -You can configure this behavior using the `http_options={"location": ...}` flag -in {mod}`serve.start `: +# ... -- "HeadOnly": start one HTTP server on the head node. Serve - assumes the head node is the node you executed serve.start - on. This is the default. -- "EveryNode": start one HTTP server per node. -- "NoServer" or `None`: disable HTTP server. +PredictorDeployment.deploy(..., http_adapter=User) +# Or: +DAGDriver.bind(other_node, http_adapter=User) -:::{note} -Using the "EveryNode" option, you can point a cloud load balancer to the -instance group of Ray cluster to achieve high availability of Serve's HTTP -proxies. -::: +``` +### List of Built-in Adapters -### Enabling CORS and other HTTP middlewares +Here is a list of adapters; please feel free to [contribute more](https://github.com/ray-project/ray/issues/new/choose)! -Serve supports arbitrary [Starlette middlewares](https://www.starlette.io/middleware/) -and custom middlewares in Starlette format. The example below shows how to enable -[Cross-Origin Resource Sharing (CORS)](https://developer.mozilla.org/en-US/docs/Web/HTTP/CORS). -You can follow the same pattern for other Starlette middlewares. +(serve-ndarray-schema)= + +```{eval-rst} +.. automodule:: ray.serve.http_adapters + :members: json_to_ndarray, image_to_ndarray, starlette_request, json_request, pandas_read_json, json_to_multi_ndarray -```python -from starlette.middleware import Middleware -from starlette.middleware.cors import CORSMiddleware - -client = serve.start( - http_options={"middlewares": [ - Middleware( - CORSMiddleware, allow_origins=["*"], allow_methods=["*"]) - ]}) ``` diff --git a/doc/source/serve/index.md b/doc/source/serve/index.md index 6dc15a111a9e1..8f0286bc684d4 100644 --- a/doc/source/serve/index.md +++ b/doc/source/serve/index.md @@ -8,8 +8,6 @@ :::{tip} [Get in touch with us](https://docs.google.com/forms/d/1l8HT35jXMPtxVUtQPeGoe09VGp5jcvSv0TqPgyz6lGU) if you're using or considering using Ray Serve. - -Chat with Ray Serve users and developers on our [forum](https://discuss.ray.io/). ::: ```{image} logo.svg @@ -21,36 +19,40 @@ Chat with Ray Serve users and developers on our [forum](https://discuss.ray.io/) (rayserve-overview)= Ray Serve is a scalable model serving library for building online inference APIs. -Serve is framework agnostic, so you can use a single toolkit to serve everything from deep learning models built with frameworks like [PyTorch](serve-pytorch-tutorial), - [Tensorflow, and Keras](serve-tensorflow-tutorial), to [Scikit-Learn](serve-sklearn-tutorial) models, to arbitrary Python business logic. +Serve is framework agnostic, so you can use a single toolkit to serve everything from deep learning models built with frameworks like PyTorch, Tensorflow, and Keras, to Scikit-Learn models, to arbitrary Python business logic. -Serve is particularly well suited for {ref}`serve-model-composition`, enabling you to build a complex inference service consisting of multiple ML models and business logic all in Python code. +Serve is particularly well suited for [model composition](serve-model-composition), enabling you to build a complex inference service consisting of multiple ML models and business logic all in Python code. Serve is built on top of Ray, so it easily scales to many machines and offers flexible scheduling support such as fractional GPUs so you can share resources and serve many machine learning models at low cost. -:::{tabbed} Installation +## Quickstart Install Ray Serve and its dependencies: ```bash pip install "ray[serve]" ``` -::: - -:::{tabbed} Quickstart - -To run this example, install the following: ``pip install ray["serve"]`` In this quick-start example we will define a simple "hello world" deployment, deploy it behind HTTP locally, and query it. ```{literalinclude} doc_code/quickstart.py :language: python ``` + +:::{tabbed} More examples +For more examples, select from the tabs. ::: -:::{tabbed} FastAPI integration +:::{tabbed} Model composition -To run this example, install the following: ``pip install ray["serve"]`` +In this example, we demonstrate how you can use Serve's model composition API to express a complex computation graph and deploy it as a Serve application. + +```{literalinclude} doc_code/quickstart_graph.py +:language: python +``` +::: + +:::{tabbed} FastAPI integration In this example we will use Serve's [FastAPI](https://fastapi.tiangolo.com/) integration to make use of more advanced HTTP functionality. @@ -59,9 +61,9 @@ In this example we will use Serve's [FastAPI](https://fastapi.tiangolo.com/) int ``` ::: -:::{tabbed} Serving a Hugging Face NLP model +:::{tabbed} Hugging Face model -To run this example, install the following: ``pip install ray["serve"] transformers`` +To run this example, install the following: ``pip install transformers`` In this example we will serve a pre-trained [Hugging Face transformers](https://huggingface.co/docs/transformers/index) model using Ray Serve. The model we'll use is a sentiment analysis model: it will take a text string as input and return if the text was "POSITIVE" or "NEGATIVE." @@ -122,9 +124,88 @@ Because it's built on top of Ray, you can run it anywhere Ray can: on your lapto ::: + +## How can Serve help me as a... + +:::{dropdown} Data scientist +:animate: fade-in-slide-down + +Serve makes it easy to go from a laptop to a cluster. You can test your models (and your entire deployment graph) on your local machine before deploying it to production on a cluster. You don't need to know heavyweight Kubernetes concepts or cloud configurations to use Serve. + +::: + +:::{dropdown} ML engineer +:animate: fade-in-slide-down + +Serve helps you scale out your deployment and runs them reliably and efficiently to save costs. With Serve's first-class model composition API, you can combine models together with business logic and build end-to-end user-facing applications. Additionally, Serve runs natively on Kubernetes with minimal operation overhead. +::: + +:::{dropdown} ML platform engineer +:animate: fade-in-slide-down + +Serve specializes in scalable and reliable ML model serving. As such, it can be an important plug-and-play component of your ML platform stack. +Serve supports arbitrary Python code and therefore integrates well with the MLOps ecosystem. You can use it with model optimizers (ONNX, TVM), model monitoring systems (Seldon Alibi, Arize), model registries (MLFlow, Weights and Biases), machine learning frameworks (XGBoost, Scikit-learn), data app UIs (Gradio, Streamlit), and Web API frameworks (FastAPI, gRPC). + +::: + + +## How does Serve compare to ... + +:::{dropdown} TFServing, TorchServe, ONNXRuntime +:animate: fade-in-slide-down + +Ray Serve is *framework agnostic*, so you can use it alongside any other Python framework or library. +We believe data scientists should not be bound to a particular machine learning framework. +They should be empowered to use the best tool available for the job. + +Compared to these framework-specific solutions, Ray Serve doesn't perform any model-specific optimizations to make your ML model run faster. However, you can still optimize the models yourself +and run them in Ray Serve. For example, you can run a model compiled by +[PyTorch JIT](https://pytorch.org/docs/stable/jit.html) or [ONNXRuntime](https://onnxruntime.ai/). +::: + +:::{dropdown} AWS SageMaker, Azure ML, Google Vertex AI +:animate: fade-in-slide-down + +As an open-source project, Ray Serve brings the scalability and reliability of these hosted offerings to your own infrastructure. +You can use the Ray [cluster launcher](cluster-index) to deploy Ray Serve to all major public clouds, K8s, as well as on bare-metal, on-premise machines. + +Ray Serve is not a full-fledged ML Platform. +Compared to these other offerings, Ray Serve lacks the functionality for +managing the lifecycle of your models, visualizing their performance, etc. Ray +Serve primarily focuses on model serving and providing the primitives for you to +build your own ML platform on top. + +If you are looking for end-to-end ML pipeline framework that can handle everything from data processing to serving, check out [Ray AI Runtime](air). +::: + +:::{dropdown} Seldon, KServe, Cortex +:animate: fade-in-slide-down + +You can develop Ray Serve on your laptop, deploy it on a dev box, and scale it out +to multiple machines or a Kubernetes cluster, all with minimal or no changes to code. It's a lot +easier to get started with when you don't need to provision and manage a K8s cluster. +When it's time to deploy, you can use our [Kubernetes Operator](kuberay-quickstart) +to transparently deploy your Ray Serve application to K8s. +::: + +:::{dropdown} BentoML, Comet.ml, MLflow +:animate: fade-in-slide-down + +Many of these tools are focused on serving and scaling models independently. +In contrast, Ray Serve is framework-agnostic and focuses on model composition. +As such, Ray Serve works with any model packaging and registry format. +Ray Serve also provides key features for building production-ready machine learning applications, including best-in-class autoscaling and naturally integrating with business logic. +::: + +We truly believe Serve is unique as it gives you end-to-end control +over your ML application while delivering scalability and high performance. To achieve +Serve's feature offerings with other tools, you would need to glue together multiple +frameworks like Tensorflow Serving and SageMaker, or even roll your own +micro-batching component to improve throughput. + ## Learn More -Check out {ref}`getting-started` and {ref}`serve-key-concepts`, look at the {ref}`serve-faq`, +Check out {ref}`getting-started` and {ref}`serve-key-concepts`, or head over to the {doc}`tutorials/index` to get started building your Ray Serve applications. @@ -161,8 +242,8 @@ or head over to the {doc}`tutorials/index` to get started building your Ray Serv **User Guides** ^^^ - Learn best practices for common patterns like :doc:`managing deployments `, how to call deployments :ref:`via HTTP ` or :ref:`from Python `. - Learn how to serve multiple ML models with :ref:`Model Ensemble `, and how to :ref:`monitor your Serve applications `. + Learn best practices for common patterns like :ref:`scaling and resource allocation ` and :ref:`model composition `. + Learn how to :ref:`develop Serve applications locally ` and :ref:`go to production `. +++ .. link-button:: serve-user-guides @@ -174,7 +255,7 @@ or head over to the {doc}`tutorials/index` to get started building your Ray Serv **Examples** ^^^ - Follow the tutorials to learn how to integrate Ray Serve with :ref:`TensorFlow `, :ref:`Scikit-Learn `, and :ref:`RLlib `. Learn how Ray Serve also integrates with :ref:`existing web applications ` + Follow the tutorials to learn how to integrate Ray Serve with :ref:`TensorFlow `, :ref:`Scikit-Learn `, and :ref:`RLlib `. +++ .. link-button:: serve-examples @@ -183,18 +264,6 @@ or head over to the {doc}`tutorials/index` to get started building your Ray Serv :classes: btn-outline-info btn-block --- - **Serve FAQ** - ^^^ - - Find answers to commonly asked questions in our detailed FAQ. - - +++ - .. link-button:: serve-faq - :type: ref - :text: Ray Serve FAQ - :classes: btn-outline-info btn-block - --- - **API Reference** ^^^ diff --git a/doc/source/serve/key-concepts.md b/doc/source/serve/key-concepts.md index f4c6569c82fcb..44614aa957fd6 100644 --- a/doc/source/serve/key-concepts.md +++ b/doc/source/serve/key-concepts.md @@ -7,12 +7,13 @@ ## Deployment Deployments are the central concept in Ray Serve. -They allow you to define and update your business logic or models that will handle incoming requests as well as how this is exposed over HTTP or in Python. +A deployment contains business logic or an ML model to handle incoming requests and can be scaled up to run across a Ray cluster. +At runtime, a deployment consists of a number of *replicas*, which are individual copies of the class or function that are started in separate Ray Actors (processes). +The number of replicas can be scaled up or down (or even autoscaled) to match the incoming request load. -A deployment is defined using {mod}`@serve.deployment ` on a Python class (or function for simple use cases). -You can specify arguments to be passed to the constructor when you call `Deployment.deploy()`, shown below. - -A deployment consists of a number of *replicas*, which are individual copies of the function or class that are started in separate Ray Actors (processes). +To define a deployment, use the {mod}`@serve.deployment ` decorator on a Python class (or function for simple use cases). +Then, `bind` the deployment with optional arguments to the constructor (see below). +Finally, deploy the resulting "bound deployment" using `serve.run` (or the equivalent `serve run` CLI command, see [Development Workflow](serve-dev-workflow) for details). ```python @serve.deployment @@ -21,96 +22,97 @@ class MyFirstDeployment: def __init__(self, msg): self.msg = msg - def __call__(self, request): - return self.msg - - def other_method(self, arg): + def __call__(self): return self.msg -MyFirstDeployment.deploy("Hello world!") +my_first_deployment = MyFirstDeployment.bind("Hello world!") +handle = serve.run(my_first_deployment) +print(ray.get(handle.remote())) # "Hello world!" ``` -Deployments can be exposed in two ways: over HTTP or in Python via the {ref}`servehandle-api`. -By default, HTTP requests will be forwarded to the `__call__` method of the class (or the function) and a `Starlette Request` object will be the sole argument. -You can also define a deployment that wraps a FastAPI app for more flexible handling of HTTP requests. See {ref}`serve-fastapi-http` for details. +(serve-key-concepts-query-deployment)= + +## ServeHandle (composing deployments) -To serve multiple deployments defined by the same class, use the `name` option: +Ray Serve enables flexible model composition and scaling by allowing multiple independent deployments to call into each other. +When binding a deployment, you can include references to _other bound deployments_. +Then, at runtime each of these arguments is converted to a {mod}`ServeHandle ` that can be used to query the deployment using a Python-native API. +Below is a basic example where the `Driver` deployment can call into two downstream models. ```python -MyFirstDeployment.options(name="hello_service").deploy("Hello!") -MyFirstDeployment.options(name="hi_service").deploy("Hi!") -``` +@serve.deployment +class Driver: + def __init__(self, model_a_handle, model_b_handle): + self._model_a_handle = model_a_handle + self._model_b_handle = model_b_handle -You can also list all available deployments and dynamically get references to them: + async def __call__(self, request): + ref_a = await self._model_a_handle.remote(request) + ref_b = await self._model_b_handle.remote(request) + return (await ref_a) + (await ref_b) -```python ->> serve.list_deployments() -{'A': Deployment(name=A,version=None,route_prefix=/A)} -{'MyFirstDeployment': Deployment(name=MyFirstDeployment,version=None,route_prefix=/MyFirstDeployment} -# Returns the same object as the original MyFirstDeployment object. -# This can be used to redeploy, get a handle, etc. -deployment = serve.get_deployment("MyFirstDeployment") +model_a = ModelA.bind() +model_b = ModelB.bind() + +# model_a and model_b will be passed to the Driver constructor as ServeHandles. +driver = Driver.bind(model_a, model_b) + +# Deploys model_a, model_b, and driver. +serve.run(driver) ``` -## HTTP Ingress -By default, deployments are exposed over HTTP at `http://localhost:8000/`. -The HTTP path that the deployment is available at can be changed using the `route_prefix` option. -All requests to `/{route_prefix}` and any subpaths will be routed to the deployment (using a longest-prefix match for overlapping route prefixes). +(serve-key-concepts-ingress-deployment)= +## Ingress Deployment (HTTP handling) + +A Serve application can consist of multiple deployments that can be combined to perform model composition or complex business logic. +However, there is always one "top-level" deployment, the one that will be passed to `serve.run` or `serve.build` to deploy the application. +This deployment is called the "ingress deployment" because it serves as the entrypoint for all traffic to the application. +Often, it will then route to other deployments or call into them using the `ServeHandle` API and compose the results before returning to the user. + +The ingress deployment defines the HTTP handling logic for the application. +By default, the `__call__` method of the class will be called and passed in a `Starlette` request object. +The response will be serialized as JSON, but other `Starlette` response objects can also be returned directly. Here's an example: ```python -@serve.deployment(name="http_deployment", route_prefix="/api") -class HTTPDeployment: - def __call__(self, request): - return "Hello world!" +@serve.deployment +class MostBasicIngress: + async def __call__(self, request: starlette.requests.Request) -> str: + name = await request.json()["name"] + return f"Hello {name}" ``` -After creating the deployment, it is now exposed by the HTTP server and handles requests using the specified class. +After binding the deployment and running `serve.run()`, it is now exposed by the HTTP server and handles requests using the specified class. We can query the model to verify that it's working. ```python import requests -print(requests.get("http://127.0.0.1:8000/api").text) +print(requests.get("http://127.0.0.1:8000/", json={"name": "Corey"}).text) # Hello Corey! ``` -(serve-key-concepts-query-deployment)= -## ServeHandle - -We can also query the deployment using the {mod}`ServeHandle ` interface. +For more expressive HTTP handling, Serve also comes with a built-in integration with `FastAPI`. +This allows you to use the full expressiveness of FastAPI to define more complex APIs: ```python -# To get a handle from the same script, use the Deployment object directly: -handle = HTTPDeployment.get_handle() +from fastapi import FastAPI -# To get a handle from a different script, reference it by name: -handle = serve.get_deployment("http_deployment").get_handle() +app = FastAPI() -print(ray.get(handle.remote())) +@serve.deployment +@serve.ingress(app) +class MostBasicIngress: + @app.get("/{name}") + async def say_hi(self, name: str) -> str: + return f"Hello {name}" ``` -As noted above, there are two ways to expose deployments. The first is by using the {mod}`ServeHandle ` -interface. This method allows you to access deployments within a Python script or code, making it convenient for a -Python developer. And the second is by using the HTTP request, allowing access to deployments via a web client application. - -:::{note} - Let's look at a simple end-to-end example using both ways to expose and access deployments. Your output may - vary due to random nature of how the prediction is computed; however, the example illustrates two things: - 1) how to expose and use deployments and 2) how to use replicas, to which requests are sent. Note that each pid - is a separate replica associated with each deployment name, `rep-1` and `rep-2` respectively. - - ```{literalinclude} doc_code/create_deployment.py - :end-before: __serve_example_end__ - :language: python - :start-after: __serve_example_begin__ - ``` -::: - (serve-key-concepts-deployment-graph)= + ## Deployment Graph -Building on top of the Deployment concept, Ray Serve provides a first-class API for composing models into a graph structure. +Building on top of the deployment concept, Ray Serve also provides a first-class API for composing multiple models into a graph structure and orchestrating the calls to each deployment automatically. Here's a simple example combining a preprocess function and model. @@ -118,8 +120,8 @@ Here's a simple example combining a preprocess function and model. ``` ## What's Next? -Now you have learned about the key concepts. You can dive into our [User Guides](user-guide) for more details into: -- [Creating, updating, and deleting deployments](managing-deployments) -- [Configuring HTTP ingress and integrating with FastAPI](http-guide) -- [Composing deployments using ServeHandle](handle-guide) -- [Building Deployment Graphs](deployment-graph) +Now that you have learned the key concepts, you can dive into the [User Guide](user-guide): +- [Scaling and allocating resources](scaling-and-resource-allocation) +- [Configuring HTTP logic and integrating with FastAPI](http-guide) +- [Development workflow for Serve applications](dev-workflow) +- [Composing deployments to perform model composition](model_composition) diff --git a/doc/source/serve/managing-deployments.md b/doc/source/serve/managing-deployments.md deleted file mode 100644 index 37bff024b4e02..0000000000000 --- a/doc/source/serve/managing-deployments.md +++ /dev/null @@ -1,197 +0,0 @@ -# Managing Deployments - -This section should help you: - -- create, query, update and configure deployments -- configure resources of your deployments -- specify different Python dependencies across different deployment using Runtime Environments - -:::{tip} -Get in touch with us if you're using or considering using [Ray Serve](https://docs.google.com/forms/d/1l8HT35jXMPtxVUtQPeGoe09VGp5jcvSv0TqPgyz6lGU). -::: - -```{contents} -``` - -## Updating a Deployment - -Often you want to be able to update your code or configuration options for a deployment over time. -Deployments can be updated simply by updating the code or configuration options and calling `deploy()` again. - -```python -@serve.deployment(name="my_deployment", num_replicas=1) -class SimpleDeployment: - pass - -# Creates one initial replica. -SimpleDeployment.deploy() - -# Re-deploys, creating an additional replica. -# This could be the SAME Python script, modified and re-run. -@serve.deployment(name="my_deployment", num_replicas=2) -class SimpleDeployment: - pass - -SimpleDeployment.deploy() - -# You can also use Deployment.options() to change options without redefining -# the class. This is useful for programmatically updating deployments. -SimpleDeployment.options(num_replicas=2).deploy() -``` - -By default, each call to `.deploy()` will cause a redeployment, even if the underlying code and options didn't change. -This could be detrimental if you have many deployments in a script and and only want to update one: if you re-run the script, all of the deployments will be redeployed, not just the one you updated. -To prevent this, you may provide a `version` string for the deployment as a keyword argument in the decorator or `Deployment.options()`. -If provided, the replicas will only be updated if the value of `version` is updated; if the value of `version` is unchanged, the call to `.deploy()` will be a no-op. -When a redeployment happens, Serve will perform a rolling update, bringing down at most 20% of the replicas at any given time. - -(configuring-a-deployment)= - -## Configuring a Deployment - -There are a number of things you'll likely want to do with your serving application including -scaling out or configuring the maximum number of in-flight requests for a deployment. -All of these options can be specified either in {mod}`@serve.deployment ` or in `Deployment.options()`. - -To update the config options for a running deployment, simply redeploy it with the new options set. - -### Scaling Out - -To scale out a deployment to many processes, simply configure the number of replicas. - -```python -# Create with a single replica. -@serve.deployment(num_replicas=1) -def func(*args): - pass - -func.deploy() - -# Scale up to 10 replicas. -func.options(num_replicas=10).deploy() - -# Scale back down to 1 replica. -func.options(num_replicas=1).deploy() -``` - -#### Autoscaling - -Serve also has the support for a demand-based replica autoscaler. -It reacts to traffic spikes via observing queue sizes and making scaling decisions. -To configure it, you can set the `autoscaling` field in deployment options. - - -```python -@serve.deployment( - autoscaling_config={ - "min_replicas": 1, - "max_replicas": 5, - "target_num_ongoing_requests_per_replica": 10, - }) -def func(_): - time.sleep(1) - return "" - -func.deploy() # The func deployment will now autoscale based on requests demand. -``` - -The `min_replicas` and `max_replicas` fields configure the range of replicas which the -Serve autoscaler chooses from. Deployments will start with `min_replicas` initially. - -The `target_num_ongoing_requests_per_replica` configuration specifies how aggressively the -autoscaler should react to traffic. Serve will try to make sure that each replica has roughly that number -of requests being processed and waiting in the queue. For example, if your processing time is `10ms` -and the latency constraint is `100ms`, you can have at most `10` requests ongoing per replica so -the last requests can finish within the latency constraint. We recommend you benchmark your application -code and set this number based on end to end latency objective. - - -:::{note} -The Ray Serve Autoscaler is an application-level autoscaler that sits on top of the [Ray Autoscaler](cluster-index). -Concretely, this means that the Ray Serve autoscaler asks Ray to start a number of replica actors based on the request demand. -If the Ray Autoscaler determines there aren't enough available CPUs to place these actors, it responds by adding more nodes. -Similarly, when Ray Serve scales down and terminates some replica actors, it may result in some nodes being empty, at which point the Ray autoscaler will remove those nodes. -::: - -(serve-cpus-gpus)= - -### Resource Management (CPUs, GPUs) - -To assign hardware resources per replica, you can pass resource requirements to -`ray_actor_options`. -By default, each replica requires one CPU. -To learn about options to pass in, take a look at [Resources with Actor](actor-resource-guide) guide. - -For example, to create a deployment where each replica uses a single GPU, you can do the -following: - -```python -@serve.deployment(ray_actor_options={"num_gpus": 1}) -def func(*args): - return do_something_with_my_gpu() -``` - -### Fractional Resources - -The resources specified in `ray_actor_options` can also be *fractional*. -This allows you to flexibly share resources between replicas. -For example, if you have two models and each doesn't fully saturate a GPU, you might want to have them share a GPU by allocating 0.5 GPUs each. -The same could be done to multiplex over CPUs. - -```python -@serve.deployment(name="deployment1", ray_actor_options={"num_gpus": 0.5}) -def func(*args): - return do_something_with_my_gpu() - -@serve.deployment(name="deployment2", ray_actor_options={"num_gpus": 0.5}) -def func(*args): - return do_something_with_my_gpu() -``` - -### Configuring Parallelism with OMP_NUM_THREADS - -Deep learning models like PyTorch and Tensorflow often use multithreading when performing inference. -The number of CPUs they use is controlled by the OMP_NUM_THREADS environment variable. -To [avoid contention](omp-num-thread-note), Ray sets `OMP_NUM_THREADS=1` by default because Ray workers and actors use a single CPU by default. -If you *do* want to enable this parallelism in your Serve deployment, just set OMP_NUM_THREADS to the desired value either when starting Ray or in your function/class definition: - -```bash -OMP_NUM_THREADS=12 ray start --head -OMP_NUM_THREADS=12 ray start --address=$HEAD_NODE_ADDRESS -``` - -```python -@serve.deployment -class MyDeployment: - def __init__(self, parallelism): - os.environ["OMP_NUM_THREADS"] = parallelism - # Download model weights, initialize model, etc. - -MyDeployment.deploy() -``` - -:::{note} -Some other libraries may not respect `OMP_NUM_THREADS` and have their own way to configure parallelism. -For example, if you're using OpenCV, you'll need to manually set the number of threads using `cv2.setNumThreads(num_threads)` (set to 0 to disable multi-threading). -You can check the configuration using `cv2.getNumThreads()` and `cv2.getNumberOfCPUs()`. -::: - -(managing-deployments-user-configuration)= - -### User Configuration (Experimental) - -Suppose you want to update a parameter in your model without needing to restart -the replicas in your deployment. You can do this by writing a `reconfigure` method -for the class underlying your deployment. At runtime, you can then pass in your -new parameters by setting the `user_config` option. - -The following simple example will make the usage clear: - -```{literalinclude} ../../../python/ray/serve/examples/doc/snippet_reconfigure.py -``` - -The `reconfigure` method is called when the class is created if `user_config` -is set. In particular, it's also called when new replicas are created in the -future if scale up your deployment later. The `reconfigure` method is also called -each time `user_config` is updated. - diff --git a/doc/source/serve/managing-java-deployments.md b/doc/source/serve/managing-java-deployments.md new file mode 100644 index 0000000000000..76aecac897709 --- /dev/null +++ b/doc/source/serve/managing-java-deployments.md @@ -0,0 +1,139 @@ +# Experimental Java API + +:::{warning} +Java API support is an experimental feature and subject to change. +::: + +Java is a mainstream programming language for production services. Ray Serve offers a native Java API for creating, updating, and managing deployments. You can create Ray Serve deployments using Java and call them via Python, or vice versa. + +This section helps you to: + +- create, query, and update Java deployments +- configure Java deployment resources +- manage Python deployments using the Java API + +```{contents} +``` + +## Creating a Deployment + +By specifying the full name of the class as an argument to the `Serve.deployment()` method, as shown in the code below, you can create and deploy a deployment of the class. + +```{literalinclude} ../../../java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java +:start-after: docs-create-start +:end-before: docs-create-end +:language: java +``` + +## Accessing a Deployment + +Once a deployment is deployed, you can fetch its instance by name. + +```{literalinclude} ../../../java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java +:start-after: docs-query-start +:end-before: docs-query-end +:language: java +``` + +## Updating a Deployment + +You can update a deployment's code and configuration and then redeploy it. The following example updates the `"counter"` deployment's initial value to 2. + +```{literalinclude} ../../../java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java +:start-after: docs-update-start +:end-before: docs-update-end +:language: java +``` + +## Configuring a Deployment + +Ray Serve lets you configure your deployments to: + +- scale out by increasing the number of [deployment replicas](serve-architecture-high-level-view) +- assign [replica resources](serve-cpus-gpus) such as CPUs and GPUs. + +The next two sections describe how to configure your deployments. + +### Scaling Out + +By specifying the `numReplicas` parameter, you can change the number of deployment replicas: + +```{literalinclude} ../../../java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java +:start-after: docs-scale-start +:end-before: docs-scale-end +:language: java +``` + +### Resource Management (CPUs, GPUs) + +Through the `rayActorOptions` parameter, you can reserve resources for each deployment replica, such as one GPU: + +```{literalinclude} ../../../java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java +:start-after: docs-resource-start +:end-before: docs-resource-end +:language: java +``` + +## Managing a Python Deployment + +A Python deployment can also be managed and called by the Java API. Suppose you have a Python file `counter.py` in the `/path/to/code/` directory: + +```python +from ray import serve + +@serve.deployment +class Counter(object): + def __init__(self, value): + self.value = int(value) + + def increase(self, delta): + self.value += int(delta) + return str(self.value) + +``` + +You can deploy it through the Java API and call it through a `RayServeHandle`: + +```java +import io.ray.api.Ray; +import io.ray.serve.api.Serve; +import io.ray.serve.deployment.Deployment; +import io.ray.serve.generated.DeploymentLanguage; +import java.io.File; + +public class ManagePythonDeployment { + + public static void main(String[] args) { + + System.setProperty( + "ray.job.code-search-path", + System.getProperty("java.class.path") + File.pathSeparator + "/path/to/code/"); + + Serve.start(true, false, null); + + Deployment deployment = + Serve.deployment() + .setDeploymentLanguage(DeploymentLanguage.PYTHON) + .setName("counter") + .setDeploymentDef("counter.Counter") + .setNumReplicas(1) + .setInitArgs(new Object[] {"1"}) + .create(); + deployment.deploy(true); + + System.out.println(Ray.get(deployment.getHandle().method("increase").remote("2"))); + } +} + +``` + +:::{note} +Before `Ray.init` or `Serve.start`, you need to specify a directory to find the Python code. For details, please refer to [Cross-Language Programming](cross_language). +::: + +## Future Roadmap + +In the future, Ray Serve plans to provide more Java features, such as: +- an improved Java API that matches the Python version +- HTTP ingress support +- bring-your-own Java Spring project as a deployment diff --git a/doc/source/serve/migration.md b/doc/source/serve/migration.md new file mode 100644 index 0000000000000..f19bac4cae7cb --- /dev/null +++ b/doc/source/serve/migration.md @@ -0,0 +1,104 @@ +(serve-migration)= + +# 1.x to 2.x API Migration Guide + +This section covers what to consider or change in your application when migrating from Ray versions 1.x to 2.x. + +## What has been changed? + +In Ray Serve 2.0, we released a [new deployment API](converting-to-ray-serve-deployment). The 1.x deployment API can still be used, but it will be deprecated in the future version. + + +## Migrating the 1.x Deployment + +### Migrating handle pass between deployments +In the 1.x deployment, we usually pass handle of deployment to chain the deployments. +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __raw_handle_graph_start__ +:end-before: __raw_handle_graph_end__ +:language: python +``` + +With the 2.0 deployment API, you can use the following code to update the above one. +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __graph_with_new_api_start__ +:end-before: __graph_with_new_api_end__ +:language: python +``` + +:::{note} +- `get_handle` can be replaced by `bind()` function to fulfill same functionality. +- `serve.run` will return the entry point deployment handle for your whole chained deployments. +::: + +### Migrating a single deployment to the new deployment API + +In the 1.x deployment API, we usually have the following code for deployment. +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __single_deployment_old_api_start__ +:end-before: __single_deployment_old_api_end__ +:language: python +``` + +With the 2.0 deployment API, you can use the following code to update the above one. +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __single_deployment_new_api_start__ +:end-before: __single_deployment_new_api_end__ +:language: python +``` + + +### Migrate Multiple deployment to new deployment API + +When you have multiple deployments, here is the normal code for 1.x API + +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __multi_deployments_old_api_start__ +:end-before: __multi_deployments_old_api_end__ +:language: python +``` + +With the 2.0 deployment API, you can use the following code to update the above one. + +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __multi_deployments_new_api_start__ +:end-before: __multi_deployments_new_api_end__ +:language: python +``` + + +:::{note} +- `predict` method is defined inside `DAGDriver` class as an entry point to fulfil requests +- Similar to `predict` method, `predict_with_route` method is defined inside `DAGDriver` class as an entry point to fulfil requests. +- `DAGDriver` is a special class to handle multi entry points for different deployments +- `DAGDriver.bind` can accept dictionary and each key is represented as entry point route path. +- `predict_with_route` accepts a route path as the first argument to select which model to use. +- In the example, you can also use an HTTP request to fulfill your request. Different models will bind with different route paths based on the user inputs; e.g. http://localhost:8000/model1 and http://localhost:8000/model2 +::: + + +### Migrate deployments with route prefixes + +Sometimes, you have a customized route prefix for each deployment: + +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __customized_route_old_api_start__ +:end-before: __customized_route_old_api_end__ +:language: python +``` + +With the 2.0 deployment API, you can use the following code to update the above one. + +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __customized_route_old_api_1_start__ +:end-before: __customized_route_old_api_1_end__ +:language: python +``` + +Or if you have multiple deployments and want to customize the HTTP route prefix for each model, you can use the following code: + +```{literalinclude} ../serve/doc_code/migration_example.py +:start-after: __customized_route_old_api_2_start__ +:end-before: __customized_route_old_api_2_end__ +:language: python +``` diff --git a/doc/source/serve/ml-models.md b/doc/source/serve/ml-models.md deleted file mode 100644 index f7c05894d4cd9..0000000000000 --- a/doc/source/serve/ml-models.md +++ /dev/null @@ -1,301 +0,0 @@ -(serve-ml-models-tutorial)= -# Serving ML Models - -This section should help you: - -- batch requests to optimize performance -- serve multiple models by composing deployments -- serve multiple models by making ensemble deployments - -```{contents} -``` - -(serve-batching)= - -## Request Batching - -You can also have Ray Serve batch requests for performance, which is especially important for some ML models that run on GPUs. In order to use this feature, you need to do the following two things: - -1. Use `async def` for your request handling logic to process queries concurrently. -2. Use the `@serve.batch` decorator to batch individual queries that come into the replica. The method/function that's decorated should handle a list of requests and return a list of the same length. - -```python -@serve.deployment(route_prefix="/increment") -class BatchingExample: - def __init__(self): - self.count = 0 - - @serve.batch - async def handle_batch(self, requests): - responses = [] - for request in requests: - responses.append(request.json()) - - return responses - - async def __call__(self, request): - return await self.handle_batch(request) - -BatchingExample.deploy() -``` - -Please take a look at [Batching Tutorial](serve-batch-tutorial) for a deep -dive. - -(serve-model-composition)= - -## Model Composition - -:::{note} -Serve recently added an experimental API for building deployment graphs of multiple models. -Please take a look at the [Deployment Graph API](serve-deployment-graph) and try it out! -::: - -Ray Serve supports composing individually scalable models into a single model -out of the box. For instance, you can combine multiple models to perform -stacking or ensembles. - -To define a higher-level composed model you need to do three things: - -1. Define your underlying models (the ones that you will compose together) as - Ray Serve deployments. -2. Define your composed model, using the handles of the underlying models - (see the example below). -3. Define a deployment representing this composed model and query it! - -In order to avoid synchronous execution in the composed model (e.g., it's very -slow to make calls to the composed model), you'll need to make the function -asynchronous by using an `async def`. You'll see this in the example below. - -That's it. Let's take a look at an example: - -```{literalinclude} ../../../python/ray/serve/examples/doc/snippet_model_composition.py -``` - -(serve-model-ensemble)= - -## Model Ensemble - -Ray Serve supports creating different ensemble models - -To define an ensemble of different models you need to do three things: - -1. Define your underlying sub models (the ones that make up the ensemble) as - Ray Serve deployments. -2. Define your ensemble model, using the handles of the underlying models - (see the example below). -3. Define a deployment representing this ensemble model and query it! - -In order to avoid synchronous execution in the ensemble model, you'll need to make -the function asynchronous by using an `async def`. In contrast to a composition model, -within an ensemble model, you want to call **all** sub models in parallel. This will be -achieved by sending all prediction calls to the sub models via async by using -`asyncio.wait()`. Each serve deployment used in an ensemble use case is independently -scalable via changing `num_replicas`. - -That's it. Let's take a look at an example: - -```{literalinclude} ../../../python/ray/serve/examples/doc/snippet_model_ensemble.py -``` - -## Integration with Model Registries - -Ray Serve is flexible. If you can load your model as a Python -function or class, then you can scale it up and serve it with Ray Serve. - -For example, if you are using the -[MLflow Model Registry](https://www.mlflow.org/docs/latest/model-registry.html) -to manage your models, the following wrapper -class will allow you to load a model using its MLflow `Model URI`: - -```python -import pandas as pd -import mlflow.pyfunc - -@serve.deployment -class MLflowDeployment: - def __init__(self, model_uri): - self.model = mlflow.pyfunc.load_model(model_uri=model_uri) - - async def __call__(self, request): - csv_text = await request.body() # The body contains just raw csv text. - df = pd.read_csv(csv_text) - return self.model.predict(df) - -model_uri = "model:/my_registered_model/Production" -MLflowDeployment.deploy(model_uri) -``` - -To serve multiple different MLflow models in the same program, use the `name` option: - -```python -MLflowDeployment.options(name="my_mlflow_model_1").deploy(model_uri) -``` - -:::{tip} -The above approach will work for any model registry, not just MLflow. -Namely, load the model from the registry in `__init__`, and forward the request to the model in `__call__`. -::: - -For a complete hands-on and seamless integration with MLflow, try this self-contained example on your laptop. -But first install `mlflow`. - -```bash -pip install mlflow -``` - -```python -# This brief example shows how to deploy models saved in a model registry such as -# MLflow to Ray Serve, using the simple Ray Serve deployment APIs. You can peruse -# the saved models' metrics and parameters in MLflow ui. -# -import json -import numpy as np -import pandas as pd -import requests -import os -import tempfile - -from sklearn.datasets import load_iris -from sklearn.ensemble import GradientBoostingClassifier -from mlflow.tracking import MlflowClient - -from ray import serve -import mlflow - - -def create_and_save_model(): - # load Iris data - iris_data = load_iris() - data, target, target_names = (iris_data['data'], - iris_data['target'], - iris_data['target_names']) - - # Instantiate a model - model = GradientBoostingClassifier() - - # Training and validation split - np.random.shuffle(data), np.random.shuffle(target) - train_x, train_y = data[:100], target[:100] - val_x, val_y = data[100:], target[100:] - - # Create labels list as file - LABEL_PATH = os.path.join(tempfile.gettempdir(), "iris_labels.json") - with open(LABEL_PATH, "w") as f: - json.dump(target_names.tolist(), f) - - # Train the model and save our label list as an MLflow artifact - # mlflow.sklearn.autolog automatically logs all parameters and metrics during - # the training. - mlflow.sklearn.autolog() - with mlflow.start_run() as run: - model.fit(train_x, train_y) - # Log label list as a artifact - mlflow.log_artifact(LABEL_PATH, artifact_path="labels") - return run.info.run_id - -# -# Create our Ray Serve deployment class -# - - -@serve.deployment(route_prefix="/regressor") -class BoostingModel: - def __init__(self, uri): - # Load the model and label artifact from the local - # Mlflow model registry as a PyFunc Model - self.model = mlflow.pyfunc.load_model(model_uri=uri) - - # Download the artifact list of labels - local_dir = "/tmp/artifact_downloads" - if not os.path.exists(local_dir): - os.mkdir(local_dir) - client = MlflowClient() - local_path = f"{client.download_artifacts(run_id, 'labels', local_dir)}/iris_labels.json" - with open(local_path, "r") as f: - self.label_list = json.load(f) - - async def __call__(self, starlette_request): - payload = await starlette_request.json() - print(f"Worker: received Starlette request with data: {payload}") - - # Get the input vector from the payload - input_vector = [ - payload["sepal length"], - payload["sepal width"], - payload["petal length"], - payload["petal width"], - ] - - # Convert the input vector in a Pandas DataFrame for prediction since - # an MLflow PythonFunc model, model.predict(...), takes pandas DataFrame - prediction = self.model.predict(pd.DataFrame([input_vector]))[0] - human_name = self.label_list[prediction] - return {"result": human_name} - - -if __name__ == '__main__': - - # Train and save the model artifacts in MLflow. - # Here our MLflow model registry is local file - # directory ./mlruns - run_id = create_and_save_model() - - # Start the Ray Serve instance - serve.start() - # Construct model uri to load the model from our model registry - uri = f"runs:/{run_id}/model" - # Deploy our model. - BoostingModel.deploy(uri) - - # Send in a request for labels types virginica, setosa, versicolor - sample_request_inputs = [{ - "sepal length": 6.3, - "sepal width": 3.3, - "petal length": 6.0, - "petal width": 2.5}, - { - "sepal length": 5.1, - "sepal width": 3.5, - "petal length": 1.4, - "petal width": 0.2}, - { - "sepal length": 6.4, - "sepal width": 3.2, - "petal length": 4.5, - "petal width": 1.5}, - ] - for input_request in sample_request_inputs: - response = requests.get("http://localhost:8000/regressor", - json=input_request) - print(response.text) - - print("Launch MLflow ui to see the model parameters, metrics, and artifacts: `mlflow ui` from current directory.") - - #output - #{ - # "result": "versicolor" - #} - #{ - # "result": "virginica" - #} - #{ - # "result": "setosa" - #} - # - # Launch MLflow ui to see the model parameters, metrics, and artifacts: `mlflow ui` from current directory. -``` - -For an even more hands-off and seamless integration with MLflow, check out the -[Ray Serve MLflow deployment plugin](https://github.com/ray-project/mlflow-ray-serve). A full -tutorial is available [here](https://github.com/mlflow/mlflow/tree/master/examples/ray_serve). - -## Framework-Specific Tutorials - -Ray Serve seamlessly integrates with popular Python ML libraries. -Below are tutorials with some of these frameworks to help get you started. - -- [PyTorch Tutorial](serve-pytorch-tutorial) -- [Scikit-Learn Tutorial](serve-sklearn-tutorial) -- [Keras and Tensorflow Tutorial](serve-tensorflow-tutorial) -- [RLlib Tutorial](serve-rllib-tutorial) diff --git a/doc/source/serve/model_composition.md b/doc/source/serve/model_composition.md new file mode 100644 index 0000000000000..17d33916413af --- /dev/null +++ b/doc/source/serve/model_composition.md @@ -0,0 +1,398 @@ +(serve-model-composition)= + +# Model Composition + +This section helps you: + +* compose multiple deployments containing ML logic or business logic into a single application +* independently scale and configure each of your ML models and business logic steps +* connect your Ray Serve deployments together with the **deployment graph** API + +(serve-handle-explainer)= + +## Composing Deployments using ServeHandles + +You can call deployment methods from within other deployments using the [ServeHandle](serve-key-concepts-query-deployment). This lets you divide your application's steps (such as preprocessing, model inference, and post-processing) into independent deployments that can be independently scaled and configured. + +To use the `ServeHandle`, use {mod}`handle.remote ` to send requests to a deployment. +These requests can be ordinary Python args and kwargs that are passed directly to the method. This method call returns a Ray `ObjectRef` whose result can be waited for or retrieved using `await` or `ray.get`. + +(serve-model-composition-serve-handles)= +### Model Composition Example + +Here's an example: + +```{literalinclude} doc_code/model_composition/class_nodes.py +:start-after: __hello_start__ +:end-before: __hello_end__ +:language: python +:linenos: true +``` + +In line 40, the `LanguageClassifier` deployment takes in the `spanish_responder` and `french_responder` as constructor arguments. At runtime, these arguments are converted into `ServeHandles`. `LanguageClassifier` can then call the `spanish_responder` and `french_responder`'s deployment methods using this handle. + +For example, the `LanguageClassifier`'s `__call__` method uses the HTTP request's values to decide whether to respond in Spanish or French. It then forwards the request's name to the `spanish_responder` or the `french_responder` on lines 17 and 19 using the `ServeHandles`. The calls are formatted as: + +```python +await self.spanish_responder.say_hello.remote(name) +``` + +This call has a few parts: +* `await` lets us issue an asynchronous request through the `ServeHandle`. +* `self.spanish_responder` is the `SpanishResponder` handle taken in through the constructor. +* `say_hello` is the `SpanishResponder` method to invoke. +* `remote` indicates that this is a `ServeHandle` call to another deployment. This is required when invoking a deployment's method through another deployment. It needs to be added to the method name. +* `name` is the argument for `say_hello`. You can pass any number of arguments or keyword arguments here. + +This call returns a reference to the result– not the result itself. This pattern allows the call to execute asynchronously. To get the actual result, `await` the reference. `await` blocks until the asynchronous call executes, and then it returns the result. In this example, line 23 calls `await ref` and returns the resulting string. **Note that getting the result needs two `await` statements in total**. First, the script must `await` the `ServeHandle` call itself to retrieve a reference. Then it must `await` the reference to get the final result. + +(serve-model-composition-await-warning)= +:::{warning} +You can use the `ray.get(ref)` method to get the return value of remote `ServeHandle` calls. However, calling `ray.get` from inside a deployment is an antipattern. It blocks the deployment from executing any other code until the call is finished. Using `await` lets the deployment process other requests while waiting for the `ServeHandle` call to finish. You should use `await` instead of `ray.get` inside deployments. +::: + +You can copy the `hello.py` script above and run it with `serve run`. Make sure to run the command from a directory containing `hello.py`, so it can locate the script: + +```console +$ serve run hello:language_classifier +``` + +You can use this client script to interact with the example: + +```{literalinclude} doc_code/model_composition/class_nodes.py +:start-after: __hello_client_start__ +:end-before: __hello_client_end__ +:language: python +``` + +While the `serve run` command is running, open a separate terminal window and run this script: + +```console +$ python hello_client.py + +Hola Dora +``` + +:::{note} +Composition lets you break apart your application and independently scale each part. For instance, suppose this `LanguageClassifier` application's requests were 75% Spanish and 25% French. You could scale your `SpanishResponder` to have 3 replicas and your `FrenchResponder` to have 1 replica, so you could meet your workload's demand. This flexibility also applies to reserving resources like CPUs and GPUs, as well as any other configurations you can set for each deployment. + +With composition, you can avoid application-level bottlenecks when serving models and business logic steps that use different types and amounts of resources. +::: + +### ServeHandle Deep Dive + +Conceptually, a `ServeHandle` is a client-side load balancer, routing requests to any replicas of a given deployment. Also, it performs buffering internally so it won't overwhelm the replicas. +Using the current number of requests buffered, it informs the autoscaler to scale up the number of replicas. + +![architecture-diagram-of-serve-handle](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/serve-handle-explainer.png) + +`ServeHandle`s take request parameters and returns a future object of type [`ray.ObjectRef`](objects-in-ray), whose value will be filled with the result object. Because of the internal buffering, the time from submitting a request to getting a `ray.ObjectRef` can vary. + +Because of this variability, Serve offers two types of handles to ensure the buffering period is handled efficiently. We offer synchronous and asynchronous versions of the handle: +- `RayServeSyncHandle` directly returns a `ray.ObjectRef`. It blocks the current thread until the request is matched to a replica. +- `RayServeDeploymentHandle` returns an `asyncio.Task` upon submission. The `asyncio.Task` can be awaited to resolve to a `ray.ObjectRef`. While the current request is buffered, other requests can be processed concurrently. + +`serve.run` deploys a deployment graph and returns the entrypoint node’s handle (the node you passed as argument to `serve.run`). The return type is a `RayServeSyncHandle`. This is useful for interacting with and testing the newly created deployment graph. + +```{literalinclude} ../serve/doc_code/handle_guide.py +:start-after: __begin_sync_handle__ +:end-before: __end_sync_handle__ +:language: python +``` + +In all other cases, `RayServeDeploymentHandle` is the default because the API is more performant than its blocking counterpart. For example, when implementing a dynamic dispatch node in deployment graph, the handle is asynchronous. + +```{literalinclude} ../serve/doc_code/handle_guide.py +:start-after: __begin_async_handle__ +:end-before: __end_async_handle__ +:language: python +``` + +The result of `deployment_handle.remote()` can also be passed directly as an argument to other downstream handles, without having to await on it. + +```{literalinclude} ../serve/doc_code/handle_guide.py +:start-after: __begin_async_handle_chain__ +:end-before: __end_async_handle_chain__ +:language: python +``` + +In both types of `ServeHandle`, you can call a specific method by using the `.method_name` accessor. For example: + +```{literalinclude} ../serve/doc_code/handle_guide.py +:start-after: __begin_handle_method__ +:end-before: __end_handle_method__ +:language: python +``` + +:::{note} +`ray.ObjectRef` corresponds to the result of a request submission. To retrieve the result, you can use the synchronous Ray Core API `ray.get(ref)` or the async API `await ref`. To wait for the result to be available without retrieving it, you can use the synchronous API `ray.wait([ref])` or the async API `await asyncio.wait([ref])`. You can mix and match these calls, but we recommend using async APIs to increase concurrency. +::: + +(serve-model-composition-deployment-graph)= +## Deployment Graph API + +:::{note} +The call graph is in **alpha**, so its APIs are subject to change. +::: + +For more advanced composition patterns, it can be useful to surface the relationships between deployments, instead of hiding them inside individual deployment definitions. + +Ray Serve's **deployment graph API** lets you specify how to route requests through your deployments, so you can explicitly create a dependency graph. It also has additional features like HTTP adapters and input routing that help you build more expressive graphs. + +### Binding Deployments + +The basic building block for all deployment graphs is the `DeploymentNode`. One type of `DeploymentNode` is the `ClassNode`. You can create `ClassNodes` by binding class-based deployments to their constructor's arguments with the `bind` method. This may sound familiar because you've already been doing this whenever you bind and run class-based deployments, such as in the [Calling Deployments using ServeHandles](serve-model-composition-serve-handles) section. + +As another example: + +```{literalinclude} doc_code/model_composition/class_nodes.py +:start-after: __echo_class_start__ +:end-before: __echo_class_end__ +:language: python +``` + +`echo.py` defines three `ClassNodes`: `foo_node`, `bar_node`, and `baz_node`. The nodes are defined by invoking `bind` on the `EchoClass` deployment. They have different behaviors because they use different arguments in the `bind` call. + +Note that all three of these nodes were created from the same `EchoClass` deployment. Class deployments are essentially factories for `ClassNodes`. A single class deployment can produce multiple `ClassNodes` through multiple `bind` statements. + +There are two options to run a node: + +1. `serve.run(node)`: This Python call can be added to your Python script to run a particular node. This call starts a Ray cluster (if one isn't already running), deploys the node to it, and then returns. You can call this function multiple times in the same script on different `DeploymentNodes`. Each time, it tears down any deployments it previously deployed and deploy the passed-in node's deployment. After the script exits, the cluster and any nodes deployed by `serve.run` are torn down. + +2. `serve run module:node`: This CLI command starts a Ray cluster and runs the node at the import path `module:node`. It then blocks, allowing you to open a separate terminal window and issue requests to the running deployment. You can stop the `serve run` command with `ctrl-c`. + +When you run a node, you are deploying the node's deployment and its bound arguments. Ray Serve creates a deployment in Ray and instantiates your deployment's class using the arguments. By default, you can send requests to your deployment at `http://localhost:8000`. These requests are converted to Starlette `request` objects and passed to your class's `__call__` method. + +:::{note} +Additionally, when you run a node, the deployment's configurations (which you can set in the `@serve.deployment` decorator, through an `options` call, or a [Serve config file](serve-in-production-config-file)) still apply to the deployment. You can use this to independently scale and configure your graph's deployments by, for instance, setting different `num_replicas`, `num_cpus`, or `num_gpus` values for different deployments. +::: + +You can try this example out using the `serve run` CLI: + +```console +$ serve run echo:foo_node +``` + +Here's a client script that can send requests to your node: + +```{literalinclude} doc_code/model_composition/class_nodes.py +:start-after: __echo_client_start__ +:end-before: __echo_client_end__ +:language: python +``` + +While the deployment is running with `serve run`, open a separate terminal window and issue a request to it with the `echo_client.py` script: + +``` +$ python echo_client.py + +foo +``` + +(deployment-graph-call-graph)= +### Building the Call Graph: MethodNodes and FunctionNodes + +After defining your `ClassNodes`, you can specify how HTTP requests should be processed using the call graph. As an example, let's look at a deployment graph that implements this chain of arithmetic operations: + +``` +output = request + 2 - 1 + 3 +``` + +Here's the graph: + +(deployment-graph-arithmetic-graph)= +```{literalinclude} doc_code/model_composition/arithmetic.py +:start-after: __graph_start__ +:end-before: __graph_end__ +:language: python +:linenos: true +``` + +Lines 29 and 30 bind two `ClassNodes` from the `AddCls` deployment. Line 32 starts the call graph: + +```python +with InputNode() as http_request: + request_number = unpack_request.bind(http_request) + add_2_output = add_2.add.bind(request_number) + subtract_1_output = subtract_one_fn.bind(add_2_output) + add_3_output = add_3.add.bind(subtract_1_output) +``` + +The `with` statement (known as a "context manager" in Python) initializes a special Ray Serve-provided object called an `InputNode`. This isn't a `DeploymentNode` like `ClassNodes`, `MethodNodes`, or `FunctionNodes`. Rather, it's the input of the graph. In this case, that input is an HTTP request. In a [later section](deployment-graph-drivers-http-adapters), you'll learn how to change this input using another Ray Serve-provided object called the `DAGDriver`. + +(deployment-graph-call-graph-input-node-note)= +:::{note} +The `InputNode` tells Ray Serve where to send the graph input at runtime. In this example, for instance, `http_request` is an `InputNode` object, so you can't call `request` methods like `.json()` on it directly in the context manager. However, during runtime, Ray Serve passes incoming HTTP requests directly into the same functions and methods that `http_request` is passed into, so those functions and methods can call `request` methods like `.json()` on the `request` object that gets passed in. +::: + +You can use the `InputNode` to indicate which node(s) the graph input should be passed into by passing the `InputNode` into `bind` calls within the context manager. In this example, the `http_request` is passed to only one node, `unpack_request`. The output of that bind call, `request_number`, is a `FunctionNode`. `FunctionNodes` are produced when deployments containing functions are bound to arguments for that function using `bind`. `request_number` represents the output of `unpack_request` when called on incoming HTTP requests. `unpack_request`, which is defined on line 26, processes the HTTP request's JSON body and returns a number that can be passed into arithmetic operations. + +:::{tip} +If you don't want to manually unpack HTTP requests, check out this guide's section on [HTTP adapters](deployment-graph-drivers-http-adapters), which can handle unpacking for you. +::: + +The graph then passes `request_number` into a `bind` call on `add_2`'s `add` method. The output of this call, `add_2_output` is a `MethodNode`. `MethodNodes` are produced when `ClassNode` methods are bound to arguments using `bind`. In this case, `add_2_output` represents the result of adding 2 to the number in the request. + +The rest of the call graph uses another `FunctionNode` and `MethodNode` to finish the chain of arithmetic. `add_2_output` is bound to the `subtract_one_fn` deployment, producing the `subtract_1_output` `FunctionNode`. Then, the `subtract_1_output` is bound to the `add_3.add` method, producing the `add_3_output` `MethodNode`. This `add_3_output` `MethodNode` represents the final output from the chain of arithmetic operations. + +To run the call graph, you need to use a driver. Drivers are deployments that process the call graph that you've written and route incoming requests through your deployments based on that graph. Ray Serve provides a driver called `DAGDriver` used on line 38: + +```python +deployment_graph = DAGDriver.bind(add_3_output) +``` + +Generally, the `DAGDriver` needs to be bound to the `FunctionNode` or `MethodNode` representing the final output of a graph. This `bind` call returns a `ClassNode` that you can run in `serve.run` or `serve run`. Running this `ClassNode` also deploys the rest of the graph's deployments. + +:::{note} +The `DAGDriver` can also be bound to `ClassNodes`. This is useful if you construct a deployment graph where `ClassNodes` invoke other `ClassNodes`' methods. In this case, you should pass in the "root" `ClassNode` to `DAGDriver` (i.e. the one that you would otherwise pass into `serve.run`). Check out the [Calling Deployments using ServeHandles](serve-model-composition-serve-handles) section for more info. +::: + +You can test this example using this client script: + +```{literalinclude} doc_code/model_composition/arithmetic.py +:start-after: __graph_client_start__ +:end-before: __graph_client_end__ +:language: python +``` + +Start the graph in the terminal: + +```console +$ serve run arithmetic:graph +``` + +In a separate terminal window, run the client script to make requests to the graph: + +```console +$ python arithmetic_client.py + +9 +``` + +(deployment-graph-drivers-http-adapters)= +### Drivers and HTTP Adapters + +Ray Serve provides the `DAGDriver`, which routes HTTP requests through your call graph. As mentioned in [the call graph section](deployment-graph-call-graph), the `DAGDriver` takes in a `DeploymentNode` and it produces a `ClassNode` that you can run. + +The `DAGDriver` also has an optional keyword argument: `http_adapter`. [HTTP adapters](serve-http-adapters) are functions that get run on the HTTP request before it's passed into the graph. Ray Serve provides a handful of these adapters, so you can rely on them to conveniently handle the HTTP parsing while focusing your attention on the graph itself. + +For instance, you can use the Ray Serve-provided `json_request` adapter to simplify the [arithmetic call graph](deployment-graph-arithmetic-graph) by eliminating the `unpack_request` function. You can replace lines 29 through 38 with this graph: + +(http-adapter-arithmetic-example)= +```{literalinclude} doc_code/model_composition/arithmetic.py +:start-after: __adapter_graph_start__ +:end-before: __adapter_graph_end__ +:language: python +``` + +Without an `http_adapter`, an `InputNode` [represents an HTTP request](deployment-graph-call-graph-input-node-note), and at runtime, incoming HTTP `request` objects are passed into the same functions and methods that the `InputNode` is passed into. When you set an `http_adapter`, the `InputNode` represents the `http_adapter`'s output. + +At runtime: + +1. Ray Serve sends each HTTP `request` object to the `DAGDriver`. +2. The `DAGDriver` calls the `http_adapter` function on each request. +3. The `DAGDriver` passes the `http_adapter` output to the same function and methods that the `InputNode` is passed into, kicking off the request's journey through the call graph. + +In the example above, the `InputNode` represents the number packaged inside the request's JSON body instead of the HTTP request itself. You can pass the JSON directly into the graph instead of first unpacking it from the request. + +See [the guide](serve-http-adapters) on `http_adapters` to learn more. + +(deployment-graph-call-graph-testing)= +### Testing the Graph with the Python API + +The `serve.run` function returns a handle that you can use to test your graph in Python, without using HTTP requests. + +To test your graph, + +1. Call `serve.run` on your graph and store the returned handle. +2. Call `handle.predict.remote(input)`. **The `input` argument becomes the input represented by `InputNode`**. Make sure to refactor your call graph accordingly, since it takes in this input directly, instead of an HTTP request. You can use an [HTTP adapter](deployment-graph-drivers-http-adapters) to make sure the graph you're testing matches the one you ultimately deploy. +3. `predict.remote` returns a reference to the result, so the graph can execute asynchronously. Call `ray.get` on this reference to get the final result. + +As an example, you can continue rewriting the [arithmetic graph example](http-adapter-arithmetic-example) from above to use `predict.remote`. You can add testing code to the example: + +```{literalinclude} doc_code/model_composition/arithmetic.py +:start-after: __test_graph_start__ +:end-before: __test_graph_end__ +:language: python +``` + +Note that the graph itself is still the same. The only change is the testing code added after it. You can run this Python script directly now to test the graph: + +``` +$ python arithmetic.py + +9 +``` + +### Visualizing the Graph + +You can render an illustration of your deployment graph to see its nodes and their connection. + +Make sure you have `pydot` and `graphviz` to follow this section: + +::::{tabbed} MacOS +``` +pip install -U pydot && brew install graphviz +``` +:::: + +::::{tabbed} Windows +``` +pip install -U pydot && winget install graphviz +``` +:::: + +::::{tabbed} Linux +``` +pip install -U pydot && sudo apt-get install -y graphviz +``` +:::: + +Here's an example graph: + +```{literalinclude} doc_code/model_composition/deployment_graph_viz.py +:language: python +``` + +The `ray.dag.vis_utils._dag_to_dot` method takes in a `DeploymentNode` and produces a graph visualization. You can see the string form of the visualization by running the script: + +```console +$ python deployment_graph_viz.py + +digraph G { +rankdir=LR; +INPUT_ATTRIBUTE_NODE -> forward; +INPUT_NODE -> INPUT_ATTRIBUTE_NODE; +Model -> forward; +} + +digraph G { +rankdir=LR; +forward -> combine; +INPUT_ATTRIBUTE_NODE -> forward; +INPUT_NODE -> INPUT_ATTRIBUTE_NODE; +Model -> forward; +forward_1 -> combine; +INPUT_ATTRIBUTE_NODE_1 -> forward_1; +INPUT_NODE -> INPUT_ATTRIBUTE_NODE_1; +Model_1 -> forward_1; +INPUT_ATTRIBUTE_NODE_2 -> combine; +INPUT_NODE -> INPUT_ATTRIBUTE_NODE_2; +} +``` + +You can render these strings in `graphviz` tools such as [https://dreampuf.github.io/GraphvizOnline](https://dreampuf.github.io/GraphvizOnline). + +When the script visualizes `m1_output`, it shows a partial execution path of the entire graph: + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/visualize_partial.svg) + +This path includes only the dependencies needed to generate `m1_output`. + +On the other hand, when the script visualizes the final graph output, `combine_output`, it captures all nodes used in execution since they're all required to create the final output. + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/visualize_full.svg) + +## Next Steps + +To learn more about deployment graphs, check out some [deployment graph patterns](serve-deployment-graph-patterns-overview) you can incorporate into your own graph! diff --git a/doc/source/serve/monitoring.md b/doc/source/serve/monitoring.md deleted file mode 100644 index 48837a4d89d5d..0000000000000 --- a/doc/source/serve/monitoring.md +++ /dev/null @@ -1,266 +0,0 @@ -(serve-monitoring)= - -# Debugging & Monitoring - -This section should help you understand how to debug and monitor your Serve application. - -## Ray Dashboard - -A high-level way to monitor your Ray Serve application is via the Ray Dashboard. -See the [Ray Dashboard documentation](ray-dashboard) for a detailed overview, including instructions on how to view the dashboard. - -Below is an example of what the Ray Dashboard might look like for a Serve deployment: - -```{image} https://raw.githubusercontent.com/ray-project/Images/master/docs/dashboard/serve-dashboard.png -:align: center -``` - -Here you can see the Serve controller actor, an HTTP proxy actor, and all of the replicas for each Serve deployment. -To learn about the function of the controller and proxy actors, see the [Serve Architecture page](serve-architecture). -In this example pictured above, we have a single-node cluster with a deployment named Counter with `num_replicas=2`. - -## Logging - -:::{note} -For an overview of logging in Ray, see [Ray Logging](ray-logging). -::: - -Ray Serve uses Python's standard `logging` facility with the `"ray.serve"` named logger. -By default, logs are emitted from actors both to `stderr` and on disk on each node at `/tmp/ray/session_latest/logs/serve/`. -This includes both system-level logs from the Serve controller and HTTP proxy as well as access logs and custom user logs produced from within deployment replicas. - -In development, logs are streamed to the driver Ray program (the program that calls `.deploy()` or `serve.run`, or the `serve run` CLI command) that deployed the deployments, so it's most convenient to keep the driver running for debugging. -For example, let's run a basic Serve application and view the logs that are emitted. -You can run this in an interactive shell like IPython to follow along. - -First we call `serve.start()`: - -```python -from ray import serve - -serve.start() -``` - -This produces a few INFO-level log messages about startup from the Serve controller. - -```bash -2022-04-02 09:10:49,906 INFO services.py:1460 -- View the Ray dashboard at http://127.0.0.1:8265 -(ServeController pid=67312) INFO 2022-04-02 09:10:51,386 controller 67312 checkpoint_path.py:17 - Using RayInternalKVStore for controller checkpoint and recovery. -(ServeController pid=67312) INFO 2022-04-02 09:10:51,492 controller 67312 http_state.py:108 - Starting HTTP proxy with name 'SERVE_CONTROLLER_ACTOR:xlehoa:SERVE_PROXY_ACTOR-node:127.0.0.1-0' on node 'node:127.0.0.1-0' listening on '127.0.0.1:8000' -``` - -Next, let's create a simple deployment that logs a custom log message when it's queried: - -```python -import logging - -logger = logging.getLogger("ray.serve") - -@serve.deployment(route_prefix="/") -class SayHello: - def __call__(self, *args): - logger.info("Hello world!") - return "hi" - -SayHello.deploy() -``` - -Running this code block, we first get some log messages from the controller saying that a new replica of the deployment is being created: - -```bash -(ServeController pid=67312) INFO 2022-04-02 09:16:13,323 controller 67312 deployment_state.py:1198 - Adding 1 replicas to deployment 'SayHello'. -``` - -Then when we query the deployment, we get both a default access log as well as our custom `"Hello world!"` message. -Note that these log lines are tagged with the deployment name followed by a unique identifier for the specific replica. -These can be parsed by a logging stack such as ELK or Loki to enable searching logs by deployment and replica. - -```bash -handle = SayHello.get_handle() -ray.get(handle.remote()) -(SayHello pid=67352) INFO 2022-04-02 09:20:08,975 SayHello SayHello#LBINMh :8 - Hello world! -(SayHello pid=67352) INFO 2022-04-02 09:20:08,975 SayHello SayHello#LBINMh replica.py:466 - HANDLE __call__ OK 0.3ms -``` - -Querying the deployment over HTTP produces a similar access log message from the HTTP proxy: - -```bash -curl -X GET http://localhost:8000/ -(HTTPProxyActor pid=67315) INFO 2022-04-02 09:20:08,976 http_proxy 127.0.0.1 http_proxy.py:310 - GET / 200 2.6ms -(SayHello pid=67352) INFO 2022-04-02 09:20:08,975 SayHello SayHello#LBINMh :8 - Hello world! -(SayHello pid=67352) INFO 2022-04-02 09:20:08,975 SayHello SayHello#LBINMh replica.py:466 - HANDLE __call__ OK 0.3ms -``` - -You can also be able to view all of these log messages in the files in `/tmp/ray/session_latest/logs/serve/`. - -To silence the replica-level logs or otherwise configure logging, configure the `"ray.serve"` logger *from inside the deployment constructor:* - -```python -import logging - -logger = logging.getLogger("ray.serve") - -@serve.deployment -class Silenced: - def __init__(self): - logger.setLevel(logging.ERROR) -``` - -This will prevent the replica INFO-level logs from being written to STDOUT or to files on disk. -You can also use your own custom logger, in which case you'll need to configure the behavior to write to STDOUT/STDERR, files on disk, or both. - -### Tutorial: Ray Serve with Loki - -Here is a quick walkthrough of how to explore and filter your logs using [Loki](https://grafana.com/oss/loki/). -Setup and configuration is very easy on Kubernetes, but in this tutorial we'll just set things up manually. - -First, install Loki and Promtail using the instructions on . -It will be convenient to save the Loki and Promtail executables in the same directory, and to navigate to this directory in your terminal before beginning this walkthrough. - -Now let's get our logs into Loki using Promtail. - -Save the following file as `promtail-local-config.yaml`: - -```yaml -server: - http_listen_port: 9080 - grpc_listen_port: 0 - -positions: - filename: /tmp/positions.yaml - -clients: - - url: http://localhost:3100/loki/api/v1/push - -scrape_configs: -- job_name: ray -static_configs: - - labels: - job: ray - __path__: /tmp/ray/session_latest/logs/serve/*.* -``` - -The relevant part for Ray is the `static_configs` field, where we have indicated the location of our log files with `__path__`. -The expression `*.*` will match all files, but not directories, which cause an error with Promtail. - -We will run Loki locally. Grab the default config file for Loki with the following command in your terminal: - -```shell -wget https://raw.githubusercontent.com/grafana/loki/v2.1.0/cmd/loki/loki-local-config.yaml -``` - -Now start Loki: - -```shell -./loki-darwin-amd64 -config.file=loki-local-config.yaml -``` - -Here you may need to replace `./loki-darwin-amd64` with the path to your Loki executable file, which may have a different name depending on your operating system. - -Start Promtail and pass in the path to the config file we saved earlier: - -```shell -./promtail-darwin-amd64 -config.file=promtail-local-config.yaml -``` - -As above, you may need to replace `./promtail-darwin-amd64` with the appropriate filename and path. - -Now we are ready to start our Ray Serve deployment. Start a long-running Ray cluster and Ray Serve instance in your terminal: - -```shell -ray start --head -serve start -``` - -Now run the following Python script to deploy a basic Serve deployment with a Serve deployment logger: - -```{literalinclude} ../../../python/ray/serve/examples/doc/deployment_logger.py -``` - -Now [install and run Grafana](https://grafana.com/docs/grafana/latest/installation/) and navigate to `http://localhost:3000`, where you can log in with the default username "admin" and default password "admin". -On the welcome page, click "Add your first data source" and click "Loki" to add Loki as a data source. - -Now click "Explore" in the left-side panel. You are ready to run some queries! - -To filter all these Ray logs for the ones relevant to our deployment, use the following [LogQL](https://grafana.com/docs/loki/latest/logql/) query: - -```shell -{job="ray"} |= "Counter" -``` - -You should see something similar to the following: - -```{image} https://raw.githubusercontent.com/ray-project/Images/master/docs/serve/loki-serve.png -:align: center -``` - -## Metrics - -Ray Serve exposes important system metrics like the number of successful and -errored requests through the [Ray metrics monitoring infrastructure](ray-metrics). By default, -the metrics are exposed in Prometheus format on each node. - -The following metrics are exposed by Ray Serve: - -```{eval-rst} -.. list-table:: - :header-rows: 1 - - * - Name - - Description - * - ``serve_deployment_request_counter`` - - The number of queries that have been processed in this replica. - * - ``serve_deployment_error_counter`` - - The number of exceptions that have occurred in the deployment. - * - ``serve_deployment_replica_starts`` - - The number of times this replica has been restarted due to failure. - * - ``serve_deployment_queuing_latency_ms`` - - The latency for queries in the replica's queue waiting to be processed. - * - ``serve_deployment_processing_latency_ms`` - - The latency for queries to be processed. - * - ``serve_replica_queued_queries`` - - The current number of queries queued in the deployment replicas. - * - ``serve_replica_processing_queries`` - - The current number of queries being processed. - * - ``serve_num_http_requests`` - - The number of HTTP requests processed. - * - ``serve_num_http_error_requests`` - - The number of non-200 HTTP responses. - * - ``serve_num_router_requests`` - - The number of requests processed by the router. - * - ``serve_handle_request_counter`` - - The number of requests processed by this ServeHandle. - * - ``serve_deployment_queued_queries`` - - The number of queries for this deployment waiting to be assigned to a replica. - * - ``serve_num_deployment_http_error_requests`` - - The number of non-200 HTTP responses returned by each deployment. -``` - -To see this in action, run `ray start --head --metrics-export-port=8080` in your terminal, and then run the following script: - -```{literalinclude} ../../../python/ray/serve/examples/doc/snippet_metrics.py -``` - -In your web browser, navigate to `localhost:8080`. -In the output there, you can search for `serve_` to locate the metrics above. -The metrics are updated once every ten seconds, and you will need to refresh the page to see the new values. - -For example, after running the script for some time and refreshing `localhost:8080` you might see something that looks like: - -``` -ray_serve_deployment_processing_latency_ms_count{...,deployment="f",...} 99.0 -ray_serve_deployment_processing_latency_ms_sum{...,deployment="f",...} 99279.30498123169 -``` - -which indicates that the average processing latency is just over one second, as expected. - -You can even define a [custom metric](application-level-metrics) to use in your deployment, and tag it with the current deployment or replica. -Here's an example: - -```{literalinclude} ../../../python/ray/serve/examples/doc/snippet_custom_metric.py -:end-before: __custom_metrics_deployment_end__ -:start-after: __custom_metrics_deployment_start__ -``` - -See the -[Ray Metrics documentation](ray-metrics) for more details, including instructions for scraping these metrics using Prometheus. diff --git a/doc/source/serve/package-ref.md b/doc/source/serve/package-ref.md index 58bde2f253526..0206a81339f7e 100644 --- a/doc/source/serve/package-ref.md +++ b/doc/source/serve/package-ref.md @@ -1,22 +1,20 @@ (serve-api)= # Ray Serve API -## Core APIs +(core-apis)= -```{eval-rst} -.. autofunction:: ray.serve.start -``` +## Core APIs ```{eval-rst} -.. autofunction:: ray.serve.deployment +.. autofunction:: ray.serve.run ``` ```{eval-rst} -.. autofunction:: ray.serve.list_deployments +.. autofunction:: ray.serve.start ``` ```{eval-rst} -.. autofunction:: ray.serve.get_deployment +.. autofunction:: ray.serve.deployment ``` ```{eval-rst} @@ -29,7 +27,7 @@ ```{eval-rst} .. autoclass:: ray.serve.deployment.Deployment - :members: deploy, delete, options, get_handle + :members: deploy, delete, options, get_handle, bind ``` (servehandle-api)= @@ -47,169 +45,27 @@ .. autofunction:: ray.serve.batch(max_batch_size=10, batch_wait_timeout_s=0.0) ``` -## Serve REST API - -### REST API - -#### `GET "/api/serve/deployments/"` - -Gets latest config that Serve has received. This config represents the current goal state for the Serve application. Starts a Serve application on the Ray cluster if it's not already running. See the [config schema](serve-rest-api-config-schema) for the response's JSON schema. - -**Example Request**: - -``` -GET /api/serve/deployments/ HTTP 1.1 -Host: http://localhost:8265/ -Accept: application/json -``` - -**Example Response**: - -```http -HTTP/1.1 200 OK -Content-Type: application/json - -{ - "import_path": "fruit.deployment_graph", - "runtime_env": { - "working_dir": "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" - }, - "deployments": [ - {"name": "MangoStand", "user_config": {"price": 1}}, - {"name": "OrangeStand", "user_config": {"price": 2}}, - {"name": "PearStand", "user_config": {"price": 3}} - ] -} -``` - -#### `PUT "/api/serve/deployments/"` - -Declaratively deploys the Serve application. Starts Serve on the Ray cluster if it's not already running. See the [config schema](serve-rest-api-config-schema) for the request's JSON schema. - -**Example Request**: - -``` -PUT /api/serve/deployments/ HTTP 1.1 -Host: http://localhost:8265/ -Accept: application/json - -{ - "import_path": "fruit.deployment_graph", - "runtime_env": { - "working_dir": "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" - }, - "deployments": [ - {"name": "MangoStand", "user_config": {"price": 1}}, - {"name": "OrangeStand", "user_config": {"price": 2}}, - {"name": "PearStand", "user_config": {"price": 3}} - ] -} -``` - -**Example Response** - - -```http -HTTP/1.1 200 OK -Content-Type: application/json -``` - -#### `GET "/api/serve/deployments/status"` +## Serve CLI and REST API -Gets the Serve application's current status, including all the deployment statuses. This config represents the current goal state for the Serve application. Starts a Serve application on the Ray cluster if it's not already running. See the [status schema](serve-rest-api-status-schema) for the response's JSON schema. +Check out the [CLI](serve-cli) and [REST API](serve-rest-api) for running, debugging, inspecting, and deploying Serve applications in production: -**Example Request**: +```{toctree} +:maxdepth: 1 +:name: serve-non-python-api +serve_cli +rest_api ``` -GET /api/serve/deployments/ HTTP 1.1 -Host: http://localhost:8265/ -Accept: application/json -``` - -**Example Response** - -```http -HTTP/1.1 200 OK -Content-Type: application/json - -{ - "app_status": { - "status": "RUNNING", - "message": "", - "deployment_timestamp": 1855994527.146304 - }, - "deployment_statuses": [ - { - "name": "MangoStand", - "status": "HEALTHY", - "message": "" - }, - { - "name": "OrangeStand", - "status": "HEALTHY", - "message": "" - }, - { - "name": "PearStand", - "status": "HEALTHY", - "message": "" - }, - { - "name": "FruitMarket", - "status": "HEALTHY", - "message": "" - }, - { - "name": "DAGDriver", - "status": "HEALTHY", - "message": "" - } - ] -} -``` - -#### `DELETE "/api/serve/deployments/"` -Shuts down the Serve application running on the Ray cluster. Has no -effect if Serve is not running on the Ray cluster. - -**Example Request**: - -``` -DELETE /api/serve/deployments/ HTTP 1.1 -Host: http://localhost:8265/ -Accept: application/json -``` - -**Example Response** - -```http -HTTP/1.1 200 OK -Content-Type: application/json -``` - -(serve-rest-api-config-schema)= - -### Config Schema - -```{eval-rst} -.. autopydantic_model:: ray.serve.schema.ServeApplicationSchema - -``` - -(serve-rest-api-status-schema)= - -### Status Schema +## Deployment Graph APIs ```{eval-rst} -.. autopydantic_model:: ray.serve.schema.ServeStatusSchema - +.. autofunction:: ray.serve.api.build ``` -## Serve CLI - -```{eval-rst} -.. click:: ray.serve.scripts:cli - :prog: serve - :show-nested: -``` +% TODO(architkulkarni): This just compiles to "alias of Deployment(name=DAGDriver,version=None,route_prefix=/)" +% in the docs, find out how to make Sphinx correctly autodocument this class. +% ```{eval-rst} +% .. autoclass:: ray.serve.drivers.DAGDriver +% :members: predict, predict_with_route +% ``` diff --git a/doc/source/serve/performance.md b/doc/source/serve/performance.md index 101fa231192ce..d497f4c4a65bf 100644 --- a/doc/source/serve/performance.md +++ b/doc/source/serve/performance.md @@ -2,12 +2,11 @@ This section should help you: -- understand the performance characteristics of Ray Serve -- find ways to debug and tune the performance of your Serve deployment +- understand Ray Serve's performance characteristics +- find ways to debug and tune your Serve application's performance :::{note} -While this section offers some tips and tricks to improve the performance of your Serve deployment, -the [architecture doc](serve-architecture) is helpful to gain a deeper understanding of these contexts and parameters. +This section offers some tips and tricks to improve your Ray Serve application's performance. Check out the [architecture page](serve-architecture) for helpful context, including an overview of the HTTP proxy actor and deployment replica actors. ::: ```{contents} @@ -18,63 +17,141 @@ the [architecture doc](serve-architecture) is helpful to gain a deeper understan We are continuously benchmarking Ray Serve. The metrics we care about are latency, throughput, and scalability. We can confidently say: - Ray Serve’s latency overhead is single digit milliseconds, around 1-2 milliseconds on average. -- For throughput, Serve achieves about 3-4k queries per second on a single machine (8 cores) using 1 http proxy and 8 replicas performing noop requests. +- For throughput, Serve achieves about 3-4k queries per second on a single machine (8 cores) using 1 HTTP proxy actor and 8 replicas performing no-op requests. - It is horizontally scalable so you can add more machines to increase the overall throughput. Ray Serve is built on top of Ray, - so its scalability is bounded by Ray’s scalability. Please check out Ray’s [scalability envelope](https://github.com/ray-project/ray/blob/master/release/benchmarks/README.md) + so its scalability is bounded by Ray’s scalability. Please see Ray’s [scalability envelope](https://github.com/ray-project/ray/blob/master/release/benchmarks/README.md) to learn more about the maximum number of nodes and other limitations. -You can check out our [microbenchmark instruction](https://github.com/ray-project/ray/blob/master/python/ray/serve/benchmarks/README.md) -to benchmark on your hardware. +We run long-running benchmarks nightly: + +```{eval-rst} +.. list-table:: + :header-rows: 1 + + * - Benchmark + - Description + - Cluster Details + - Performance Numbers + * - `Single Deployment `_ + - Runs 10 minute `wrk `_ trial on a single no-op deployment with 1000 replicas. + - Head node: AWS EC2 m5.8xlarge. 32 worker nodes: AWS EC2 m5.8xlarge. + - * per_thread_latency_avg_ms = 22.41 + * per_thread_latency_max_ms = 1400.0 + * per_thread_avg_tps = 55.75 + * per_thread_max_tps = 121.0 + * per_node_avg_tps = 553.17 + * per_node_avg_transfer_per_sec_KB = 83.19 + * cluster_total_thoughput = 10954456 + * cluster_total_transfer_KB = 1647441.9199999997 + * cluster_total_timeout_requests = 0 + * cluster_max_P50_latency_ms = 8.84 + * cluster_max_P75_latency_ms = 35.31 + * cluster_max_P90_latency_ms = 49.69 + * cluster_max_P99_latency_ms = 56.5 + * - `Multiple Deployments `_ + - Runs 10 minute `wrk `_ trial on 10 deployments with 100 replicas each. Each deployment recursively sends queries to up to 5 other deployments. + - Head node: AWS EC2 m5.8xlarge. 32 worker nodes: AWS EC2 m5.8xlarge. + - * per_thread_latency_avg_ms = 0.0 + * per_thread_latency_max_ms = 0.0 + * per_thread_avg_tps = 0.0 + * per_thread_max_tps = 0.0 + * per_node_avg_tps = 0.35 + * per_node_avg_transfer_per_sec_KB = 0.05 + * cluster_total_thoughput = 6964 + * cluster_total_transfer_KB = 1047.28 + * cluster_total_timeout_requests = 6964.0 + * cluster_max_P50_latency_ms = 0.0 + * cluster_max_P75_latency_ms = 0.0 + * cluster_max_P90_latency_ms = 0.0 + * cluster_max_P99_latency_ms = 0.0 + * - `Deployment Graph: Ensemble `_ + - Runs 10 node ensemble, constructed with a call graph, that performs basic arithmetic at each node. Ensemble pattern routes the input to 10 different nodes, and their outputs are combined to produce the final output. Simulates 4 clients making 20 requests each. + - Head node: AWS EC2 m5.8xlarge. 0 Worker nodes. + - * throughput_mean_tps = 8.75 + * throughput_std_tps = 0.43 + * latency_mean_ms = 126.15 + * latency_std_ms = 18.35 +``` + +:::{note} +The performance numbers above come from a recent run of the nightly benchmarks. +::: + + + +Check out [our benchmark workloads'](https://github.com/ray-project/ray/tree/f6735f90c72581baf83a9cea7cbbe3ea2f6a56d8/release/serve_tests/workloads) source code directly to get a better sense of what they test. You can see which cluster templates each benchmark uses [here](https://github.com/ray-project/ray/blob/8eca6ae852e2d23bcf49680fef6f0384a1b63564/release/release_tests.yaml#L2328-L2576) (under the `cluster_compute` key), and you can see what type of nodes each template spins up [here](https://github.com/ray-project/ray/tree/8beb887bbed31ecea3d2813b61833b81c45712e1/release/serve_tests). + +You can check out our [microbenchmark instructions](https://github.com/ray-project/ray/blob/master/python/ray/serve/benchmarks/README.md) +to benchmark Ray Serve on your hardware. + +(serve-performance-batching-requests)= + +## Request Batching + +Serve offers a request batching feature that can improve your service throughput without sacrificing latency. This is possible because ML models can utilize efficient vectorized computation to process a batch of request at a time. Batching is also necessary when your model is expensive to use and you want to maximize the utilization of hardware. + +Machine Learning (ML) frameworks such as Tensorflow, PyTorch, and Scikit-Learn support evaluating multiple samples at the same time. +Ray Serve allows you to take advantage of this feature via dynamic request batching. +When a request arrives, Serve puts the request in a queue. This queue buffers the requests to form a batch. The deployment picks up the batch and evaluates it. After the evaluation, the resulting batch will be split up, and each response is returned individually. + +### Enable batching for your deployment +You can enable batching by using the {mod}`ray.serve.batch` decorator. Let's take a look at a simple example by modifying the `MyModel` class to accept a batch. +```{literalinclude} doc_code/batching_guide.py +--- +start-after: __single_sample_begin__ +end-before: __single_sample_end__ +--- +``` + +The batching decorators expect you to make the following changes in your method signature: +- The method is declared as an async method because the decorator batches in asyncio event loop. +- The method accepts a list of its original input types as input. For example, `arg1: int, arg2: str` should be changed to `arg1: List[int], arg2: List[str]`. +- The method returns a list. The length of the return list and the input list must be of equal lengths for the decorator to split the output evenly and return a corresponding response back to its respective request. + +```{literalinclude} doc_code/batching_guide.py +--- +start-after: __batch_begin__ +end-before: __batch_end__ +emphasize-lines: 6-9 +--- +``` + +You can supply two optional parameters to the decorators. +- `batch_wait_timeout_s` controls how long Serve should wait for a batch once the first request arrives. +- `max_batch_size` controls the size of the batch. +Once the first request arrives, the batching decorator will wait for a full batch (up to `max_batch_size`) until `batch_wait_timeout_s` is reached. If the timeout is reached, the batch will be sent to the model regardless the batch size. + +### Tips for fine-tuning batching parameters + +`max_batch_size` ideally should be a power of 2 (2, 4, 8, 16, ...) because CPUs and GPUs are both optimized for data of these shapes. Large batch sizes incur a high memory cost as well as latency penalty for the first few requests. + +`batch_wait_timeout_s` should be set considering the end to end latency SLO (Service Level Objective). For example, if your latency target is 150ms, and the model takes 100ms to evaluate the batch, the `batch_wait_timeout_s` should be set to a value much lower than 150ms - 100ms = 50ms. + +When using batching in a Serve Deployment Graph, the relationship between an upstream node and a downstream node might affect the performance as well. Consider a chain of two models where first model sets `max_batch_size=8` and second model sets `max_batch_size=6`. In this scenario, when the first model finishes a full batch of 8, the second model will finish one batch of 6 and then to fill the next batch, which will initially only be partially filled with 8 - 6 = 2 requests, incurring latency costs. The batch size of downstream models should ideally be multiples or divisors of the upstream models to ensure the batches play well together. ## Debugging performance issues The performance issue you're most likely to encounter is high latency and/or low throughput for requests. -If you have set up [monitoring](serve-monitoring) with Ray and Ray Serve, you will likely observe that -`serve_num_router_requests` is constant while your load increases -`serve_deployment_queuing_latency_ms` is spiking up as queries queue up in the background +Once you set up [monitoring](serve-monitoring) with Ray and Ray Serve, these issues may appear as: -Given the symptom, there are several ways to fix it. +* `serve_num_router_requests` staying constant while your load increases +* `serve_deployment_processing_latency_ms` spiking up as queries queue up in the background -### Choosing the right hardware +There are handful of ways to address these issues: -Make sure you are using the right hardware and resources. -Are you using GPUs (`ray_actor_options={“num_gpus”: 1}`) or 1+ cores (`ray_actor_options={“num_cpus”: 2}`, and setting `OMP_NUM_THREADS`) -to increase the performance of your deep learning framework? +1. Make sure you are using the right hardware and resources: + * Are you reserving GPUs for your deployment replicas using `ray_actor_options` (e.g. `ray_actor_options={“num_gpus”: 1}`)? + * Are you reserving one or more cores for your deployment replicas using `ray_actor_options` (e.g. `ray_actor_options={“num_cpus”: 2}`)? + * Are you setting [OMP_NUM_THREADS](serve-omp-num-threads) to increase the performance of your deep learning framework? +2. Consider using `async` methods in your callable. See [the section below](serve-performance-async-methods). +3. Consider batching your requests. See [the section below](serve-performance-batching-requests). -### Async functions +(serve-performance-async-methods)= +### Using `async` methods -Are you using `async def` in your callable? If you are using asyncio and +Are you using `async def` in your callable? If you are using `asyncio` and hitting the same queuing issue mentioned above, you might want to increase -`max_concurrent_queries`. Serve sets a low number by default so the client gets -proper backpressure. You can increase the value in the Deployment decorator. - -### Batching - -If your deployment can process a batch at a time at a sublinear latency -(for example, if it takes 1ms to process 1 query and 5ms to process 10 of them) -then batching is your best approach. Check out the [batching guide](serve-batching) to -make your deployment accept batches (especially for GPU-based ML inference). You might want to tune your `max_batch_size` and `batch_wait_timeout` in the `@serve.batch` decorator to maximize the benefits: - -- `max_batch_size` specifies how big the batch should be. Generally, - we recommend choosing the largest batch size your function can handle - AND the performance improvement is no longer sublinear. Take a dummy - example: suppose it takes 1ms to process 1 query, 5ms to process 10 queries, - and 6ms to process 11 queries. Here you should set the batch size to to 10 - because adding more queries won’t improve the performance. -- `batch_wait_timeout` specifies how the maximum amount of time to wait before - a batch should be processed, even if it’s not full. It should be set according - to `batch-wait-timeout + full batch processing time ~= expected latency`. The idea - here is to have the first query wait for the longest possible time to achieve high throughput. - This means you should set `batch_wait_timeout` as large as possible without exceeding your desired expected latency in the equation above. - -### Scaling HTTP servers - -Sometimes it’s not about your code: Serve’s HTTP server can become the bottleneck. -If you observe that the CPU utilization for HTTPProxy actor spike up to 100%, the HTTP server is the bottleneck. -Serve only starts a single HTTP server on the Ray head node by default. -This single HTTP server can handle about 3k queries per second. -If your workload exceeds this number, you might want to consider starting one -HTTP server per Ray node to spread the load by `serve.start(http_options={“location”: “EveryNode”})`. -This configuration tells Serve to spawn one HTTP server per node. -You should put an external load balancer in front of it. +`max_concurrent_queries`. Serve sets a low number (100) by default so the client gets +proper backpressure. You can increase the value in the deployment decorator; e.g. +`@serve.deployment(max_concurrent_queries=1000)`. diff --git a/doc/source/serve/production-guide/config.md b/doc/source/serve/production-guide/config.md new file mode 100644 index 0000000000000..0f852c36c112b --- /dev/null +++ b/doc/source/serve/production-guide/config.md @@ -0,0 +1,201 @@ +(serve-in-production-config-file)= + +# Serve Config Files (`serve build`) + +This section should help you: + +- understand the Serve config file format. +- understand how to generate and update a config file for a Serve application. + +This config file can be used with the [serve deploy](serve-in-production-deploying) command CLI or embedded in a [RayService] custom resource in Kubernetes to deploy and update your application in production. +The file is written in YAML and has the following format: + +```yaml +import_path: ... + +runtime_env: ... + +deployments: + + - name: ... + num_replicas: ... + ... + + - name: + ... + + ... +``` + +The file contains the following fields: + +- An `import_path`, which is the path to your top-level Serve deployment (or the same path passed to `serve run`). The most minimal config file consists of only an `import_path`. +- A `runtime_env` that defines the environment that the application will run in. This is used to package application dependencies such as `pip` packages (see {ref}`Runtime Environments ` for supported fields). Note that the `import_path` must be available _within_ the `runtime_env` if it's specified. +- A list of `deployments`. This is optional and allows you to override the `@serve.deployment` settings specified in the deployment graph code. Each entry in this list must include the deployment `name`, which must match one in the code. If this section is omitted, Serve launches all deployments in the graph with the settings specified in the code. + +Below is an equivalent config for the [`FruitStand` example](serve-in-production-example): + +```yaml +import_path: fruit:deployment_graph + +runtime_env: {} + +deployments: + + - name: FruitMarket + num_replicas: 2 + + - name: MangoStand + user_config: + price: 3 + + - name: OrangeStand + user_config: + price: 2 + + - name: PearStand + user_config: + price: 4 + + - name: DAGDriver +``` + +The file uses the same `fruit:deployment_graph` import path that was used with `serve run` and it has five entries in the `deployments` list– one for each deployment. All the entries contain a `name` setting and some other configuration options such as `num_replicas` or `user_config`. + +:::{tip} +Each individual entry in the `deployments` list is optional. In the example config file above, we could omit the `PearStand`, including its `name` and `user_config`, and the file would still be valid. When we deploy the file, the `PearStand` deployment will still be deployed, using the configurations set in the `@serve.deployment` decorator from the deployment graph's code. +::: + +We can also auto-generate this config file from the code. The `serve build` command takes an import path to your deployment graph and it creates a config file containing all the deployments and their settings from the graph. You can tweak these settings to manage your deployments in production. + +Using the `FruitStand` deployment graph example: + +```console +$ ls +fruit.py + +$ serve build fruit:deployment_graph -o fruit_config.yaml + +$ ls +fruit.py +fruit_config.yaml +``` + +(fruit-config-yaml)= + +The `fruit_config.yaml` file contains: + +```yaml +import_path: fruit:deployment_graph + +runtime_env: {} + +deployments: + +- name: MangoStand + num_replicas: 2 + route_prefix: null + max_concurrent_queries: 100 + user_config: + price: 3 + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +- name: OrangeStand + num_replicas: 1 + route_prefix: null + max_concurrent_queries: 100 + user_config: + price: 2 + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +- name: PearStand + num_replicas: 1 + route_prefix: null + max_concurrent_queries: 100 + user_config: + price: 4 + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +- name: FruitMarket + num_replicas: 2 + route_prefix: null + max_concurrent_queries: 100 + user_config: null + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +- name: DAGDriver + num_replicas: 1 + route_prefix: / + max_concurrent_queries: 100 + user_config: null + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +``` + +Note that the `runtime_env` field will always be empty when using `serve build` and must be set manually. + +## Overriding deployment settings + +Settings from `@serve.deployment` can be overriden with this Serve config file. The order of priority is (from highest to lowest): + +1. Config File +2. Deployment graph code (either through the `@serve.deployment` decorator or a `.set_options()` call) +3. Serve defaults + +For example, if a deployment's `num_replicas` is specified in the config file and their graph code, Serve will use the config file's value. If it's only specified in the code, Serve will use the code value. If the user doesn't specify it anywhere, Serve will use a default (which is `num_replicas=1`). + +Keep in mind that this override order is applied separately to each individual setting. +For example, if a user has a deployment `ExampleDeployment` with the following decorator: + +```python +@serve.deployment( + num_replicas=2, + max_concurrent_queries=15, +) +class ExampleDeployment: + ... +``` + +and the following config file: + +```yaml +... + +deployments: + + - name: ExampleDeployment + num_replicas: 5 + +... +``` + +Serve will set `num_replicas=5`, using the config file value, and `max_concurrent_queries=15`, using the code value (since `max_concurrent_queries` wasn't specified in the config file). All other deployment settings use Serve defaults since the user didn't specify them in the code or the config. + +:::{tip} +Remember that `ray_actor_options` counts as a single setting. The entire `ray_actor_options` dictionary in the config file overrides the entire `ray_actor_options` dictionary from the graph code. If there are individual options within `ray_actor_options` (e.g. `runtime_env`, `num_gpus`, `memory`) that are set in the code but not in the config, Serve still won't use the code settings if the config has a `ray_actor_options` dictionary. It will treat these missing options as though the user never set them and will use defaults instead. This dictionary overriding behavior also applies to `user_config`. +::: diff --git a/doc/source/serve/production-guide/failures.md b/doc/source/serve/production-guide/failures.md new file mode 100644 index 0000000000000..f98eaa7b87d99 --- /dev/null +++ b/doc/source/serve/production-guide/failures.md @@ -0,0 +1,40 @@ +# Failure Recovery + +Ray Serve is resilient to any component failures within the Ray cluster out of the box. +You can read more about how worker node and process failures are handled in: {ref}`serve-ft-detail`. + +(serve-health-checking)= + +## Health checking + +By default, each Serve deployment replica is periodically health-checked and restarted on failure. + +You can customize this behavior to perform an application-level health check or to adjust the frequency/timeout. +To define a custom healthcheck, define a `check_health` method on your deployment class. +This method should take no arguments and return no result, and it should raise an exception if the replica should be considered unhealthy. +The raised exception will be logged by the Serve controller if the health check fails. +You can also customize how frequently the health check is run and the timeout after which a replica is marked unhealthy in the deployment options. + +> ```python +> @serve.deployment(health_check_period_s=10, health_check_timeout_s=30) +> class MyDeployment: +> def __init__(self, db_addr: str): +> self._my_db_connection = connect_to_db(db_addr) +> +> def __call__(self, request): +> return self._do_something_cool() +> +> # Will be called by Serve to check the health of the replica. +> def check_health(self): +> if not self._my_db_connection.is_connected(): +> # The specific type of exception is not important. +> raise RuntimeError("uh-oh, DB connection is broken.") +> ``` + +## Head node failures + +By default the Ray head node is a single point of failure: if it crashes, the entire cluster crashes and needs to be restarted. +When running on Kubernetes, the `RayService` controller health-checks the cluster and restarts it if this occurs, but this still results in some downtime. +In Ray 2.0, [KubeRay](https://ray-project.github.io/kuberay/) has added experimental support for [GCS fault tolerance](https://ray-project.github.io/kuberay/guidance/gcs-ft/#ray-gcs-fault-tolerancegcs-ft-experimental), preventing the Ray cluster from crashing if the head node goes down. +While the head node is recovering, Serve applications can still handle traffic but cannot be updated or recover from other failures (e.g., actors or worker nodes crashing). +Once the GCS is recovered, the cluster will return to normal behavior. diff --git a/doc/source/serve/production-guide/index.md b/doc/source/serve/production-guide/index.md new file mode 100644 index 0000000000000..d3f742e6a4733 --- /dev/null +++ b/doc/source/serve/production-guide/index.md @@ -0,0 +1,119 @@ +(serve-in-production)= + +# Production Guide + +The recommended way to run Ray Serve in production is on Kubernetes using the [KubeRay](kuberay-quickstart) [RayService] custom resource. +The RayService custom resource automatically handles important production requirements such as health checking, status reporting, failure recovery, and upgrades. +If you're not running on Kubernetes, you can also run Ray Serve on a Ray cluster directly using the Serve CLI. + +This section will walk you through a quickstart of how to generate a Serve config file and deploy it using the Serve CLI. +For more details, you can check out the other pages in the production guide: +- Understand the [Serve config file format](serve-in-production-config-file). +- Understand how to [deploy on VMs using the Serve CLI](serve-in-production-deploying). +- Understand how to [deploy on Kubernetes using KubeRay](serve-in-production-kubernetes). +- Understand how to [monitor running Serve applications](serve-monitoring). + +(serve-in-production-example)= + +## Working example: FruitStand application + +Throughout the production guide, we will use the following Serve application as a working example. +The application takes in requests containing a list of two values, a fruit name and an amount, and returns the total price for the batch of fruits. + +```{literalinclude} ../doc_code/production_fruit_example.py +:language: python +:start-after: __fruit_example_begin__ +:end-before: __fruit_example_end__ +``` + +Save this code locally in `fruit.py` to follow along. +In development, we would likely use the `serve run` command to iteratively run, develop, and repeat (see the [Development Workflow](serve-dev-workflow) for more information). +When we're ready to go to production, we will generate a structured [config file](serve-in-production-config-file) that acts as the single source of truth for the application. + +This config file can be generated using `serve build`: +``` +$ serve build fruit:deployment_graph -o fruit_config.yaml +``` + +The generated version of this file contains an `import_path`, `runtime_env`, and configuration options for each deployment in the application. +A minimal version of the config looks as follows (save this config locally in `fruit_config.yaml` to follow along): + +```yaml +import_path: fruit:deployment_graph + +runtime_env: {} + +deployments: + +- name: MangoStand + num_replicas: 2 + +- name: OrangeStand + num_replicas: 1 + +- name: PearStand + num_replicas: 1 + +- name: FruitMarket + num_replicas: 2 + +- name: DAGDriver + num_replicas: 1 +``` + +You can use `serve deploy` to deploy the application to a local Ray cluster and `serve status` to get the status at runtime: + +```console +# Start a local Ray cluster. +ray start --head + +# Deploy the FruitStand application to the local Ray cluster. +serve deploy fruit_config.yaml +2022-08-16 12:51:22,043 SUCC scripts.py:180 -- +Sent deploy request successfully! + * Use `serve status` to check deployments' statuses. + * Use `serve config` to see the running app's config. + +$ serve status +app_status: + status: RUNNING + message: '' + deployment_timestamp: 1660672282.0406542 +deployment_statuses: +- name: MangoStand + status: HEALTHY + message: '' +- name: OrangeStand + status: HEALTHY + message: '' +- name: PearStand + status: HEALTHY + message: '' +- name: FruitMarket + status: HEALTHY + message: '' +- name: DAGDriver + status: HEALTHY + message: '' +``` + +You can test the application using `curl`: + +```console +$ curl -H "Content-Type: application/json" -d '["PEAR", 2]' "http://localhost:8000/" +8 +``` + +To update the application, modify the config file and use `serve deploy` again. + +## Next Steps + +This section provided a quickstart on how to generate and use a Serve config file. +For a deeper dive into how to deploy, update, and monitor Serve applications, see the following pages: +- Learn the details of the [Serve config file format](serve-in-production-config-file). +- Learn how to [deploy on VMs using the Serve CLI](serve-in-production-deploying). +- Learn how to [deploy on Kubernetes using KubeRay](serve-in-production-kubernetes). +- Learn how to [monitor running Serve applications](serve-monitoring). + +[KubeRay]: https://ray-project.github.io/kuberay/ +[RayService]: https://ray-project.github.io/kuberay/guidance/rayservice/ diff --git a/doc/source/serve/production-guide/kubernetes.md b/doc/source/serve/production-guide/kubernetes.md new file mode 100644 index 0000000000000..cda5f02522bb3 --- /dev/null +++ b/doc/source/serve/production-guide/kubernetes.md @@ -0,0 +1,252 @@ +(serve-in-production-kubernetes)= + +# Deploying on Kubernetes + +This section should help you: + +- understand how to install and use the [KubeRay] operator. +- understand how to deploy a Ray Serve application using a [RayService]. +- understand how to monitor and update your application. + +The recommended way to deploy Ray Serve is on Kubernetes, providing the best of both worlds: the user experience and scalable compute of Ray Serve and operational benefits of Kubernetes. +This also allows you to integrate with existing applications that may be running on Kubernetes. +The recommended practice when running on Kubernetes is to use the [RayService] controller that's provided as part of [KubeRay]. The RayService controller automatically handles important production requirements such as health checking, status reporting, failure recovery, and upgrades. + +A [RayService] CR encapsulates a multi-node Ray Cluster and a Serve application that runs on top of it into a single Kubernetes manifest. +Deploying, upgrading, and getting the status of the application can be done using standard `kubectl` commands. +This section walks through how to deploy, monitor, and upgrade the [`FruitStand` example](serve-in-production-example) on Kubernetes. + +:::{warning} +Although it's actively developed and maintained, [KubeRay] is still considered alpha, or experimental, so some APIs may be subject to change. +::: + +## Installing the KubeRay operator + +This guide assumes that you have a running Kubernetes cluster and have `kubectl` configured to run commands on it. +See the [Kubernetes documentation](https://kubernetes.io/docs/setup/) or the [KubeRay quickstart guide](kuberay-quickstart) if you need help getting started. + +The first step is to install the `KubeRay` operator into your Kubernetes cluster. +This creates a pod that runs the `KubeRay` controller. The `KubeRay` controller manages resources based on the `RayService` CRs you create. + +Install the operator using `kubectl apply` and check that the controller pod is running: +```console +$ kubectl create -k "github.com/ray-project/kuberay/ray-operator/config/default?ref=v0.3.0&timeout=90s" +$ kubectl get deployments -n ray-system +NAME READY UP-TO-DATE AVAILABLE AGE +kuberay-apiserver 1/1 1 1 13s +kuberay-operator 1/1 1 1 13s + +$ kubectl get pods -n ray-system +NAME READY STATUS RESTARTS AGE +kuberay-apiserver-799bc6dd95-787w7 1/1 Running 0 42s +kuberay-operator-68c75b5d5f-m8xd7 1/1 Running 0 42s +``` + +For more details, see the [KubeRay quickstart guide](kuberay-quickstart). + +## Deploying a Serve application + +Once the KubeRay controller is running, you can manage your Ray Serve application by creating and updating a `RayService` custom resource (CR). +`RayService` custom resources consist of the following: +- a `KubeRay` `RayCluster` config defining the cluster that the Serve application runs on. +- a Ray Serve [config](serve-in-production-config-file) defining the Serve application to run on the cluster. + +When the `RayService` is created, the `KubeRay` controller first creates a Ray cluster using the provided configuration. +Then, once the cluster is running, it deploys the Serve application to the cluster using the [REST API](serve-in-production-deploying). +The controller also creates a Kubernetes Service that can be used to route traffic to the Serve application. + +Let's see this in action by deploying the [`FruitStand` example](serve-in-production-example). +The Serve config for the example is embedded into [this example `RayService` CR](https://github.com/ray-project/kuberay/blob/release-0.3/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml). +To follow along, save this CR locally in a file named `ray_v1alpha1_rayservice.yaml`: + +:::{note} +The example `RayService` uses very small resource requests because it's only for demonstration. +In production, you'll want to provide more resources to the cluster. +Learn more about how to configure KubeRay clusters [here](kuberay-config). +::: + +```console +$ curl -o ray_v1alpha1_rayservice.yaml https://raw.githubusercontent.com/ray-project/kuberay/release-0.3/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +``` + +To deploy the example, we simply `kubectl apply` the CR. +This creates the underlying Ray cluster, consisting of a head and worker node pod (see [Ray Clusters Key Concepts](../../cluster/key-concepts.rst) for more details on Ray clusters), as well as the service that can be used to query our application: + +```console +$ kubectl apply -f ray_v1alpha1_rayservice.yaml + +$ kubectl get rayservices +NAME AGE +rayservice-sample 7s + +$ kubectl get pods +NAME READY STATUS RESTARTS AGE +rayservice-sample-raycluster-qd2vl-worker-small-group-bxpp6 1/1 Running 0 24m +rayservice-sample-raycluster-qd2vl-head-45hj4 1/1 Running 0 24m + +$ kubectl get services +NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE +kubernetes ClusterIP 10.100.0.1 443/TCP 62d +# Services used internally by the KubeRay controller. +rayservice-sample-head-svc ClusterIP 10.100.34.24 6379/TCP,8265/TCP,10001/TCP,8000/TCP,52365/TCP 24m +rayservice-sample-raycluster-qd2vl-dashboard-svc ClusterIP 10.100.109.177 52365/TCP 24m +rayservice-sample-raycluster-qd2vl-head-svc ClusterIP 10.100.180.221 6379/TCP,8265/TCP,10001/TCP,8000/TCP,52365/TCP 24m +# The Serve service that we will use to send queries to the application. +rayservice-sample-serve-svc ClusterIP 10.100.39.92 8000/TCP 24m +``` + +Note that the `rayservice-sample-serve-svc` above is the one that can be used to send queries to the Serve application -- this will be used in the next section. + +## Querying the application + +Once the `RayService` is running, we can query it over HTTP using the service created by the KubeRay controller. +This service can be queried directly from inside the cluster, but to access it from your laptop you'll need to configure a [Kubernetes ingress](kuberay-networking) or use port forwarding as below: + +```console +$ kubectl port-forward service/rayservice-sample-serve-svc 8000 +$ curl -X POST -H 'Content-Type: application/json' localhost:8000 -d '["MANGO", 2]' +6 +``` + +## Getting the status of the application + +As the `RayService` is running, the `KubeRay` controller continually monitors it and writes relevant status updates to the CR. +You can view the status of the application using `kubectl describe`. +This includes the status of the cluster, events such as health check failures or restarts, and the application-level statuses reported by [`serve status`](serve-in-production-inspecting). + +```console +$ kubectl get rayservices +NAME AGE +rayservice-sample 7s + +$ kubectl describe rayservice rayservice-sample +... +Status: + Active Service Status: + App Status: + Last Update Time: 2022-08-16T20:52:41Z + Status: RUNNING + Dashboard Status: + Health Last Update Time: 2022-08-16T20:52:41Z + Is Healthy: true + Last Update Time: 2022-08-16T20:52:41Z + Ray Cluster Name: rayservice-sample-raycluster-9ghjw + Ray Cluster Status: + Available Worker Replicas: 2 + Desired Worker Replicas: 1 + Endpoints: + Client: 10001 + Dashboard: 8265 + Dashboard - Agent: 52365 + Gcs - Server: 6379 + Serve: 8000 + Last Update Time: 2022-08-16T20:51:14Z + Max Worker Replicas: 5 + Min Worker Replicas: 1 + State: ready + Serve Deployment Statuses: + Health Last Update Time: 2022-08-16T20:52:41Z + Last Update Time: 2022-08-16T20:52:41Z + Name: MangoStand + Status: HEALTHY + Health Last Update Time: 2022-08-16T20:52:41Z + Last Update Time: 2022-08-16T20:52:41Z + Name: OrangeStand + Status: HEALTHY + Health Last Update Time: 2022-08-16T20:52:41Z + Last Update Time: 2022-08-16T20:52:41Z + Name: PearStand + Status: HEALTHY + Health Last Update Time: 2022-08-16T20:52:41Z + Last Update Time: 2022-08-16T20:52:41Z + Name: FruitMarket + Status: HEALTHY + Health Last Update Time: 2022-08-16T20:52:41Z + Last Update Time: 2022-08-16T20:52:41Z + Name: DAGDriver + Status: HEALTHY + Pending Service Status: + App Status: + Dashboard Status: + Ray Cluster Status: + Service Status: Running +Events: + Type Reason Age From Message + ---- ------ ---- ---- ------- + Normal WaitForDashboard 5m44s (x2 over 5m44s) rayservice-controller Service "rayservice-sample-raycluster-9ghjw-dashboard-svc" not found + Normal WaitForServeDeploymentReady 4m37s (x17 over 5m42s) rayservice-controller Put "http://rayservice-sample-raycluster-9ghjw-dashboard-svc.default.svc.cluster.local:52365/api/serve/deployments/": context deadline exceeded (Client.Timeout exceeded while awaiting headers) + Normal WaitForServeDeploymentReady 4m35s (x6 over 5m38s) rayservice-controller Put "http://rayservice-sample-raycluster-9ghjw-dashboard-svc.default.svc.cluster.local:52365/api/serve/deployments/": dial tcp 10.121.3.243:52365: i/o timeout (Client.Timeout exceeded while awaiting headers) + Normal Running 44s (x129 over 94s) rayservice-controller The Serve applicaton is now running and healthy. +``` + +## Updating the application + +To update the `RayService`, modify the manifest and apply it use `kubectl apply`. +There are two types of updates that can occur: +- *Application-level updates*: when only the Serve config options are changed, the update is applied _in-place_ on the same Ray cluster. This enables [lightweight updates](serve-in-production-lightweight-update) such as scaling a deployment up or down or modifying autoscaling parameters. +- *Cluster-level updates*: when the `RayCluster` config options are changed, such as updating the container image for the cluster, it may result in a cluster-level update. In this case, a new cluster is started, and the application is deployed to it. Once the new cluster is ready, the Kubernetes service is updated to point to the new cluster and the previous cluster is terminated. There should not be any downtime for the application, but note that this requires the Kubernetes cluster to be large enough to schedule both Ray clusters. + +### Example: Serve config update + +In the `FruitStand` example above, let's change the price of a mango in the Serve config to 4: + +```console + - name: MangoStand + numReplicas: 1 + userConfig: | + price: 4 +``` + +Now to update the application we apply the modified manifest: + +```console +$ kubectl apply -f ray_v1alpha1_rayservice.yaml + +$ kubectl describe rayservice rayservice-sample +... + serveDeploymentStatuses: + - healthLastUpdateTime: "2022-07-18T21:51:37Z" + lastUpdateTime: "2022-07-18T21:51:41Z" + name: MangoStand + status: UPDATING +... +``` + +If we query the application, we can see that we now get a different result reflecting the updated price: + +```console +$ curl -X POST -H 'Content-Type: application/json' localhost:8000 -d '["MANGO", 2]' +8 +``` + +### Updating the RayCluster config + +The process of updating the RayCluster config is the same as updating the Serve config. +For example, we can update the number of worker nodes to 2 in the manifest: + +```console +workerGroupSpecs: + # the number of pods in the worker group. + - replicas: 2 +``` + +```console +$ kubectl apply -f ray_v1alpha1_rayservice.yaml + +$ kubectl describe rayservice rayservice-sample +... + pendingServiceStatus: + appStatus: {} + dashboardStatus: + healthLastUpdateTime: "2022-07-18T21:54:53Z" + lastUpdateTime: "2022-07-18T21:54:54Z" + rayClusterName: rayservice-sample-raycluster-bshfr + rayClusterStatus: {} +... +``` + +In the status, you can see that the `RayService` is preparing a pending cluster. +After the pending cluster is healthy, it becomes the active cluster and the previous cluster is terminated. + +[KubeRay]: https://ray-project.github.io/kuberay/ +[RayService]: https://ray-project.github.io/kuberay/guidance/rayservice/ diff --git a/doc/source/serve/production-guide/monitoring.md b/doc/source/serve/production-guide/monitoring.md new file mode 100644 index 0000000000000..48c8aa4433c48 --- /dev/null +++ b/doc/source/serve/production-guide/monitoring.md @@ -0,0 +1,300 @@ +(serve-monitoring)= + +# Monitoring Ray Serve + +This section helps you debug and monitor your Serve applications by: + +* viewing the Ray dashboard +* using Ray logging and Loki +* inspecting built-in Ray Serve metrics + +## Ray dashboard + +You can use the Ray dashboard to get a high-level overview of your Ray cluster and Ray Serve application's states. +This includes details such as: +* the number of deployment replicas currently running +* logs for your Serve controller, deployment replicas, and HTTP proxies +* the Ray nodes (i.e. machines) running in your Ray cluster. + +You can access the Ray dashboard at port 8265 at your cluster's URI. +For example, if you're running Ray Serve locally, you can access the dashboard by going to `http://localhost:8265` in your browser. + +You can view important information about your application here. +For example, you can inspect your deployment replicas by navigating to the Ray dashboard's "Actors" tab while your Serve application is running: + +```{image} https://raw.githubusercontent.com/ray-project/Images/master/docs/dashboard/serve-dashboard-2-replicas.png +:align: center +``` + +In this example, there's a single-node cluster running a deployment named `Translator`. This example Serve application uses four [Ray actors](actor-guide): + +* 1 Serve controller +* 1 HTTP proxy +* 2 `Translator` deployment replicas + +This page includes additional useful information like each actor's process ID (PID) and a link to each actor's logs, which includes their `logging` and `print` statements. You can also see whether any particular actor is alive or dead to help you debug potential cluster failures. For example, the image indicates that the Serve controller is currently dead and likely undergoing recovery. + +:::{tip} +To learn more about the Serve controller actor, the HTTP proxy actor(s), the deployment replicas, and how they all work together, check out the [Serve Architecture](serve-architecture) documentation. +::: + +For a detailed overview of the Ray dashboard, see the [dashboard documentation](ray-dashboard). + +## Ray logging + +To understand system-level behavior and to surface application-level details during runtime, you can leverage Ray logging. + +Ray Serve uses Python's standard `logging` module with a logger named `"ray.serve"`. +By default, logs are emitted from actors both to `stderr` and on disk on each node at `/tmp/ray/session_latest/logs/serve/`. +This includes both system-level logs from the Serve controller and HTTP proxy as well as access logs and custom user logs produced from within deployment replicas. + +In development, logs are streamed to the driver Ray program (the Python script that calls `serve.run()` or the `serve run` CLI command), so it's convenient to keep the driver running while debugging. + +For example, let's run a basic Serve application and view the logs that it emits. + +First, let's create a simple deployment that logs a custom log message when it's queried: + +```{literalinclude} ../doc_code/monitoring/monitoring.py +:start-after: __start__ +:end-before: __end__ +:language: python +``` + +Run this deployment using the `serve run` CLI command: + +```console +$ serve run monitoring:say_hello + +2022-08-10 22:58:55,963 INFO scripts.py:294 -- Deploying from import path: "monitoring:say_hello". +2022-08-10 22:58:57,886 INFO worker.py:1481 -- Started a local Ray instance. View the dashboard at http://127.0.0.1:8265. +(ServeController pid=63881) INFO 2022-08-10 22:58:59,365 controller 63881 http_state.py:129 - Starting HTTP proxy with name 'SERVE_CONTROLLER_ACTOR:SERVE_PROXY_ACTOR-1252fc7fbbb16ca6a80c45cbb5fe4ef182030b95aa60b62604151168' on node '1252fc7fbbb16ca6a80c45cbb5fe4ef182030b95aa60b62604151168' listening on '127.0.0.1:8000' +The new client HTTP config differs from the existing one in the following fields: ['location']. The new HTTP config is ignored. +(ServeController pid=63881) INFO 2022-08-10 22:58:59,999 controller 63881 deployment_state.py:1232 - Adding 1 replicas to deployment 'SayHello'. +(HTTPProxyActor pid=63883) INFO: Started server process [63883] +2022-08-10 22:59:00,979 SUCC scripts.py:315 -- Deployed successfully. +``` + +`serve run` prints a few log messages immediately. Note that a few of these messages start with identifiers such as + +``` +(ServeController pid=63881) +``` + +These messages are logs from Ray Serve [actors](actor-guide). They describe which actor (Serve controller, HTTP proxy, or deployment replica) created the log and what its process ID is (which is useful when distinguishing between different deployment replicas or HTTP proxies). The rest of these log messages are the actual log statements generated by the actor. + +While `serve run` is running, we can query the deployment in a separate terminal window: + +``` +curl -X GET http://localhost:8000/ +``` + +This causes the HTTP proxy and deployment replica to print log statements to the terminal running `serve run`: + +```console +(HTTPProxyActor pid=63883) INFO 2022-08-10 23:10:08,005 http_proxy 127.0.0.1 http_proxy.py:315 - GET / 200 2.4ms +(ServeReplica:SayHello pid=63885) INFO 2022-08-10 23:10:08,004 SayHello SayHello#JYbzqP monitoring.py:15 - Hello world! +(ServeReplica:SayHello pid=63885) INFO 2022-08-10 23:10:08,004 SayHello SayHello#JYbzqP replica.py:482 - HANDLE __call__ OK 0.2ms +``` + +A copy of these logs are stored at `/tmp/ray/session_latest/logs/serve/`. You can parse these stored logs with a logging stack such as ELK or [Loki](serve-logging-loki) to search them by deployment or replica. + +To silence the replica-level logs or otherwise configure logging, configure the `"ray.serve"` logger **inside the deployment constructor**: + +```python +import logging + +logger = logging.getLogger("ray.serve") + +@serve.deployment +class Silenced: + def __init__(self): + logger.setLevel(logging.ERROR) +``` + +This controls which logs are written to STDOUT or files on disk. +In addition to the standard Python logger, Serve supports custom logging. Custom logging lets you control what messages are written to STDOUT/STDERR, files on disk, or both. + +For a detailed overview of logging in Ray, see [Ray Logging](ray-logging). + +(serve-logging-loki)= +### Filtering logs with Loki + +You can explore and filter your logs using [Loki](https://grafana.com/oss/loki/). +Setup and configuration are straightforward on Kubernetes, but as a tutorial, let's set up Loki manually. + +For this walkthrough, you need both Loki and Promtail, which are both supported by [Grafana Labs](https://grafana.com). Follow the installation instructions at Grafana's website to get executables for [Loki](https://grafana.com/docs/loki/latest/installation/) and [Promtail](https://grafana.com/docs/loki/latest/clients/promtail/). +For convenience, save the Loki and Promtail executables in the same directory, and then navigate to this directory in your terminal. + +Now let's get your logs into Loki using Promtail. + +Save the following file as `promtail-local-config.yaml`: + +```yaml +server: + http_listen_port: 9080 + grpc_listen_port: 0 + +positions: + filename: /tmp/positions.yaml + +clients: + - url: http://localhost:3100/loki/api/v1/push + +scrape_configs: + - job_name: ray + static_configs: + - labels: + job: ray + __path__: /tmp/ray/session_latest/logs/serve/*.* +``` + +The relevant part for Ray Serve is the `static_configs` field, where we have indicated the location of our log files with `__path__`. +The expression `*.*` will match all files, but it won't match directories since they cause an error with Promtail. + +We'll run Loki locally. Grab the default config file for Loki with the following command in your terminal: + +```shell +wget https://raw.githubusercontent.com/grafana/loki/v2.1.0/cmd/loki/loki-local-config.yaml +``` + +Now start Loki: + +```shell +./loki-darwin-amd64 -config.file=loki-local-config.yaml +``` + +Here you may need to replace `./loki-darwin-amd64` with the path to your Loki executable file, which may have a different name depending on your operating system. + +Start Promtail and pass in the path to the config file we saved earlier: + +```shell +./promtail-darwin-amd64 -config.file=promtail-local-config.yaml +``` + +Once again, you may need to replace `./promtail-darwin-amd64` with your Promtail executable. + +Run the following Python script to deploy a basic Serve deployment with a Serve deployment logger and to make some requests: + +```{literalinclude} ../doc_code/monitoring/deployment_logger.py +:start-after: __start__ +:end-before: __end__ +:language: python +``` + +Now [install and run Grafana](https://grafana.com/docs/grafana/latest/installation/) and navigate to `http://localhost:3000`, where you can log in with default credentials: + +* Username: admin +* Password: admin + +On the welcome page, click "Add your first data source" and click "Loki" to add Loki as a data source. + +Now click "Explore" in the left-side panel. You are ready to run some queries! + +To filter all these Ray logs for the ones relevant to our deployment, use the following [LogQL](https://grafana.com/docs/loki/latest/logql/) query: + +```shell +{job="ray"} |= "Counter" +``` + +You should see something similar to the following: + +```{image} https://raw.githubusercontent.com/ray-project/Images/master/docs/serve/loki-serve.png +:align: center +``` + +You can use Loki to filter your Ray Serve logs and gather insights quicker. + +## Built-in Ray Serve metrics + +You can leverage built-in Ray Serve metrics to get a closer look at your application's performance. + +Ray Serve exposes important system metrics like the number of successful and +failed requests through the [Ray metrics monitoring infrastructure](ray-metrics). By default, the metrics are exposed in Prometheus format on each node. + +:::{note} +Different metrics are collected when Deployments are called +via Python `ServeHandle` and when they are called via HTTP. + +See the list of metrics below marked for each. +::: + +The following metrics are exposed by Ray Serve: + +```{eval-rst} +.. list-table:: + :header-rows: 1 + + * - Name + - Description + * - ``serve_deployment_request_counter`` [**] + - The number of queries that have been processed in this replica. + * - ``serve_deployment_error_counter`` [**] + - The number of exceptions that have occurred in the deployment. + * - ``serve_deployment_replica_starts`` [**] + - The number of times this replica has been restarted due to failure. + * - ``serve_deployment_processing_latency_ms`` [**] + - The latency for queries to be processed. + * - ``serve_replica_processing_queries`` [**] + - The current number of queries being processed. + * - ``serve_num_http_requests`` [*] + - The number of HTTP requests processed. + * - ``serve_num_http_error_requests`` [*] + - The number of non-200 HTTP responses. + * - ``serve_num_router_requests`` [*] + - The number of requests processed by the router. + * - ``serve_handle_request_counter`` [**] + - The number of requests processed by this ServeHandle. + * - ``serve_deployment_queued_queries`` [*] + - The number of queries for this deployment waiting to be assigned to a replica. + * - ``serve_num_deployment_http_error_requests`` [*] + - The number of non-200 HTTP responses returned by each deployment. +``` +[*] - only available when using HTTP calls +[**] - only available when using Python `ServeHandle` calls + +To see this in action, first run the following command to start Ray and set up the metrics export port: + +```bash +ray start --head --metrics-export-port=8080 +``` + +Then run the following script: + +```{literalinclude} ../doc_code/monitoring/metrics_snippet.py +:start-after: __start__ +:end-before: __end__ +:language: python +``` + +The requests will loop until canceled with `ctrl-c`. + +While this script is running, go to `localhost:8080` in your web browser. +In the output there, you can search for `serve_` to locate the metrics above. +The metrics are updated once every ten seconds, so you need to refresh the page to see new values. + +For example, after running the script for some time and refreshing `localhost:8080` you should find metrics similar to the following: + +``` +ray_serve_deployment_processing_latency_ms_count{..., replica="sleeper#jtzqhX"} 48.0 +ray_serve_deployment_processing_latency_ms_sum{..., replica="sleeper#jtzqhX"} 48160.6719493866 +``` + +which indicates that the average processing latency is just over one second, as expected. + +You can even define a [custom metric](application-level-metrics) for your deployment and tag it with deployment or replica metadata. +Here's an example: + +```{literalinclude} ../doc_code/monitoring/custom_metric_snippet.py +:start-after: __start__ +:end-before: __end__ +``` + +The emitted logs include: + +``` +# HELP ray_my_counter The number of odd-numbered requests to this deployment. +# TYPE ray_my_counter gauge +ray_my_counter{..., deployment="MyDeployment"} 5.0 +``` + +See the [Ray Metrics documentation](ray-metrics) for more details, including instructions for scraping these metrics using Prometheus. diff --git a/doc/source/serve/production-guide/rest-api.md b/doc/source/serve/production-guide/rest-api.md new file mode 100644 index 0000000000000..4e2ecd7283084 --- /dev/null +++ b/doc/source/serve/production-guide/rest-api.md @@ -0,0 +1,321 @@ +(serve-in-production-deploying)= + +# Deploying on VMs + +You can deploy your Serve application to production on a Ray cluster using the Ray Serve CLI. +`serve deploy` takes in a config file path and it deploys that file to a Ray cluster over HTTP. +This could either be a local, single-node cluster as in this example or a remote, multi-node cluster started with the [Ray Cluster Launcher](cloud-vm-index). + +This section should help you: + +- understand how to deploy a Ray Serve config file using the CLI. +- understand how to update your application using the CLI. +- understand how to deploy to a remote cluster started with the [Ray Cluster Launcher](cloud-vm-index). + +Let's start by deploying the [config for the `FruitStand` example](fruit-config-yaml): + +```console +$ ls +fruit.py +fruit_config.yaml + +$ ray start --head +... + +$ serve deploy fruit_config.yaml +2022-06-20 17:26:31,106 SUCC scripts.py:139 -- +Sent deploy request successfully! + * Use `serve status` to check deployments' statuses. + * Use `serve config` to see the running app's config. +``` + +`ray start --head` starts a long-lived Ray cluster locally. `serve deploy fruit_config.yaml` deploys the `fruit_config.yaml` file to this local cluster. To stop your Ray cluster, you can run the CLI command `ray stop`. + +The message `Sent deploy request successfully!` means: +* The Ray cluster has received your config file successfully. +* It will start a new Serve application if one hasn't already started. +* The Serve application will deploy the deployments from your deployment graph, updated with the configurations from your config file. + +It does **not** mean that your Serve application, including your deployments, has already started running successfully. This happens asynchronously as the Ray cluster attempts to update itself to match the settings from your config file. Check out the [next section](serve-in-production-inspecting) to learn more about how to get the current status. + +## Adding a runtime environment + +The import path (e.g., `fruit:deployment_graph`) must be importable by Serve at runtime. +When running locally, this might be in your current working directory. +However, when running on a cluster you also need to make sure the path is importable. +You can achieve this either by building the code into the cluster's container image (see [Cluster Configuration](kuberay-config) for more details) or by using a `runtime_env` with a [remote URI](remote-uris) that hosts the code in remote storage. + +As an example, we have [pushed a copy of the FruitStand deployment graph to GitHub](https://github.com/ray-project/test_dag/blob/40d61c141b9c37853a7014b8659fc7f23c1d04f6/fruit.py). You can use this config file to deploy the `FruitStand` deployment graph to your own Ray cluster even if you don't have the code locally: + +```yaml +import_path: fruit:deployment_graph + +runtime_env: + working_dir: "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" +``` + +:::{note} +As a side note, you could also package your deployment graph into a standalone Python package that can be imported using a [PYTHONPATH](https://docs.python.org/3.10/using/cmdline.html#envvar-PYTHONPATH) to provide location independence on your local machine. However, it's still best practice to use a `runtime_env`, to ensure consistency across all machines in your cluster. +::: + +(serve-in-production-remote-cluster)= + +## Using a remote cluster + +By default, `serve deploy` deploys to a cluster running locally. However, you should also use `serve deploy` whenever you want to deploy your Serve application to a remote cluster. `serve deploy` takes in an optional `--address/-a` argument where you can specify your remote Ray cluster's dashboard agent address. This address should be of the form: + +``` +[RAY_CLUSTER_URI]:[DASHBOARD_AGENT_PORT] +``` + +As an example, the address for the local cluster started by `ray start --head` is `http://127.0.0.1:52365`. We can explicitly deploy to this address using the command + +```console +$ serve deploy config_file.yaml -a http://127.0.0.1:52365 +``` + +The Ray dashboard agent's default port is 52365. You can set it to a different value using the `--dashboard-agent-listen-port` argument when running `ray start`." + +:::{note} +If the port 52365 (or whichever port you specify with `--dashboard-agent-listen-port`) is unavailable when Ray starts, the dashboard agent’s HTTP server will fail. However, the dashboard agent and Ray will continue to run. +You can check if an agent’s HTTP server is running by sending a curl request: `curl http://{node_ip}:{dashboard_agent_port}/api/serve/deployments/`. If the request succeeds, the server is running on that node. If the request fails, the server is not running on that node. To launch the server on that node, terminate the process occupying the dashboard agent’s port, and restart Ray on that node. +::: + +:::{tip} +By default, all the Serve CLI commands assume that you're working with a local cluster. All Serve CLI commands, except `serve start` and `serve run` use the Ray agent address associated with a local cluster started by `ray start --head`. However, if the `RAY_AGENT_ADDRESS` environment variable is set, these Serve CLI commands will default to that value instead. + +Similarly, `serve start` and `serve run`, use the Ray head node address associated with a local cluster by default. If the `RAY_ADDRESS` environment variable is set, they will use that value instead. + +You can check `RAY_AGENT_ADDRESS`'s value by running: + +```console +$ echo $RAY_AGENT_ADDRESS +``` + +You can set this variable by running the CLI command: + +```console +$ export RAY_AGENT_ADDRESS=[YOUR VALUE] +``` + +You can unset this variable by running the CLI command: + +```console +$ unset RAY_AGENT_ADDRESS +``` + +Check for this variable in your environment to make sure you're using your desired Ray agent address. +::: + +(serve-in-production-inspecting)= + +## Inspecting the application with `serve config` and `serve status` + +The Serve CLI also offers two commands to help you inspect your Serve application in production: `serve config` and `serve status`. +If you're working with a remote cluster, `serve config` and `serve status` also offer an `--address/-a` argument to access your cluster. Check out [the previous section](serve-in-production-remote-cluster) for more info on this argument. + +`serve config` gets the latest config file the Ray cluster received. This config file represents the Serve application's goal state. The Ray cluster will constantly attempt to reach and maintain this state by deploying deployments, recovering failed replicas, and more. + +Using the `fruit_config.yaml` example from [an earlier section](fruit-config-yaml): + +```console +$ ray start --head +$ serve deploy fruit_config.yaml +... + +$ serve config +import_path: fruit:deployment_graph + +runtime_env: {} + +deployments: + +- name: MangoStand + num_replicas: 2 + route_prefix: null +... +``` + +`serve status` gets your Serve application's current status. It's divided into two parts: the `app_status` and the `deployment_statuses`. + +The `app_status` contains three fields: +* `status`: a Serve application has four possible statuses: + * `"NOT_STARTED"`: no application has been deployed on this cluster. + * `"DEPLOYING"`: the application is currently carrying out a `serve deploy` request. It is deploying new deployments or updating existing ones. + * `"RUNNING"`: the application is at steady-state. It has finished executing any previous `serve deploy` requests, and it is attempting to maintain the goal state set by the latest `serve deploy` request. + * `"DEPLOY_FAILED"`: the latest `serve deploy` request has failed. +* `message`: provides context on the current status. +* `deployment_timestamp`: a unix timestamp of when Serve received the last `serve deploy` request. This is calculated using the `ServeController`'s local clock. + +The `deployment_statuses` contains a list of dictionaries representing each deployment's status. Each dictionary has three fields: +* `name`: the deployment's name. +* `status`: a Serve deployment has three possible statuses: + * `"UPDATING"`: the deployment is updating to meet the goal state set by a previous `deploy` request. + * `"HEALTHY"`: the deployment is at the latest requests goal state. + * `"UNHEALTHY"`: the deployment has either failed to update, or it has updated and has become unhealthy afterwards. This may be due to an error in the deployment's constructor, a crashed replica, or a general system or machine error. +* `message`: provides context on the current status. + +You can use the `serve status` command to inspect your deployments after they are deployed and throughout their lifetime. + +Using the `fruit_config.yaml` example from [an earlier section](fruit-config-yaml): + +```console +$ ray start --head +$ serve deploy fruit_config.yaml +... + +$ serve status +app_status: + status: RUNNING + message: '' + deployment_timestamp: 1655771534.835145 +deployment_statuses: +- name: MangoStand + status: HEALTHY + message: '' +- name: OrangeStand + status: HEALTHY + message: '' +- name: PearStand + status: HEALTHY + message: '' +- name: FruitMarket + status: HEALTHY + message: '' +- name: DAGDriver + status: HEALTHY + message: '' +``` + +`serve status` can also be used with KubeRay ({ref}`kuberay-index`), a Kubernetes operator for Ray Serve, to help deploy your Serve applications with Kubernetes. There's also work in progress to provide closer integrations between some of the features from this document, like `serve status`, with Kubernetes to provide a clearer Serve deployment story. + +(serve-in-production-updating)= + +## Updating the Serve application + +You can update your Serve applications once they're in production by updating the settings in your config file and redeploying it using the `serve deploy` command. In the redeployed config file, you can add new deployment settings or remove old deployment settings. This is because `serve deploy` is **idempotent**, meaning your Serve application's config always matches (or honors) the latest config you deployed successfully – regardless of what config files you deployed before that. + +(serve-in-production-lightweight-update)= + +## Lightweight Config Updates + +Lightweight config updates modify running deployment replicas without tearing them down and restarting them, so there's less downtime as the deployments update. For each deployment, modifying `num_replicas`, `autoscaling_config`, and/or `user_config` is considered a lightweight config update, and won't tear down the replicas for that deployment. + +:::{note} +Lightweight config updates are only possible for deployments that are included as entries under `deployments` in the config file. If a deployment is not included in the config file, replicas of that deployment will be torn down and brought up again each time you redeploy with `serve deploy`. +::: + +### Updating User Config +Let's use the `FruitStand` deployment graph [from an earlier section](fruit-config-yaml) as an example. All the individual fruit deployments contain a `reconfigure()` method. This method allows us to issue lightweight updates to our deployments by updating the `user_config`. + +First let's deploy the graph. Make sure to stop any previous Ray cluster using the CLI command `ray stop` for this example: + +```console +$ ray start --head +$ serve deploy fruit_config.yaml +... + +$ python + +>>> import requests +>>> requests.post("http://localhost:8000/", json=["MANGO", 2]).json() + +6 +``` + +Now, let's update the price of mangos in our deployment. We can change the `price` attribute in the `MangoStand` deployment to `5` in our config file: + +```yaml +import_path: fruit:deployment_graph + +runtime_env: {} + +deployments: + +- name: MangoStand + num_replicas: 2 + route_prefix: null + max_concurrent_queries: 100 + user_config: + # price: 3 (Outdated price) + price: 5 + autoscaling_config: null + graceful_shutdown_wait_loop_s: 2.0 + graceful_shutdown_timeout_s: 20.0 + health_check_period_s: 10.0 + health_check_timeout_s: 30.0 + ray_actor_options: null + +... +``` + +Without stopping the Ray cluster, we can redeploy our graph using `serve deploy`: + +```console +$ serve deploy fruit_config.yaml +... +``` + +We can inspect our deployments with `serve status`. Once the `app_status`'s `status` returns to `"RUNNING"`, we can try our requests one more time: + +```console +$ serve status +app_status: + status: RUNNING + message: '' + deployment_timestamp: 1655776483.457707 +deployment_statuses: +- name: MangoStand + status: HEALTHY + message: '' +- name: OrangeStand + status: HEALTHY + message: '' +- name: PearStand + status: HEALTHY + message: '' +- name: FruitMarket + status: HEALTHY + message: '' +- name: DAGDriver + status: HEALTHY + message: '' + +$ python + +>>> import requests +>>> requests.post("http://localhost:8000/", json=["MANGO", 2]).json() + +10 +``` + +The price has updated! The same request now returns `10` instead of `6`, reflecting the new price. + +## Code Updates + +Similarly, you can update any other setting in any deployment in the config file. If a deployment setting other than `num_replicas`, `autoscaling_config`, or `user_config` is changed, it is considered a code update, and the deployment replicas will be restarted. Note that the following modifications are all considered "changes", and will trigger tear down of replicas: +* changing an existing setting +* adding an override setting that was previously not present in the config file +* removing a setting from the config file + +Note also that changing `import_path` or `runtime_env` is considered a code update for all deployments, and will tear down all running deployments and restart them. + +:::{warning} +Although you can update your Serve application by deploying an entirely new deployment graph using a different `import_path` and a different `runtime_env`, this is NOT recommended in production. + +The best practice for large-scale code updates is to start a new Ray cluster, deploy the updated code to it using `serve deploy`, and then switch traffic from your old cluster to the new one. +::: + +## Best practices + +This section summarizes the best practices when deploying to production using the Serve CLI: + +* Use `serve run` to manually test and improve your deployment graph locally. +* Use `serve build` to create a Serve config file for your deployment graph. + * Put your deployment graph's code in a remote repository and manually configure the `working_dir` or `py_modules` fields in your Serve config file's `runtime_env` to point to that repository. +* Use `serve deploy` to deploy your graph and its deployments to your Ray cluster. After the deployment is finished, you can start serving traffic from your cluster. +* Use `serve status` to track your Serve application's health and deployment progress. +* Use `serve config` to check the latest config that your Serve application received. This is its goal state. +* Make lightweight configuration updates (e.g. `num_replicas` or `user_config` changes) by modifying your Serve config file and redeploying it with `serve deploy`. +* Make heavyweight code updates (e.g. `runtime_env` changes) by starting a new Ray cluster, updating your Serve config file, and deploying the file with `serve deploy` to the new cluster. Once the new deployment is finished, switch your traffic to the new cluster. + diff --git a/doc/source/serve/production.md b/doc/source/serve/production.md deleted file mode 100644 index e58923883aee8..0000000000000 --- a/doc/source/serve/production.md +++ /dev/null @@ -1,554 +0,0 @@ -(serve-in-production)= - -# Putting Ray Serve Deployment Graphs in Production - -This section should help you: - -- develop and test your Serve deployment graph locally -- understand the Serve config file format -- deploy, inspect, and update your Serve application in production - -```{contents} -``` - -(serve-in-production-testing)= - -## Developing and Testing Your Serve Deployment Graph with `serve run` - -You can test your Serve deployment graph using the Serve CLI's `serve run` command. The `serve run` command launches a temporary Ray cluster, deploys the graph to it, and blocks. Then, you can send HTTP requests to test your application. When your graph receives and processes these requests, it will output `print` and `logging` statements to the terminal. Once you're finished testing your graph, you can type `ctrl-C` to kill the temporary Ray cluster and tear down your graph. You can use this pattern to quickly run, debug, and iterate on your Serve deployment graph. - -Let's use this graph as an example: - -```{literalinclude} ../serve/doc_code/production_fruit_example.py -:language: python -:start-after: __fruit_example_begin__ -:end-before: __fruit_example_end__ -``` - -This graph is located in the `fruit.py` file and stored in the `deployment_graph` variable. It takes in requests containing a list of two values: a fruit name and an amount. It returns the total price for the batch of fruits. - -To run the deployment graph, we first navigate to the same directory containing the `fruit.py` file and then run `serve run fruit.deployment_graph`. `fruit.deployment_graph` is the deployment graph's import path (assuming we are running `serve run` in the same directory as `fruit.py`). - -```console -# Terminal Window 1 - -$ ls -fruit.py - -$ serve run fruit.deployment_graph -2022-06-21 13:07:01,966 INFO scripts.py:253 -- Deploying from import path: "fruit.deployment_graph". -2022-06-21 13:07:03,774 INFO services.py:1477 -- View the Ray dashboard at http://127.0.0.1:8265 -... -2022-06-21 13:07:08,076 SUCC scripts.py:266 -- Deployed successfully. -``` - -We can test this graph by opening a new terminal window and making requests with Python's [requests](https://requests.readthedocs.io/en/latest/) library. - -```console -# Terminal Window 2 - -$ python3 - ->>> import requests ->>> requests.post("http://localhost:8000/", json=["PEAR", 2]).json() - 8 -``` - -Once we're finished, we can close the Python interpreter by running `quit()` and terminate the Ray cluster by typing `ctrl-C` int the terminal running `serve run`. This will tear down the deployments and then the cluster. - -(serve-in-production-config-file)= - -## Creating Your Serve Config File with `serve build` - -You can create a Serve config file to manage your deployment graphs' configurations in production. The Serve CLI can "deploy" this file, using the [serve deploy](serve-in-production-deploying) command. This will deploy or update your deployment graphs in production. The file is written in YAML and has the following format: - -```yaml -import_path: ... - -runtime_env: ... - -deployments: - - - name: ... - num_replicas: ... - ... - - - name: - ... - - ... -``` - -The `import_path` is the deployment graph's import path. When you deploy your config file, Serve will import your deployment graph using this path. Similarly, the `runtime_env` is the deployment graph's runtime environment. Serve will import the deployment graph inside this environment. - -The `deployments` section is optional. If it's omitted, Serve will launch the deployment graph (and all its deployments). The graph will run with any deployment settings specified in the `@serve.deployment` decorators from the graph's code. If you want to override these decorator settings from the code, you can include a `deployments` section in the file. You can add an entry of deployment settings to the `deployments` list. The only required setting in each list entry is the deployment `name`, which must match one of the deployments from the graph's code. You can include any settings from the `@serve.deployment` decorator inside the entry, **except** `init_args` and `init_kwargs`, which must be set in the graph's code itself. - -For example, let's take the `FruitStand` deployment graph from the [previous section](serve-in-production-testing). An equivalent config would be: - -```yaml -import_path: fruit.deployment_graph - -runtime_env: {} - -deployments: - - - name: FruitMarket - num_replicas: 2 - - - name: MangoStand - user_config: - price: 3 - - - name: OrangeStand - user_config: - price: 2 - - - name: PearStand - user_config: - price: 4 - - - name: DAGDriver -``` - -The file uses the same `fruit.deployment_graph` import path, and it has five entries in the `deployments` list– one for each deployment. All the entries contain a `name` setting (the only required setting when including an entry) as well as additional settings (such as `num_replicas` or `user_config`) depending on the deployment. - -Note how this config specifies the same settings as the `@serve.deployment` decorators from the deployment graph's code. We can change or add to these settings to override the settings from the decorators. - -:::{tip} -Each individual entry in the `deployments` list is optional. In the example config file above, we could omit the `PearStand`, including its `name` and `user_config`, and the file would still be valid. When we deploy the file, the `PearStand` deployment will still be deployed, using the configurations set in the `@serve.deployment` decorator from the deployment graph's code. -::: - -We can also auto-generate this config file. The `serve build` command takes an import path to your deployment graph, and it creates a config file containing all the deployments and their settings from the graph. You can tweak these settings to manage you deployments in production. - -Using the `FruitStand` deployment graph example: - -```console -$ ls -fruit.py - -$ serve build fruit.deployment_graph -o fruit_config.yaml - -$ ls -fruit.py -fruit_config.yaml -``` - -(fruit-config-yaml)= - -The `fruit_config.yaml` file contains: - -```yaml -import_path: fruit.deployment_graph - -runtime_env: {} - -deployments: - -- name: MangoStand - num_replicas: 2 - route_prefix: null - max_concurrent_queries: 100 - user_config: - price: 3 - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -- name: OrangeStand - num_replicas: 1 - route_prefix: null - max_concurrent_queries: 100 - user_config: - price: 2 - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -- name: PearStand - num_replicas: 1 - route_prefix: null - max_concurrent_queries: 100 - user_config: - price: 4 - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -- name: FruitMarket - num_replicas: 2 - route_prefix: null - max_concurrent_queries: 100 - user_config: null - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -- name: DAGDriver - num_replicas: 1 - route_prefix: / - max_concurrent_queries: 100 - user_config: null - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -``` - -Note that the `runtime_env` field will always be empty when using `serve build`. That field must be set manually. - -### Overriding Deployment Settings - -Settings from `@serve.deployment` can be overriden with this Serve config file. The order of priority is (from highest to lowest): - -1. Config File -2. Deployment graph code (either through the `@serve.deployment` decorator or a `.set_options()` call) -3. Serve defaults - -For example, if a deployment's `num_replicas` is specified in the config file and their graph code, Serve will use the config file's value. If it's only specified in the code, Serve will use the code value. If the user doesn't specify it anywhere, Serve will use a default (which is `num_replicas=1`). - -Keep in mind that this override order is at the settings-level. For example, if a user has a deployment `ExampleDeployment` with the following decorator: - -```python -@serve.deployment( - num_replicas=2, - max_concurrent_queries=15, -) -class ExampleDeployment: - ... -``` - -and the following config file: - -```yaml -... - -deployments: - - - name: ExampleDeployment - num_replicas: 5 - -... -``` - -Serve will set `num_replicas=5`, using the config file value, and `max_concurrent_queries=15`, using the code value (since `max_concurrent_queries` wasn't specified in the config file). All other deployment settings use Serve defaults since the user didn't specify them in the code or the config. - -:::{tip} -Remember that `ray_actor_options` is an independent setting. The entire `ray_actor_options` dictionary in the config file overrides the entire `ray_actor_options` dictionary from the graph code. If there are individual options within `ray_actor_options` (e.g. `runtime_env`, `num_gpus`, `memory`) that are set in the code but not in the config, Serve still won't use the code settings if the config has a `ray_actor_options` dictionary. It will treat these missing options as though the user never set them (and use defaults instead) since the entire `ray_actor_options` dictionary in the config overrides the one in the code. This dictionary overriding behavior also applies to `user_config`. -::: - -(serve-in-production-deploying)= - -## Deploying Your Serve Application to Production with `serve deploy` - -You can deploy your Serve application to production using the config file and the `serve deploy` CLI command. `serve deploy` takes in a config file path, and it deploys that file to a Ray cluster. - -Let's deploy the [fruit_config.yaml](fruit-config-yaml) file from the previous section: - -```console -$ ls -fruit.py -fruit_config.yaml - -$ ray start --head -... - -$ serve deploy fruit_config.yaml -2022-06-20 17:26:31,106 SUCC scripts.py:139 -- -Sent deploy request successfully! - * Use `serve status` to check deployments' statuses. - * Use `serve config` to see the running app's config. -``` - -`ray start --head` starts a long-lived Ray cluster locally. `serve deploy fruit_config.yaml` deploys the `fruit_config.yaml` file to this local cluster. To stop your Ray cluster, you can run the CLI command `ray stop`. - -The message `Sent deploy request successfully!` means: -* The Ray cluster has received your config file successfully. -* It will start a new Serve application if one hasn't already started. -* The Serve application will deploy the deployments from your deployment graph, updated with the configurations from your config file. - -It does **not** mean that your Serve application, including your deployments, has already started running successfully. This happens asynchronously as the Ray cluster attempts to update itself to match the settings from your config file. Check out the [next section](serve-in-production-inspecting) to learn more about how to inspect your deployments. - -### Adding a Runtime Environment - -If you start Ray and deploy your deployment graph from a directory that doesn't contain the graph code, your deployments will fail to run. This happens because your import path is generally location-dependent. For example, the import path `fruit.deployment_graph` assumes the current directory contains the `fruit.py` module, which contains a `deployment_graph` object. - -To make your config file location-independent, you can push your deployment graph code to [a remote repository and add that repository to your config file's `runtime_env` field](remote-uris). When Serve runs your deployment graph, it will pull the code from the remote repository rather than use a local copy. **This is a best practice** because it lets you deploy your config file from any machine in any directory and share the file with other developers, making it a more standalone artifact. - -As an example, we have [pushed a copy of the FruitStand deployment graph to GitHub](https://github.com/ray-project/test_dag/blob/c620251044717ace0a4c19d766d43c5099af8a77/fruit.py). You can use this config file to deploy the `FruitStand` deployment graph to your own Ray cluster even if you don't have the code locally: - -```yaml -import_path: fruit.deployment_graph - -runtime_env: - working_dir: "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" -``` - -:::{note} -As a side note, you could also package your deployment graph into a standalone Python package that can be imported using a [PYTHONPATH](https://docs.python.org/3.10/using/cmdline.html#envvar-PYTHONPATH) to provide location independence on your local machine. However, it's still best practice to use a `runtime_env`, to ensure consistency across all machines in your cluster. -::: - -(serve-in-production-remote-cluster)= - -### Using a Remote Cluster - -By default, `serve deploy` deploys to a cluster running locally. However, you should also use `serve deploy` whenever you want to deploy your Serve application to a remote cluster. `serve deploy` takes in an optional `--address/-a` argument where you can specify your remote Ray cluster's dashboard agent address. This address should be of the form: - -``` -[YOUR_RAY_CLUSTER_URI]:[DASHBOARD AGENT PORT] -``` - -As an example, the address for the local cluster started by `ray start --head` is `http://127.0.0.1:52365`. We can explicitly deploy to this address using the command - -```console -$ serve deploy config_file.yaml -a http://127.0.0.1:52365 -``` - -The Ray dashboard agent's default port is 52365. You can set it to a different value using the `--dashboard-agent-listen-port` argument when running `ray start`." - -:::{note} -If the port 52365 (or whichever port you specify with `--dashboard-agent-listen-port`) is unavailable when Ray starts, the dashboard agent’s HTTP server will fail. However, the dashboard agent and Ray will continue to run. -You can check if an agent’s HTTP server is running by sending a curl request: `curl http://{node_ip}:{dashboard_agent_port}/api/serve/deployments/`. If the request succeeds, the server is running on that node. If the request fails, the server is not running on that node. To launch the server on that node, terminate the process occupying the dashboard agent’s port, and restart Ray on that node. -::: - -:::{tip} -By default, all the Serve CLI commands assume that you're working with a local cluster. All Serve CLI commands, except `serve start` and `serve run` use the Ray agent address associated with a local cluster started by `ray start --head`. However, if the `RAY_AGENT_ADDRESS` environment variable is set, these Serve CLI commands will default to that value instead. - -Similarly, `serve start` and `serve run`, use the Ray head node address associated with a local cluster by default. If the `RAY_ADDRESS` environment variable is set, they will use that value instead. - -You can check `RAY_AGENT_ADDRESS`'s value by running: - -```console -$ echo $RAY_AGENT_ADDRESS -``` - -You can set this variable by running the CLI command: - -```console -$ export RAY_AGENT_ADDRESS=[YOUR VALUE] -``` - -You can unset this variable by running the CLI command: - -```console -$ unset RAY_AGENT_ADDRESS -``` - -Check for this variable in your environment to make sure you're using your desired Ray agent address. -::: - -(serve-in-production-inspecting)= - -## Inspecting Your Serve Application in Production with `serve config` and `serve status` - -The Serve CLI offers two commands to help you inspect your Serve application in production: `serve config` and `serve status`. - -If you're working with a remote cluster, `serve config` and `serve status` also offer an `--address/-a` argument to access your cluster. Check out [the previous section](serve-in-production-remote-cluster) for more info on this argument. - -(serve-in-production-config-command)= - -### `serve config` - -`serve config` gets the latest config file the Ray cluster received. This config file represents the Serve application's goal state. The Ray cluster will constantly attempt to reach and maintain this state by deploying deployments, recovering failed replicas, and more. - -Using the `fruit_config.yaml` example from [an earlier section](fruit-config-yaml): - -```console -$ ray start --head -$ serve deploy fruit_config.yaml -... - -$ serve config -import_path: fruit.deployment_graph - -runtime_env: {} - -deployments: - -- name: MangoStand - num_replicas: 2 - route_prefix: null -... -``` - -(serve-in-production-status-command)= - -### `serve status` - -`serve status` gets your Serve application's current status. It's divided into two parts: the `app_status` and the `deployment_statuses`. - -The `app_status` contains three fields: -* `status`: a Serve application has three possible statuses: - * `"DEPLOYING"`: the application is currently carrying out a `serve deploy` request. It is deploying new deployments or updating existing ones. - * `"RUNNING"`: the application is at steady-state. It has finished executing any previous `serve deploy` requests, and it is attempting to maintain the goal state set by the latest `serve deploy` request. - * `"DEPLOY_FAILED"`: the latest `serve deploy` request has failed. -* `message`: provides context on the current status. -* `deployment_timestamp`: a unix timestamp of when Serve received the last `serve deploy` request. This is calculated using the `ServeController`'s local clock. - -The `deployment_statuses` contains a list of dictionaries representing each deployment's status. Each dictionary has three fields: -* `name`: the deployment's name. -* `status`: a Serve deployment has three possible statuses: - * `"UPDATING"`: the deployment is updating to meet the goal state set by a previous `deploy` request. - * `"HEALTHY"`: the deployment is at the latest requests goal state. - * `"UNHEALTHY"`: the deployment has either failed to update, or it has updated and has become unhealthy afterwards. This may be due to an error in the deployment's constructor, a crashed replica, or a general system or machine error. -* `message`: provides context on the current status. - -You can use the `serve status` command to inspect your deployments after they are deployed and throughout their lifetime. - -Using the `fruit_config.yaml` example from [an earlier section](fruit-config-yaml): - -```console -$ ray start --head -$ serve deploy fruit_config.yaml -... - -$ serve status -app_status: - status: RUNNING - message: '' - deployment_timestamp: 1655771534.835145 -deployment_statuses: -- name: MangoStand - status: HEALTHY - message: '' -- name: OrangeStand - status: HEALTHY - message: '' -- name: PearStand - status: HEALTHY - message: '' -- name: FruitMarket - status: HEALTHY - message: '' -- name: DAGDriver - status: HEALTHY - message: '' -``` - -`serve status` can also be used with KubeRay, a Kubernetes operator for Ray Serve, to help deploy your Serve applications with Kubernetes. There's also work in progress to provide closer integrations between some of the features from this document, like `serve status`, with Kubernetes to provide a clearer Serve deployment story. - -(serve-in-production-updating)= - -## Updating Your Serve Application in Production - -You can also update your Serve applications once they're in production. You can update the settings in your config file and redeploy it using the `serve deploy` command. - -Let's use the `FruitStand` deployment graph [from an earlier section](fruit-config-yaml) as an example. All the individual fruit deployments contain a `reconfigure()` method. [This method allows us to issue lightweight updates](managing-deployments-user-configuration) to our deployments by updating the `user_config`. These updates don't need to tear down the running deployments, meaning there's less downtime as the deployments update. - -First let's deploy the graph. Make sure to stop any previous Ray cluster using the CLI command `ray stop` for this example: - -```console -$ ray start --head -$ serve deploy fruit_config.yaml -... - -$ python - ->>> import requests ->>> requests.post("http://localhost:8000/", json=["MANGO", 2]).json() - -6 -``` - -Now, let's update the price of mangos in our deployment. We can change the `price` attribute in the `MangoStand` deployment to `5` in our config file: - -```yaml -import_path: fruit.deployment_graph - -runtime_env: {} - -deployments: - -- name: MangoStand - num_replicas: 2 - route_prefix: null - max_concurrent_queries: 100 - user_config: - # price: 3 (Outdated price) - price: 5 - autoscaling_config: null - graceful_shutdown_wait_loop_s: 2.0 - graceful_shutdown_timeout_s: 20.0 - health_check_period_s: 10.0 - health_check_timeout_s: 30.0 - ray_actor_options: null - -... -``` - -Without stopping the Ray cluster, we can redeploy our graph using `serve deploy`: - -```console -$ serve deploy fruit_config.yaml -... -``` - -We can inspect our deployments with `serve status`. Once the `app_status`'s `status` returns to `"RUNNING"`, we can try our requests one more time: - -```console -$ serve status -app_status: - status: RUNNING - message: '' - deployment_timestamp: 1655776483.457707 -deployment_statuses: -- name: MangoStand - status: HEALTHY - message: '' -- name: OrangeStand - status: HEALTHY - message: '' -- name: PearStand - status: HEALTHY - message: '' -- name: FruitMarket - status: HEALTHY - message: '' -- name: DAGDriver - status: HEALTHY - message: '' - -$ python - ->>> import requests ->>> requests.post("http://localhost:8000/", json=["MANGO", 2]).json() - -10 -``` - -The price has updated! The same request now returns `10` instead of `6`, reflecting the new price. - -You can update any setting in any deployment in the config file similarly. You can also add new deployment settings or remove old deployment settings from the config. This is because `serve deploy` is **idempotent**. Your Serve application's will match the one specified in the latest config you deployed– regardless of what config files you deployed before that. - -:::{warning} -Although you can update your Serve application by deploying an entirely new deployment graph using a different `import_path` and a different `runtime_env`, this is NOT recommended in production. - -The best practice for large-scale code updates is to start a new Ray cluster, deploy the updated code to it using `serve deploy`, and then switch traffic from your old cluster to the new one. -::: - -## Best Practices - -This section summarizes the best practices when deploying to production: - -* Use `serve run` to manually test and improve your deployment graph locally. -* Use `serve build` to create a Serve config file for your deployment graph. - * Put your deployment graph's code in a remote repository and manually configure the `working_dir` or `py_modules` fields in your Serve config file's `runtime_env` to point to that repository. -* Use `serve deploy` to deploy your graph and its deployments to your Ray cluster. After the deployment is finished, you can start serving traffic from your cluster. -* Use `serve status` to track your Serve application's health and deployment progress. -* Use `serve config` to check the latest config that your Serve application received. This is its goal state. -* Make lightweight configuration updates (e.g. `num_replicas` or `user_config` changes) by modifying your Serve config file and redeploying it with `serve deploy`. -* Make heavyweight code updates (e.g. `runtime_env` changes) by starting a new Ray cluster, updating your Serve config file, and deploying the file with `serve deploy` to the new cluster. Once the new deployment is finished, switch your traffic to the new cluster. diff --git a/doc/source/serve/rest_api.md b/doc/source/serve/rest_api.md new file mode 100644 index 0000000000000..5e5f0b9fa268c --- /dev/null +++ b/doc/source/serve/rest_api.md @@ -0,0 +1,154 @@ +(serve-rest-api)= + +# Serve REST API + +## REST API + +``` +GET /api/serve/deployments/ HTTP 1.1 +Host: http://localhost:52365/ +Accept: application/json +``` + +**Example Response**: + +```http +HTTP/1.1 200 OK +Content-Type: application/json + +{ + "import_path": "fruit.deployment_graph", + "runtime_env": { + "working_dir": "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" + }, + "deployments": [ + {"name": "MangoStand", "user_config": {"price": 1}}, + {"name": "OrangeStand", "user_config": {"price": 2}}, + {"name": "PearStand", "user_config": {"price": 3}} + ] +} +``` + +### `PUT "/api/serve/deployments/"` + +Declaratively deploys the Serve application. Starts Serve on the Ray cluster if it's not already running. See the [config schema](serve-rest-api-config-schema) for the request's JSON schema. + +**Example Request**: + +``` +PUT /api/serve/deployments/ HTTP 1.1 +Host: http://localhost:52365/ +Accept: application/json + +{ + "import_path": "fruit.deployment_graph", + "runtime_env": { + "working_dir": "https://github.com/ray-project/serve_config_examples/archive/HEAD.zip" + }, + "deployments": [ + {"name": "MangoStand", "user_config": {"price": 1}}, + {"name": "OrangeStand", "user_config": {"price": 2}}, + {"name": "PearStand", "user_config": {"price": 3}} + ] +} +``` + +**Example Response** + + +```http +HTTP/1.1 200 OK +Content-Type: application/json +``` + +### `GET "/api/serve/deployments/status"` + +Gets the Serve application's current status, including all the deployment statuses. This config represents the current goal state for the Serve application. Starts a Serve application on the Ray cluster if it's not already running. See the [status schema](serve-rest-api-status-schema) for the response's JSON schema. + +**Example Request**: + +``` +GET /api/serve/deployments/ HTTP 1.1 +Host: http://localhost:52365/ +Accept: application/json +``` + +**Example Response** + +```http +HTTP/1.1 200 OK +Content-Type: application/json + +{ + "app_status": { + "status": "RUNNING", + "message": "", + "deployment_timestamp": 1855994527.146304 + }, + "deployment_statuses": [ + { + "name": "MangoStand", + "status": "HEALTHY", + "message": "" + }, + { + "name": "OrangeStand", + "status": "HEALTHY", + "message": "" + }, + { + "name": "PearStand", + "status": "HEALTHY", + "message": "" + }, + { + "name": "FruitMarket", + "status": "HEALTHY", + "message": "" + }, + { + "name": "DAGDriver", + "status": "HEALTHY", + "message": "" + } + ] +} +``` + +### `DELETE "/api/serve/deployments/"` + +Shuts down the Serve application running on the Ray cluster. Has no +effect if Serve is not running on the Ray cluster. + +**Example Request**: + +``` +DELETE /api/serve/deployments/ HTTP 1.1 +Host: http://localhost:52365/ +Accept: application/json +``` + +**Example Response** + +```http +HTTP/1.1 200 OK +Content-Type: application/json +``` + +(serve-rest-api-config-schema)= + +## Config Schema + +```{eval-rst} +.. autopydantic_model:: ray.serve.schema.ServeApplicationSchema + +``` + +(serve-rest-api-status-schema)= + +## Status Schema + +```{eval-rst} +.. autopydantic_model:: ray.serve.schema.ServeStatusSchema + +``` \ No newline at end of file diff --git a/doc/source/serve/scaling-and-resource-allocation.md b/doc/source/serve/scaling-and-resource-allocation.md new file mode 100644 index 0000000000000..a7d6b79d444f4 --- /dev/null +++ b/doc/source/serve/scaling-and-resource-allocation.md @@ -0,0 +1,150 @@ +(serve-scaling-and-resource-allocation)= + +# Scaling and Resource Allocation + +This guide helps you to: + +- scale your deployments horizontally by specifying a number of replicas +- scale up and down automatically to react to changing traffic +- allocate hardware resources (CPUs, GPUs, etc) for each deployment + +(scaling-out-a-deployment)= + +## Scaling horizontally with `num_replicas` + +Each deployment consists of one or more [replicas](serve-architecture-high-level-view). +The number of replicas is specified by the `num_replicas` field in the deployment options. +By default, `num_replicas` is 1. + +```{literalinclude} ../serve/doc_code/managing_deployments.py +:start-after: __scaling_out_start__ +:end-before: __scaling_out_end__ +:language: python +``` + +(ray-serve-autoscaling)= + +## Autoscaling + +Serve also supports a demand-based replica autoscaler. It adjusts to traffic spikes by observing queue sizes and making scaling decisions to add or remove replicas. +To configure it, you can set the `autoscaling_config` field in deployment options. + +```{literalinclude} ../serve/doc_code/managing_deployments.py +:start-after: __autoscaling_start__ +:end-before: __autoscaling_end__ +:language: python +``` + +The `min_replicas` and `max_replicas` fields configure the range of replicas which the +Serve autoscaler chooses from. Deployments will start with `min_replicas` initially. + +The `target_num_ongoing_requests_per_replica` configuration specifies how aggressively the +autoscaler should react to traffic. Serve will try to make sure that each replica has roughly that number +of requests being processed and waiting in the queue. For example, if your processing time is `10ms` +and the latency constraint is `100ms`, you can have at most `10` requests ongoing per replica so +the last requests can finish within the latency constraint. We recommend you benchmark your application +code and set this number based on end to end latency objective. + +:::{note} +The Ray Serve Autoscaler is an application-level autoscaler that sits on top of the [Ray Autoscaler](cluster-index). +Concretely, this means that the Ray Serve autoscaler asks Ray to start a number of replica actors based on the request demand. +If the Ray Autoscaler determines there aren't enough available CPUs to place these actors, it responds by requesting more Ray nodes. +The underlying cloud provider will then respond by adding more nodes. +Similarly, when Ray Serve scales down and terminates some replica actors, it will try to do so in a way that results in the most nodes having no Ray actors or tasks running on them, at which point the Ray autoscaler will remove those nodes. +::: + +To learn about the architecture underlying Ray Serve Autoscaling, see {ref}`serve-autoscaling-architecture`. + +### `autoscaling_config` parameters + +There are several user-specified parameters the autoscaling algorithm takes into consideration when deciding the target number of replicas for your deployment: + +**min_replicas[default_value=1]**: The minimum number of replicas for the deployment. ``min_replicas`` will also be the initial number of replicas when the deployment is deployed. +:::{note} +Ray Serve Autoscaling allows the `min_replicas` to be 0 when starting your deployment; the scale up will be started when you start sending traffic. There will be a cold start time as the Ray ServeHandle waits (blocks) for available replicas to assign the request. +::: +**max_replicas[default_value=1]**: The maximum number of replicas for the deployment. Ray Serve Autoscaling will rely on the Ray Autoscaler to scale up more nodes when the currently available cluster resources (CPUs, GPUs, etc.) are not enough to support more replicas. + +**target_num_ongoing_requests_per_replica[default_value=1]**: How many ongoing requests are expected to run concurrently per replica. The autoscaler scales up if the value is lower than the current number of ongoing requests per replica. Similarly, the autoscaler scales down if it's higher than the current number of ongoing requests. Scaling happens quicker if there's a high disparity between this value and the current number of ongoing requests. +:::{note} +- It is always recommended to load test your workloads. For example, if the use case is latency sensitive, you can lower the `target_num_ongoing_requests_per_replica` number to maintain high performance. +- Internally, the autoscaler will decide to scale up or down by comparing `target_num_ongoing_requests_per_replica` to the number of `RUNNING` and `PENDING` tasks on each replica. +- `target_num_ongoing_requests_per_replica` is only a target value used for autoscaling (not a hard limit), the real ongoing requests number can be higher than the config. +::: + +**downscale_delay_s[default_value=600.0]**: How long the cluster needs to wait before scaling down replicas. + +**upscale_delay_s[default_value=30.0]**: How long the cluster needs to wait before scaling up replicas. + +:::{note} +`downscale_delay_s` and `upscale_delay_s` control the frequency of doing autoscaling work. For example, if your application takes a long time to do initialization work, you can increase `downscale_delay_s` to make the downscaling happen slowly. +::: + +**smoothing_factor[default_value=1.0]**: The multiplicative factor to speed up or slow down each autoscaling step. For example, when the application has high traffic volume in short period of time, you can increase `smoothing_factor` to scale up the resource quickly. You can think of this as a "gain" factor to amplify the response of the autoscaling algorithm. + +**metrics_interval_s[default_value=10]**: This is control how often each replica sends metrics to the autoscaler. (Normally you don't need to change this config.) + +(serve-cpus-gpus)= + +## Resource Management (CPUs, GPUs) + +You may want to specify a deployment's resource requirements to reserve cluster resources like GPUs. To assign hardware resources per replica, you can pass resource requirements to +`ray_actor_options`. +By default, each replica reserves one CPU. +To learn about options to pass in, take a look at the [Resources with Actors guide](actor-resource-guide). + +For example, to create a deployment where each replica uses a single GPU, you can do the +following: + +```python +@serve.deployment(ray_actor_options={"num_gpus": 1}) +def func(*args): + return do_something_with_my_gpu() +``` + +(serve-fractional-resources-guide)= + +### Fractional CPUs and Fractional GPUs + +Suppose you have two models and each doesn't fully saturate a GPU. You might want to have them share a GPU by allocating 0.5 GPUs each. + +To do this, the resources specified in `ray_actor_options` can be *fractional*. +For example, if you have two models and each doesn't fully saturate a GPU, you might want to have them share a GPU by allocating 0.5 GPUs each. + +```python +@serve.deployment(ray_actor_options={"num_gpus": 0.5}) +def func_1(*args): + return do_something_with_my_gpu() + +@serve.deployment(ray_actor_options={"num_gpus": 0.5}) +def func_2(*args): + return do_something_with_my_gpu() +``` + +In this example, each replica of each deployment will be allocated 0.5 GPUs. The same can be done to multiplex over CPUs, using `"num_cpus"`. + +(serve-omp-num-threads)= + +## Configuring Parallelism with OMP_NUM_THREADS + +Deep learning models like PyTorch and Tensorflow often use multithreading when performing inference. +The number of CPUs they use is controlled by the `OMP_NUM_THREADS` environment variable. +To [avoid contention](omp-num-thread-note), Ray sets `OMP_NUM_THREADS=1` by default because Ray tasks and actors use a single CPU by default. +If you *do* want to enable this parallelism in your Serve deployment, just set `OMP_NUM_THREADS` to the desired value either when starting Ray or in your function/class definition: + +```bash +OMP_NUM_THREADS=12 ray start --head +OMP_NUM_THREADS=12 ray start --address=$HEAD_NODE_ADDRESS +``` + +```{literalinclude} ../serve/doc_code/managing_deployments.py +:start-after: __configure_parallism_start__ +:end-before: __configure_parallism_end__ +:language: python +``` + +:::{note} +Some other libraries may not respect `OMP_NUM_THREADS` and have their own way to configure parallelism. +For example, if you're using OpenCV, you'll need to manually set the number of threads using `cv2.setNumThreads(num_threads)` (set to 0 to disable multi-threading). +You can check the configuration using `cv2.getNumThreads()` and `cv2.getNumberOfCPUs()`. +::: diff --git a/doc/source/serve/serve_cli.md b/doc/source/serve/serve_cli.md new file mode 100644 index 0000000000000..7f3f2705c7c7d --- /dev/null +++ b/doc/source/serve/serve_cli.md @@ -0,0 +1,9 @@ +(serve-cli)= + +# Serve CLI + +```{eval-rst} +.. click:: ray.serve.scripts:cli + :prog: serve + :show-nested: +``` \ No newline at end of file diff --git a/doc/source/serve/tutorials/batch.md b/doc/source/serve/tutorials/batch.md index fe0e023ca118c..24d3983cb4410 100644 --- a/doc/source/serve/tutorials/batch.md +++ b/doc/source/serve/tutorials/batch.md @@ -2,8 +2,8 @@ # Batching Tutorial -In this guide, we will deploy a simple vectorized adder that takes -a batch of queries and adds them at once. In particular, we show: +In this guide, we will deploy a simple text generator that takes in +a batch of queries and processes them at once. In particular, we show: - How to implement and deploy a Ray Serve deployment that accepts batches. - How to configure the batch size. @@ -18,9 +18,11 @@ This tutorial should help the following use cases: inference with batching can increase the *throughput* of the model as well as *utilization* of the hardware. -Let's import Ray Serve and some other helpers. -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_batch.py +## Define the Deployment +Open a new Python file called `tutorial_batch.py`. First, let's import Ray Serve and some other helpers. + +```{literalinclude} ../doc_code/tutorial_batch.py :end-before: __doc_import_end__ :start-after: __doc_import_begin__ ``` @@ -62,44 +64,126 @@ finishes, a larger batch may be executed. This behavior can be tuned using the timeout may improve throughput at the cost of latency under low load. ::: -Let's define a deployment that takes in a list of requests, extracts the input value, -converts them into an array, and uses NumPy to add 1 to each element. +Let's define a deployment that takes in a list of input strings and runs +vectorized text generation on the inputs. -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_batch.py +```{literalinclude} ../doc_code/tutorial_batch.py :end-before: __doc_define_servable_end__ :start-after: __doc_define_servable_begin__ ``` -Let's deploy it. Note that in the `@serve.batch` decorator, we are specifying -the maximum batch size via `max_batch_size=4`. This option limits +Let's prepare to deploy the deployment. Note that in the `@serve.batch` decorator, we +are specifying the maximum batch size via `max_batch_size=4`. This option limits the maximum possible batch size that will be executed at once. -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_batch.py +```{literalinclude} ../doc_code/tutorial_batch.py :end-before: __doc_deploy_end__ :start-after: __doc_deploy_begin__ ``` +## Deploy the Deployment +Deploy the deployment by running the following through the terminal. +```console +$ serve run tutorial_batch:generator +``` + Let's define a [Ray remote task](ray-remote-functions) to send queries in -parallel. As you can see, the first batch has a batch size of 1, and the subsequent -queries have a batch size of 4. Even though each query is issued independently, -Ray Serve was able to evaluate them in batches. +parallel. While Serve is running, open a separate terminal window, and run the +following in an interactive Python shell or a separate Python script: -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_batch.py -:end-before: __doc_query_end__ -:start-after: __doc_query_begin__ +```python +import ray +import requests +import numpy as np + +@ray.remote +def send_query(text): + resp = requests.get("http://localhost:8000/?text={}".format(text)) + return resp.text + +# Let's use Ray to send all queries in parallel +texts = [ + 'Once upon a time,', + 'Hi my name is Lewis and I like to', + 'My name is Mary, and my favorite', + 'My name is Clara and I am', + 'My name is Julien and I like to', + 'Today I accidentally', + 'My greatest wish is to', + 'In a galaxy far far away', + 'My best talent is', +] +results = ray.get([send_query.remote(text) for text in texts]) +print("Result returned:", results) ``` +You should get an output like the following. As you can see, the first batch has a +batch size of 1, and the subsequent queries have a batch size of 4. Even though each +query is issued independently, Ray Serve was able to evaluate them in batches. +```python +(pid=...) Our input array has length: 1 +(pid=...) Our input array has length: 4 +(pid=...) Our input array has length: 4 +Result returned: [ + 'Once upon a time, when I got to look at and see the work of my parents (I still can\'t stand them,) they said, "Boys, you\'re going to like it if you\'ll stay away from him or make him look', + + "Hi my name is Lewis and I like to look great. When I'm not playing against, it's when I play my best and always feel most comfortable. I get paid by the same people who make my games, who work hardest for me.", + + "My name is Mary, and my favorite person in these two universes, the Green Lantern and the Red Lantern, are the same, except they're two of the Green Lanterns, but they also have their own different traits. Now their relationship is known", + + 'My name is Clara and I am married and live in Philadelphia. I am an English language teacher and translator. I am passionate about the issues that have so inspired me and my journey. My story begins with the discovery of my own child having been born', + + 'My name is Julien and I like to travel with my son on vacations... In fact I really prefer to spend more time with my son."\n\nIn 2011, the following year he was diagnosed with terminal Alzheimer\'s disease, and since then,', + + "Today I accidentally got lost and went on another tour in August. My story was different, but it had so many emotions that it made me happy. I'm proud to still be able to go back to Oregon for work.\n\nFor the longest", + + 'My greatest wish is to return your loved ones to this earth where they can begin their own free and prosperous lives. This is true only on occasion as it is not intended or even encouraged to be so.\n\nThe Gospel of Luke 8:29', + + 'In a galaxy far far away, the most brilliant and powerful beings known would soon enter upon New York, setting out to restore order to the state. When the world turned against them, Darth Vader himself and Obi-Wan Kenobi, along with the Jedi', + + 'My best talent is that I can make a movie with somebody who really has a big and strong voice. I do believe that they would be great writers. I can tell you that to make sure."\n\n\nWith this in mind, "Ghostbusters' +] +``` + +## Deploy the Deployment using Python API What if you want to evaluate a whole batch in Python? Ray Serve allows you to send queries via the Python API. A batch of queries can either come from the web server -or the Python API. Learn more [here](serve-handle-explainer). +or the Python API. -To query the deployment via the Python API, we can use `Deployment.get_handle` to receive -a handle to the corresponding deployment. To enqueue a query, you can call -`handle.method.remote(data)`. This call returns immediately -with a [Ray ObjectRef](ray-object-refs). You can call `ray.get` to retrieve -the result. +To query the deployment via the Python API, we can use `serve.run()`, which is part +of the Python API, instead of running `serve run` from the console. Add the following +to the Python script `tutorial_batch.py`: -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_batch.py -:end-before: __doc_query_handle_end__ -:start-after: __doc_query_handle_begin__ +```python +handle = serve.run(generator) ``` + +Generally, to enqueue a query, you can call `handle.method.remote(data)`. This call +returns immediately with a [Ray ObjectRef](ray-object-refs). You can call `ray.get` to +retrieve the result. Add the following to the same Python script. + +```python +input_batch = [ + 'Once upon a time,', + 'Hi my name is Lewis and I like to', + 'My name is Mary, and my favorite', + 'My name is Clara and I am', + 'My name is Julien and I like to', + 'Today I accidentally', + 'My greatest wish is to', + 'In a galaxy far far away', + 'My best talent is', +] +print("Input batch is", input_batch) + +import ray +result_batch = ray.get([handle.handle_batch.remote(batch) for batch in input_batch]) +print("Result batch is", result_batch) +``` + +Finally, let's run the script. +```console +$ python tutorial_batch.py +``` + +You should get a similar output like before! \ No newline at end of file diff --git a/doc/source/serve/tutorials/deployment-graph-patterns.md b/doc/source/serve/tutorials/deployment-graph-patterns.md new file mode 100644 index 0000000000000..d1005e72588db --- /dev/null +++ b/doc/source/serve/tutorials/deployment-graph-patterns.md @@ -0,0 +1,9 @@ +(serve-deployment-graph-patterns-overview)= + +# Deployment Graph Patterns + +These are some common [deployment graph](serve-model-composition-deployment-graph) patterns that can support complex workloads: + +- [Linear Pipeline](deployment-graph-patterns/linear_pipeline.md) +- [Branching Input](deployment-graph-patterns/branching_input.md) +- [Conditional](deployment-graph-patterns/conditional.md) diff --git a/doc/source/serve/tutorials/deployment-graph-patterns/branching_input.md b/doc/source/serve/tutorials/deployment-graph-patterns/branching_input.md new file mode 100644 index 0000000000000..081131854366b --- /dev/null +++ b/doc/source/serve/tutorials/deployment-graph-patterns/branching_input.md @@ -0,0 +1,36 @@ +(deployment-graph-pattern-branching-input)= + +# Pattern: Branching Input + +This [deployment graph pattern](serve-deployment-graph-patterns-overview) lets you pass the same input to multiple deployments in parallel. You can then aggregate these deployments' intermediate outputs in another deployment. + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/deployment_graph_combine_two_nodes_with_passing_same_input_parallel.svg) + +## Code + +```{literalinclude} ../../doc_code/deployment_graph_patterns/branching_input.py +:language: python +:start-after: __graph_start__ +:end-before: __graph_end__ +``` + +## Execution + +This graph includes two `Model` nodes, with `weights` of 0 and 1. It passes the input into the two `Models`, and they add their own weights to it. Then, it uses the `combine` deployment to add the two `Model` deployments' outputs together. + +The resulting calculation is: + +``` +input = 1 +output1 = input + weight_1 = 0 + 1 = 1 +output2 = input + weight_2 = 1 + 1 = 2 +combine_output = output1 + output2 = 1 + 2 = 3 +``` + +The final output is 3: + +``` +$ python branching_input.py + +3 +``` diff --git a/doc/source/serve/tutorials/deployment-graph-patterns/conditional.md b/doc/source/serve/tutorials/deployment-graph-patterns/conditional.md new file mode 100644 index 0000000000000..baa62c432ffe0 --- /dev/null +++ b/doc/source/serve/tutorials/deployment-graph-patterns/conditional.md @@ -0,0 +1,72 @@ +# Pattern: Conditional + +This [deployment graph pattern](serve-deployment-graph-patterns-overview) allows you to control your graph's flow using conditionals. You can use this pattern to introduce a dynamic path for your requests to flow through. + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/control_flow_based_on_user_inputs.svg) + +## Code + +```{literalinclude} ../../doc_code/deployment_graph_patterns/conditional.py +:language: python +:start-after: __graph_start__ +:end-before: __graph_end__ +``` + +:::{note} +`combine` takes in intermediate values from the call graph as the individual arguments, `value1` and `value2`. You can also aggregate and pass these intermediate values as a list argument. However, this list contains references to the values, rather than the values themselves. You must explicitly use `await` to get the actual values before using them. Use `await` instead of `ray.get` to avoid [blocking the deployment](serve-model-composition-await-warning). + +For example: +```python +dag = combine.bind([output1, output2], user_input[1]) +... +@serve.deployment +async def combine(value_refs, combine_type): + values = await value_refs + value1, value2 = values +... +``` +::: + +## Execution + +The graph creates two `Model` nodes, with `weights` of 0 and 1. It then takes the `user_input` and unpacks it into two parts: a number and an operation. + +:::{note} +`handle.predict.remote()` can take an arbitrary number of arguments. These arguments can be unpacked by indexing into the `InputNode`. For example, + +```python +with InputNode() as user_input: + input_number, input_operation = user_input[0], user_input[1] +``` +::: + +It passes the number into the two `Model` nodes, similar to the [branching input](deployment-graph-pattern-branching-input) pattern. Then it passes the requested operation, as well as the intermediate outputs, to the `combine` deployment to get a final result. + +The example script makes two requests to the graph, both with a number input of 1. The resulting calculations are + +`max`: + +``` +input = 1 +output1 = input + weight_1 = 0 + 1 = 1 +output2 = input + weight_2 = 1 + 1 = 2 +combine_output = max(output1, output2) = max(1, 2) = 2 +``` + +`sum`: + +``` +input = 1 +output1 = input + weight_1 = 0 + 1 = 1 +output2 = input + weight_2 = 1 + 1 = 2 +combine_output = output1 + output2 = 1 + 2 = 3 +``` + +The final outputs are 2 and 3: + +``` +$ python conditional.py + +2 +3 +``` diff --git a/doc/source/serve/tutorials/deployment-graph-patterns/linear_pipeline.md b/doc/source/serve/tutorials/deployment-graph-patterns/linear_pipeline.md new file mode 100644 index 0000000000000..02ed781b4427c --- /dev/null +++ b/doc/source/serve/tutorials/deployment-graph-patterns/linear_pipeline.md @@ -0,0 +1,25 @@ +# Pattern: Linear Pipeline + +This [deployment graph pattern](serve-deployment-graph-patterns-overview) is a linear pipeline of deployments. The request flows from each deployment to the next, getting transformed each time. + +![pic](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/deployment-graph/chain_nodes_same_class_different_args.svg) + +## Code + +```{literalinclude} ../../doc_code/deployment_graph_patterns/linear_pipeline.py +:language: python +:start-after: __graph_start__ +:end-before: __graph_end__ +``` + +## Execution + +This graph has three nodes, which are all instances of the `Model` deployment. Each `Model` is constructed with a different `weight`, and its `forward` method adds that `weight` to the input. + +The call graph calls each deployment's `forward` method, one after another, which adds all the `Model`'s `weights` to the input. The code executes the graph on an input of 0 and after adding all the weights (0, 1, and 2), it gets a final `sum` of 3: + +```console +$ python linear_pipeline.py + +3 +``` diff --git a/doc/source/serve/tutorials/gradio-integration.md b/doc/source/serve/tutorials/gradio-integration.md index 3a75e9b05daad..e4f907e9b8d14 100644 --- a/doc/source/serve/tutorials/gradio-integration.md +++ b/doc/source/serve/tutorials/gradio-integration.md @@ -1,7 +1,6 @@ # Scaling your Gradio app with Ray Serve -In this guide, we will show you how to scale up your [Gradio](https://gradio.app/) application using Ray Serve. There is no need to change the internal architecture of your Gradio app; instead, we will neatly wrap it with Ray Serve and then scale it up to access more resources. - +In this guide, we will show you how to scale up your [Gradio](https://gradio.app/) application using Ray Serve. Keeping the internal architecture of your Gradio app intact (no changes), we simply wrap the app within Ray Serve as a deployment and scale it to access more resources. ## Dependencies To follow this tutorial, you will need Ray Serve and Gradio. If you haven't already, install them by running: @@ -9,22 +8,22 @@ To follow this tutorial, you will need Ray Serve and Gradio. If you haven't alre $ pip install "ray[serve]" $ pip install gradio ``` -For the purposes of this tutorial, we will be working with Gradio apps that run text summarization and text generation models. **Note that you can substitute this Gradio app for any Gradio app of your own!** +For this tutorial, we will use Gradio apps that run text summarization and generation models and use [HuggingFace's Pipelines](https://huggingface.co/docs/transformers/main_classes/pipelines) to access these models. **Note that you can substitute this Gradio app for any Gradio app of your own!** -We will be using [HuggingFace's Pipelines](https://huggingface.co/docs/transformers/main_classes/pipelines) to access the model. First, let's install the transformers module. +First, let's install the transformers module. ```console $ pip install transformers ``` ## Quickstart: Deploy your Gradio app with Ray Serve -This example will show you an easy, straightforward way to deploy your app onto Ray Serve. Start by creating a new Python file named `demo.py` and import `GradioServer` from Ray Serve for deploying your Gradio app, `gradio`, and `transformers.pipeline` for loading text summarization models. +This section shows you an easy way to deploy your app onto Ray Serve. First, create a new Python file named `demo.py`. Second, import `GradioServer` from Ray Serve to deploy your Gradio app later, `gradio`, and `transformers.pipeline` to load text summarization models. ```{literalinclude} ../../../../python/ray/serve/examples/doc/gradio-integration.py :start-after: __doc_import_begin__ :end-before: __doc_import_end__ ``` -Then, we construct the (optional) Gradio app `io`: +Then, we construct the (optional) Gradio app `io`. This application takes in text and uses the [T5 Small](https://huggingface.co/t5-small) text summarization model loaded using [HuggingFace's Pipelines](https://huggingface.co/docs/transformers/main_classes/pipelines) to summarize that text. :::{note} Remember you can substitute this with your own Gradio app if you want to try scaling up your own Gradio app! ::: @@ -33,21 +32,16 @@ Remember you can substitute this with your own Gradio app if you want to try sca :end-before: __doc_gradio_app_end__ ``` +### Deploying Gradio Server +In order to deploy your Gradio app onto Ray Serve, you need to wrap your Gradio app in a Serve [deployment](serve-key-concepts-deployment). `GradioServer` acts as that wrapper. It serves your Gradio app remotely on Ray Serve so that it can process and respond to HTTP requests. + +Replicas in a deployment are copies of your program running on Ray Serve, where each replica runs on a separate Ray cluster node's worker process. More replicas scales your deployment by serving more client requests. By wrapping your application in `GradioServer`, you can increase the number of replicas of your application or increase the number of CPUs and/or GPUs available to each replica. -### Understanding `GradioServer` -In order to deploy your Gradio app onto Ray Serve, you need to wrap your Gradio app in a Serve [deployment](serve-key-concepts-deployment). `GradioServer` acts as that wrapper. It serves your Gradio app remotely on Ray Serve so that it can process and respond to HTTP requests. :::{note} -`GradioServer` is simply `GradioIngress` but wrapped in a Serve deployment. +`GradioServer` is simply `GradioIngress` but wrapped in a Serve deployment. You can use `GradioServer` for the simple wrap-and-deploy use case, but as you will see in the next section, you can use `GradioIngress` to define your own Gradio Server for more customized use cases. ::: -```{literalinclude} ../../../../python/ray/serve/gradio_integrations.py -:start-after: __doc_gradio_ingress_begin__ -:end-before: __doc_gradio_ingress_end__ -``` - -### Deploy your Gradio Server -Replicas in a deployment are copies of your program living on Ray Serve, and more replicas means your deployment can serve more client requests. You can increase the number of replicas of your application or increase the number of CPUs and/or GPUs available to each replica. -Then, using either the example we created above, or an existing Gradio app (of type `Interface`, `Block`, `Parallel`, etc.), wrap it in your Gradio Server. +Using either the example app `io` we created above or an existing Gradio app (of type `Interface`, `Block`, `Parallel`, etc.), wrap it in your Gradio Server. ```{literalinclude} ../../../../python/ray/serve/examples/doc/gradio-integration.py :start-after: __doc_app_begin__ @@ -62,11 +56,11 @@ $ serve run demo:app Now you can access your Gradio app at `http://localhost:8000`! This is what it should look like: ![Gradio Result](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/gradio_result.png) -See [Putting Ray Serve Deployment Graphs in Production](https://docs.ray.io/en/master/serve/production.html#id1) for more information on how to deploy your app in production. +See the [Production Guide](serve-in-production) for more information on how to deploy your app in production. ## Parallelizing models with Ray Serve -You can run multiple models in parallel with Ray Serve by utilizing the [deployment graph](deployment-graph-e2e-tutorial) in Ray Serve. +You can run multiple models in parallel with Ray Serve by utilizing the [deployment graph](serve-model-composition-deployment-graph) in Ray Serve. ### Original Approach Suppose you want to run the following program. @@ -84,9 +78,9 @@ This is how you would do it normally: ### Parallelize using Ray Serve -With Ray Serve, we can parallelize the two text generation models by wrapping each model in a separate Ray Serve [deployment](serve-key-concepts-deployment). Deployments are defined by decorating a Python class or function with `@serve.deployment`, and usually wrap the models that you want to deploy on Ray Serve and handle incoming requests. +With Ray Serve, we can parallelize the two text generation models by wrapping each model in a separate Ray Serve [deployment](serve-key-concepts-deployment). Deployments are defined by decorating a Python class or function with `@serve.deployment`, and they usually wrap the models that you want to deploy on Ray Serve to handle incoming requests. -First, let's import our dependencies. Note that we need to import `GradioIngress` instead of `GradioServer` like before since we're now building a customized `MyGradioServer` that can run models in parallel. +Let's walk through a few steps to achieve parallelism. First, let's import our dependencies. Note that we need to import `GradioIngress` instead of `GradioServer` like before since we're now building a customized `MyGradioServer` that can run models in parallel. ```{literalinclude} ../../../../python/ray/serve/examples/doc/gradio-integration-parallel.py :start-after: __doc_import_begin__ @@ -113,16 +107,17 @@ Lastly, we link everything together: ``` :::{note} -This will bind your two text generation models (wrapped in Serve deployments) to `MyGradioServer._d1` and `MyGradioServer._d2`, forming a [deployment graph](deployment-graph-e2e-tutorial). Thus, we have built our Gradio Interface `io` such that it calls `MyGradioServer.fanout()`, which simply sends requests to your two text generation models that are deployed on Ray Serve. +This will bind your two text generation models (wrapped in Serve deployments) to `MyGradioServer._d1` and `MyGradioServer._d2`, forming a [deployment graph](serve-model-composition-deployment-graph). Thus, we have built our Gradio Interface `io` such that it calls `MyGradioServer.fanout()`, which simply sends requests to your two text generation models that are deployed on Ray Serve. ::: -Now, you can run your scalable app, and the two text generation models will run in parallel on Ray Serve! Run your Gradio app: +Now, you can run your scalable app, and the two text generation models will run in parallel on Ray Serve. +Run your Gradio app with the following command: ```console $ serve run demo:app ``` -Access your Gradio app at http://localhost:8000. This is what it should look like: +Access your Gradio app at `http://localhost:8000`, and you should see the following interactive interface: ![Gradio Result](https://raw.githubusercontent.com/ray-project/images/master/docs/serve/gradio_result_parallel.png) -See [Putting Ray Serve Deployment Graphs in Production](https://docs.ray.io/en/master/serve/production.html#id1) for more information on how to deploy your app in production. \ No newline at end of file +See the [Production Guide](serve-in-production) for more information on how to deploy your app in production. diff --git a/doc/source/serve/tutorials/index.md b/doc/source/serve/tutorials/index.md index ccd2a3422681a..6986b30788163 100644 --- a/doc/source/serve/tutorials/index.md +++ b/doc/source/serve/tutorials/index.md @@ -9,15 +9,9 @@ Ray Serve functionality and how to integrate different modeling frameworks. :maxdepth: '-1' :name: serve-tutorials -tensorflow -pytorch -sklearn +serve-ml-models batch -web-server-integration rllib gradio-integration +java ``` - -Other Topics: - -- {doc}`../deploying-serve` diff --git a/doc/source/serve/tutorials/java.md b/doc/source/serve/tutorials/java.md new file mode 100644 index 0000000000000..75863f39d612a --- /dev/null +++ b/doc/source/serve/tutorials/java.md @@ -0,0 +1,113 @@ +(serve-java-tutorial)= + +# Java Tutorial + +To use Java Ray Serve, you need the following dependency in your pom.xml. + +```xml + + io.ray + ray-serve + ${ray.version} + provided + +``` + +> NOTE: After installing Ray via Python, the Java jar of Ray Serve is included locally. The `provided` scope could ensure the Java code using Ray Serve can be compiled and will not cause version conflicts when deployed on the cluster. + +## Example Model + +Our example use case is derived from production workflow of a financial application. The application needs to compute the best strategy to interact with different banks for a single task. + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/Strategy.java +:end-before: docs-strategy-end +:language: java +:start-after: docs-strategy-start +``` + +This `Strategy` class is used to calculate the indicators of a number of banks. + +* The `calc` method is the entry of the calculation. The input parameters are the time interval of calculation and the map of the banks and their indicators. As we can see, the `calc` method contains a two-tier `for` loop, traversing each indicator list of each bank, and calling the `calcBankIndicators` method to calculate the indicators of the specified bank. + +- There is another layer of `for` loop in the `calcBankIndicators` method, which traverses each indicator, and then calls the `calcIndicator` method to calculate the specific indicator of the bank. +- The `calcIndicator` method is a specific calculation logic based on the bank, the specified time interval and the indicator. + +This is the code that uses the `Strategy` class: + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalc.java +:end-before: docs-strategy-calc-end +:language: java +:start-after: docs-strategy-calc-start +``` + +When the scale of banks and indicators expands, the three-tier `for` loop will slow down the calculation. Even if the thread pool is used to calculate each indicator in parallel, we may encounter a single machine performance bottleneck. Moreover, this `Strategy` object cannot be reused as a resident service. + +## Converting to a Ray Serve Deployment + +Through Ray Serve, the core computing logic of `Strategy` can be deployed as a scalable distributed computing service. + +First, we can extract the indicator calculation of each institution into a separate `StrategyOnRayServe` class: + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyOnRayServe.java +:end-before: docs-strategy-end +:language: java +:start-after: docs-strategy-start +``` + +Next, we start the Ray Serve runtime and deploy `StrategyOnRayServe` as a deployment. + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java +:end-before: docs-deploy-end +:language: java +:start-after: docs-deploy-start +``` + +The `Deployment.create` makes a Deployment object named "strategy." After executing `Deployment.deploy`, this "strategy" deployment is deployed in the instance of Ray Serve with four replicas, and we can access it for distributed parallel computing. + +## Testing the Ray Serve Deployment + +Now we can test the "strategy" deployment using RayServeHandle inside Ray: + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java +:end-before: docs-calc-end +:language: java +:start-after: docs-calc-start +``` + +At present, the calculation of each bank's each indicator is still executed serially, and sent to Ray for execution. We can make the calculation concurrent, which not only improves the calculation efficiency, but also solves the bottleneck of single machine. + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java +:end-before: docs-parallel-calc-end +:language: java +:start-after: docs-parallel-calc-start +``` + +Now, we can use `StrategyCalcOnRayServe` like the example in the `main` method: + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java +:end-before: docs-main-end +:language: java +:start-after: docs-main-start +``` + +## Calling Ray Serve Deployment with HTTP + +Another way to test or call a deployment is through the HTTP request. But there are now two limitations for the Java deployments: + +- The HTTP requests can only be processed by the `call` method of the user class. + +- The `call` method could only have one input parameter, and the type of the input parameter and the returned value can only be `String`. + +If we want to call the "strategy" deployment via HTTP, the class can be rewritten like this: + +```{literalinclude} ../../../../java/serve/src/test/java/io/ray/serve/docdemo/HttpStrategyOnRayServe.java +:end-before: docs-strategy-end +:language: java +:start-after: docs-strategy-start +``` + +After deploying this deployment, we can access it through `curl` command: + +```shell +curl -d '{"time":1641038674, "bank":"test_bank", "indicator":"test_indicator"}' http://127.0.0.1:8000/strategy +``` diff --git a/doc/source/serve/tutorials/pytorch.md b/doc/source/serve/tutorials/pytorch.md deleted file mode 100644 index e357505f1dc2a..0000000000000 --- a/doc/source/serve/tutorials/pytorch.md +++ /dev/null @@ -1,48 +0,0 @@ -(serve-pytorch-tutorial)= - -# PyTorch Tutorial - -In this guide, we will load and serve a PyTorch Resnet Model. -In particular, we show: - -- How to load the model from PyTorch's pre-trained modelzoo. -- How to parse the JSON request, transform the payload and evaluated in the model. - -Please see the [Key Concepts](serve-key-concepts) to learn more general information about Ray Serve. - -This tutorial requires Pytorch and Torchvision installed in your system. Ray Serve -is framework agnostic and works with any version of PyTorch. - -```bash -pip install torch torchvision -``` - -Let's import Ray Serve and some other helpers. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_pytorch.py -:end-before: __doc_import_end__ -:start-after: __doc_import_begin__ -``` - -Services are just defined as normal classes with `__init__` and `__call__` methods. -The `__call__` method will be invoked per request. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_pytorch.py -:end-before: __doc_define_servable_end__ -:start-after: __doc_define_servable_begin__ -``` - -Now that we've defined our services, let's deploy the model to Ray Serve. We will -define a Serve deployment that will be exposed over an HTTP route. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_pytorch.py -:end-before: __doc_deploy_end__ -:start-after: __doc_deploy_begin__ -``` - -Let's query it! - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_pytorch.py -:end-before: __doc_query_end__ -:start-after: __doc_query_begin__ -``` diff --git a/doc/source/serve/tutorials/rllib.md b/doc/source/serve/tutorials/rllib.md index 7c1b54b5e8bc1..cd14182418afe 100644 --- a/doc/source/serve/tutorials/rllib.md +++ b/doc/source/serve/tutorials/rllib.md @@ -72,7 +72,7 @@ pass them into the restored `Algorithm` using the `compute_single_action` method from starlette.requests import Request -@serve.deployment(route_prefix="/cartpole-ppo") +@serve.deployment class ServePPOModel: def __init__(self, checkpoint_path) -> None: # Re-create the originally used config. @@ -94,20 +94,19 @@ class ServePPOModel: :::{tip} Although we used a single input and `Algorithm.compute_single_action(...)` here, you -can process a batch of input using Ray Serve's [batching](serve-batching) feature +can process a batch of input using Ray Serve's [batching](serve-performance-batching-requests) feature and use `Algorithm.compute_actions(...)` to process a batch of inputs. ::: Now that we've defined our `ServePPOModel` service, let's deploy it to Ray Serve. -The deployment will be exposed through the `/cartpole-ppo` route. ```{code-cell} python3 :tags: [hide-output] -serve.start() -ServePPOModel.deploy(checkpoint_path) +ppo_model = ServePPOModel.bind(checkpoint_path) +serve.run(ppo_model) ``` -Note that the `checkpoint_path` that we passed to the `deploy()` method will be passed to +Note that the `checkpoint_path` that we passed to the `bind()` method will be passed to the `__init__` method of the `ServePPOModel` class that we defined above. Now that the model is deployed, let's query it! @@ -123,7 +122,7 @@ for _ in range(5): print(f"-> Sending observation {obs}") resp = requests.get( - "http://localhost:8000/cartpole-ppo", json={"observation": obs.tolist()} + "http://localhost:8000/", json={"observation": obs.tolist()} ) print(f"<- Received response {resp.json()}") ``` diff --git a/doc/source/serve/tutorials/serve-ml-models.md b/doc/source/serve/tutorials/serve-ml-models.md new file mode 100644 index 0000000000000..9a9e0c6f62e4f --- /dev/null +++ b/doc/source/serve/tutorials/serve-ml-models.md @@ -0,0 +1,256 @@ +(serve-ml-models-tutorial)= + +# Serving ML Models (Tensorflow, PyTorch, Scikit-Learn, others) + +In this guide, we will show you how to train models from various machine learning frameworks and deploy them to Ray Serve. + + +Please see the [Key Concepts](serve-key-concepts) to learn more general information about Ray Serve. + + +::::{tabbed} Keras and Tensorflow + + +Let's train and deploy a simple Tensorflow neural net. +In particular, we will show: + +- How to train a Tensorflow model and load the model from your file system in your Ray Serve deployment. +- How to parse the JSON request and make a prediction. + +Ray Serve is framework agnostic -- you can use any version of Tensorflow. +However, for this tutorial, we will use Tensorflow 2 and Keras. We will also need `requests` to send HTTP requests to your model deployment. If you haven't already, please install Tensorflow 2 and requests by running: + +```console +$ pip install "tensorflow>=2.0" requests +``` + +Open a new Python file called `tutorial_tensorflow.py`. First, let's import Ray Serve and some other helpers. + +```{literalinclude} ../doc_code/tutorial_tensorflow.py +:start-after: __doc_import_begin__ +:end-before: __doc_import_end__ +``` + +Next, let's train a simple MNIST model using Keras. + +```{literalinclude} ../doc_code/tutorial_tensorflow.py +:start-after: __doc_train_model_begin__ +:end-before: __doc_train_model_end__ +``` + +Next, we define a class `TFMnistModel` that will accept HTTP requests and run the MNIST model that we trained. It is decorated with `@serve.deployment` to make it a deployment object so it can be deployed onto Ray Serve. Note that the Serve deployment is exposed over an HTTP route, and by default the `__call__` method is invoked when a request is sent to your deployment over HTTP. + +```{literalinclude} ../doc_code/tutorial_tensorflow.py +:start-after: __doc_define_servable_begin__ +:end-before: __doc_define_servable_end__ +``` + +:::{note} +When `TFMnistModel` is deployed and instantiated, it will load the Tensorflow model from your file system so that it can be ready to run inference on the model and serve requests later. +::: + +Now that we've defined our Serve deployment, let's prepare it so that it can be deployed. + +```{literalinclude} ../doc_code/tutorial_tensorflow.py +:start-after: __doc_deploy_begin__ +:end-before: __doc_deploy_end__ +``` + +:::{note} +`TFMnistModel.bind(TRAINED_MODEL_PATH)` binds the argument `TRAINED_MODEL_PATH` to our deployment and returns a `DeploymentNode` object (wrapping an `TFMnistModel` deployment object) that can then be used to connect with other `DeploymentNodes` to form a more complex [deployment graph](serve-model-composition-deployment-graph). +::: + +Finally, we can deploy our model to Ray Serve through the terminal. +```console +$ serve run tutorial_tensorflow:mnist_model +``` + +Let's query it! While Serve is running, open a separate terminal window, and run the following in an interactive Python shell or a separate Python script: + +```python +import requests +import numpy as np + +resp = requests.get( + "http://localhost:8000/", json={"array": np.random.randn(28 * 28).tolist()} +) +print(resp.json()) +``` + +You should get an output like the following (the exact prediction may vary): + +```bash +{ + "prediction": [[-1.504277229309082, ..., -6.793371200561523]], + "file": "/tmp/mnist_model.h5" +} +``` +:::: + +::::{tabbed} Pytorch + +Let's load and deploy a PyTorch Resnet Model. +In particular, we will show: + +- How to load the model from PyTorch's pre-trained modelzoo. +- How to parse the JSON request, transform the payload and make a prediction. + +This tutorial will require PyTorch and Torchvision. Ray Serve is framework agnostic and works with any version of PyTorch. We will also need `requests` to send HTTP requests to your model deployment. If you haven't already, please install them by running: + +```console +$ pip install torch torchvision requests +``` + +Open a new Python file called `tutorial_pytorch.py`. First, let's import Ray Serve and some other helpers. + +```{literalinclude} ../doc_code/tutorial_pytorch.py +:start-after: __doc_import_begin__ +:end-before: __doc_import_end__ +``` + +We define a class `ImageModel` that parses the input data, transforms the images, and runs the ResNet18 model loaded from `torchvision`. It is decorated with `@serve.deployment` to make it a deployment object so it can be deployed onto Ray Serve. Note that the Serve deployment is exposed over an HTTP route, and by default the `__call__` method is invoked when a request is sent to your deployment over HTTP. + +```{literalinclude} ../doc_code/tutorial_pytorch.py +:start-after: __doc_define_servable_begin__ +:end-before: __doc_define_servable_end__ +``` + +:::{note} +When `ImageModel` is deployed and instantiated, it will load the resnet18 model from `torchvision` so that it can be ready to run inference on the model and serve requests later. +::: + +Now that we've defined our Serve deployment, let's prepare it so that it can be deployed. + +```{literalinclude} ../doc_code/tutorial_pytorch.py +:start-after: __doc_deploy_begin__ +:end-before: __doc_deploy_end__ +``` + +:::{note} +`ImageModel.bind()` returns a `DeploymentNode` object (wrapping an `ImageModel` deployment object) that can then be used to connect with other `DeploymentNodes` to form a more complex [deployment graph](serve-model-composition-deployment-graph). +::: + +Finally, we can deploy our model to Ray Serve through the terminal. +```console +$ serve run tutorial_pytorch:image_model +``` + +Let's query it! While Serve is running, open a separate terminal window, and run the following in an interactive Python shell or a separate Python script: + +```python +import requests + +ray_logo_bytes = requests.get( + "https://raw.githubusercontent.com/ray-project/" + "ray/master/doc/source/images/ray_header_logo.png" +).content + +resp = requests.post("http://localhost:8000/", data=ray_logo_bytes) +print(resp.json()) +``` + +You should get an output like the following (the exact number may vary): + +```bash +{'class_index': 919} +``` +:::: + +::::{tabbed} Scikit-Learn + +Let's train and deploy a simple Scikit-Learn classifier. +In particular, we will show: + +- How to load the Scikit-Learn model from file system in your Ray Serve definition. +- How to parse the JSON request and make a prediction. + +Ray Serve is framework agnostic. You can use any version of sklearn. We will also need `requests` to send HTTP requests to your model deployment. If you haven't already, please install scikit-learn and requests by running: + +```console +$ pip install scikit-learn requests +``` + +Open a new Python file called `tutorial_sklearn.py`. Let's import Ray Serve and some other helpers. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_import_begin__ +:end-before: __doc_import_end__ +``` + +**Train a Classifier** + +We will train a classifier with the [iris dataset](https://scikit-learn.org/stable/auto_examples/datasets/plot_iris_dataset.html). + + +First, let's instantiate a `GradientBoostingClassifier` loaded from Scikit-Learn. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_instantiate_model_begin__ +:end-before: __doc_instantiate_model_end__ +``` + +Next, load the iris dataset and split the data into training and validation sets. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_data_begin__ +:end-before: __doc_data_end__ +``` + +We then train the model and save it to file. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_train_model_begin__ +:end-before: __doc_train_model_end__ +``` + +**Deploy with Ray Serve** + +Finally, we are ready to deploy the classifier using Ray Serve! + +We define a class `BoostingModel` that runs inference on the `GradientBoosingClassifier` model we trained and returns the resulting label. It is decorated with `@serve.deployment` to make it a deployment object so it can be deployed onto Ray Serve. Note that the Serve deployment is exposed over an HTTP route, and by default the `__call__` method is invoked when a request is sent to your deployment over HTTP. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_define_servable_begin__ +:end-before: __doc_define_servable_end__ +``` + +:::{note} +When `BoostingModel` is deployed and instantiated, it will load the classifier model that we trained from your file system so that it can be ready to run inference on the model and serve requests later. +::: + +Now that we've defined our Serve deployment, let's prepare it so that it can be deployed. + +```{literalinclude} ../doc_code/tutorial_sklearn.py +:start-after: __doc_deploy_begin__ +:end-before: __doc_deploy_end__ +``` + +:::{note} +`BoostingModel.bind(MODEL_PATH, LABEL_PATH)` binds the arguments `MODEL_PATH` and `LABEL_PATH` to our deployment and returns a `DeploymentNode` object (wrapping an `BoostingModel` deployment object) that can then be used to connect with other `DeploymentNodes` to form a more complex [deployment graph](serve-model-composition-deployment-graph). +::: + +Finally, we can deploy our model to Ray Serve through the terminal. +```console +$ serve run tutorial_sklearn:boosting_model +``` + +Let's query it! While Serve is running, open a separate terminal window, and run the following in an interactive Python shell or a separate Python script: + +```python +import requests + +sample_request_input = { + "sepal length": 1.2, + "sepal width": 1.0, + "petal length": 1.1, + "petal width": 0.9, +} +response = requests.get("http://localhost:8000/", json=sample_request_input) +print(response.text) +``` + +You should get an output like the following (the exact prediction may vary): +```python +{"result": "versicolor"} +``` +:::: \ No newline at end of file diff --git a/doc/source/serve/tutorials/sklearn.md b/doc/source/serve/tutorials/sklearn.md deleted file mode 100644 index 900a10818c735..0000000000000 --- a/doc/source/serve/tutorials/sklearn.md +++ /dev/null @@ -1,54 +0,0 @@ -(serve-sklearn-tutorial)= - -# Scikit-Learn Tutorial - -In this guide, we will train and deploy a simple Scikit-Learn classifier. -In particular, we show: - -- How to load the model from file system in your Ray Serve definition -- How to parse the JSON request and evaluated in sklearn model - -Please see the [Key Concepts](serve-key-concepts) to learn more general information about Ray Serve. - -Ray Serve is framework agnostic. You can use any version of sklearn. - -```bash -pip install scikit-learn -``` - -Let's import Ray Serve and some other helpers. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_sklearn.py -:end-before: __doc_import_end__ -:start-after: __doc_import_begin__ -``` - -We will train a logistic regression with the iris dataset. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_sklearn.py -:end-before: __doc_train_model_end__ -:start-after: __doc_train_model_begin__ -``` - -Services are just defined as normal classes with `__init__` and `__call__` methods. -The `__call__` method will be invoked per request. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_sklearn.py -:end-before: __doc_define_servable_end__ -:start-after: __doc_define_servable_begin__ -``` - -Now that we've defined our services, let's deploy the model to Ray Serve. We will -define a Serve deployment that will be exposed over an HTTP route. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_sklearn.py -:end-before: __doc_deploy_end__ -:start-after: __doc_deploy_begin__ -``` - -Let's query it! - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_sklearn.py -:end-before: __doc_query_end__ -:start-after: __doc_query_begin__ -``` diff --git a/doc/source/serve/tutorials/tensorflow.md b/doc/source/serve/tutorials/tensorflow.md deleted file mode 100644 index 18cf7e6fa324c..0000000000000 --- a/doc/source/serve/tutorials/tensorflow.md +++ /dev/null @@ -1,56 +0,0 @@ -(serve-tensorflow-tutorial)= - -# Keras and Tensorflow Tutorial - -In this guide, we will train and deploy a simple Tensorflow neural net. -In particular, we show: - -- How to load the model from file system in your Ray Serve definition -- How to parse the JSON request and evaluated in Tensorflow - -Please see the [Key Concepts](serve-key-concepts) to learn more general information about Ray Serve. - -Ray Serve is framework agnostic -- you can use any version of Tensorflow. -However, for this tutorial, we use Tensorflow 2 and Keras. Please make sure you have -Tensorflow 2 installed. - -```bash -pip install "tensorflow>=2.0" -``` - -Let's import Ray Serve and some other helpers. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_tensorflow.py -:end-before: __doc_import_end__ -:start-after: __doc_import_begin__ -``` - -We will train a simple MNIST model using Keras. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_tensorflow.py -:end-before: __doc_train_model_end__ -:start-after: __doc_train_model_begin__ -``` - -Services are just defined as normal classes with `__init__` and `__call__` methods. -The `__call__` method will be invoked per request. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_tensorflow.py -:end-before: __doc_define_servable_end__ -:start-after: __doc_define_servable_begin__ -``` - -Now that we've defined our services, let's deploy the model to Ray Serve. We will -define a Serve deployment that will be exposed over an HTTP route. - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_tensorflow.py -:end-before: __doc_deploy_end__ -:start-after: __doc_deploy_begin__ -``` - -Let's query it! - -```{literalinclude} ../../../../python/ray/serve/examples/doc/tutorial_tensorflow.py -:end-before: __doc_query_end__ -:start-after: __doc_query_begin__ -``` diff --git a/doc/source/serve/tutorials/web-server-integration.md b/doc/source/serve/tutorials/web-server-integration.md deleted file mode 100644 index 164ea7d18934d..0000000000000 --- a/doc/source/serve/tutorials/web-server-integration.md +++ /dev/null @@ -1,33 +0,0 @@ -(serve-web-server-integration-tutorial)= - -# Integration with Existing Web Servers - -In this guide, you will learn how to use Ray Serve to scale up your existing web application. The key feature of Ray Serve that makes this possible is the Python-native {ref}`servehandle-api`, which allows you keep using your same Python web server while offloading your heavy computation to Ray Serve. - -We give two examples, one using a [FastAPI](https://fastapi.tiangolo.com/) web server and another using an [AIOHTTP](https://docs.aiohttp.org/en/stable/) web server, but the same approach will work with any Python web server. - -## Scaling Up a FastAPI Application - -Ray Serve has a native integration with FastAPI - please see {ref}`serve-fastapi-http`. - -## Scaling Up an AIOHTTP Application - -In this section, we'll integrate Ray Serve with an [AIOHTTP](https://docs.aiohttp.org/en/stable/) web server run using [Gunicorn](https://gunicorn.org/). You'll need to install AIOHTTP and gunicorn with the command `pip install aiohttp gunicorn`. - -First, here is the script that deploys Ray Serve: - -```{literalinclude} ../../../../python/ray/serve/examples/doc/aiohttp/aiohttp_deploy_serve.py -``` - -Next is the script that defines the AIOHTTP server: - -```{literalinclude} ../../../../python/ray/serve/examples/doc/aiohttp/aiohttp_app.py -``` - -Here's how to run this example: - -1. Run `ray start --head` to start a local Ray cluster in the background. -2. In the directory where the example files are saved, run `python aiohttp_deploy_serve.py` to deploy our Ray Serve deployment. -3. Run `gunicorn aiohttp_app:app --worker-class aiohttp.GunicornWebWorker` to start the AIOHTTP app using gunicorn. -4. To test out the server, run `curl localhost:8000/dummy-model`. This should output `Model received data: dummy input`. -5. For cleanup, you can press Ctrl-C to stop the Gunicorn server, and run `ray stop` to stop the background Ray cluster. diff --git a/doc/source/serve/user-guide.md b/doc/source/serve/user-guide.md index 2b81198559f07..e09ee3bafe524 100644 --- a/doc/source/serve/user-guide.md +++ b/doc/source/serve/user-guide.md @@ -3,16 +3,11 @@ If you’re new to Ray Serve, we recommend starting with the [Ray Serve Quick Start](getting_started). -This user guide will help you navigate the Ray Serve project and show you how achieve several tasks, for instance -you will learn -- [Create/Update/Delete Serve Deployments](managing-deployments) -- [Handling Dependencies](handling-dependencies) -- [Configuring HTTP and Bring Your own FastAPI app](http-guide) -- [Using HTTP Adapters](http-adapters) -- [Composing Deployments](handle-guide) -- [Serving ML Models](ml-models) -- [Using Deployment Graph](deployment-graph) -- [Deploying Ray Serve](deploying-serve) -- [Debugging/Monitoring](monitoring) +This user guide will help you navigate the Ray Serve project and show you how to achieve several key tasks: +- [HTTP Handling](http-guide) +- [Scaling and Resource Allocation](scaling-and-resource-allocation) +- [Model Composition](serve-model-composition) +- [Development Workflow](dev-workflow) +- [Production Guide](serve-in-production) - [Performance Tuning](performance) -- [Autoscaling](autoscaling) +- [Handling Dependencies](handling-dependencies) diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/HttpStrategyOnRayServe.java b/java/serve/src/test/java/io/ray/serve/docdemo/HttpStrategyOnRayServe.java new file mode 100644 index 0000000000000..dc4bcb58766d8 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/HttpStrategyOnRayServe.java @@ -0,0 +1,20 @@ +package io.ray.serve.docdemo; + +// docs-strategy-start +import com.google.gson.Gson; +import java.util.Map; + +public class HttpStrategyOnRayServe { + + private Gson gson = new Gson(); + + public String call(String dataJson) { + Map data = gson.fromJson(dataJson, Map.class); + long time = (long) data.get("time"); + String bank = (String) data.get("bank"); + String indicator = (String) data.get("indicator"); + // do bank data calculation + return bank + "-" + indicator + "-" + time; // Demo; + } +} +// docs-strategy-end diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java b/java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java new file mode 100644 index 0000000000000..67d2bc73ea154 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/ManageDeployment.java @@ -0,0 +1,78 @@ +package io.ray.serve.docdemo; + +import io.ray.serve.api.Serve; +import io.ray.serve.deployment.Deployment; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class ManageDeployment { + + // docs-create-start + public static class Counter { + + private AtomicInteger value; + + public Counter(Integer value) { + this.value = new AtomicInteger(value); + } + + public String call(String delta) { + return String.valueOf(value.addAndGet(Integer.valueOf(delta))); + } + } + + public void create() { + Serve.deployment() + .setName("counter") + .setDeploymentDef(Counter.class.getName()) + .setInitArgs(new Object[] {1}) + .setNumReplicas(2) + .create() + .deploy(true); + } + // docs-create-end + + // docs-query-start + public void query() { + Deployment deployment = Serve.getDeployment("counter"); + } + // docs-query-end + + // docs-update-start + public void update() { + Serve.deployment() + .setName("counter") + .setDeploymentDef(Counter.class.getName()) + .setInitArgs(new Object[] {2}) + .setNumReplicas(2) + .create() + .deploy(true); + } + // docs-update-end + + // docs-scale-start + public void scaleOut() { + Deployment deployment = Serve.getDeployment("counter"); + + // Scale up to 10 replicas. + deployment.options().setNumReplicas(10).create().deploy(true); + + // Scale down to 1 replica. + deployment.options().setNumReplicas(1).create().deploy(true); + } + // docs-scale-end + + // docs-resource-start + public void manageResource() { + Map rayActorOptions = new HashMap<>(); + rayActorOptions.put("num_gpus", 1); + Serve.deployment() + .setName("counter") + .setDeploymentDef(Counter.class.getName()) + .setRayActorOptions(rayActorOptions) + .create() + .deploy(true); + } + // docs-resource-end +} diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/ManagePythonDeployment.java b/java/serve/src/test/java/io/ray/serve/docdemo/ManagePythonDeployment.java new file mode 100644 index 0000000000000..075f3128eb674 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/ManagePythonDeployment.java @@ -0,0 +1,30 @@ +package io.ray.serve.docdemo; + +import io.ray.api.Ray; +import io.ray.serve.api.Serve; +import io.ray.serve.deployment.Deployment; +import java.io.File; + +public class ManagePythonDeployment { + + public static void main(String[] args) { + + System.setProperty( + "ray.job.code-search-path", + System.getProperty("java.class.path") + File.pathSeparator + "/path/to/code/"); + + Serve.start(true, false, null); + + Deployment deployment = + Serve.deployment() + // .setDeploymentLanguage(DeploymentLanguage.PYTHON) + .setName("counter") + .setDeploymentDef("counter.Counter") + .setNumReplicas(1) + .setInitArgs(new Object[] {"1"}) + .create(); + deployment.deploy(true); + + System.out.println(Ray.get(deployment.getHandle().method("increase").remote("2"))); + } +} diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/Strategy.java b/java/serve/src/test/java/io/ray/serve/docdemo/Strategy.java new file mode 100644 index 0000000000000..62fee2b0cabb7 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/Strategy.java @@ -0,0 +1,35 @@ +package io.ray.serve.docdemo; + +// docs-strategy-start +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +public class Strategy { + + public List calc(long time, Map>> banksAndIndicators) { + List results = new ArrayList<>(); + for (Entry>> e : banksAndIndicators.entrySet()) { + String bank = e.getKey(); + for (List indicators : e.getValue()) { + results.addAll(calcBankIndicators(time, bank, indicators)); + } + } + return results; + } + + public List calcBankIndicators(long time, String bank, List indicators) { + List results = new ArrayList<>(); + for (String indicator : indicators) { + results.add(calcIndicator(time, bank, indicator)); + } + return results; + } + + public String calcIndicator(long time, String bank, String indicator) { + // do bank data calculation + return bank + "-" + indicator + "-" + time; // Demo; + } +} +// docs-strategy-end diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalc.java b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalc.java new file mode 100644 index 0000000000000..10e0333b2bfd1 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalc.java @@ -0,0 +1,28 @@ +package io.ray.serve.docdemo; + +// docs-strategy-calc-start +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class StrategyCalc { + + public static void main(String[] args) { + long time = System.currentTimeMillis(); + String bank1 = "demo_bank_1"; + String bank2 = "demo_bank_2"; + String indicator1 = "demo_indicator_1"; + String indicator2 = "demo_indicator_2"; + Map>> banksAndIndicators = new HashMap<>(); + banksAndIndicators.put(bank1, Arrays.asList(Arrays.asList(indicator1, indicator2))); + banksAndIndicators.put( + bank2, Arrays.asList(Arrays.asList(indicator1), Arrays.asList(indicator2))); + + Strategy strategy = new Strategy(); + List results = strategy.calc(time, banksAndIndicators); + + System.out.println(results); + } +} +// docs-strategy-calc-end diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java new file mode 100644 index 0000000000000..e9b5d9c81dd79 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyCalcOnRayServe.java @@ -0,0 +1,92 @@ +package io.ray.serve.docdemo; + +import io.ray.api.ObjectRef; +import io.ray.serve.api.Serve; +import io.ray.serve.deployment.Deployment; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +public class StrategyCalcOnRayServe { + + // docs-deploy-start + public void deploy() { + Serve.start(true, false, null); + + Deployment deployment = + Serve.deployment() + .setName("strategy") + .setDeploymentDef(StrategyOnRayServe.class.getName()) + .setNumReplicas(4) + .create(); + deployment.deploy(true); + } + // docs-deploy-end + + // docs-calc-start + public List calc(long time, Map>> banksAndIndicators) { + Deployment deployment = Serve.getDeployment("strategy"); + + List results = new ArrayList<>(); + for (Entry>> e : banksAndIndicators.entrySet()) { + String bank = e.getKey(); + for (List indicators : e.getValue()) { + for (String indicator : indicators) { + results.add( + (String) + deployment + .getHandle() + .method("calcIndicator") + .remote(time, bank, indicator) + .get()); + } + } + } + return results; + } + // docs-calc-end + + // docs-parallel-calc-start + public List parallelCalc(long time, Map>> banksAndIndicators) { + Deployment deployment = Serve.getDeployment("strategy"); + + List results = new ArrayList<>(); + List> refs = new ArrayList<>(); + for (Entry>> e : banksAndIndicators.entrySet()) { + String bank = e.getKey(); + for (List indicators : e.getValue()) { + for (String indicator : indicators) { + refs.add(deployment.getHandle().method("calcIndicator").remote(time, bank, indicator)); + } + } + } + for (ObjectRef ref : refs) { + results.add((String) ref.get()); + } + return results; + } + // docs-parallel-calc-end + + // docs-main-start + public static void main(String[] args) { + long time = System.currentTimeMillis(); + String bank1 = "demo_bank_1"; + String bank2 = "demo_bank_2"; + String indicator1 = "demo_indicator_1"; + String indicator2 = "demo_indicator_2"; + Map>> banksAndIndicators = new HashMap<>(); + banksAndIndicators.put(bank1, Arrays.asList(Arrays.asList(indicator1, indicator2))); + banksAndIndicators.put( + bank2, Arrays.asList(Arrays.asList(indicator1), Arrays.asList(indicator2))); + + StrategyCalcOnRayServe strategy = new StrategyCalcOnRayServe(); + strategy.deploy(); + List results = strategy.parallelCalc(time, banksAndIndicators); + + System.out.println(results); + } + // docs-main-end +} diff --git a/java/serve/src/test/java/io/ray/serve/docdemo/StrategyOnRayServe.java b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyOnRayServe.java new file mode 100644 index 0000000000000..e2c22b49c2b82 --- /dev/null +++ b/java/serve/src/test/java/io/ray/serve/docdemo/StrategyOnRayServe.java @@ -0,0 +1,11 @@ +package io.ray.serve.docdemo; + +// docs-strategy-start +public class StrategyOnRayServe { + + public String calcIndicator(long time, String bank, String indicator) { + // do bank data calculation + return bank + "-" + indicator + "-" + time; // Demo; + } +} +// docs-strategy-end diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 13d5da48f4d4c..96a2f002ea29b 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -520,10 +520,11 @@ def build(target: Union[ClassNode, FunctionNode]) -> Application: Serve application consisting of one or more deployments. This is intended to be used for production scenarios and deployed via the Serve REST API or CLI, so there are some restrictions placed on the deployments: - 1) All of the deployments must be importable. That is, they cannot be - defined in __main__ or inline defined. The deployments will be - imported in production using the same import path they were here. - 2) All arguments bound to the deployment must be JSON-serializable. + + 1) All of the deployments must be importable. That is, they cannot be + defined in __main__ or inline defined. The deployments will be + imported in production using the same import path they were here. + 2) All arguments bound to the deployment must be JSON-serializable. The returned Application object can be exported to a dictionary or YAML config.