Skip to content

Commit

Permalink
Java deploy deployments through application.
Browse files Browse the repository at this point in the history
Signed-off-by: chuhan.ly <chuhan.ly@antgroup.com>
  • Loading branch information
chuhan.ly committed Oct 1, 2023
1 parent 957fa6a commit d322507
Show file tree
Hide file tree
Showing 26 changed files with 587 additions and 310 deletions.
64 changes: 55 additions & 9 deletions java/serve/src/main/java/io/ray/serve/api/Serve.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,20 @@
import io.ray.serve.deployment.DeploymentRoute;
import io.ray.serve.exception.RayServeException;
import io.ray.serve.generated.ActorNameList;
import io.ray.serve.generated.StatusOverview;
import io.ray.serve.handle.DeploymentHandle;
import io.ray.serve.poll.LongPollClientFactory;
import io.ray.serve.replica.ReplicaContext;
import io.ray.serve.util.CollectionUtil;
import io.ray.serve.util.MessageFormatter;
import io.ray.serve.util.ServeProtoUtil;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -305,6 +308,7 @@ public static Deployment getDeployment(String name) {
*
* @return
*/
@Deprecated
public static Map<String, Deployment> listDeployments() {
Map<String, DeploymentRoute> infos = getGlobalClient().listDeployments();
if (infos == null || infos.size() == 0) {
Expand Down Expand Up @@ -343,23 +347,65 @@ public static Optional<DeploymentHandle> run(

List<Deployment> deployments = Graph.build(target.getInternalDagNode(), name);
Deployment ingress = Graph.getAndValidateIngressDeployment(deployments);

for (Deployment deployment : deployments) {
client.deploy(
deployment.getName(),
deployment.getReplicaConfig(),
deployment.getDeploymentConfig(),
deployment.getVersion(),
routePrefix,
deployment.getUrl(),
blocking);
// Overwrite route prefix
if (StringUtils.isNotBlank(routePrefix)
&& StringUtils.isNotBlank(deployment.getRoutePrefix())) {
Preconditions.checkArgument(
routePrefix.startsWith("/"), "The route_prefix must start with a forward slash ('/')");
deployment.setRoutePrefix(routePrefix);
}
deployment
.getDeploymentConfig()
.setVersion(
StringUtils.isNotBlank(deployment.getVersion())
? deployment.getVersion()
: RandomStringUtils.randomAlphabetic(6));
}

client.deployApplication(name, deployments, blocking);

return Optional.ofNullable(ingress)
.map(ingressDeployment -> client.getHandle(ingressDeployment.getName(), true));
.map(ingressDeployment -> client.getHandle(ingressDeployment.getName(), name, true));
}

private static void init() {
System.setProperty("ray.job.namespace", Constants.SERVE_NAMESPACE);
Ray.init();
}

public static DeploymentHandle getAppHandle(String name) {
ServeControllerClient client = getGlobalClient();
String ingress =
(String)
((PyActorHandle) client.getController())
.task(PyActorMethod.of("get_ingress_deployment_name"), name)
.remote()
.get();

if (StringUtils.isBlank(ingress)) {
throw new RayServeException(
MessageFormatter.format("Application '{}' does not exist.", ingress));
}
return client.getHandle(ingress, name, false);
}

public static void delete(String name) {
delete(name, true);
}

/**
* Delete an application by its name. Deletes the app with all corresponding deployments.
*
* @param name
* @param blocking
*/
public static void delete(String name, boolean blocking) {
getGlobalClient().deleteApps(Arrays.asList(name), blocking);
}

public static StatusOverview status(String name) {
return getGlobalClient().getServeStatus(name);
}
}
Loading

0 comments on commit d322507

Please sign in to comment.