Skip to content

Commit

Permalink
1. Delete DAGDriver and InputNode. 2. Add DeploymentHandle and Deploy…
Browse files Browse the repository at this point in the history
…mentResponse demo. 3. Adjusting the way Python deploys deployment for other languages.

Signed-off-by: chuhan.ly <[email protected]>
  • Loading branch information
chuhan.ly committed Sep 14, 2023
1 parent b21bb7a commit eb2a07a
Showing 1 changed file with 29 additions and 133 deletions.
162 changes: 29 additions & 133 deletions reps/2023-08-18-serve-java-dag-api.md
Original file line number Diff line number Diff line change
Expand Up @@ -15,19 +15,16 @@ def preprocess(inp: int) -> int:

@serve.deployment
class Model:
def __init__(self, increment: int):
def __init__(self, preprocess_handle: RayServeHandle, increment: int):
self.preprocess_handle: DeploymentHandle = preprocess_handle.options(use_new_handle_api=True)
self.increment = increment

def predict(self, inp: int) -> int:
return inp + self.increment
async def predict(self, inp: int) -> int:
preprocessed = await self.preprocess_handle.remote(inp)
return preprocessed + self.increment


with InputNode() as inp:
model = Model.bind(increment=2)
output = model.predict.bind(preprocess.bind(inp))
serve_dag = DAGDriver.bind(output)

handle = serve.run(serve_dag)
app = Model.bind(preprocess.bind(), increment=2)
handle = serve.run(app)
assert ray.get(handle.predict.remote(1)) == 4

```
Expand All @@ -37,14 +34,14 @@ Main `ray` project. A part of java/serve.
## Stewardship

### Required Reviewers
@sihanwang41
@sihanwang41 @edoakes

### Shepherd of the Proposal (should be a senior committer)
@sihanwang41
@sihanwang41 @edoakes

## Design and Architecture

### Update Java user API to be consistent with Python
### Update Java User API to be Consistent with Python
A standard Java deployment demo is shown below:
```java
// Demo 1
Expand Down Expand Up @@ -73,27 +70,27 @@ public class DeploymentDemo {

```
In this demo, a DAG node is defined through the `bind` method of the Deployment, and it is deployed using the `Serve.run` API.
Furthermore, a Deployment can bind other Deployments, and users can use the Deployment input parameters in a similar way to `RayServeHandle`. For example:
Furthermore, a Deployment can bind other Deployments, and users can use the Deployment input parameters in a similar way to `DeploymentHandle`. For example:
```java
// Demo 2
import io.ray.api.ObjectRef;
import io.ray.serve.api.Serve;
import io.ray.serve.deployment.Application;
import io.ray.serve.handle.RayServeHandle;
import io.ray.serve.handle.DeploymentHandle;
import io.ray.serve.handle.DeploymentResponse;

public class Driver {
private RayServeHandle modelAHandle;
private RayServeHandle modelBHandle;
private DeploymentHandle modelAHandle;
private DeploymentHandle modelBHandle;

public Driver(RayServeHandle modelAHandle, RayServeHandle modelBHandle) {
public Driver(DeploymentHandle modelAHandle, DeploymentHandle modelBHandle) {
this.modelAHandle = modelAHandle;
this.modelBHandle = modelBHandle;
}

public String call(String request) {
ObjectRef<Object> refA = modelAHandle.remote(request);
ObjectRef<Object> refB = modelBHandle.remote(request);
return (String) refA.get() + refB.get();
DeploymentResponse responseA = modelAHandle.remote(request);
DeploymentResponse responseB = modelBHandle.remote(request);
return (String) responseA.result() + responseB.result();
}

public static class ModelA {
Expand All @@ -119,123 +116,22 @@ public class Driver {
}

```
In this example, modelA and modelB are defined as two Deployments, and driver is instantiated with the corresponding `RayServeHandle` of these two Deployments. When `call` is executed, both models are invoked. Additionally, more complex graphs can be composed, for example:
```python
def preprocess(inp: int) -> int:
return inp + 1

```

```java
// Demo 3
import io.ray.serve.api.Serve;
import io.ray.serve.deployment.Application;
import io.ray.serve.deployment.DAGDriver;
import io.ray.serve.deployment.InputNode;
import io.ray.serve.generated.DeploymentLanguage;
import io.ray.serve.handle.RayServeHandle;
import java.util.concurrent.atomic.AtomicInteger;
import org.testng.Assert;

public class Model {
private AtomicInteger increment;

public Model(int increment) {
this.increment = new AtomicInteger(increment);
}

public int predict(int inp) {
return inp + increment.get();
}

public static void main(String[] args) throws Exception {
try (InputNode inp = new InputNode()) {
Application model =
Serve.deployment()
.setDeploymentDef(Model.class.getName())
.bind(2);
Application pyPreprocess =
Serve.deployment()
.setDeploymentDef("deployment_graph.preprocess")
.setLanguage(DeploymentLanguage.PYTHON)
.bind(inp);
Application output = model.method("predict").bind(pyPreprocess);
Application serveDag = DAGDriver.bind(output);

RayServeHandle handle = Serve.run(serveDag);
Assert.assertEquals(handle.method("predict").remote(1).get(), 4);
}
}
}

```
In this case, two deployments are defined:
* model: a Java Deployment where the `predict` method takes an integer input and performs addition with the initialized value.
* pyPreprocess: a Python deployment that adds one to the input parameter.

During the graph composition, the output of pyPreprocess will be used as input to the `model.predict` method. `DAGDriver` acts as the Ingress Deployment and orchestrates the entire graph. Finally, the graph is deployed using `Serve.run`.

One more thing to note is the usage of `InputNode`. In Python, `InputNode` is very flexible and can represent a list, a dict, or structured object. However, in Java, it is difficult to simulate the invocation of arbitrary objects using `InputNode`, so we have made some compromises. We can simulate the invocation of a List or a Map using the `InputNode.get` method. As for structured objects, the only option is to pass the entire `InputNode` as a parameter. Here's an example:
```java
// Demo 4
import io.ray.serve.api.Serve;
import io.ray.serve.deployment.Application;
import io.ray.serve.deployment.DAGDriver;
import io.ray.serve.deployment.InputNode;
import io.ray.serve.generated.DeploymentLanguage;
import io.ray.serve.handle.RayServeHandle;

public class Model {

private int weight;

public Model() {}

public Model(int weight) {
this.weight = weight;
}
In this example, the modelA and modelB are defined as two Deployments, and the driver is instantiated with the corresponding `DeploymentHandle` of these two Deployments. When `call` is executed, both models are invoked. Additionally, it is evident that `DeploymentHandle.remote` returns `DeploymentResponse` instead of `ObjectRef`. The result can be accessed through `DeploymentResponse.result`.

public int forward(int input) {
return weight + input;
}

public static void main(String[] args) throws Exception {
Application m1 = Serve.deployment().setDeploymentDef(Model.class.getName()).bind(1);
Application m2 = Serve.deployment().setDeploymentDef(Model.class.getName()).bind(2);

try (InputNode userInput = InputNode.create()) {
Application m1Output = m1.method("forward").bind(userInput.get(0));
Application m2Output = m2.method("forward").bind(userInput.get(1));
Application combineOutput =
Serve.deployment()
.setDeploymentDef("deployment_graph.combine")
.setLanguage(DeploymentLanguage.PYTHON)
.bind(m1Output, m2Output, userInput.get(2));
### Deploying Deployments of the Other Languages through Python API
In another REP ([Add Cpp Deployment in Ray Serve](https://github.com/ray-project/enhancements/pull/34)), it is mentioned how to deploy C++ deployments through Python. Deploying Java deployments through Python is similar. Since Java and C++ do not have the decorator mechanism like Python, a straightforward way is to directly use the `serve.deployment` API (with the addition of a new `language` parameter):

Application graph = DAGDriver.bind(combineOutput);
RayServeHandle handle = Serve.run(graph);
}
}
}
```python
deployment = serve.deployment('io.ray.serve.ExampleDeployment', name='my_deployment', language=JAVA)

```
### Cross-language transmission of DAGDriver
In the above examples, both Demo 1 and Demo 2 are deployments of regular Java applications, which are relatively easy to implement. However, for Demo 3 and Demo 4, they involve Python `DAGDriver`, where the input of `DAGDriver` may contain `RayServeDAGHandle` that carries information for graph execution. In order to fully support graph execution orchestration through Python `DAGDriver`, it would require support for cross-language transmission of several internal core types, such as `RayServeDAGHandle`, `DeploymentMethodExecutorNode`, `DeploymentFunctionExecutorNode`, `InputAttributeNode`, and `InputNode`. This could be a significant change and needs further evaluation.

### Deploying deployments of other languages through Python API
In another REP ([Add Cpp Deployment in Ray Serve](https://github.com/ray-project/enhancements/pull/34)), it is mentioned how to deploy C++ deployments through Python. Deploying Java deployments through Python is similar. Since Java and C++ do not have the decorator mechanism like Python, a straightforward way is to directly instantiate the corresponding Deployment object:
### Deploying through the Config File
// TODO

```python
deployment_config = DeploymentConfig()
deployment_config.deployment_language = JAVA # or CPP
### Serve Handle C++ Core

deployment = Deployment(_func_or_class='io.ray.serve.ExampleDeployment', name='my_deployment', config=config)
```
Alternatively, you can directly use the deployment API (with the addition of a new `language` parameter):
```python
deployment = serve.deployment(_func_or_class='io.ray.serve.ExampleDeployment', name='my_deployment', language=JAVA)
In the design of C++ Deployment, it also includes the C++ implementation of Serve Handle. After the implementation, it can be reused as the core of Serve Handle by other languages (Python and Java) to avoid maintaining duplicate logic in the three languages. For the complete design, we will continue to supplement it in the "[Cpp Deployment Design](https://github.com/ray-project/enhancements/pull/34)" or another new document.

```
## Compatibility, Deprecation, and Migration Plan
In Java, the old API will be marked with the @Deprecated annotation, for example:
```java
Expand Down Expand Up @@ -263,4 +159,4 @@ Related test cases will be provided under ray/java/serve, and they will cover th
## (Optional) Follow-on Work
- Modify the Ray Serve Java API to support the usage of DAGs.
- Optimize the code by removing unused components and improving cross-language parameter handling.
- Support the usage of streaming.
- Support the usage of streaming.

0 comments on commit eb2a07a

Please sign in to comment.