From 254e64f161b042e41a37a06bf6869de03730534d Mon Sep 17 00:00:00 2001 From: Basasuya Date: Tue, 26 Jul 2022 21:12:46 +0800 Subject: [PATCH 1/7] [api server] enable job spec server --- apiserver/cmd/main.go | 2 + apiserver/pkg/client/job.go | 35 + apiserver/pkg/manager/client_manager.go | 7 + apiserver/pkg/manager/resource_manager.go | 132 ++- apiserver/pkg/model/converter.go | 32 +- apiserver/pkg/server/job_server.go | 99 ++ apiserver/pkg/util/cluster.go | 87 +- apiserver/pkg/util/job.go | 63 ++ proto/go_client/job.pb.go | 807 ++++++++++++++ proto/go_client/job.pb.gw.go | 587 ++++++++++ proto/go_client/job_grpc.pb.go | 256 +++++ proto/hack/generate.sh | 10 + proto/job.proto | 127 +++ proto/kuberay_api.swagger.json | 1178 +++++++++++++++++++++ proto/swagger/job.swagger.json | 427 ++++++++ 15 files changed, 3799 insertions(+), 50 deletions(-) create mode 100644 apiserver/pkg/client/job.go create mode 100644 apiserver/pkg/server/job_server.go create mode 100644 apiserver/pkg/util/job.go create mode 100644 proto/go_client/job.pb.go create mode 100644 proto/go_client/job.pb.gw.go create mode 100644 proto/go_client/job_grpc.pb.go create mode 100644 proto/job.proto create mode 100644 proto/kuberay_api.swagger.json create mode 100644 proto/swagger/job.swagger.json diff --git a/apiserver/cmd/main.go b/apiserver/cmd/main.go index 168882b458d..2889553eab3 100644 --- a/apiserver/cmd/main.go +++ b/apiserver/cmd/main.go @@ -58,6 +58,7 @@ func startRpcServer(resourceManager *manager.ResourceManager) { grpc.MaxRecvMsgSize(math.MaxInt32)) api.RegisterClusterServiceServer(s, server.NewClusterServer(resourceManager, &server.ClusterServerOptions{CollectMetrics: *collectMetricsFlag})) api.RegisterComputeTemplateServiceServer(s, server.NewComputeTemplateServer(resourceManager, &server.ComputeTemplateServerOptions{CollectMetrics: *collectMetricsFlag})) + api.RegisterRayJobServiceServer(s, server.NewRayJobServer(resourceManager)) // Register reflection service on gRPC server. reflection.Register(s) @@ -92,6 +93,7 @@ func startHttpProxy() { ) registerHttpHandlerFromEndpoint(api.RegisterClusterServiceHandlerFromEndpoint, "ClusterService", ctx, runtimeMux) registerHttpHandlerFromEndpoint(api.RegisterComputeTemplateServiceHandlerFromEndpoint, "ComputeTemplateService", ctx, runtimeMux) + registerHttpHandlerFromEndpoint(api.RegisterRayJobServiceHandlerFromEndpoint, "JobService", ctx, runtimeMux) // Create a top level mux to include both Http gRPC servers and other endpoints like metrics topMux := http.NewServeMux() diff --git a/apiserver/pkg/client/job.go b/apiserver/pkg/client/job.go new file mode 100644 index 00000000000..2effe2d4e23 --- /dev/null +++ b/apiserver/pkg/client/job.go @@ -0,0 +1,35 @@ +package client + +import ( + "time" + + "github.com/golang/glog" + "github.com/ray-project/kuberay/apiserver/pkg/util" + rayclusterclient "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned" + rayiov1alpha1 "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/typed/ray/v1alpha1" + "sigs.k8s.io/controller-runtime/pkg/client/config" +) + +type JobClientInterface interface { + RayJobClient(namespace string) rayiov1alpha1.RayJobInterface +} + +type RayJobClient struct { + client rayiov1alpha1.RayV1alpha1Interface +} + +func (cc RayJobClient) RayJobClient(namespace string) rayiov1alpha1.RayJobInterface { + return cc.client.RayJobs(namespace) +} + +func NewRayJobClientOrFatal(initConnectionTimeout time.Duration, options util.ClientOptions) JobClientInterface { + cfg, err := config.GetConfig() + if err != nil { + glog.Fatalf("Failed to create RayCluster client. Error: %v", err) + } + cfg.QPS = options.QPS + cfg.Burst = options.Burst + + rayJobClient := rayclusterclient.NewForConfigOrDie(cfg).RayV1alpha1() + return &RayJobClient{client: rayJobClient} +} diff --git a/apiserver/pkg/manager/client_manager.go b/apiserver/pkg/manager/client_manager.go index e0f8edcbd25..c4ed661246f 100644 --- a/apiserver/pkg/manager/client_manager.go +++ b/apiserver/pkg/manager/client_manager.go @@ -10,6 +10,7 @@ import ( type ClientManagerInterface interface { ClusterClient() client.ClusterClientInterface + RayJobClient() client.JobClientInterface KubernetesClient() client.KubernetesClientInterface Time() util.TimeInterface } @@ -18,6 +19,7 @@ type ClientManagerInterface interface { type ClientManager struct { // Kubernetes clients clusterClient client.ClusterClientInterface + jobClient client.JobClientInterface kubernetesClient client.KubernetesClientInterface // auxiliary tools time util.TimeInterface @@ -27,6 +29,10 @@ func (c *ClientManager) ClusterClient() client.ClusterClientInterface { return c.clusterClient } +func (c *ClientManager) RayJobClient() client.JobClientInterface { + return c.jobClient +} + func (c *ClientManager) KubernetesClient() client.KubernetesClientInterface { return c.kubernetesClient } @@ -52,6 +58,7 @@ func (c *ClientManager) init() { // TODO: Potentially, we may need storage layer clients to help persist the data. // 2. kubernetes client initialization c.clusterClient = client.NewRayClusterClientOrFatal(initConnectionTimeout, defaultKubernetesClientConfig) + c.jobClient = client.NewRayJobClientOrFatal(initConnectionTimeout, defaultKubernetesClientConfig) c.kubernetesClient = client.CreateKubernetesCoreOrFatal(initConnectionTimeout, defaultKubernetesClientConfig) klog.Infof("Client manager initialized successfully") diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 8d15dcafbde..0a813e6eac1 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -30,6 +30,10 @@ type ResourceManagerInterface interface { GetComputeTemplate(ctx context.Context, name string, namespace string) (*v1.ConfigMap, error) ListComputeTemplates(ctx context.Context, namespace string) ([]*v1.ConfigMap, error) DeleteComputeTemplate(ctx context.Context, name string, namespace string) error + CreateJob(ctx context.Context, apiCluster *api.Cluster) (*v1alpha1.RayJob, error) + GetJob(ctx context.Context, jobName string, namespace string) (*v1alpha1.RayJob, error) + ListJobs(ctx context.Context, namespace string) ([]*v1alpha1.RayJob, error) + ListAllJobs(ctx context.Context) ([]*v1alpha1.RayJob, error) } type ResourceManager struct { @@ -48,6 +52,10 @@ func (r *ResourceManager) getRayClusterClient(namespace string) rayiov1alpha1.Ra return r.clientManager.ClusterClient().RayClusterClient(namespace) } +func (r *ResourceManager) getRayJobClient(namespace string) rayiov1alpha1.RayJobInterface { + return r.clientManager.RayJobClient().RayJobClient(namespace) +} + func (r *ResourceManager) getKubernetesConfigMapClient(namespace string) clientv1.ConfigMapInterface { return r.clientManager.KubernetesClient().ConfigMapClient(namespace) } @@ -63,7 +71,7 @@ func (r *ResourceManager) getKubernetesNamespaceClient() clientv1.NamespaceInter // clusters func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Cluster) (*v1alpha1.RayCluster, error) { // populate cluster map - computeTemplateDict, err := r.populateComputeTemplate(ctx, apiCluster) + computeTemplateDict, err := r.populateComputeTemplate(ctx, apiCluster.ClusterSpec, apiCluster.Namespace) if err != nil { return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiCluster.Namespace, apiCluster.Name) } @@ -83,11 +91,11 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu return newRayCluster, nil } -func (r *ResourceManager) populateComputeTemplate(ctx context.Context, cluster *api.Cluster) (map[string]*api.ComputeTemplate, error) { +func (r *ResourceManager) populateComputeTemplate(ctx context.Context, clusterSpec *api.ClusterSpec, nameSpace string) (map[string]*api.ComputeTemplate, error) { dict := map[string]*api.ComputeTemplate{} // populate head compute template - name := cluster.ClusterSpec.HeadGroupSpec.ComputeTemplate - configMap, err := r.GetComputeTemplate(ctx, name, cluster.Namespace) + name := clusterSpec.HeadGroupSpec.ComputeTemplate + configMap, err := r.GetComputeTemplate(ctx, name, nameSpace) if err != nil { return nil, err } @@ -95,10 +103,10 @@ func (r *ResourceManager) populateComputeTemplate(ctx context.Context, cluster * dict[name] = computeTemplate // populate worker compute template - for _, spec := range cluster.ClusterSpec.WorkerGroupSpec { + for _, spec := range clusterSpec.WorkerGroupSpec { name := spec.ComputeTemplate if _, exist := dict[name]; !exist { - configMap, err := r.GetComputeTemplate(ctx, name, cluster.Namespace) + configMap, err := r.GetComputeTemplate(ctx, name, nameSpace) if err != nil { return nil, err } @@ -181,6 +189,101 @@ func (r *ResourceManager) DeleteCluster(ctx context.Context, clusterName string, return nil } +func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob, jobNamespace string) (*v1alpha1.RayJob, error) { + computeTemplateMap := make(map[string]*api.ComputeTemplate) + var err error + + if apiJob.ClusterSpec != nil && len(apiJob.ClusterSelector) == 0 { + // populate cluster map + computeTemplateMap, err = r.populateComputeTemplate(ctx, apiJob.ClusterSpec, jobNamespace) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", jobNamespace, apiJob.JobId) + } + } + + // convert *api.Cluster to v1alpha1.RayCluster + rayJob := util.NewRayJob(apiJob, computeTemplateMap) + + + newRayJob, err := r.getRayJobClient(jobNamespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to create a job for (%s/%s)", jobNamespace, apiJob.JobId) + } + + return newRayJob, nil +} + +func (r *ResourceManager) GetJob(ctx context.Context, jobName string, namespace string) (*v1alpha1.RayJob, error) { + client := r.getRayJobClient(namespace) + return getJobByName(ctx, client, jobName) +} + +func (r *ResourceManager) ListJobs(ctx context.Context, namespace string) ([]*v1alpha1.RayJob, error) { + labelSelector := metav1.LabelSelector{ + MatchLabels: map[string]string{ + util.KubernetesManagedByLabelKey: util.ComponentName, + }, + } + rayJobList, err := r.getRayJobClient(namespace).List(ctx, metav1.ListOptions{ + LabelSelector: labels.Set(labelSelector.MatchLabels).String(), + }) + if err != nil { + return nil, util.Wrap(err, fmt.Sprintf("List RayCluster failed in %s", namespace)) + } + + var result []*v1alpha1.RayJob + length := len(rayJobList.Items) + for i := 0; i < length; i++ { + result = append(result, &rayJobList.Items[i]) + } + + return result, nil +} + +func (r *ResourceManager) ListAllJobs(ctx context.Context) ([]*v1alpha1.RayJob, error) { + namespaces, err := r.getKubernetesNamespaceClient().List(ctx, metav1.ListOptions{}) + if err != nil { + return nil, util.Wrap(err, "Failed to fetch all Kubernetes namespaces") + } + + var result []*v1alpha1.RayJob + labelSelector := metav1.LabelSelector{ + MatchLabels: map[string]string{ + util.KubernetesManagedByLabelKey: util.ComponentName, + }, + } + for _, namespace := range namespaces.Items { + rayJobList, err := r.getRayJobClient(namespace.Name).List(ctx, metav1.ListOptions{ + LabelSelector: labels.Set(labelSelector.MatchLabels).String(), + }) + if err != nil { + return nil, util.Wrap(err, fmt.Sprintf("List RayCluster failed in %s", namespace.Name)) + } + + length := len(rayJobList.Items) + for i := 0; i < length; i++ { + result = append(result, &rayJobList.Items[i]) + } + } + return result, nil +} + +func (r *ResourceManager) DeleteJob(ctx context.Context, jobName string, namespace string) error { + client := r.getRayJobClient(namespace) + job, err := getJobByName(ctx, client, jobName) + if err != nil { + return util.Wrap(err, "Get job failure") + } + + // Delete Kubernetes resources + if err := client.Delete(ctx, job.Name, metav1.DeleteOptions{}); err != nil { + // API won't need to delete the ray cluster CR + return util.NewInternalServerError(err, "Failed to delete cluster %v.", jobName) + } + + return nil +} + // Compute Runtimes func (r *ResourceManager) CreateComputeTemplate(ctx context.Context, runtime *api.ComputeTemplate) (*v1.ConfigMap, error) { _, err := r.GetComputeTemplate(ctx, runtime.Name, runtime.Namespace) @@ -277,6 +380,23 @@ func getClusterByName(ctx context.Context, client rayiov1alpha1.RayClusterInterf return cluster, nil } +// getJobByName returns the Kubernetes RayJob object by given name and client +func getJobByName(ctx context.Context, client rayiov1alpha1.RayJobInterface, name string) (*v1alpha1.RayJob, error) { + job, err := client.Get(ctx, name, metav1.GetOptions{}) + if err != nil { + if errors.IsNotFound(err) { + return nil, util.NewNotFoundError(err, "Job %s not found", name) + } + + return nil, util.Wrap(err, "Get Job failed") + } + if managedBy, ok := job.Labels[util.KubernetesManagedByLabelKey]; !ok || managedBy != util.ComponentName { + return nil, fmt.Errorf("RayCluster with name %s not managed by %s", name, util.ComponentName) + } + + return job, nil +} + // getComputeTemplateByName returns the Kubernetes configmap object by given name and client func getComputeTemplateByName(ctx context.Context, client clientv1.ConfigMapInterface, name string) (*v1.ConfigMap, error) { runtime, err := client.Get(ctx, name, metav1.GetOptions{}) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 907589e475c..88ed53b00b4 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -31,9 +31,7 @@ func FromCrdToApiCluster(cluster *v1alpha1.RayCluster, events []v1.Event) *api.C } // loop container and find the resource - pbCluster.ClusterSpec = &api.ClusterSpec{} - pbCluster.ClusterSpec.HeadGroupSpec = PopulateHeadNodeSpec(cluster.Spec.HeadGroupSpec) - pbCluster.ClusterSpec.WorkerGroupSpec = PopulateWorkerNodeSpec(cluster.Spec.WorkerGroupSpecs) + pbCluster.ClusterSpec = PopulateRayClusterSpec(cluster.Spec) // parse events for _, event := range events { @@ -58,6 +56,13 @@ func FromCrdToApiCluster(cluster *v1alpha1.RayCluster, events []v1.Event) *api.C return pbCluster } +func PopulateRayClusterSpec(spec v1alpha1.RayClusterSpec) *api.ClusterSpec { + clusterSpec := &api.ClusterSpec{} + clusterSpec.HeadGroupSpec = PopulateHeadNodeSpec(spec.HeadGroupSpec) + clusterSpec.WorkerGroupSpec = PopulateWorkerNodeSpec(spec.WorkerGroupSpecs) + return clusterSpec +} + func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { headNodeSpec := &api.HeadGroupSpec{ RayStartParams: spec.RayStartParams, @@ -111,3 +116,24 @@ func FromKubeToAPIComputeTemplates(configMaps []*v1.ConfigMap) []*api.ComputeTem } return apiComputeTemplates } + +func FromCrdToApiJobs(jobs []*v1alpha1.RayJob) []*api.RayJob { + apiJobs := make([]*api.RayJob, 0) + for _, job := range jobs { + apiJobs = append(apiJobs, FromCrdToApiJob(job)) + } + return apiJobs +} + +func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { + pbJob := &api.RayJob{ + Entrypoint: job.Spec.Entrypoint, + Metadata: job.Spec.Metadata, + RuntimeEnv: job.Spec.RuntimeEnv, + JobId: job.Spec.JobId, + ShutdownAfterJobFinishes: job.Spec.ShutdownAfterJobFinishes, + ClusterSelector: job.Spec.ClusterSelector, + ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), + } + return pbJob +} \ No newline at end of file diff --git a/apiserver/pkg/server/job_server.go b/apiserver/pkg/server/job_server.go new file mode 100644 index 00000000000..44ecb0f5b60 --- /dev/null +++ b/apiserver/pkg/server/job_server.go @@ -0,0 +1,99 @@ +package server + +import ( + "context" + "github.com/ray-project/kuberay/apiserver/pkg/manager" + "github.com/ray-project/kuberay/apiserver/pkg/model" + "github.com/ray-project/kuberay/apiserver/pkg/util" + api "github.com/ray-project/kuberay/proto/go_client" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/types/known/emptypb" +) + +// implements `type RayJobServiceServer interface` in job_grpc.pb.go +// RayJobServer is the server API for RayJobServer service. + +func NewRayJobServer(resourceManager *manager.ResourceManager) *RayJobServer { + return &RayJobServer{resourceManager: resourceManager} +} + + +// TODO(basasuya) add event outputs +type RayJobServer struct { + resourceManager *manager.ResourceManager + api.UnimplementedRayJobServiceServer +} + + +func (s *RayJobServer) CreateRayJob(ctx context.Context, request *api.CreateRayJobRequest) (*api.RayJob, error) { + // use the namespace in the request to override the namespace in the job definition + request.Job.Namespace = request.Namespace + + job, err := s.resourceManager.CreateJob(ctx, request.Job, request.Namespace) + if err != nil { + return nil, util.Wrap(err, "Create Job failed.") + } + + return model.FromCrdToApiJob(job), nil +} + +func (s *RayJobServer) GetRayJob(ctx context.Context, request *api.GetRayJobRequest) (*api.RayJob, error) { + if request.Name == "" { + return nil, util.NewInvalidInputError("job name is empty. Please specify a valid value.") + } + + if request.Namespace == "" { + return nil, util.NewInvalidInputError("job namespace is empty. Please specify a valid value.") + } + + job, err := s.resourceManager.GetJob(ctx, request.Name, request.Namespace) + if err != nil { + return nil, util.Wrap(err, "Get cluster failed.") + } + + return model.FromCrdToApiJob(job), nil + return nil, status.Errorf(codes.Unimplemented, "method GetRayJob not implemented") +} + +func (s *RayJobServer) ListRayJobs(ctx context.Context, request *api.ListRayJobsRequest) (*api.ListRayJobsResponse, error) { + if request.Namespace == "" { + return nil, util.NewInvalidInputError("job namespace is empty. Please specify a valid value.") + } + + jobs, err := s.resourceManager.ListJobs(ctx, request.Namespace) + if err != nil { + return nil, util.Wrap(err, "List jobs failed.") + } + + return &api.ListRayJobsResponse{ + Jobs: model.FromCrdToApiJobs(jobs), + }, nil +} + +func (s *RayJobServer) ListAllRayJobs(ctx context.Context, request *api.ListAllRayJobsRequest) (*api.ListAllRayJobsResponse, error) { + jobs, err := s.resourceManager.ListAllJobs(ctx) + if err != nil { + return nil, util.Wrap(err, "List jobs failed.") + } + + return &api.ListAllRayJobsResponse{ + Jobs: model.FromCrdToApiJobs(jobs), + }, nil +} + +func (s *RayJobServer) DeleteRayJob(ctx context.Context, request *api.DeleteRayJobRequest) (*emptypb.Empty, error) { + if request.Name == "" { + return nil, util.NewInvalidInputError("job name is empty. Please specify a valid value.") + } + + if request.Namespace == "" { + return nil, util.NewInvalidInputError("job namespace is empty. Please specify a valid value.") + } + + if err := s.resourceManager.DeleteJob(ctx, request.Name, request.Namespace); err != nil { + return nil, err + } + + return &emptypb.Empty{}, nil +} diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index dbbd5869c6a..c8ac18f011e 100644 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -19,10 +19,6 @@ type RayCluster struct { // NewRayCluster creates a RayCluster. // func NewRayCluster(apiCluster *api.Cluster, clusterRuntime *api.ClusterRuntime, computeRuntime *api.ComputeRuntime) *RayCluster { func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) *RayCluster { - // figure out how to build this - computeTemplate := computeTemplateMap[apiCluster.ClusterSpec.HeadGroupSpec.ComputeTemplate] - headPodTemplate := buildHeadPodTemplate(apiCluster, apiCluster.ClusterSpec.HeadGroupSpec, computeTemplate) - headReplicas := int32(1) rayCluster := &rayclusterapi.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Name: apiCluster.Name, @@ -30,21 +26,47 @@ func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.C Labels: buildRayClusterLabels(apiCluster), Annotations: buildRayClusterAnnotations(apiCluster), }, - Spec: rayclusterapi.RayClusterSpec{ - RayVersion: apiCluster.Version, - HeadGroupSpec: rayclusterapi.HeadGroupSpec{ - ServiceType: v1.ServiceType(apiCluster.ClusterSpec.HeadGroupSpec.ServiceType), - Template: headPodTemplate, - Replicas: &headReplicas, - RayStartParams: apiCluster.ClusterSpec.HeadGroupSpec.RayStartParams, - }, - WorkerGroupSpecs: []rayclusterapi.WorkerGroupSpec{}, + Spec: *buildRayClusterSpec(apiCluster.Version, apiCluster.ClusterSpec, computeTemplateMap), + } + + return &RayCluster{rayCluster} +} + +func buildRayClusterLabels(cluster *api.Cluster) map[string]string { + labels := map[string]string{} + labels[RayClusterNameLabelKey] = cluster.Name + labels[RayClusterUserLabelKey] = cluster.User + labels[RayClusterVersionLabelKey] = cluster.Version + labels[RayClusterEnvironmentLabelKey] = cluster.Environment.String() + labels[KubernetesApplicationNameLabelKey] = ApplicationName + labels[KubernetesManagedByLabelKey] = ComponentName + return labels +} + +func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { + annotations := map[string]string{} + // TODO: Add optional annotations + return annotations +} + +func buildRayClusterSpec(imageVersion string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) *rayclusterapi.RayClusterSpec { + computeTemplate := computeTemplateMap[clusterSpec.HeadGroupSpec.ComputeTemplate] + headPodTemplate := buildHeadPodTemplate(imageVersion, clusterSpec.HeadGroupSpec, computeTemplate) + headReplicas := int32(1) + rayClusterSpec := &rayclusterapi.RayClusterSpec{ + RayVersion: imageVersion, + HeadGroupSpec: rayclusterapi.HeadGroupSpec{ + ServiceType: v1.ServiceType(clusterSpec.HeadGroupSpec.ServiceType), + Template: headPodTemplate, + Replicas: &headReplicas, + RayStartParams: clusterSpec.HeadGroupSpec.RayStartParams, }, + WorkerGroupSpecs: []rayclusterapi.WorkerGroupSpec{}, } - for _, spec := range apiCluster.ClusterSpec.WorkerGroupSpec { - computeTemplate := computeTemplateMap[spec.ComputeTemplate] - workerPodTemplate := buildWorkerPodTemplate(apiCluster, spec, computeTemplate) + for _, spec := range clusterSpec.WorkerGroupSpec { + computeTemplate = computeTemplateMap[spec.ComputeTemplate] + workerPodTemplate := buildWorkerPodTemplate(imageVersion, spec, computeTemplate) minReplicas := spec.Replicas maxReplicas := spec.Replicas @@ -64,27 +86,10 @@ func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.C Template: workerPodTemplate, } - rayCluster.Spec.WorkerGroupSpecs = append(rayCluster.Spec.WorkerGroupSpecs, workerNodeSpec) + rayClusterSpec.WorkerGroupSpecs = append(rayClusterSpec.WorkerGroupSpecs, workerNodeSpec) } - return &RayCluster{rayCluster} -} - -func buildRayClusterLabels(cluster *api.Cluster) map[string]string { - labels := map[string]string{} - labels[RayClusterNameLabelKey] = cluster.Name - labels[RayClusterUserLabelKey] = cluster.User - labels[RayClusterVersionLabelKey] = cluster.Version - labels[RayClusterEnvironmentLabelKey] = cluster.Environment.String() - labels[KubernetesApplicationNameLabelKey] = ApplicationName - labels[KubernetesManagedByLabelKey] = ComponentName - return labels -} - -func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { - annotations := map[string]string{} - // TODO: Add optional annotations - return annotations + return rayClusterSpec } func buildNodeGroupAnnotations(computeTemplate *api.ComputeTemplate, image string) map[string]string { @@ -94,10 +99,10 @@ func buildNodeGroupAnnotations(computeTemplate *api.ComputeTemplate, image strin return annotations } -func buildHeadPodTemplate(cluster *api.Cluster, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) v1.PodTemplateSpec { - image := constructRayImage(RayClusterDefaultImageRepository, cluster.Version) - if len(cluster.ClusterSpec.HeadGroupSpec.Image) != 0 { - image = cluster.ClusterSpec.HeadGroupSpec.Image +func buildHeadPodTemplate(imageVersion string, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) v1.PodTemplateSpec { + image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) + if len(spec.Image) != 0 { + image = spec.Image } // calculate resources @@ -182,10 +187,10 @@ func constructRayImage(containerImage string, version string) string { return fmt.Sprintf("%s:%s", containerImage, version) } -func buildWorkerPodTemplate(cluster *api.Cluster, spec *api.WorkerGroupSpec, computeRuntime *api.ComputeTemplate) v1.PodTemplateSpec { +func buildWorkerPodTemplate(imageVersion string, spec *api.WorkerGroupSpec, computeRuntime *api.ComputeTemplate) v1.PodTemplateSpec { // If user doesn't provide the image, let's use the default image instead. // TODO: verify the versions in the range - image := constructRayImage(RayClusterDefaultImageRepository, cluster.Version) + image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) if len(spec.Image) != 0 { image = spec.Image } diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go new file mode 100644 index 00000000000..3089427994b --- /dev/null +++ b/apiserver/pkg/util/job.go @@ -0,0 +1,63 @@ +package util + +import ( + api "github.com/ray-project/kuberay/proto/go_client" + rayalphaapi "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +type RayJob struct { + *rayalphaapi.RayJob +} + +const rayJobDefaultVersion = "1.13" + +// NewRayJob creates a RayJob. +func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTemplate) *RayJob { + var clusterSpec rayalphaapi.RayClusterSpec + + if apiJob.ClusterSpec != nil && len(apiJob.ClusterSelector) == 0 { + clusterSpec = *buildRayClusterSpec(rayJobDefaultVersion, apiJob.ClusterSpec, computeTemplateMap) + } + + rayJob := &rayalphaapi.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + Name: apiJob.Name, + Namespace: apiJob.Namespace, + Labels: buildRayJobLabels(apiJob), + Annotations: buildRayJobAnnotations(apiJob), + }, + Spec: rayalphaapi.RayJobSpec{ + Entrypoint: apiJob.Entrypoint, + Metadata: apiJob.Metadata, + RuntimeEnv: apiJob.RuntimeEnv, + ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, + JobId: apiJob.JobId, + RayClusterSpec: clusterSpec, + ClusterSelector: apiJob.ClusterSelector, + }, + Status: rayalphaapi.RayJobStatus{}, + } + + return &RayJob{ + rayJob, + } +} + +func (j *RayJob) Get() *rayalphaapi.RayJob { + return j.RayJob +} + +func buildRayJobLabels(job *api.RayJob) map[string]string { + labels := map[string]string{} + labels[RayClusterNameLabelKey] = job.Name + labels[RayClusterUserLabelKey] = job.User + labels[KubernetesApplicationNameLabelKey] = ApplicationName + labels[KubernetesManagedByLabelKey] = ComponentName + return labels +} + +func buildRayJobAnnotations(cluster *api.RayJob) map[string]string { + annotations := map[string]string{} + return annotations +} \ No newline at end of file diff --git a/proto/go_client/job.pb.go b/proto/go_client/job.pb.go new file mode 100644 index 00000000000..b00e33dff6f --- /dev/null +++ b/proto/go_client/job.pb.go @@ -0,0 +1,807 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.26.0 +// protoc v3.17.3 +// source: job.proto + +package go_client + +import ( + _ "github.com/grpc-ecosystem/grpc-gateway/v2/protoc-gen-openapiv2/options" + _ "google.golang.org/genproto/googleapis/api/annotations" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + emptypb "google.golang.org/protobuf/types/known/emptypb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type CreateRayJobRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The compute template to be created. + Job *RayJob `protobuf:"bytes,1,opt,name=job,proto3" json:"job,omitempty"` + // The namespace of the compute template to be created + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *CreateRayJobRequest) Reset() { + *x = CreateRayJobRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateRayJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateRayJobRequest) ProtoMessage() {} + +func (x *CreateRayJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CreateRayJobRequest.ProtoReflect.Descriptor instead. +func (*CreateRayJobRequest) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{0} +} + +func (x *CreateRayJobRequest) GetJob() *RayJob { + if x != nil { + return x.Job + } + return nil +} + +func (x *CreateRayJobRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type GetRayJobRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The name of the RayJob to be retrieved. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the compute template to be retrieved. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *GetRayJobRequest) Reset() { + *x = GetRayJobRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetRayJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetRayJobRequest) ProtoMessage() {} + +func (x *GetRayJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetRayJobRequest.ProtoReflect.Descriptor instead. +func (*GetRayJobRequest) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{1} +} + +func (x *GetRayJobRequest) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *GetRayJobRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type ListRayJobsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The namespace of the compute templates to be retrieved. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` // TODO: support paganation later +} + +func (x *ListRayJobsRequest) Reset() { + *x = ListRayJobsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListRayJobsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListRayJobsRequest) ProtoMessage() {} + +func (x *ListRayJobsRequest) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListRayJobsRequest.ProtoReflect.Descriptor instead. +func (*ListRayJobsRequest) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{2} +} + +func (x *ListRayJobsRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +type ListRayJobsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Jobs []*RayJob `protobuf:"bytes,1,rep,name=jobs,proto3" json:"jobs,omitempty"` +} + +func (x *ListRayJobsResponse) Reset() { + *x = ListRayJobsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListRayJobsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListRayJobsResponse) ProtoMessage() {} + +func (x *ListRayJobsResponse) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListRayJobsResponse.ProtoReflect.Descriptor instead. +func (*ListRayJobsResponse) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{3} +} + +func (x *ListRayJobsResponse) GetJobs() []*RayJob { + if x != nil { + return x.Jobs + } + return nil +} + +type ListAllRayJobsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ListAllRayJobsRequest) Reset() { + *x = ListAllRayJobsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllRayJobsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllRayJobsRequest) ProtoMessage() {} + +func (x *ListAllRayJobsRequest) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListAllRayJobsRequest.ProtoReflect.Descriptor instead. +func (*ListAllRayJobsRequest) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{4} +} + +type ListAllRayJobsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Jobs []*RayJob `protobuf:"bytes,1,rep,name=jobs,proto3" json:"jobs,omitempty"` +} + +func (x *ListAllRayJobsResponse) Reset() { + *x = ListAllRayJobsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllRayJobsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllRayJobsResponse) ProtoMessage() {} + +func (x *ListAllRayJobsResponse) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListAllRayJobsResponse.ProtoReflect.Descriptor instead. +func (*ListAllRayJobsResponse) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{5} +} + +func (x *ListAllRayJobsResponse) GetJobs() []*RayJob { + if x != nil { + return x.Jobs + } + return nil +} + +type DeleteRayJobRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The name of the compute template to be deleted. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the compute template to be deleted. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` +} + +func (x *DeleteRayJobRequest) Reset() { + *x = DeleteRayJobRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeleteRayJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeleteRayJobRequest) ProtoMessage() {} + +func (x *DeleteRayJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeleteRayJobRequest.ProtoReflect.Descriptor instead. +func (*DeleteRayJobRequest) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{6} +} + +func (x *DeleteRayJobRequest) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *DeleteRayJobRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +// RayJob +type RayJob struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Required input field. Unique job name provided by user. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Required input field. job namespace provided by user + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` + // Required field. This field indicates the user who owns the job. + User string `protobuf:"bytes,3,opt,name=user,proto3" json:"user,omitempty"` + // The entrypoint of the RayJob + Entrypoint string `protobuf:"bytes,4,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` + // Metadata is data to store along with this job. + Metadata map[string]string `protobuf:"bytes,5,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // RuntimeEnv is base64 encoded string which stores runtime configuration + RuntimeEnv string `protobuf:"bytes,6,opt,name=runtime_env,json=runtimeEnv,proto3" json:"runtime_env,omitempty"` + // If jobId is not set, a new jobId will be auto-generated. + JobId string `protobuf:"bytes,7,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + // If set to true, the rayCluster will be deleted after the rayJob finishes + ShutdownAfterJobFinishes bool `protobuf:"varint,8,opt,name=ShutdownAfterJobFinishes,proto3" json:"ShutdownAfterJobFinishes,omitempty"` + // The label selector to choose exiting clusters + ClusterSelector map[string]string `protobuf:"bytes,9,rep,name=cluster_selector,json=clusterSelector,proto3" json:"cluster_selector,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // The cluster template + ClusterSpec *ClusterSpec `protobuf:"bytes,10,opt,name=cluster_spec,json=clusterSpec,proto3" json:"cluster_spec,omitempty"` + // logs + Logs string `protobuf:"bytes,11,opt,name=logs,proto3" json:"logs,omitempty"` +} + +func (x *RayJob) Reset() { + *x = RayJob{} + if protoimpl.UnsafeEnabled { + mi := &file_job_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RayJob) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RayJob) ProtoMessage() {} + +func (x *RayJob) ProtoReflect() protoreflect.Message { + mi := &file_job_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RayJob.ProtoReflect.Descriptor instead. +func (*RayJob) Descriptor() ([]byte, []int) { + return file_job_proto_rawDescGZIP(), []int{7} +} + +func (x *RayJob) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *RayJob) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *RayJob) GetUser() string { + if x != nil { + return x.User + } + return "" +} + +func (x *RayJob) GetEntrypoint() string { + if x != nil { + return x.Entrypoint + } + return "" +} + +func (x *RayJob) GetMetadata() map[string]string { + if x != nil { + return x.Metadata + } + return nil +} + +func (x *RayJob) GetRuntimeEnv() string { + if x != nil { + return x.RuntimeEnv + } + return "" +} + +func (x *RayJob) GetJobId() string { + if x != nil { + return x.JobId + } + return "" +} + +func (x *RayJob) GetShutdownAfterJobFinishes() bool { + if x != nil { + return x.ShutdownAfterJobFinishes + } + return false +} + +func (x *RayJob) GetClusterSelector() map[string]string { + if x != nil { + return x.ClusterSelector + } + return nil +} + +func (x *RayJob) GetClusterSpec() *ClusterSpec { + if x != nil { + return x.ClusterSpec + } + return nil +} + +func (x *RayJob) GetLogs() string { + if x != nil { + return x.Logs + } + return "" +} + +var File_job_proto protoreflect.FileDescriptor + +var file_job_proto_rawDesc = []byte{ + 0x0a, 0x09, 0x6a, 0x6f, 0x62, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x05, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, + 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x6f, 0x70, 0x65, 0x6e, 0x61, 0x70, + 0x69, 0x76, 0x32, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0d, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x54, 0x0a, 0x13, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x03, 0x6a, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, + 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x22, 0x44, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x32, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, + 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x38, 0x0a, 0x13, + 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, + 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0x3b, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x47, 0x0a, 0x13, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xb6, 0x04, 0x0a, 0x06, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, + 0x1f, 0x0a, 0x0b, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x65, 0x6e, 0x76, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x45, 0x6e, 0x76, + 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x53, 0x68, 0x75, 0x74, 0x64, + 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, 0x73, + 0x68, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x53, 0x68, 0x75, 0x74, 0x64, + 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, 0x73, + 0x68, 0x65, 0x73, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x12, 0x35, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x70, + 0x65, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, 0x63, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x6f, 0x67, + 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x1a, 0x3b, 0x0a, + 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, + 0x04, 0x0a, 0x0d, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x12, 0x5b, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x20, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x1a, 0x22, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x32, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x6e, 0x0a, + 0x09, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, + 0x6f, 0x62, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, + 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x78, 0x0a, + 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x61, 0x70, + 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x6a, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x41, + 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, + 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, + 0x6f, 0x62, 0x73, 0x12, 0x7d, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, + 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x2a, + 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x7d, 0x42, 0x54, 0x5a, 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, + 0x2f, 0x72, 0x61, 0x79, 0x2d, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x6b, 0x75, 0x62, + 0x65, 0x72, 0x61, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x5f, 0x63, 0x6c, + 0x69, 0x65, 0x6e, 0x74, 0x92, 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, + 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_job_proto_rawDescOnce sync.Once + file_job_proto_rawDescData = file_job_proto_rawDesc +) + +func file_job_proto_rawDescGZIP() []byte { + file_job_proto_rawDescOnce.Do(func() { + file_job_proto_rawDescData = protoimpl.X.CompressGZIP(file_job_proto_rawDescData) + }) + return file_job_proto_rawDescData +} + +var file_job_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_job_proto_goTypes = []interface{}{ + (*CreateRayJobRequest)(nil), // 0: proto.CreateRayJobRequest + (*GetRayJobRequest)(nil), // 1: proto.GetRayJobRequest + (*ListRayJobsRequest)(nil), // 2: proto.ListRayJobsRequest + (*ListRayJobsResponse)(nil), // 3: proto.ListRayJobsResponse + (*ListAllRayJobsRequest)(nil), // 4: proto.ListAllRayJobsRequest + (*ListAllRayJobsResponse)(nil), // 5: proto.ListAllRayJobsResponse + (*DeleteRayJobRequest)(nil), // 6: proto.DeleteRayJobRequest + (*RayJob)(nil), // 7: proto.RayJob + nil, // 8: proto.RayJob.MetadataEntry + nil, // 9: proto.RayJob.ClusterSelectorEntry + (*ClusterSpec)(nil), // 10: proto.ClusterSpec + (*emptypb.Empty)(nil), // 11: google.protobuf.Empty +} +var file_job_proto_depIdxs = []int32{ + 7, // 0: proto.CreateRayJobRequest.job:type_name -> proto.RayJob + 7, // 1: proto.ListRayJobsResponse.jobs:type_name -> proto.RayJob + 7, // 2: proto.ListAllRayJobsResponse.jobs:type_name -> proto.RayJob + 8, // 3: proto.RayJob.metadata:type_name -> proto.RayJob.MetadataEntry + 9, // 4: proto.RayJob.cluster_selector:type_name -> proto.RayJob.ClusterSelectorEntry + 10, // 5: proto.RayJob.cluster_spec:type_name -> proto.ClusterSpec + 0, // 6: proto.RayJobService.CreateRayJob:input_type -> proto.CreateRayJobRequest + 1, // 7: proto.RayJobService.GetRayJob:input_type -> proto.GetRayJobRequest + 2, // 8: proto.RayJobService.ListRayJobs:input_type -> proto.ListRayJobsRequest + 4, // 9: proto.RayJobService.ListAllRayJobs:input_type -> proto.ListAllRayJobsRequest + 6, // 10: proto.RayJobService.DeleteRayJob:input_type -> proto.DeleteRayJobRequest + 7, // 11: proto.RayJobService.CreateRayJob:output_type -> proto.RayJob + 7, // 12: proto.RayJobService.GetRayJob:output_type -> proto.RayJob + 3, // 13: proto.RayJobService.ListRayJobs:output_type -> proto.ListRayJobsResponse + 5, // 14: proto.RayJobService.ListAllRayJobs:output_type -> proto.ListAllRayJobsResponse + 11, // 15: proto.RayJobService.DeleteRayJob:output_type -> google.protobuf.Empty + 11, // [11:16] is the sub-list for method output_type + 6, // [6:11] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_job_proto_init() } +func file_job_proto_init() { + if File_job_proto != nil { + return + } + file_cluster_proto_init() + if !protoimpl.UnsafeEnabled { + file_job_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateRayJobRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetRayJobRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListRayJobsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListRayJobsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListAllRayJobsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListAllRayJobsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteRayJobRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_job_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RayJob); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_job_proto_rawDesc, + NumEnums: 0, + NumMessages: 10, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_job_proto_goTypes, + DependencyIndexes: file_job_proto_depIdxs, + MessageInfos: file_job_proto_msgTypes, + }.Build() + File_job_proto = out.File + file_job_proto_rawDesc = nil + file_job_proto_goTypes = nil + file_job_proto_depIdxs = nil +} diff --git a/proto/go_client/job.pb.gw.go b/proto/go_client/job.pb.gw.go new file mode 100644 index 00000000000..ebf5e09e3c5 --- /dev/null +++ b/proto/go_client/job.pb.gw.go @@ -0,0 +1,587 @@ +// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT. +// source: job.proto + +/* +Package go_client is a reverse proxy. + +It translates gRPC into RESTful JSON APIs. +*/ +package go_client + +import ( + "context" + "io" + "net/http" + + "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" + "github.com/grpc-ecosystem/grpc-gateway/v2/utilities" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" +) + +// Suppress "imported and not used" errors +var _ codes.Code +var _ io.Reader +var _ status.Status +var _ = runtime.String +var _ = utilities.NewDoubleArray +var _ = metadata.Join + +var ( + filter_RayJobService_CreateRayJob_0 = &utilities.DoubleArray{Encoding: map[string]int{"job": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + +func request_RayJobService_CreateRayJob_0(ctx context.Context, marshaler runtime.Marshaler, client RayJobServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateRayJobRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq.Job); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_RayJobService_CreateRayJob_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.CreateRayJob(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_RayJobService_CreateRayJob_0(ctx context.Context, marshaler runtime.Marshaler, server RayJobServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq CreateRayJobRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq.Job); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_RayJobService_CreateRayJob_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.CreateRayJob(ctx, &protoReq) + return msg, metadata, err + +} + +func request_RayJobService_GetRayJob_0(ctx context.Context, marshaler runtime.Marshaler, client RayJobServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRayJobRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + msg, err := client.GetRayJob(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_RayJobService_GetRayJob_0(ctx context.Context, marshaler runtime.Marshaler, server RayJobServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq GetRayJobRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + msg, err := server.GetRayJob(ctx, &protoReq) + return msg, metadata, err + +} + +func request_RayJobService_ListRayJobs_0(ctx context.Context, marshaler runtime.Marshaler, client RayJobServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListRayJobsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := client.ListRayJobs(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_RayJobService_ListRayJobs_0(ctx context.Context, marshaler runtime.Marshaler, server RayJobServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListRayJobsRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + msg, err := server.ListRayJobs(ctx, &protoReq) + return msg, metadata, err + +} + +func request_RayJobService_ListAllRayJobs_0(ctx context.Context, marshaler runtime.Marshaler, client RayJobServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListAllRayJobsRequest + var metadata runtime.ServerMetadata + + msg, err := client.ListAllRayJobs(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_RayJobService_ListAllRayJobs_0(ctx context.Context, marshaler runtime.Marshaler, server RayJobServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListAllRayJobsRequest + var metadata runtime.ServerMetadata + + msg, err := server.ListAllRayJobs(ctx, &protoReq) + return msg, metadata, err + +} + +func request_RayJobService_DeleteRayJob_0(ctx context.Context, marshaler runtime.Marshaler, client RayJobServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteRayJobRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + msg, err := client.DeleteRayJob(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_RayJobService_DeleteRayJob_0(ctx context.Context, marshaler runtime.Marshaler, server RayJobServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq DeleteRayJobRequest + var metadata runtime.ServerMetadata + + var ( + val string + ok bool + err error + _ = err + ) + + val, ok = pathParams["namespace"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") + } + + protoReq.Namespace, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) + } + + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") + } + + protoReq.Name, err = runtime.String(val) + if err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "name", err) + } + + msg, err := server.DeleteRayJob(ctx, &protoReq) + return msg, metadata, err + +} + +// RegisterRayJobServiceHandlerServer registers the http handlers for service RayJobService to "mux". +// UnaryRPC :call RayJobServiceServer directly. +// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. +// Note that using this registration option will cause many gRPC library features to stop working. Consider using RegisterRayJobServiceHandlerFromEndpoint instead. +func RegisterRayJobServiceHandlerServer(ctx context.Context, mux *runtime.ServeMux, server RayJobServiceServer) error { + + mux.Handle("POST", pattern_RayJobService_CreateRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/proto.RayJobService/CreateRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_RayJobService_CreateRayJob_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_CreateRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_GetRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/proto.RayJobService/GetRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs/{name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_RayJobService_GetRayJob_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_GetRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_ListRayJobs_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/proto.RayJobService/ListRayJobs", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_RayJobService_ListRayJobs_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_ListRayJobs_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_ListAllRayJobs_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/proto.RayJobService/ListAllRayJobs", runtime.WithHTTPPathPattern("/apis/v1alpha2/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_RayJobService_ListAllRayJobs_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_ListAllRayJobs_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_RayJobService_DeleteRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/proto.RayJobService/DeleteRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs/{name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_RayJobService_DeleteRayJob_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_DeleteRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +// RegisterRayJobServiceHandlerFromEndpoint is same as RegisterRayJobServiceHandler but +// automatically dials to "endpoint" and closes the connection when "ctx" gets done. +func RegisterRayJobServiceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) { + conn, err := grpc.Dial(endpoint, opts...) + if err != nil { + return err + } + defer func() { + if err != nil { + if cerr := conn.Close(); cerr != nil { + grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr) + } + return + } + go func() { + <-ctx.Done() + if cerr := conn.Close(); cerr != nil { + grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr) + } + }() + }() + + return RegisterRayJobServiceHandler(ctx, mux, conn) +} + +// RegisterRayJobServiceHandler registers the http handlers for service RayJobService to "mux". +// The handlers forward requests to the grpc endpoint over "conn". +func RegisterRayJobServiceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error { + return RegisterRayJobServiceHandlerClient(ctx, mux, NewRayJobServiceClient(conn)) +} + +// RegisterRayJobServiceHandlerClient registers the http handlers for service RayJobService +// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "RayJobServiceClient". +// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "RayJobServiceClient" +// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in +// "RayJobServiceClient" to call the correct interceptors. +func RegisterRayJobServiceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client RayJobServiceClient) error { + + mux.Handle("POST", pattern_RayJobService_CreateRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req, "/proto.RayJobService/CreateRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_RayJobService_CreateRayJob_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_CreateRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_GetRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req, "/proto.RayJobService/GetRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs/{name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_RayJobService_GetRayJob_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_GetRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_ListRayJobs_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req, "/proto.RayJobService/ListRayJobs", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_RayJobService_ListRayJobs_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_ListRayJobs_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("GET", pattern_RayJobService_ListAllRayJobs_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req, "/proto.RayJobService/ListAllRayJobs", runtime.WithHTTPPathPattern("/apis/v1alpha2/jobs")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_RayJobService_ListAllRayJobs_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_ListAllRayJobs_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + mux.Handle("DELETE", pattern_RayJobService_DeleteRayJob_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req, "/proto.RayJobService/DeleteRayJob", runtime.WithHTTPPathPattern("/apis/v1alpha2/namespaces/{namespace}/jobs/{name}")) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_RayJobService_DeleteRayJob_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_RayJobService_DeleteRayJob_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + + return nil +} + +var ( + pattern_RayJobService_CreateRayJob_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha2", "jobs"}, "")) + + pattern_RayJobService_GetRayJob_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"apis", "v1alpha2", "namespaces", "namespace", "jobs", "name"}, "")) + + pattern_RayJobService_ListRayJobs_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4}, []string{"apis", "v1alpha2", "namespaces", "namespace", "jobs"}, "")) + + pattern_RayJobService_ListAllRayJobs_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha2", "jobs"}, "")) + + pattern_RayJobService_DeleteRayJob_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3, 2, 4, 1, 0, 4, 1, 5, 5}, []string{"apis", "v1alpha2", "namespaces", "namespace", "jobs", "name"}, "")) +) + +var ( + forward_RayJobService_CreateRayJob_0 = runtime.ForwardResponseMessage + + forward_RayJobService_GetRayJob_0 = runtime.ForwardResponseMessage + + forward_RayJobService_ListRayJobs_0 = runtime.ForwardResponseMessage + + forward_RayJobService_ListAllRayJobs_0 = runtime.ForwardResponseMessage + + forward_RayJobService_DeleteRayJob_0 = runtime.ForwardResponseMessage +) diff --git a/proto/go_client/job_grpc.pb.go b/proto/go_client/job_grpc.pb.go new file mode 100644 index 00000000000..7d7b5120ad0 --- /dev/null +++ b/proto/go_client/job_grpc.pb.go @@ -0,0 +1,256 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. + +package go_client + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + emptypb "google.golang.org/protobuf/types/known/emptypb" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// RayJobServiceClient is the client API for RayJobService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type RayJobServiceClient interface { + // Creates a new compute template. + CreateRayJob(ctx context.Context, in *CreateRayJobRequest, opts ...grpc.CallOption) (*RayJob, error) + // Finds a specific compute template by its name and namespace. + GetRayJob(ctx context.Context, in *GetRayJobRequest, opts ...grpc.CallOption) (*RayJob, error) + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListRayJobs(ctx context.Context, in *ListRayJobsRequest, opts ...grpc.CallOption) (*ListRayJobsResponse, error) + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListAllRayJobs(ctx context.Context, in *ListAllRayJobsRequest, opts ...grpc.CallOption) (*ListAllRayJobsResponse, error) + // Deletes a compute template by its name and namespace + DeleteRayJob(ctx context.Context, in *DeleteRayJobRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) +} + +type rayJobServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewRayJobServiceClient(cc grpc.ClientConnInterface) RayJobServiceClient { + return &rayJobServiceClient{cc} +} + +func (c *rayJobServiceClient) CreateRayJob(ctx context.Context, in *CreateRayJobRequest, opts ...grpc.CallOption) (*RayJob, error) { + out := new(RayJob) + err := c.cc.Invoke(ctx, "/proto.RayJobService/CreateRayJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rayJobServiceClient) GetRayJob(ctx context.Context, in *GetRayJobRequest, opts ...grpc.CallOption) (*RayJob, error) { + out := new(RayJob) + err := c.cc.Invoke(ctx, "/proto.RayJobService/GetRayJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rayJobServiceClient) ListRayJobs(ctx context.Context, in *ListRayJobsRequest, opts ...grpc.CallOption) (*ListRayJobsResponse, error) { + out := new(ListRayJobsResponse) + err := c.cc.Invoke(ctx, "/proto.RayJobService/ListRayJobs", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rayJobServiceClient) ListAllRayJobs(ctx context.Context, in *ListAllRayJobsRequest, opts ...grpc.CallOption) (*ListAllRayJobsResponse, error) { + out := new(ListAllRayJobsResponse) + err := c.cc.Invoke(ctx, "/proto.RayJobService/ListAllRayJobs", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *rayJobServiceClient) DeleteRayJob(ctx context.Context, in *DeleteRayJobRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := c.cc.Invoke(ctx, "/proto.RayJobService/DeleteRayJob", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// RayJobServiceServer is the server API for RayJobService service. +// All implementations must embed UnimplementedRayJobServiceServer +// for forward compatibility +type RayJobServiceServer interface { + // Creates a new compute template. + CreateRayJob(context.Context, *CreateRayJobRequest) (*RayJob, error) + // Finds a specific compute template by its name and namespace. + GetRayJob(context.Context, *GetRayJobRequest) (*RayJob, error) + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListRayJobs(context.Context, *ListRayJobsRequest) (*ListRayJobsResponse, error) + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListAllRayJobs(context.Context, *ListAllRayJobsRequest) (*ListAllRayJobsResponse, error) + // Deletes a compute template by its name and namespace + DeleteRayJob(context.Context, *DeleteRayJobRequest) (*emptypb.Empty, error) + mustEmbedUnimplementedRayJobServiceServer() +} + +// UnimplementedRayJobServiceServer must be embedded to have forward compatible implementations. +type UnimplementedRayJobServiceServer struct { +} + +func (UnimplementedRayJobServiceServer) CreateRayJob(context.Context, *CreateRayJobRequest) (*RayJob, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateRayJob not implemented") +} +func (UnimplementedRayJobServiceServer) GetRayJob(context.Context, *GetRayJobRequest) (*RayJob, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetRayJob not implemented") +} +func (UnimplementedRayJobServiceServer) ListRayJobs(context.Context, *ListRayJobsRequest) (*ListRayJobsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListRayJobs not implemented") +} +func (UnimplementedRayJobServiceServer) ListAllRayJobs(context.Context, *ListAllRayJobsRequest) (*ListAllRayJobsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListAllRayJobs not implemented") +} +func (UnimplementedRayJobServiceServer) DeleteRayJob(context.Context, *DeleteRayJobRequest) (*emptypb.Empty, error) { + return nil, status.Errorf(codes.Unimplemented, "method DeleteRayJob not implemented") +} +func (UnimplementedRayJobServiceServer) mustEmbedUnimplementedRayJobServiceServer() {} + +// UnsafeRayJobServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to RayJobServiceServer will +// result in compilation errors. +type UnsafeRayJobServiceServer interface { + mustEmbedUnimplementedRayJobServiceServer() +} + +func RegisterRayJobServiceServer(s grpc.ServiceRegistrar, srv RayJobServiceServer) { + s.RegisterService(&RayJobService_ServiceDesc, srv) +} + +func _RayJobService_CreateRayJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateRayJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RayJobServiceServer).CreateRayJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.RayJobService/CreateRayJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RayJobServiceServer).CreateRayJob(ctx, req.(*CreateRayJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RayJobService_GetRayJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRayJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RayJobServiceServer).GetRayJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.RayJobService/GetRayJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RayJobServiceServer).GetRayJob(ctx, req.(*GetRayJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RayJobService_ListRayJobs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListRayJobsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RayJobServiceServer).ListRayJobs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.RayJobService/ListRayJobs", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RayJobServiceServer).ListRayJobs(ctx, req.(*ListRayJobsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RayJobService_ListAllRayJobs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListAllRayJobsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RayJobServiceServer).ListAllRayJobs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.RayJobService/ListAllRayJobs", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RayJobServiceServer).ListAllRayJobs(ctx, req.(*ListAllRayJobsRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _RayJobService_DeleteRayJob_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DeleteRayJobRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RayJobServiceServer).DeleteRayJob(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.RayJobService/DeleteRayJob", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RayJobServiceServer).DeleteRayJob(ctx, req.(*DeleteRayJobRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// RayJobService_ServiceDesc is the grpc.ServiceDesc for RayJobService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var RayJobService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "proto.RayJobService", + HandlerType: (*RayJobServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "CreateRayJob", + Handler: _RayJobService_CreateRayJob_Handler, + }, + { + MethodName: "GetRayJob", + Handler: _RayJobService_GetRayJob_Handler, + }, + { + MethodName: "ListRayJobs", + Handler: _RayJobService_ListRayJobs_Handler, + }, + { + MethodName: "ListAllRayJobs", + Handler: _RayJobService_ListAllRayJobs_Handler, + }, + { + MethodName: "DeleteRayJob", + Handler: _RayJobService_DeleteRayJob_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "job.proto", +} diff --git a/proto/hack/generate.sh b/proto/hack/generate.sh index 8ab8728ed55..828a81e0b9c 100755 --- a/proto/hack/generate.sh +++ b/proto/hack/generate.sh @@ -19,3 +19,13 @@ protoc -I. \ # Move *.pb.go and *.gw.go to go_client folder. cp ${TMP_OUTPUT}/github.com/ray-project/kuberay/proto/go_client/* ./go_client + +# Generate a single swagger json file from the swagger json files of all models. +# Note: use proto/swagger/{cluster,config,error}.swagger.json +# Note: swagger files are generate to source folder directly. No files in ${TMP_OUTPUT} +jq -s 'reduce .[] as $item ({}; . * $item) | .info.title = "KubeRay API" | .info.description = "This file contains REST API specification for KubeRay. The file is autogenerated from the swagger definition." | .info.version = "'0.3.0'" | .info.license = { "name": "Apache 2.0", "url": "https://raw.githubusercontent.com/ray-project/kuberay/master/LICENSE" }' \ + /go/src/github.com/ray-project/kuberay/proto/swagger/cluster.swagger.json \ + /go/src/github.com/ray-project/kuberay/proto/swagger/config.swagger.json \ + /go/src/github.com/ray-project/kuberay/proto/swagger/error.swagger.json \ + /go/src/github.com/ray-project/kuberay/proto/swagger/job.swagger.json \ + > "/go/src/github.com/ray-project/kuberay/proto/kuberay_api.swagger.json" \ No newline at end of file diff --git a/proto/job.proto b/proto/job.proto new file mode 100644 index 00000000000..b8b14c0820a --- /dev/null +++ b/proto/job.proto @@ -0,0 +1,127 @@ +syntax = "proto3"; + +option go_package = "github.com/ray-project/kuberay/proto/go_client"; +package proto; + +import "google/api/annotations.proto"; +import "google/protobuf/empty.proto"; +import "protoc-gen-openapiv2/options/annotations.proto"; +import "cluster.proto"; + + +option (grpc.gateway.protoc_gen_openapiv2.options.openapiv2_swagger) = { + schemes: HTTP; + responses: { + key: "default"; + value: { + schema: { + json_schema: { + ref: ".api.Status"; + } + } + } + } +}; + +service RayJobService { + // Creates a new compute template. + rpc CreateRayJob(CreateRayJobRequest) returns (RayJob) { + option (google.api.http) = { + post: "/apis/v1alpha2/jobs" + body: "job" + }; + } + + // Finds a specific compute template by its name and namespace. + rpc GetRayJob(GetRayJobRequest) returns (RayJob) { + option (google.api.http) = { + get: "/apis/v1alpha2/namespaces/{namespace}/jobs/{name}" + }; + } + + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + rpc ListRayJobs(ListRayJobsRequest) returns (ListRayJobsResponse) { + option (google.api.http) = { + get: "/apis/v1alpha2/namespaces/{namespace}/jobs" + }; + } + + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + rpc ListAllRayJobs(ListAllRayJobsRequest) returns (ListAllRayJobsResponse) { + option (google.api.http) = { + get: "/apis/v1alpha2/jobs" + }; + } + + // Deletes a compute template by its name and namespace + rpc DeleteRayJob(DeleteRayJobRequest) returns (google.protobuf.Empty) { + option (google.api.http) = { + delete: "/apis/v1alpha2/namespaces/{namespace}/jobs/{name}" + }; + } +} + +message CreateRayJobRequest { + // The compute template to be created. + RayJob job = 1; + // The namespace of the compute template to be created + string namespace = 2; +} + +message GetRayJobRequest { + // The name of the RayJob to be retrieved. + string name = 1; + // The namespace of the compute template to be retrieved. + string namespace = 2; +} + +message ListRayJobsRequest { + // The namespace of the compute templates to be retrieved. + string namespace = 1; + // TODO: support paganation later +} + +message ListRayJobsResponse { + repeated RayJob jobs = 1; +} + +message ListAllRayJobsRequest { + // TODO: support paganation later +} + +message ListAllRayJobsResponse { + repeated RayJob jobs = 1; +} + +message DeleteRayJobRequest { + // The name of the compute template to be deleted. + string name = 1; + // The namespace of the compute template to be deleted. + string namespace = 2; +} + +// RayJob +message RayJob { + // Required input field. Unique job name provided by user. + string name = 1; + // Required input field. job namespace provided by user + string namespace = 2; + // Required field. This field indicates the user who owns the job. + string user = 3; + // The entrypoint of the RayJob + string entrypoint = 4; + // Metadata is data to store along with this job. + map metadata = 5; + // RuntimeEnv is base64 encoded string which stores runtime configuration + string runtime_env = 6; + // If jobId is not set, a new jobId will be auto-generated. + string job_id = 7; + // If set to true, the rayCluster will be deleted after the rayJob finishes + bool ShutdownAfterJobFinishes = 8; + // The label selector to choose exiting clusters + map cluster_selector = 9; + // The cluster template + ClusterSpec cluster_spec = 10; + // logs + string logs = 11; +} diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json new file mode 100644 index 00000000000..b183fe1f0dc --- /dev/null +++ b/proto/kuberay_api.swagger.json @@ -0,0 +1,1178 @@ +{ + "swagger": "2.0", + "info": { + "title": "KubeRay API", + "version": "0.3.0", + "description": "This file contains REST API specification for KubeRay. The file is autogenerated from the swagger definition.", + "license": { + "name": "Apache 2.0", + "url": "https://raw.githubusercontent.com/ray-project/kuberay/master/LICENSE" + } + }, + "tags": [ + { + "name": "RayJobService" + } + ], + "schemes": [ + "http" + ], + "consumes": [ + "application/json" + ], + "produces": [ + "application/json" + ], + "paths": { + "/apis/v1alpha2/clusters": { + "get": { + "summary": "Finds all Clusters in all namespaces. Supports pagination, and sorting on certain fields.", + "operationId": "ClusterService_ListAllClusters", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListAllClustersResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "tags": [ + "ClusterService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/clusters": { + "get": { + "summary": "Finds all Clusters in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "ClusterService_ListCluster", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListClustersResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the clusters to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ClusterService" + ] + }, + "post": { + "summary": "Creates a new Cluster.", + "operationId": "ClusterService_CreateCluster", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoCluster" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the cluster to be created.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "body", + "description": "The cluster to be created.", + "in": "body", + "required": true, + "schema": { + "$ref": "#/definitions/protoCluster" + } + } + ], + "tags": [ + "ClusterService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/clusters/{name}": { + "get": { + "summary": "Finds a specific Cluster by ID.", + "operationId": "ClusterService_GetCluster", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoCluster" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the cluster to be retrieved.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the cluster to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ClusterService" + ] + }, + "delete": { + "summary": "Deletes an cluster without deleting the cluster's runs and jobs. To\navoid unexpected behaviors, delete an cluster's runs and jobs before\ndeleting the cluster.", + "operationId": "ClusterService_DeleteCluster", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "properties": {} + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the cluster to be deleted.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the cluster to be deleted.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ClusterService" + ] + } + }, + "/apis/v1alpha2/compute_templates": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "ComputeTemplateService_ListAllComputeTemplates", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListAllComputeTemplatesResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "tags": [ + "ComputeTemplateService" + ] + } + }, + "/apis/v1alpha2/image_templates": { + "post": { + "summary": "Creates a new ImageTemplate.", + "operationId": "ImageTemplateService_CreateImageTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoImageTemplate" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "body", + "description": "The image template to be created.", + "in": "body", + "required": true, + "schema": { + "$ref": "#/definitions/protoImageTemplate" + } + }, + { + "name": "namespace", + "description": "The namespace of the image template to be created.", + "in": "query", + "required": false, + "type": "string" + } + ], + "tags": [ + "ImageTemplateService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/compute_templates": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "ComputeTemplateService_ListComputeTemplates", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListComputeTemplatesResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute templates to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ComputeTemplateService" + ] + }, + "post": { + "summary": "Creates a new compute template.", + "operationId": "ComputeTemplateService_CreateComputeTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoComputeTemplate" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be created", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "body", + "description": "The compute template to be created.", + "in": "body", + "required": true, + "schema": { + "$ref": "#/definitions/protoComputeTemplate" + } + } + ], + "tags": [ + "ComputeTemplateService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/compute_templates/{name}": { + "get": { + "summary": "Finds a specific compute template by its name and namespace.", + "operationId": "ComputeTemplateService_GetComputeTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoComputeTemplate" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be retrieved.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the ComputeTemplate to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ComputeTemplateService" + ] + }, + "delete": { + "summary": "Deletes a compute template by its name and namespace", + "operationId": "ComputeTemplateService_DeleteComputeTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "properties": {} + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ComputeTemplateService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/image_templates": { + "get": { + "summary": "Finds all ImageTemplates. Supports pagination, and sorting on certain fields.", + "operationId": "ImageTemplateService_ListImageTemplates", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListImageTemplatesResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the image templates to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ImageTemplateService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/image_templates/{name}": { + "get": { + "summary": "Finds a specific ImageTemplate by ID.", + "operationId": "ImageTemplateService_GetImageTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoImageTemplate" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the image template to be retrieved.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the image template to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ImageTemplateService" + ] + }, + "delete": { + "summary": "Deletes an ImageTemplate.", + "operationId": "ImageTemplateService_DeleteImageTemplate", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "properties": {} + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the image template to be deleted.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the image template to be deleted.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "ImageTemplateService" + ] + } + }, + "/apis/v1alpha2/jobs": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "RayJobService_ListAllRayJobs", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListAllRayJobsResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "tags": [ + "RayJobService" + ] + }, + "post": { + "summary": "Creates a new compute template.", + "operationId": "RayJobService_CreateRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "body", + "description": "The compute template to be created.", + "in": "body", + "required": true, + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + { + "name": "namespace", + "description": "The namespace of the compute template to be created.", + "in": "query", + "required": false, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/jobs": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "RayJobService_ListRayJobs", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListRayJobsResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute templates to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/jobs/{name}": { + "get": { + "summary": "Finds a specific compute template by its name and namespace.", + "operationId": "RayJobService_GetRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be retrieved.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the RayJob to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + }, + "delete": { + "summary": "Deletes a compute template by its name and namespace", + "operationId": "RayJobService_DeleteRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "properties": {} + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + } + }, + "definitions": { + "ClusterEnvironment": { + "type": "string", + "enum": [ + "DEV", + "TESTING", + "STAGING", + "PRODUCTION" + ], + "default": "DEV", + "description": "Optional field." + }, + "VolumeVolumeType": { + "type": "string", + "enum": [ + "PERSISTENT_VOLUME_CLAIM", + "HOST_PATH" + ], + "default": "PERSISTENT_VOLUME_CLAIM" + }, + "googlerpcStatus": { + "type": "object", + "properties": { + "code": { + "type": "integer", + "format": "int32" + }, + "message": { + "type": "string" + }, + "details": { + "type": "array", + "items": { + "$ref": "#/definitions/protobufAny" + } + } + } + }, + "protoCluster": { + "type": "object", + "properties": { + "name": { + "type": "string", + "description": "Required input field. Unique cluster name provided by user." + }, + "namespace": { + "type": "string", + "title": "Required input field. Cluster's namespace provided by user" + }, + "user": { + "type": "string", + "description": "Required field. This field indicates the user who owns the cluster." + }, + "version": { + "type": "string", + "title": "Optional input field. Ray cluster version" + }, + "environment": { + "$ref": "#/definitions/ClusterEnvironment" + }, + "clusterSpec": { + "$ref": "#/definitions/protoClusterSpec", + "title": "Required field. This field indicates ray cluster configuration" + }, + "createdAt": { + "type": "string", + "format": "date-time", + "description": "Output. The time that the cluster created." + }, + "deletedAt": { + "type": "string", + "format": "date-time", + "description": "Output. The time that the cluster deleted." + }, + "clusterState": { + "type": "string", + "title": "Output. The status to show the cluster status.state" + }, + "events": { + "type": "array", + "items": { + "$ref": "#/definitions/protoClusterEvent" + }, + "description": "Output. The list related to the cluster." + }, + "serviceEndpoint": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Output. The service endpoint of the cluster" + } + } + }, + "protoClusterEvent": { + "type": "object", + "properties": { + "id": { + "type": "string", + "description": "Output. Unique Event Id." + }, + "name": { + "type": "string", + "description": "Output. Human readable name for event." + }, + "createdAt": { + "type": "string", + "format": "date-time", + "description": "Output. The creation time of the event." + }, + "firstTimestamp": { + "type": "string", + "format": "date-time", + "description": "Output. The last time the event occur." + }, + "lastTimestamp": { + "type": "string", + "format": "date-time", + "title": "Output. The first time the event occur" + }, + "reason": { + "type": "string", + "description": "Output. The reason for the transition into the object's current status." + }, + "message": { + "type": "string", + "description": "Output. A human-readable description of the status of this operation." + }, + "type": { + "type": "string", + "title": "Output. Type of this event (Normal, Warning), new types could be added in the future" + }, + "count": { + "type": "integer", + "format": "int32", + "description": "Output. The number of times this event has occurred." + } + } + }, + "protoClusterSpec": { + "type": "object", + "properties": { + "headGroupSpec": { + "$ref": "#/definitions/protoHeadGroupSpec", + "title": "The head group configuration" + }, + "workerGroupSpec": { + "type": "array", + "items": { + "$ref": "#/definitions/protoWorkerGroupSpec" + }, + "title": "The worker group configurations" + } + } + }, + "protoHeadGroupSpec": { + "type": "object", + "properties": { + "computeTemplate": { + "type": "string", + "title": "Optional. The computeTemplate of head node group" + }, + "image": { + "type": "string", + "title": "Optional field. This field will be used to retrieve right ray container" + }, + "serviceType": { + "type": "string", + "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" + }, + "rayStartParams": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. The ray start params of head node group" + }, + "volumes": { + "type": "array", + "items": { + "$ref": "#/definitions/protoVolume" + }, + "title": "Optional. The volumes mount to head pod" + } + } + }, + "protoListAllClustersResponse": { + "type": "object", + "properties": { + "clusters": { + "type": "array", + "items": { + "$ref": "#/definitions/protoCluster" + }, + "description": "A list of clusters returned." + } + } + }, + "protoListClustersResponse": { + "type": "object", + "properties": { + "clusters": { + "type": "array", + "items": { + "$ref": "#/definitions/protoCluster" + }, + "description": "A list of clusters returned." + } + } + }, + "protoVolume": { + "type": "object", + "properties": { + "mountPath": { + "type": "string" + }, + "volumeType": { + "$ref": "#/definitions/VolumeVolumeType" + }, + "name": { + "type": "string" + }, + "source": { + "type": "string" + }, + "readOnly": { + "type": "boolean" + } + } + }, + "protoWorkerGroupSpec": { + "type": "object", + "properties": { + "groupName": { + "type": "string", + "title": "Required. Group name of the current worker group" + }, + "computeTemplate": { + "type": "string", + "title": "Optional. The computeTemplate of head node group" + }, + "image": { + "type": "string", + "title": "Optional field. This field will be used to retrieve right ray container" + }, + "replicas": { + "type": "integer", + "format": "int32", + "title": "Required. Desired replicas of the worker group" + }, + "minReplicas": { + "type": "integer", + "format": "int32", + "title": "Optional. Min replicas of the worker group" + }, + "maxReplicas": { + "type": "integer", + "format": "int32", + "title": "Optional. Max replicas of the worker group" + }, + "rayStartParams": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. The ray start parames of worker node group" + }, + "volumes": { + "type": "array", + "items": { + "$ref": "#/definitions/protoVolume" + }, + "title": "Optional. The volumes mount to worker pods" + } + } + }, + "protobufAny": { + "type": "object", + "properties": { + "typeUrl": { + "type": "string", + "description": "A URL/resource name that uniquely identifies the type of the serialized\nprotocol buffer message. This string must contain at least\none \"/\" character. The last segment of the URL's path must represent\nthe fully qualified name of the type (as in\n`path/google.protobuf.Duration`). The name should be in a canonical form\n(e.g., leading \".\" is not accepted).\n\nIn practice, teams usually precompile into the binary all types that they\nexpect it to use in the context of Any. However, for URLs which use the\nscheme `http`, `https`, or no scheme, one can optionally set up a type\nserver that maps type URLs to message definitions as follows:\n\n* If no scheme is provided, `https` is assumed.\n* An HTTP GET on the URL must yield a [google.protobuf.Type][]\n value in binary format, or produce an error.\n* Applications are allowed to cache lookup results based on the\n URL, or have them precompiled into a binary to avoid any\n lookup. Therefore, binary compatibility needs to be preserved\n on changes to types. (Use versioned type names to manage\n breaking changes.)\n\nNote: this functionality is not currently available in the official\nprotobuf release, and it is not used for type URLs beginning with\ntype.googleapis.com.\n\nSchemes other than `http`, `https` (or the empty scheme) might be\nused with implementation specific semantics." + }, + "value": { + "type": "string", + "format": "byte", + "description": "Must be a valid serialized protocol buffer of the above specified type." + } + }, + "description": "`Any` contains an arbitrary serialized protocol buffer message along with a\nURL that describes the type of the serialized message.\n\nProtobuf library provides support to pack/unpack Any values in the form\nof utility functions or additional generated methods of the Any type.\n\nExample 1: Pack and unpack a message in C++.\n\n Foo foo = ...;\n Any any;\n any.PackFrom(foo);\n ...\n if (any.UnpackTo(&foo)) {\n ...\n }\n\nExample 2: Pack and unpack a message in Java.\n\n Foo foo = ...;\n Any any = Any.pack(foo);\n ...\n if (any.is(Foo.class)) {\n foo = any.unpack(Foo.class);\n }\n\n Example 3: Pack and unpack a message in Python.\n\n foo = Foo(...)\n any = Any()\n any.Pack(foo)\n ...\n if any.Is(Foo.DESCRIPTOR):\n any.Unpack(foo)\n ...\n\n Example 4: Pack and unpack a message in Go\n\n foo := &pb.Foo{...}\n any, err := anypb.New(foo)\n if err != nil {\n ...\n }\n ...\n foo := &pb.Foo{}\n if err := any.UnmarshalTo(foo); err != nil {\n ...\n }\n\nThe pack methods provided by protobuf library will by default use\n'type.googleapis.com/full.type.name' as the type URL and the unpack\nmethods only use the fully qualified type name after the last '/'\nin the type URL, for example \"foo.bar.com/x/y.z\" will yield type\nname \"y.z\".\n\n\nJSON\n====\nThe JSON representation of an `Any` value uses the regular\nrepresentation of the deserialized, embedded message, with an\nadditional field `@type` which contains the type URL. Example:\n\n package google.profile;\n message Person {\n string first_name = 1;\n string last_name = 2;\n }\n\n {\n \"@type\": \"type.googleapis.com/google.profile.Person\",\n \"firstName\": ,\n \"lastName\": \n }\n\nIf the embedded message type is well-known and has a custom JSON\nrepresentation, that representation will be embedded adding a field\n`value` which holds the custom JSON in addition to the `@type`\nfield. Example (for message [google.protobuf.Duration][]):\n\n {\n \"@type\": \"type.googleapis.com/google.protobuf.Duration\",\n \"value\": \"1.212s\"\n }" + }, + "protoComputeTemplate": { + "type": "object", + "properties": { + "name": { + "type": "string", + "title": "The name of the compute template" + }, + "namespace": { + "type": "string", + "title": "The namespace of the compute template" + }, + "cpu": { + "type": "integer", + "format": "int64", + "title": "Number of cpus" + }, + "memory": { + "type": "integer", + "format": "int64", + "title": "Number of memory" + }, + "gpu": { + "type": "integer", + "format": "int64", + "title": "Number of gpus" + }, + "gpuAccelerator": { + "type": "string", + "title": "The detail gpu accelerator type" + } + }, + "title": "ComputeTemplate can be reused by any compute units like worker group, workspace, image build job, etc" + }, + "protoImageTemplate": { + "type": "object", + "properties": { + "name": { + "type": "string", + "title": "The ID of the image template" + }, + "namespace": { + "type": "string", + "title": "The namespace of the image template" + }, + "baseImage": { + "type": "string", + "title": "The base container image to be used for image building" + }, + "pipPackages": { + "type": "array", + "items": { + "type": "string" + }, + "title": "The pip packages to install" + }, + "condaPackages": { + "type": "array", + "items": { + "type": "string" + }, + "title": "The conda packages to install" + }, + "systemPackages": { + "type": "array", + "items": { + "type": "string" + }, + "title": "The system packages to install" + }, + "environmentVariables": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "The environment variables to set" + }, + "customCommands": { + "type": "string", + "title": "The post install commands to execute" + }, + "image": { + "type": "string", + "title": "Output. The result image generated" + } + }, + "title": "ImageTemplate can be used by worker group and workspce.\nThey can be distinguish by different entrypoints" + }, + "protoListAllComputeTemplatesResponse": { + "type": "object", + "properties": { + "computeTemplates": { + "type": "array", + "items": { + "$ref": "#/definitions/protoComputeTemplate" + } + } + } + }, + "protoListComputeTemplatesResponse": { + "type": "object", + "properties": { + "computeTemplates": { + "type": "array", + "items": { + "$ref": "#/definitions/protoComputeTemplate" + } + } + } + }, + "protoListImageTemplatesResponse": { + "type": "object", + "properties": { + "imageTemplates": { + "type": "array", + "items": { + "$ref": "#/definitions/protoImageTemplate" + }, + "description": "A list of Compute returned." + } + } + }, + "protoListAllRayJobsResponse": { + "type": "object", + "properties": { + "jobs": { + "type": "array", + "items": { + "$ref": "#/definitions/protoRayJob" + } + } + } + }, + "protoListRayJobsResponse": { + "type": "object", + "properties": { + "jobs": { + "type": "array", + "items": { + "$ref": "#/definitions/protoRayJob" + } + } + } + }, + "protoRayJob": { + "type": "object", + "properties": { + "name": { + "type": "string", + "description": "Required input field. Unique job name provided by user." + }, + "namespace": { + "type": "string", + "title": "Required input field. job namespace provided by user" + }, + "user": { + "type": "string", + "description": "Required field. This field indicates the user who owns the job." + }, + "entrypoint": { + "type": "string", + "title": "The entrypoint of the RayJob" + }, + "metadata": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Metadata is data to store along with this job." + }, + "runtimeEnv": { + "type": "string", + "title": "RuntimeEnv is base64 encoded string which stores runtime configuration" + }, + "jobId": { + "type": "string", + "description": "If jobId is not set, a new jobId will be auto-generated." + }, + "ShutdownAfterJobFinishes": { + "type": "boolean", + "title": "If set to true, the rayCluster will be deleted after the rayJob finishes" + }, + "clusterSelector": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "The label selector to choose exiting clusters" + }, + "clusterSpec": { + "$ref": "#/definitions/protoClusterSpec", + "title": "The cluster template" + }, + "logs": { + "type": "string", + "title": "logs" + } + }, + "title": "RayJob" + } + } +} diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json new file mode 100644 index 00000000000..8d741b05f0a --- /dev/null +++ b/proto/swagger/job.swagger.json @@ -0,0 +1,427 @@ +{ + "swagger": "2.0", + "info": { + "title": "job.proto", + "version": "version not set" + }, + "tags": [ + { + "name": "RayJobService" + } + ], + "schemes": [ + "http" + ], + "consumes": [ + "application/json" + ], + "produces": [ + "application/json" + ], + "paths": { + "/apis/v1alpha2/jobs": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "RayJobService_ListAllRayJobs", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListAllRayJobsResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "tags": [ + "RayJobService" + ] + }, + "post": { + "summary": "Creates a new compute template.", + "operationId": "RayJobService_CreateRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "body", + "description": "The compute template to be created.", + "in": "body", + "required": true, + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + { + "name": "namespace", + "description": "The namespace of the compute template to be created.", + "in": "query", + "required": false, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/jobs": { + "get": { + "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "operationId": "RayJobService_ListRayJobs", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoListRayJobsResponse" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute templates to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + }, + "/apis/v1alpha2/namespaces/{namespace}/jobs/{name}": { + "get": { + "summary": "Finds a specific compute template by its name and namespace.", + "operationId": "RayJobService_GetRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "$ref": "#/definitions/protoRayJob" + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be retrieved.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the RayJob to be retrieved.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + }, + "delete": { + "summary": "Deletes a compute template by its name and namespace", + "operationId": "RayJobService_DeleteRayJob", + "responses": { + "200": { + "description": "A successful response.", + "schema": { + "properties": {} + } + }, + "default": { + "description": "An unexpected error response.", + "schema": { + "$ref": "#/definitions/googlerpcStatus" + } + } + }, + "parameters": [ + { + "name": "namespace", + "description": "The namespace of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + }, + { + "name": "name", + "description": "The name of the compute template to be deleted.", + "in": "path", + "required": true, + "type": "string" + } + ], + "tags": [ + "RayJobService" + ] + } + } + }, + "definitions": { + "VolumeVolumeType": { + "type": "string", + "enum": [ + "PERSISTENT_VOLUME_CLAIM", + "HOST_PATH" + ], + "default": "PERSISTENT_VOLUME_CLAIM" + }, + "googlerpcStatus": { + "type": "object", + "properties": { + "code": { + "type": "integer", + "format": "int32" + }, + "message": { + "type": "string" + }, + "details": { + "type": "array", + "items": { + "$ref": "#/definitions/protobufAny" + } + } + } + }, + "protoClusterSpec": { + "type": "object", + "properties": { + "headGroupSpec": { + "$ref": "#/definitions/protoHeadGroupSpec", + "title": "The head group configuration" + }, + "workerGroupSpec": { + "type": "array", + "items": { + "$ref": "#/definitions/protoWorkerGroupSpec" + }, + "title": "The worker group configurations" + } + } + }, + "protoHeadGroupSpec": { + "type": "object", + "properties": { + "computeTemplate": { + "type": "string", + "title": "Optional. The computeTemplate of head node group" + }, + "image": { + "type": "string", + "title": "Optional field. This field will be used to retrieve right ray container" + }, + "serviceType": { + "type": "string", + "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" + }, + "rayStartParams": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. The ray start params of head node group" + }, + "volumes": { + "type": "array", + "items": { + "$ref": "#/definitions/protoVolume" + }, + "title": "Optional. The volumes mount to head pod" + } + } + }, + "protoListAllRayJobsResponse": { + "type": "object", + "properties": { + "jobs": { + "type": "array", + "items": { + "$ref": "#/definitions/protoRayJob" + } + } + } + }, + "protoListRayJobsResponse": { + "type": "object", + "properties": { + "jobs": { + "type": "array", + "items": { + "$ref": "#/definitions/protoRayJob" + } + } + } + }, + "protoRayJob": { + "type": "object", + "properties": { + "name": { + "type": "string", + "description": "Required input field. Unique job name provided by user." + }, + "namespace": { + "type": "string", + "title": "Required input field. job namespace provided by user" + }, + "user": { + "type": "string", + "description": "Required field. This field indicates the user who owns the job." + }, + "entrypoint": { + "type": "string", + "title": "The entrypoint of the RayJob" + }, + "metadata": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Metadata is data to store along with this job." + }, + "runtimeEnv": { + "type": "string", + "title": "RuntimeEnv is base64 encoded string which stores runtime configuration" + }, + "jobId": { + "type": "string", + "description": "If jobId is not set, a new jobId will be auto-generated." + }, + "ShutdownAfterJobFinishes": { + "type": "boolean", + "title": "If set to true, the rayCluster will be deleted after the rayJob finishes" + }, + "clusterSelector": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "The label selector to choose exiting clusters" + }, + "clusterSpec": { + "$ref": "#/definitions/protoClusterSpec", + "title": "The cluster template" + }, + "logs": { + "type": "string", + "title": "logs" + } + }, + "title": "RayJob" + }, + "protoVolume": { + "type": "object", + "properties": { + "mountPath": { + "type": "string" + }, + "volumeType": { + "$ref": "#/definitions/VolumeVolumeType" + }, + "name": { + "type": "string" + }, + "source": { + "type": "string" + }, + "readOnly": { + "type": "boolean" + } + } + }, + "protoWorkerGroupSpec": { + "type": "object", + "properties": { + "groupName": { + "type": "string", + "title": "Required. Group name of the current worker group" + }, + "computeTemplate": { + "type": "string", + "title": "Optional. The computeTemplate of head node group" + }, + "image": { + "type": "string", + "title": "Optional field. This field will be used to retrieve right ray container" + }, + "replicas": { + "type": "integer", + "format": "int32", + "title": "Required. Desired replicas of the worker group" + }, + "minReplicas": { + "type": "integer", + "format": "int32", + "title": "Optional. Min replicas of the worker group" + }, + "maxReplicas": { + "type": "integer", + "format": "int32", + "title": "Optional. Max replicas of the worker group" + }, + "rayStartParams": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. The ray start parames of worker node group" + }, + "volumes": { + "type": "array", + "items": { + "$ref": "#/definitions/protoVolume" + }, + "title": "Optional. The volumes mount to worker pods" + } + } + }, + "protobufAny": { + "type": "object", + "properties": { + "typeUrl": { + "type": "string", + "description": "A URL/resource name that uniquely identifies the type of the serialized\nprotocol buffer message. This string must contain at least\none \"/\" character. The last segment of the URL's path must represent\nthe fully qualified name of the type (as in\n`path/google.protobuf.Duration`). The name should be in a canonical form\n(e.g., leading \".\" is not accepted).\n\nIn practice, teams usually precompile into the binary all types that they\nexpect it to use in the context of Any. However, for URLs which use the\nscheme `http`, `https`, or no scheme, one can optionally set up a type\nserver that maps type URLs to message definitions as follows:\n\n* If no scheme is provided, `https` is assumed.\n* An HTTP GET on the URL must yield a [google.protobuf.Type][]\n value in binary format, or produce an error.\n* Applications are allowed to cache lookup results based on the\n URL, or have them precompiled into a binary to avoid any\n lookup. Therefore, binary compatibility needs to be preserved\n on changes to types. (Use versioned type names to manage\n breaking changes.)\n\nNote: this functionality is not currently available in the official\nprotobuf release, and it is not used for type URLs beginning with\ntype.googleapis.com.\n\nSchemes other than `http`, `https` (or the empty scheme) might be\nused with implementation specific semantics." + }, + "value": { + "type": "string", + "format": "byte", + "description": "Must be a valid serialized protocol buffer of the above specified type." + } + }, + "description": "`Any` contains an arbitrary serialized protocol buffer message along with a\nURL that describes the type of the serialized message.\n\nProtobuf library provides support to pack/unpack Any values in the form\nof utility functions or additional generated methods of the Any type.\n\nExample 1: Pack and unpack a message in C++.\n\n Foo foo = ...;\n Any any;\n any.PackFrom(foo);\n ...\n if (any.UnpackTo(\u0026foo)) {\n ...\n }\n\nExample 2: Pack and unpack a message in Java.\n\n Foo foo = ...;\n Any any = Any.pack(foo);\n ...\n if (any.is(Foo.class)) {\n foo = any.unpack(Foo.class);\n }\n\n Example 3: Pack and unpack a message in Python.\n\n foo = Foo(...)\n any = Any()\n any.Pack(foo)\n ...\n if any.Is(Foo.DESCRIPTOR):\n any.Unpack(foo)\n ...\n\n Example 4: Pack and unpack a message in Go\n\n foo := \u0026pb.Foo{...}\n any, err := anypb.New(foo)\n if err != nil {\n ...\n }\n ...\n foo := \u0026pb.Foo{}\n if err := any.UnmarshalTo(foo); err != nil {\n ...\n }\n\nThe pack methods provided by protobuf library will by default use\n'type.googleapis.com/full.type.name' as the type URL and the unpack\nmethods only use the fully qualified type name after the last '/'\nin the type URL, for example \"foo.bar.com/x/y.z\" will yield type\nname \"y.z\".\n\n\nJSON\n====\nThe JSON representation of an `Any` value uses the regular\nrepresentation of the deserialized, embedded message, with an\nadditional field `@type` which contains the type URL. Example:\n\n package google.profile;\n message Person {\n string first_name = 1;\n string last_name = 2;\n }\n\n {\n \"@type\": \"type.googleapis.com/google.profile.Person\",\n \"firstName\": \u003cstring\u003e,\n \"lastName\": \u003cstring\u003e\n }\n\nIf the embedded message type is well-known and has a custom JSON\nrepresentation, that representation will be embedded adding a field\n`value` which holds the custom JSON in addition to the `@type`\nfield. Example (for message [google.protobuf.Duration][]):\n\n {\n \"@type\": \"type.googleapis.com/google.protobuf.Duration\",\n \"value\": \"1.212s\"\n }" + } + } +} From 19c9fa553bb4bf235cc34b077dc68d7fc594c243 Mon Sep 17 00:00:00 2001 From: Basasuya Date: Tue, 26 Jul 2022 23:45:09 +0800 Subject: [PATCH 2/7] fix for lint --- apiserver/pkg/manager/client_manager.go | 4 ++-- apiserver/pkg/manager/resource_manager.go | 1 - apiserver/pkg/model/converter.go | 14 +++++++------- apiserver/pkg/server/job_server.go | 2 -- apiserver/pkg/util/job.go | 14 +++++++------- 5 files changed, 16 insertions(+), 19 deletions(-) diff --git a/apiserver/pkg/manager/client_manager.go b/apiserver/pkg/manager/client_manager.go index c4ed661246f..cd4c9155c23 100644 --- a/apiserver/pkg/manager/client_manager.go +++ b/apiserver/pkg/manager/client_manager.go @@ -10,7 +10,7 @@ import ( type ClientManagerInterface interface { ClusterClient() client.ClusterClientInterface - RayJobClient() client.JobClientInterface + RayJobClient() client.JobClientInterface KubernetesClient() client.KubernetesClientInterface Time() util.TimeInterface } @@ -29,7 +29,7 @@ func (c *ClientManager) ClusterClient() client.ClusterClientInterface { return c.clusterClient } -func (c *ClientManager) RayJobClient() client.JobClientInterface { +func (c *ClientManager) RayJobClient() client.JobClientInterface { return c.jobClient } diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 0a813e6eac1..02ec166ce09 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -204,7 +204,6 @@ func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob, job // convert *api.Cluster to v1alpha1.RayCluster rayJob := util.NewRayJob(apiJob, computeTemplateMap) - newRayJob, err := r.getRayJobClient(jobNamespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) if err != nil { return nil, util.NewInternalServerError(err, "Failed to create a job for (%s/%s)", jobNamespace, apiJob.JobId) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 88ed53b00b4..342175956c0 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -127,13 +127,13 @@ func FromCrdToApiJobs(jobs []*v1alpha1.RayJob) []*api.RayJob { func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { pbJob := &api.RayJob{ - Entrypoint: job.Spec.Entrypoint, - Metadata: job.Spec.Metadata, - RuntimeEnv: job.Spec.RuntimeEnv, - JobId: job.Spec.JobId, + Entrypoint: job.Spec.Entrypoint, + Metadata: job.Spec.Metadata, + RuntimeEnv: job.Spec.RuntimeEnv, + JobId: job.Spec.JobId, ShutdownAfterJobFinishes: job.Spec.ShutdownAfterJobFinishes, - ClusterSelector: job.Spec.ClusterSelector, - ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), + ClusterSelector: job.Spec.ClusterSelector, + ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), } return pbJob -} \ No newline at end of file +} diff --git a/apiserver/pkg/server/job_server.go b/apiserver/pkg/server/job_server.go index 44ecb0f5b60..3aff9412a1a 100644 --- a/apiserver/pkg/server/job_server.go +++ b/apiserver/pkg/server/job_server.go @@ -18,14 +18,12 @@ func NewRayJobServer(resourceManager *manager.ResourceManager) *RayJobServer { return &RayJobServer{resourceManager: resourceManager} } - // TODO(basasuya) add event outputs type RayJobServer struct { resourceManager *manager.ResourceManager api.UnimplementedRayJobServiceServer } - func (s *RayJobServer) CreateRayJob(ctx context.Context, request *api.CreateRayJobRequest) (*api.RayJob, error) { // use the namespace in the request to override the namespace in the job definition request.Job.Namespace = request.Namespace diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 3089427994b..2bbf9a5b480 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -28,13 +28,13 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem Annotations: buildRayJobAnnotations(apiJob), }, Spec: rayalphaapi.RayJobSpec{ - Entrypoint: apiJob.Entrypoint, - Metadata: apiJob.Metadata, - RuntimeEnv: apiJob.RuntimeEnv, + Entrypoint: apiJob.Entrypoint, + Metadata: apiJob.Metadata, + RuntimeEnv: apiJob.RuntimeEnv, ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, - JobId: apiJob.JobId, - RayClusterSpec: clusterSpec, - ClusterSelector: apiJob.ClusterSelector, + JobId: apiJob.JobId, + RayClusterSpec: clusterSpec, + ClusterSelector: apiJob.ClusterSelector, }, Status: rayalphaapi.RayJobStatus{}, } @@ -60,4 +60,4 @@ func buildRayJobLabels(job *api.RayJob) map[string]string { func buildRayJobAnnotations(cluster *api.RayJob) map[string]string { annotations := map[string]string{} return annotations -} \ No newline at end of file +} From 18e293206f90ec81cda4657da58c4187ea9cb631 Mon Sep 17 00:00:00 2001 From: Basasuya Date: Wed, 27 Jul 2022 18:03:43 +0800 Subject: [PATCH 3/7] fix for review --- apiserver/cmd/main.go | 2 +- apiserver/pkg/client/cluster.go | 4 +- apiserver/pkg/client/job.go | 4 +- apiserver/pkg/manager/client_manager.go | 4 +- apiserver/pkg/manager/resource_manager.go | 15 +- apiserver/pkg/model/converter.go | 6 + apiserver/pkg/server/job_server.go | 20 +- apiserver/pkg/util/cluster.go | 20 +- apiserver/pkg/util/job.go | 2 +- proto/go_client/job.pb.go | 288 ++++++++++++---------- proto/hack/generate.sh | 2 +- proto/job.proto | 9 +- proto/kuberay_api.swagger.json | 41 ++- proto/swagger/job.swagger.json | 39 ++- 14 files changed, 289 insertions(+), 167 deletions(-) diff --git a/apiserver/cmd/main.go b/apiserver/cmd/main.go index 2889553eab3..b9ac99c11f2 100644 --- a/apiserver/cmd/main.go +++ b/apiserver/cmd/main.go @@ -58,7 +58,7 @@ func startRpcServer(resourceManager *manager.ResourceManager) { grpc.MaxRecvMsgSize(math.MaxInt32)) api.RegisterClusterServiceServer(s, server.NewClusterServer(resourceManager, &server.ClusterServerOptions{CollectMetrics: *collectMetricsFlag})) api.RegisterComputeTemplateServiceServer(s, server.NewComputeTemplateServer(resourceManager, &server.ComputeTemplateServerOptions{CollectMetrics: *collectMetricsFlag})) - api.RegisterRayJobServiceServer(s, server.NewRayJobServer(resourceManager)) + api.RegisterRayJobServiceServer(s, server.NewRayJobServer(resourceManager, &server.JobServerOptions{CollectMetrics: *collectMetricsFlag})) // Register reflection service on gRPC server. reflection.Register(s) diff --git a/apiserver/pkg/client/cluster.go b/apiserver/pkg/client/cluster.go index ea562ecea24..79fd853d43b 100644 --- a/apiserver/pkg/client/cluster.go +++ b/apiserver/pkg/client/cluster.go @@ -5,7 +5,7 @@ import ( "github.com/golang/glog" "github.com/ray-project/kuberay/apiserver/pkg/util" - rayclusterclient "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned" + rayclient "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned" rayiov1alpha1 "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/typed/ray/v1alpha1" "sigs.k8s.io/controller-runtime/pkg/client/config" ) @@ -30,6 +30,6 @@ func NewRayClusterClientOrFatal(initConnectionTimeout time.Duration, options uti cfg.QPS = options.QPS cfg.Burst = options.Burst - rayClusterClient := rayclusterclient.NewForConfigOrDie(cfg).RayV1alpha1() + rayClusterClient := rayclient.NewForConfigOrDie(cfg).RayV1alpha1() return &RayClusterClient{client: rayClusterClient} } diff --git a/apiserver/pkg/client/job.go b/apiserver/pkg/client/job.go index 2effe2d4e23..3bfeadf66bc 100644 --- a/apiserver/pkg/client/job.go +++ b/apiserver/pkg/client/job.go @@ -5,7 +5,7 @@ import ( "github.com/golang/glog" "github.com/ray-project/kuberay/apiserver/pkg/util" - rayclusterclient "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned" + rayclient "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned" rayiov1alpha1 "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/typed/ray/v1alpha1" "sigs.k8s.io/controller-runtime/pkg/client/config" ) @@ -30,6 +30,6 @@ func NewRayJobClientOrFatal(initConnectionTimeout time.Duration, options util.Cl cfg.QPS = options.QPS cfg.Burst = options.Burst - rayJobClient := rayclusterclient.NewForConfigOrDie(cfg).RayV1alpha1() + rayJobClient := rayclient.NewForConfigOrDie(cfg).RayV1alpha1() return &RayJobClient{client: rayJobClient} } diff --git a/apiserver/pkg/manager/client_manager.go b/apiserver/pkg/manager/client_manager.go index cd4c9155c23..7f8ee01b3ca 100644 --- a/apiserver/pkg/manager/client_manager.go +++ b/apiserver/pkg/manager/client_manager.go @@ -10,7 +10,7 @@ import ( type ClientManagerInterface interface { ClusterClient() client.ClusterClientInterface - RayJobClient() client.JobClientInterface + JobClient() client.JobClientInterface KubernetesClient() client.KubernetesClientInterface Time() util.TimeInterface } @@ -29,7 +29,7 @@ func (c *ClientManager) ClusterClient() client.ClusterClientInterface { return c.clusterClient } -func (c *ClientManager) RayJobClient() client.JobClientInterface { +func (c *ClientManager) JobClient() client.JobClientInterface { return c.jobClient } diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 02ec166ce09..9ba0d812bf6 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -30,10 +30,11 @@ type ResourceManagerInterface interface { GetComputeTemplate(ctx context.Context, name string, namespace string) (*v1.ConfigMap, error) ListComputeTemplates(ctx context.Context, namespace string) ([]*v1.ConfigMap, error) DeleteComputeTemplate(ctx context.Context, name string, namespace string) error - CreateJob(ctx context.Context, apiCluster *api.Cluster) (*v1alpha1.RayJob, error) + CreateJob(ctx context.Context, apiJob *api.RayJob) (*v1alpha1.RayJob, error) GetJob(ctx context.Context, jobName string, namespace string) (*v1alpha1.RayJob, error) ListJobs(ctx context.Context, namespace string) ([]*v1alpha1.RayJob, error) ListAllJobs(ctx context.Context) ([]*v1alpha1.RayJob, error) + DeleteJob(ctx context.Context, jobName string, namespace string) error } type ResourceManager struct { @@ -53,7 +54,7 @@ func (r *ResourceManager) getRayClusterClient(namespace string) rayiov1alpha1.Ra } func (r *ResourceManager) getRayJobClient(namespace string) rayiov1alpha1.RayJobInterface { - return r.clientManager.RayJobClient().RayJobClient(namespace) + return r.clientManager.JobClient().RayJobClient(namespace) } func (r *ResourceManager) getKubernetesConfigMapClient(namespace string) clientv1.ConfigMapInterface { @@ -189,24 +190,24 @@ func (r *ResourceManager) DeleteCluster(ctx context.Context, clusterName string, return nil } -func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob, jobNamespace string) (*v1alpha1.RayJob, error) { +func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob) (*v1alpha1.RayJob, error) { computeTemplateMap := make(map[string]*api.ComputeTemplate) var err error if apiJob.ClusterSpec != nil && len(apiJob.ClusterSelector) == 0 { // populate cluster map - computeTemplateMap, err = r.populateComputeTemplate(ctx, apiJob.ClusterSpec, jobNamespace) + computeTemplateMap, err = r.populateComputeTemplate(ctx, apiJob.ClusterSpec, apiJob.Namespace) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", jobNamespace, apiJob.JobId) + return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiJob.Namespace, apiJob.JobId) } } // convert *api.Cluster to v1alpha1.RayCluster rayJob := util.NewRayJob(apiJob, computeTemplateMap) - newRayJob, err := r.getRayJobClient(jobNamespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) + newRayJob, err := r.getRayJobClient(apiJob.Namespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a job for (%s/%s)", jobNamespace, apiJob.JobId) + return nil, util.NewInternalServerError(err, "Failed to create a job for (%s/%s)", apiJob.Namespace, apiJob.JobId) } return newRayJob, nil diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 342175956c0..beed613bedf 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -127,6 +127,9 @@ func FromCrdToApiJobs(jobs []*v1alpha1.RayJob) []*api.RayJob { func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { pbJob := &api.RayJob{ + Name: job.Name, + Namespace: job.Namespace, + User: job.Labels[util.RayClusterUserLabelKey], Entrypoint: job.Spec.Entrypoint, Metadata: job.Spec.Metadata, RuntimeEnv: job.Spec.RuntimeEnv, @@ -134,6 +137,9 @@ func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { ShutdownAfterJobFinishes: job.Spec.ShutdownAfterJobFinishes, ClusterSelector: job.Spec.ClusterSelector, ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), + TtlSecondsAfterFinished: *job.Spec.TTLSecondsAfterFinished, + CreatedAt: ×tamp.Timestamp{Seconds: job.CreationTimestamp.Unix()}, + DeleteAt: ×tamp.Timestamp{Seconds: job.DeletionTimestamp.Unix()}, } return pbJob } diff --git a/apiserver/pkg/server/job_server.go b/apiserver/pkg/server/job_server.go index 3aff9412a1a..c7c498fcce9 100644 --- a/apiserver/pkg/server/job_server.go +++ b/apiserver/pkg/server/job_server.go @@ -6,29 +6,32 @@ import ( "github.com/ray-project/kuberay/apiserver/pkg/model" "github.com/ray-project/kuberay/apiserver/pkg/util" api "github.com/ray-project/kuberay/proto/go_client" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "google.golang.org/protobuf/types/known/emptypb" ) +type JobServerOptions struct { + CollectMetrics bool +} + // implements `type RayJobServiceServer interface` in job_grpc.pb.go // RayJobServer is the server API for RayJobServer service. -func NewRayJobServer(resourceManager *manager.ResourceManager) *RayJobServer { - return &RayJobServer{resourceManager: resourceManager} +func NewRayJobServer(resourceManager *manager.ResourceManager, options *JobServerOptions) *RayJobServer { + return &RayJobServer{resourceManager: resourceManager, options: options} } -// TODO(basasuya) add event outputs type RayJobServer struct { resourceManager *manager.ResourceManager + options *JobServerOptions api.UnimplementedRayJobServiceServer } +// Creates a new Ray Job. func (s *RayJobServer) CreateRayJob(ctx context.Context, request *api.CreateRayJobRequest) (*api.RayJob, error) { // use the namespace in the request to override the namespace in the job definition request.Job.Namespace = request.Namespace - job, err := s.resourceManager.CreateJob(ctx, request.Job, request.Namespace) + job, err := s.resourceManager.CreateJob(ctx, request.Job) if err != nil { return nil, util.Wrap(err, "Create Job failed.") } @@ -36,6 +39,7 @@ func (s *RayJobServer) CreateRayJob(ctx context.Context, request *api.CreateRayJ return model.FromCrdToApiJob(job), nil } +// Finds a specific Job by job name. func (s *RayJobServer) GetRayJob(ctx context.Context, request *api.GetRayJobRequest) (*api.RayJob, error) { if request.Name == "" { return nil, util.NewInvalidInputError("job name is empty. Please specify a valid value.") @@ -51,9 +55,9 @@ func (s *RayJobServer) GetRayJob(ctx context.Context, request *api.GetRayJobRequ } return model.FromCrdToApiJob(job), nil - return nil, status.Errorf(codes.Unimplemented, "method GetRayJob not implemented") } +// Finds all Jobs in a given namespace. func (s *RayJobServer) ListRayJobs(ctx context.Context, request *api.ListRayJobsRequest) (*api.ListRayJobsResponse, error) { if request.Namespace == "" { return nil, util.NewInvalidInputError("job namespace is empty. Please specify a valid value.") @@ -69,6 +73,7 @@ func (s *RayJobServer) ListRayJobs(ctx context.Context, request *api.ListRayJobs }, nil } +// Finds all Jobs in all namespaces. func (s *RayJobServer) ListAllRayJobs(ctx context.Context, request *api.ListAllRayJobsRequest) (*api.ListAllRayJobsResponse, error) { jobs, err := s.resourceManager.ListAllJobs(ctx) if err != nil { @@ -80,6 +85,7 @@ func (s *RayJobServer) ListAllRayJobs(ctx context.Context, request *api.ListAllR }, nil } +// Deletes an Job func (s *RayJobServer) DeleteRayJob(ctx context.Context, request *api.DeleteRayJobRequest) (*emptypb.Empty, error) { if request.Name == "" { return nil, util.NewInvalidInputError("job name is empty. Please specify a valid value.") diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index c8ac18f011e..debb58d495b 100644 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -6,20 +6,20 @@ import ( "strconv" api "github.com/ray-project/kuberay/proto/go_client" - rayclusterapi "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + rayalphaapi "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) type RayCluster struct { - *rayclusterapi.RayCluster + *rayalphaapi.RayCluster } // NewRayCluster creates a RayCluster. // func NewRayCluster(apiCluster *api.Cluster, clusterRuntime *api.ClusterRuntime, computeRuntime *api.ComputeRuntime) *RayCluster { func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) *RayCluster { - rayCluster := &rayclusterapi.RayCluster{ + rayCluster := &rayalphaapi.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Name: apiCluster.Name, Namespace: apiCluster.Namespace, @@ -49,19 +49,19 @@ func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { return annotations } -func buildRayClusterSpec(imageVersion string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) *rayclusterapi.RayClusterSpec { +func buildRayClusterSpec(imageVersion string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) *rayalphaapi.RayClusterSpec { computeTemplate := computeTemplateMap[clusterSpec.HeadGroupSpec.ComputeTemplate] headPodTemplate := buildHeadPodTemplate(imageVersion, clusterSpec.HeadGroupSpec, computeTemplate) headReplicas := int32(1) - rayClusterSpec := &rayclusterapi.RayClusterSpec{ + rayClusterSpec := &rayalphaapi.RayClusterSpec{ RayVersion: imageVersion, - HeadGroupSpec: rayclusterapi.HeadGroupSpec{ + HeadGroupSpec: rayalphaapi.HeadGroupSpec{ ServiceType: v1.ServiceType(clusterSpec.HeadGroupSpec.ServiceType), Template: headPodTemplate, Replicas: &headReplicas, RayStartParams: clusterSpec.HeadGroupSpec.RayStartParams, }, - WorkerGroupSpecs: []rayclusterapi.WorkerGroupSpec{}, + WorkerGroupSpecs: []rayalphaapi.WorkerGroupSpec{}, } for _, spec := range clusterSpec.WorkerGroupSpec { @@ -77,7 +77,7 @@ func buildRayClusterSpec(imageVersion string, clusterSpec *api.ClusterSpec, comp maxReplicas = spec.MaxReplicas } - workerNodeSpec := rayclusterapi.WorkerGroupSpec{ + workerNodeSpec := rayalphaapi.WorkerGroupSpec{ GroupName: spec.GroupName, MinReplicas: intPointer(minReplicas), MaxReplicas: intPointer(maxReplicas), @@ -392,8 +392,8 @@ func intPointer(value int32) *int32 { return &value } -// Get converts this object to a rayclusterapi.Workflow. -func (c *RayCluster) Get() *rayclusterapi.RayCluster { +// Get converts this object to a rayalphaapi.Workflow. +func (c *RayCluster) Get() *rayalphaapi.RayCluster { return c.RayCluster } diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 2bbf9a5b480..72b69e27df3 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -32,11 +32,11 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem Metadata: apiJob.Metadata, RuntimeEnv: apiJob.RuntimeEnv, ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, + TTLSecondsAfterFinished: &apiJob.TtlSecondsAfterFinished, JobId: apiJob.JobId, RayClusterSpec: clusterSpec, ClusterSelector: apiJob.ClusterSelector, }, - Status: rayalphaapi.RayJobStatus{}, } return &RayJob{ diff --git a/proto/go_client/job.pb.go b/proto/go_client/job.pb.go index b00e33dff6f..0f675763b15 100644 --- a/proto/go_client/job.pb.go +++ b/proto/go_client/job.pb.go @@ -12,6 +12,7 @@ import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" emptypb "google.golang.org/protobuf/types/known/emptypb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" ) @@ -400,8 +401,12 @@ type RayJob struct { ClusterSelector map[string]string `protobuf:"bytes,9,rep,name=cluster_selector,json=clusterSelector,proto3" json:"cluster_selector,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // The cluster template ClusterSpec *ClusterSpec `protobuf:"bytes,10,opt,name=cluster_spec,json=clusterSpec,proto3" json:"cluster_spec,omitempty"` - // logs - Logs string `protobuf:"bytes,11,opt,name=logs,proto3" json:"logs,omitempty"` + // TTLSecondsAfterFinished is the TTL to clean up RayCluster. + TtlSecondsAfterFinished int32 `protobuf:"varint,11,opt,name=ttl_seconds_after_finished,json=ttlSecondsAfterFinished,proto3" json:"ttl_seconds_after_finished,omitempty"` + // Output. The time that the job created. + CreatedAt *timestamppb.Timestamp `protobuf:"bytes,12,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` + // Output. The time that the job deleted. + DeleteAt *timestamppb.Timestamp `protobuf:"bytes,13,opt,name=delete_at,json=deleteAt,proto3" json:"delete_at,omitempty"` } func (x *RayJob) Reset() { @@ -506,11 +511,25 @@ func (x *RayJob) GetClusterSpec() *ClusterSpec { return nil } -func (x *RayJob) GetLogs() string { +func (x *RayJob) GetTtlSecondsAfterFinished() int32 { if x != nil { - return x.Logs + return x.TtlSecondsAfterFinished } - return "" + return 0 +} + +func (x *RayJob) GetCreatedAt() *timestamppb.Timestamp { + if x != nil { + return x.CreatedAt + } + return nil +} + +func (x *RayJob) GetDeleteAt() *timestamppb.Timestamp { + if x != nil { + return x.DeleteAt + } + return nil } var File_job_proto protoreflect.FileDescriptor @@ -520,115 +539,127 @@ var file_job_proto_rawDesc = []byte{ 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x6f, 0x70, 0x65, 0x6e, 0x61, 0x70, - 0x69, 0x76, 0x32, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x61, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0d, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x54, 0x0a, 0x13, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x03, 0x6a, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, - 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x22, 0x44, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, - 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x32, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, - 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, - 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x38, 0x0a, 0x13, - 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x6f, 0x70, 0x65, 0x6e, 0x61, + 0x70, 0x69, 0x76, 0x32, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x61, 0x6e, 0x6e, + 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0d, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x54, 0x0a, + 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x03, 0x6a, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, - 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, - 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, - 0x3b, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x47, 0x0a, 0x13, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xb6, 0x04, 0x0a, 0x06, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, - 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x6e, 0x74, 0x72, - 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, - 0x1f, 0x0a, 0x0b, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x65, 0x6e, 0x76, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x45, 0x6e, 0x76, - 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x53, 0x68, 0x75, 0x74, 0x64, - 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, 0x73, - 0x68, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x53, 0x68, 0x75, 0x74, 0x64, - 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, 0x73, - 0x68, 0x65, 0x73, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x43, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x0f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x12, 0x35, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x70, - 0x65, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, 0x63, 0x6c, - 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6c, 0x6f, 0x67, - 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x1a, 0x3b, 0x0a, - 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x43, 0x6c, + 0x52, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x22, 0x44, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x32, 0x0a, 0x12, 0x4c, 0x69, 0x73, + 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x38, 0x0a, + 0x13, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, + 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x41, + 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0x3b, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, + 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, + 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x47, 0x0a, + 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xd3, 0x05, 0x0a, 0x06, 0x52, 0x61, 0x79, 0x4a, 0x6f, + 0x62, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x65, 0x6e, 0x76, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x45, 0x6e, + 0x76, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x53, 0x68, 0x75, 0x74, + 0x64, 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, + 0x73, 0x68, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x53, 0x68, 0x75, 0x74, + 0x64, 0x6f, 0x77, 0x6e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4a, 0x6f, 0x62, 0x46, 0x69, 0x6e, 0x69, + 0x73, 0x68, 0x65, 0x73, 0x12, 0x4d, 0x0a, 0x10, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, + 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, - 0x04, 0x0a, 0x0d, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x12, 0x5b, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, - 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, - 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x20, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x1a, 0x22, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, - 0x68, 0x61, 0x32, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x6e, 0x0a, - 0x09, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, - 0x6f, 0x62, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, - 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x78, 0x0a, - 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x61, 0x70, - 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x6a, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x41, - 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x72, 0x79, 0x52, 0x0f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x12, 0x35, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, + 0x70, 0x65, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3b, 0x0a, 0x1a, 0x74, 0x74, + 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, + 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, 0x52, 0x17, + 0x74, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x41, 0x66, 0x74, 0x65, 0x72, 0x46, + 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, + 0x41, 0x74, 0x12, 0x37, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x74, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, 0x04, 0x0a, + 0x0d, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5b, + 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x1a, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, + 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x1a, 0x22, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, + 0x32, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x6e, 0x0a, 0x09, 0x47, + 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, + 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, + 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x78, 0x0a, 0x0b, 0x4c, + 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x61, 0x70, 0x69, 0x73, + 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, + 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x6a, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, - 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, - 0x6f, 0x62, 0x73, 0x12, 0x7d, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, - 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x2a, - 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, - 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, - 0x65, 0x7d, 0x42, 0x54, 0x5a, 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x72, 0x61, 0x79, 0x2d, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x6b, 0x75, 0x62, - 0x65, 0x72, 0x61, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x5f, 0x63, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x92, 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, - 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, 0x2f, 0x61, + 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, 0x6f, 0x62, + 0x73, 0x12, 0x7d, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, + 0x62, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x2a, 0x31, 0x2f, + 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, + 0x42, 0x54, 0x5a, 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, + 0x61, 0x79, 0x2d, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, + 0x61, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x5f, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x92, 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, + 0x75, 0x6c, 0x74, 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -656,7 +687,8 @@ var file_job_proto_goTypes = []interface{}{ nil, // 8: proto.RayJob.MetadataEntry nil, // 9: proto.RayJob.ClusterSelectorEntry (*ClusterSpec)(nil), // 10: proto.ClusterSpec - (*emptypb.Empty)(nil), // 11: google.protobuf.Empty + (*timestamppb.Timestamp)(nil), // 11: google.protobuf.Timestamp + (*emptypb.Empty)(nil), // 12: google.protobuf.Empty } var file_job_proto_depIdxs = []int32{ 7, // 0: proto.CreateRayJobRequest.job:type_name -> proto.RayJob @@ -665,21 +697,23 @@ var file_job_proto_depIdxs = []int32{ 8, // 3: proto.RayJob.metadata:type_name -> proto.RayJob.MetadataEntry 9, // 4: proto.RayJob.cluster_selector:type_name -> proto.RayJob.ClusterSelectorEntry 10, // 5: proto.RayJob.cluster_spec:type_name -> proto.ClusterSpec - 0, // 6: proto.RayJobService.CreateRayJob:input_type -> proto.CreateRayJobRequest - 1, // 7: proto.RayJobService.GetRayJob:input_type -> proto.GetRayJobRequest - 2, // 8: proto.RayJobService.ListRayJobs:input_type -> proto.ListRayJobsRequest - 4, // 9: proto.RayJobService.ListAllRayJobs:input_type -> proto.ListAllRayJobsRequest - 6, // 10: proto.RayJobService.DeleteRayJob:input_type -> proto.DeleteRayJobRequest - 7, // 11: proto.RayJobService.CreateRayJob:output_type -> proto.RayJob - 7, // 12: proto.RayJobService.GetRayJob:output_type -> proto.RayJob - 3, // 13: proto.RayJobService.ListRayJobs:output_type -> proto.ListRayJobsResponse - 5, // 14: proto.RayJobService.ListAllRayJobs:output_type -> proto.ListAllRayJobsResponse - 11, // 15: proto.RayJobService.DeleteRayJob:output_type -> google.protobuf.Empty - 11, // [11:16] is the sub-list for method output_type - 6, // [6:11] is the sub-list for method input_type - 6, // [6:6] is the sub-list for extension type_name - 6, // [6:6] is the sub-list for extension extendee - 0, // [0:6] is the sub-list for field type_name + 11, // 6: proto.RayJob.created_at:type_name -> google.protobuf.Timestamp + 11, // 7: proto.RayJob.delete_at:type_name -> google.protobuf.Timestamp + 0, // 8: proto.RayJobService.CreateRayJob:input_type -> proto.CreateRayJobRequest + 1, // 9: proto.RayJobService.GetRayJob:input_type -> proto.GetRayJobRequest + 2, // 10: proto.RayJobService.ListRayJobs:input_type -> proto.ListRayJobsRequest + 4, // 11: proto.RayJobService.ListAllRayJobs:input_type -> proto.ListAllRayJobsRequest + 6, // 12: proto.RayJobService.DeleteRayJob:input_type -> proto.DeleteRayJobRequest + 7, // 13: proto.RayJobService.CreateRayJob:output_type -> proto.RayJob + 7, // 14: proto.RayJobService.GetRayJob:output_type -> proto.RayJob + 3, // 15: proto.RayJobService.ListRayJobs:output_type -> proto.ListRayJobsResponse + 5, // 16: proto.RayJobService.ListAllRayJobs:output_type -> proto.ListAllRayJobsResponse + 12, // 17: proto.RayJobService.DeleteRayJob:output_type -> google.protobuf.Empty + 13, // [13:18] is the sub-list for method output_type + 8, // [8:13] is the sub-list for method input_type + 8, // [8:8] is the sub-list for extension type_name + 8, // [8:8] is the sub-list for extension extendee + 0, // [0:8] is the sub-list for field type_name } func init() { file_job_proto_init() } diff --git a/proto/hack/generate.sh b/proto/hack/generate.sh index 828a81e0b9c..4c92406e0a9 100755 --- a/proto/hack/generate.sh +++ b/proto/hack/generate.sh @@ -28,4 +28,4 @@ jq -s 'reduce .[] as $item ({}; . * $item) | .info.title = "KubeRay API" | .info /go/src/github.com/ray-project/kuberay/proto/swagger/config.swagger.json \ /go/src/github.com/ray-project/kuberay/proto/swagger/error.swagger.json \ /go/src/github.com/ray-project/kuberay/proto/swagger/job.swagger.json \ - > "/go/src/github.com/ray-project/kuberay/proto/kuberay_api.swagger.json" \ No newline at end of file + > "/go/src/github.com/ray-project/kuberay/proto/kuberay_api.swagger.json" diff --git a/proto/job.proto b/proto/job.proto index b8b14c0820a..1136e2d7ab1 100644 --- a/proto/job.proto +++ b/proto/job.proto @@ -5,6 +5,7 @@ package proto; import "google/api/annotations.proto"; import "google/protobuf/empty.proto"; +import "google/protobuf/timestamp.proto"; import "protoc-gen-openapiv2/options/annotations.proto"; import "cluster.proto"; @@ -122,6 +123,10 @@ message RayJob { map cluster_selector = 9; // The cluster template ClusterSpec cluster_spec = 10; - // logs - string logs = 11; + // TTLSecondsAfterFinished is the TTL to clean up RayCluster. + int32 ttl_seconds_after_finished = 11; + // Output. The time that the job created. + google.protobuf.Timestamp created_at = 12; + // Output. The time that the job deleted. + google.protobuf.Timestamp delete_at = 13; } diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index b183fe1f0dc..943e956b87d 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -196,7 +196,7 @@ }, "/apis/v1alpha2/compute_templates": { "get": { - "summary": "Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields.", + "summary": "Finds all compute templates in all namespaces. Supports pagination, and sorting on certain fields.", "operationId": "ComputeTemplateService_ListAllComputeTemplates", "responses": { "200": { @@ -696,6 +696,24 @@ "default": "DEV", "description": "Optional field." }, + "VolumeHostPathType": { + "type": "string", + "enum": [ + "DIRECTORY", + "FILE" + ], + "default": "DIRECTORY", + "description": "If indicate hostpath, we need to let user indicate which type \nthey would like to use." + }, + "VolumeMountPropagationMode": { + "type": "string", + "enum": [ + "NONE", + "HOSTTOCONTAINER", + "BIDIRECTIONAL" + ], + "default": "NONE" + }, "VolumeVolumeType": { "type": "string", "enum": [ @@ -911,6 +929,12 @@ }, "readOnly": { "type": "boolean" + }, + "hostPathType": { + "$ref": "#/definitions/VolumeHostPathType" + }, + "mountPropagationMode": { + "$ref": "#/definitions/VolumeMountPropagationMode" } } }, @@ -1167,9 +1191,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "The cluster template" }, - "logs": { + "ttlSecondsAfterFinished": { + "type": "integer", + "format": "int32", + "description": "TTLSecondsAfterFinished is the TTL to clean up RayCluster." + }, + "createdAt": { "type": "string", - "title": "logs" + "format": "date-time", + "description": "Output. The time that the job created." + }, + "deleteAt": { + "type": "string", + "format": "date-time", + "description": "Output. The time that the job deleted." } }, "title": "RayJob" diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json index 8d741b05f0a..63d49e3452e 100644 --- a/proto/swagger/job.swagger.json +++ b/proto/swagger/job.swagger.json @@ -191,6 +191,24 @@ } }, "definitions": { + "VolumeHostPathType": { + "type": "string", + "enum": [ + "DIRECTORY", + "FILE" + ], + "default": "DIRECTORY", + "description": "If indicate hostpath, we need to let user indicate which type \nthey would like to use." + }, + "VolumeMountPropagationMode": { + "type": "string", + "enum": [ + "NONE", + "HOSTTOCONTAINER", + "BIDIRECTIONAL" + ], + "default": "NONE" + }, "VolumeVolumeType": { "type": "string", "enum": [ @@ -335,9 +353,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "The cluster template" }, - "logs": { + "ttlSecondsAfterFinished": { + "type": "integer", + "format": "int32", + "description": "TTLSecondsAfterFinished is the TTL to clean up RayCluster." + }, + "createdAt": { "type": "string", - "title": "logs" + "format": "date-time", + "description": "Output. The time that the job created." + }, + "deleteAt": { + "type": "string", + "format": "date-time", + "description": "Output. The time that the job deleted." } }, "title": "RayJob" @@ -359,6 +388,12 @@ }, "readOnly": { "type": "boolean" + }, + "hostPathType": { + "$ref": "#/definitions/VolumeHostPathType" + }, + "mountPropagationMode": { + "$ref": "#/definitions/VolumeMountPropagationMode" } } }, From 5292b77d1e97b42c598cf660bd9874db7ec6f893 Mon Sep 17 00:00:00 2001 From: Basasuya Date: Wed, 27 Jul 2022 18:38:20 +0800 Subject: [PATCH 4/7] fix --- apiserver/pkg/model/converter.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index beed613bedf..4d7180bd9e3 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -127,9 +127,9 @@ func FromCrdToApiJobs(jobs []*v1alpha1.RayJob) []*api.RayJob { func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { pbJob := &api.RayJob{ - Name: job.Name, - Namespace: job.Namespace, - User: job.Labels[util.RayClusterUserLabelKey], + Name: job.Name, + Namespace: job.Namespace, + User: job.Labels[util.RayClusterUserLabelKey], Entrypoint: job.Spec.Entrypoint, Metadata: job.Spec.Metadata, RuntimeEnv: job.Spec.RuntimeEnv, @@ -137,9 +137,9 @@ func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { ShutdownAfterJobFinishes: job.Spec.ShutdownAfterJobFinishes, ClusterSelector: job.Spec.ClusterSelector, ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), - TtlSecondsAfterFinished: *job.Spec.TTLSecondsAfterFinished, - CreatedAt: ×tamp.Timestamp{Seconds: job.CreationTimestamp.Unix()}, - DeleteAt: ×tamp.Timestamp{Seconds: job.DeletionTimestamp.Unix()}, + TtlSecondsAfterFinished: *job.Spec.TTLSecondsAfterFinished, + CreatedAt: ×tamp.Timestamp{Seconds: job.CreationTimestamp.Unix()}, + DeleteAt: ×tamp.Timestamp{Seconds: job.DeletionTimestamp.Unix()}, } return pbJob } From 40f292eacfff48d56b6c52858f9bfbab41f279ac Mon Sep 17 00:00:00 2001 From: Basasuya Date: Wed, 27 Jul 2022 18:44:16 +0800 Subject: [PATCH 5/7] fix for lint --- apiserver/pkg/server/job_server.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/server/job_server.go b/apiserver/pkg/server/job_server.go index c7c498fcce9..175f31382c8 100644 --- a/apiserver/pkg/server/job_server.go +++ b/apiserver/pkg/server/job_server.go @@ -2,6 +2,7 @@ package server import ( "context" + "github.com/ray-project/kuberay/apiserver/pkg/manager" "github.com/ray-project/kuberay/apiserver/pkg/model" "github.com/ray-project/kuberay/apiserver/pkg/util" From 37dc0420b3bdc8e1f0fbbed742e4c7653e668699 Mon Sep 17 00:00:00 2001 From: Basasuya Date: Fri, 29 Jul 2022 00:05:00 +0800 Subject: [PATCH 6/7] fix for review --- apiserver/deploy/base/apiserver.yaml | 1 + apiserver/pkg/model/converter.go | 31 +++++- apiserver/pkg/server/job_server.go | 47 +++++++++ apiserver/pkg/util/job.go | 5 +- proto/go_client/job.pb.go | 138 +++++++++++++++++---------- proto/job.proto | 6 ++ proto/kuberay_api.swagger.json | 12 +++ proto/swagger/job.swagger.json | 12 +++ 8 files changed, 192 insertions(+), 60 deletions(-) diff --git a/apiserver/deploy/base/apiserver.yaml b/apiserver/deploy/base/apiserver.yaml index 1a2b9ba5f5c..d9c6fcd59ca 100644 --- a/apiserver/deploy/base/apiserver.yaml +++ b/apiserver/deploy/base/apiserver.yaml @@ -83,6 +83,7 @@ rules: - ray.io resources: - rayclusters + - rayjobs verbs: - create - delete diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 4d7180bd9e3..40d70c32936 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -2,6 +2,7 @@ package model import ( "fmt" + "github.com/golang/glog" "strconv" "github.com/golang/protobuf/ptypes/timestamp" @@ -125,21 +126,41 @@ func FromCrdToApiJobs(jobs []*v1alpha1.RayJob) []*api.RayJob { return apiJobs } -func FromCrdToApiJob(job *v1alpha1.RayJob) *api.RayJob { - pbJob := &api.RayJob{ +func FromCrdToApiJob(job *v1alpha1.RayJob) (pbJob *api.RayJob) { + defer func() { + err := recover() + if err != nil { + glog.Errorf("failed to transfer ray job, err: %v, item: %v", err, job) + } + }() + + var ttl int32 = -1 + if job.Spec.TTLSecondsAfterFinished != nil { + ttl = *job.Spec.TTLSecondsAfterFinished + } + + var deleteTime int64 = -1 + if job.DeletionTimestamp != nil { + deleteTime = job.DeletionTimestamp.Unix() + } + + pbJob = &api.RayJob{ Name: job.Name, Namespace: job.Namespace, User: job.Labels[util.RayClusterUserLabelKey], Entrypoint: job.Spec.Entrypoint, Metadata: job.Spec.Metadata, RuntimeEnv: job.Spec.RuntimeEnv, - JobId: job.Spec.JobId, + JobId: job.Status.JobId, ShutdownAfterJobFinishes: job.Spec.ShutdownAfterJobFinishes, ClusterSelector: job.Spec.ClusterSelector, ClusterSpec: PopulateRayClusterSpec(job.Spec.RayClusterSpec), - TtlSecondsAfterFinished: *job.Spec.TTLSecondsAfterFinished, + TtlSecondsAfterFinished: ttl, CreatedAt: ×tamp.Timestamp{Seconds: job.CreationTimestamp.Unix()}, - DeleteAt: ×tamp.Timestamp{Seconds: job.DeletionTimestamp.Unix()}, + DeleteAt: ×tamp.Timestamp{Seconds: deleteTime}, + JobStatus: string(job.Status.JobStatus), + JobDeploymentStatus: string(job.Status.JobDeploymentStatus), + Message: job.Status.Message, } return pbJob } diff --git a/apiserver/pkg/server/job_server.go b/apiserver/pkg/server/job_server.go index 175f31382c8..3b6ddb2ee0e 100644 --- a/apiserver/pkg/server/job_server.go +++ b/apiserver/pkg/server/job_server.go @@ -29,6 +29,10 @@ type RayJobServer struct { // Creates a new Ray Job. func (s *RayJobServer) CreateRayJob(ctx context.Context, request *api.CreateRayJobRequest) (*api.RayJob, error) { + if err := ValidateCreateJobRequest(request); err != nil { + return nil, util.Wrap(err, "Validate job request failed.") + } + // use the namespace in the request to override the namespace in the job definition request.Job.Namespace = request.Namespace @@ -102,3 +106,46 @@ func (s *RayJobServer) DeleteRayJob(ctx context.Context, request *api.DeleteRayJ return &emptypb.Empty{}, nil } + +func ValidateCreateJobRequest(request *api.CreateRayJobRequest) error { + if request.Namespace == "" { + return util.NewInvalidInputError("Namespace is empty. Please specify a valid value.") + } + + if request.Namespace != request.Job.Namespace { + return util.NewInvalidInputError("The namespace in the request is different from the namespace in the job definition.") + } + + if request.Job.Name == "" { + return util.NewInvalidInputError("Job name is empty. Please specify a valid value.") + } + + if request.Job.User == "" { + return util.NewInvalidInputError("User who create the job is empty. Please specify a valid value.") + } + + if len(request.Job.ClusterSelector) != 0 { + return nil + } + + if len(request.Job.ClusterSpec.HeadGroupSpec.ComputeTemplate) == 0 { + return util.NewInvalidInputError("HeadGroupSpec compute template is empty. Please specify a valid value.") + } + + for index, spec := range request.Job.ClusterSpec.WorkerGroupSpec { + if len(spec.GroupName) == 0 { + return util.NewInvalidInputError("WorkerNodeSpec %d group name is empty. Please specify a valid value.", index) + } + if len(spec.ComputeTemplate) == 0 { + return util.NewInvalidInputError("WorkerNodeSpec %d compute template is empty. Please specify a valid value.", index) + } + if spec.MaxReplicas == 0 { + return util.NewInvalidInputError("WorkerNodeSpec %d MaxReplicas can not be 0. Please specify a valid value.", index) + } + if spec.MinReplicas > spec.MaxReplicas { + return util.NewInvalidInputError("WorkerNodeSpec %d MinReplica > MaxReplicas. Please specify a valid value.", index) + } + } + + return nil +} diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 72b69e27df3..37998ddbcbf 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -57,7 +57,6 @@ func buildRayJobLabels(job *api.RayJob) map[string]string { return labels } -func buildRayJobAnnotations(cluster *api.RayJob) map[string]string { - annotations := map[string]string{} - return annotations +func buildRayJobAnnotations(job *api.RayJob) map[string]string { + return job.Metadata } diff --git a/proto/go_client/job.pb.go b/proto/go_client/job.pb.go index 0f675763b15..7a8b412b1cd 100644 --- a/proto/go_client/job.pb.go +++ b/proto/go_client/job.pb.go @@ -407,6 +407,12 @@ type RayJob struct { CreatedAt *timestamppb.Timestamp `protobuf:"bytes,12,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` // Output. The time that the job deleted. DeleteAt *timestamppb.Timestamp `protobuf:"bytes,13,opt,name=delete_at,json=deleteAt,proto3" json:"delete_at,omitempty"` + // Output. The current job status + JobStatus string `protobuf:"bytes,14,opt,name=job_status,json=jobStatus,proto3" json:"job_status,omitempty"` + // Output. The current job deployment status + JobDeploymentStatus string `protobuf:"bytes,15,opt,name=job_deployment_status,json=jobDeploymentStatus,proto3" json:"job_deployment_status,omitempty"` + // Output. A human-readable description of the status of this operation. + Message string `protobuf:"bytes,16,opt,name=message,proto3" json:"message,omitempty"` } func (x *RayJob) Reset() { @@ -532,6 +538,27 @@ func (x *RayJob) GetDeleteAt() *timestamppb.Timestamp { return nil } +func (x *RayJob) GetJobStatus() string { + if x != nil { + return x.JobStatus + } + return "" +} + +func (x *RayJob) GetJobDeploymentStatus() string { + if x != nil { + return x.JobDeploymentStatus + } + return "" +} + +func (x *RayJob) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + var File_job_proto protoreflect.FileDescriptor var file_job_proto_rawDesc = []byte{ @@ -572,7 +599,7 @@ var file_job_proto_rawDesc = []byte{ 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xd3, 0x05, 0x0a, 0x06, 0x52, 0x61, 0x79, 0x4a, 0x6f, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xc0, 0x06, 0x0a, 0x06, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, @@ -609,57 +636,64 @@ var file_job_proto_rawDesc = []byte{ 0x41, 0x74, 0x12, 0x37, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x74, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, 0x04, 0x0a, - 0x0d, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5b, - 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x1a, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, - 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, - 0x1a, 0x22, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, - 0x32, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x6e, 0x0a, 0x09, 0x47, - 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x47, 0x65, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, - 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, - 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x78, 0x0a, 0x0b, 0x4c, - 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x32, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x61, 0x70, 0x69, 0x73, - 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, - 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x6a, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, - 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, 0x2f, 0x61, - 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, 0x6f, 0x62, - 0x73, 0x12, 0x7d, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, - 0x62, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x2a, 0x31, 0x2f, - 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, - 0x42, 0x54, 0x5a, 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, - 0x61, 0x79, 0x2d, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, - 0x61, 0x79, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x5f, 0x63, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x92, 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, - 0x75, 0x6c, 0x74, 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x70, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x41, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6a, + 0x6f, 0x62, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x6a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x6a, 0x6f, + 0x62, 0x5f, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x6a, 0x6f, 0x62, 0x44, 0x65, + 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, + 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xc1, 0x04, 0x0a, 0x0d, 0x52, 0x61, + 0x79, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5b, 0x0a, 0x0c, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1a, 0x22, 0x13, + 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, + 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x6e, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x52, + 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x47, 0x65, + 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x22, 0x39, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, + 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, 0x62, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x78, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, + 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, + 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2c, 0x12, 0x2a, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, + 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x7d, 0x2f, 0x6a, 0x6f, + 0x62, 0x73, 0x12, 0x6a, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, + 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x1c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x41, 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, + 0x6c, 0x6c, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x73, + 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x7d, + 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, 0x4a, 0x6f, 0x62, 0x12, 0x1a, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x61, 0x79, + 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x22, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x2a, 0x31, 0x2f, 0x61, 0x70, 0x69, + 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x7d, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x42, 0x54, 0x5a, + 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x61, 0x79, 0x2d, + 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x6b, 0x75, 0x62, 0x65, 0x72, 0x61, 0x79, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x92, + 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/proto/job.proto b/proto/job.proto index 1136e2d7ab1..e0850f9f870 100644 --- a/proto/job.proto +++ b/proto/job.proto @@ -129,4 +129,10 @@ message RayJob { google.protobuf.Timestamp created_at = 12; // Output. The time that the job deleted. google.protobuf.Timestamp delete_at = 13; + // Output. The current job status + string job_status = 14; + // Output. The current job deployment status + string job_deployment_status = 15; + // Output. A human-readable description of the status of this operation. + string message = 16; } diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index 943e956b87d..9867a8f9767 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -1205,6 +1205,18 @@ "type": "string", "format": "date-time", "description": "Output. The time that the job deleted." + }, + "jobStatus": { + "type": "string", + "title": "Output. The current job status" + }, + "jobDeploymentStatus": { + "type": "string", + "title": "Output. The current job deployment status" + }, + "message": { + "type": "string", + "description": "Output. A human-readable description of the status of this operation." } }, "title": "RayJob" diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json index 63d49e3452e..16a9dad8839 100644 --- a/proto/swagger/job.swagger.json +++ b/proto/swagger/job.swagger.json @@ -367,6 +367,18 @@ "type": "string", "format": "date-time", "description": "Output. The time that the job deleted." + }, + "jobStatus": { + "type": "string", + "title": "Output. The current job status" + }, + "jobDeploymentStatus": { + "type": "string", + "title": "Output. The current job deployment status" + }, + "message": { + "type": "string", + "description": "Output. A human-readable description of the status of this operation." } }, "title": "RayJob" From 960dbafd51d321e294e656373b8433adde92af30 Mon Sep 17 00:00:00 2001 From: Basasuya Date: Fri, 29 Jul 2022 19:47:27 +0800 Subject: [PATCH 7/7] fix UT / add encoding for runtime env --- apiserver/pkg/model/converter.go | 3 ++- apiserver/pkg/util/job.go | 9 ++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 40d70c32936..53e52bf9a8e 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -2,9 +2,10 @@ package model import ( "fmt" - "github.com/golang/glog" "strconv" + "github.com/golang/glog" + "github.com/golang/protobuf/ptypes/timestamp" "github.com/ray-project/kuberay/apiserver/pkg/util" api "github.com/ray-project/kuberay/proto/go_client" diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 37998ddbcbf..33b2539f6b3 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -1,6 +1,9 @@ package util import ( + "encoding/base64" + + "github.com/golang/glog" api "github.com/ray-project/kuberay/proto/go_client" rayalphaapi "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -20,6 +23,10 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem clusterSpec = *buildRayClusterSpec(rayJobDefaultVersion, apiJob.ClusterSpec, computeTemplateMap) } + // transfer json to runtimeEnv + encodedText := base64.StdEncoding.EncodeToString([]byte(apiJob.RuntimeEnv)) + glog.Infof(encodedText) + rayJob := &rayalphaapi.RayJob{ ObjectMeta: metav1.ObjectMeta{ Name: apiJob.Name, @@ -30,7 +37,7 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem Spec: rayalphaapi.RayJobSpec{ Entrypoint: apiJob.Entrypoint, Metadata: apiJob.Metadata, - RuntimeEnv: apiJob.RuntimeEnv, + RuntimeEnv: encodedText, ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, TTLSecondsAfterFinished: &apiJob.TtlSecondsAfterFinished, JobId: apiJob.JobId,