diff --git a/apiserver/README.md b/apiserver/README.md index 9e741b4d74..c91ee87975 100644 --- a/apiserver/README.md +++ b/apiserver/README.md @@ -6,14 +6,16 @@ KubeRay APIServer provides the gRPC and HTTP API to manage kuberay resources. ### Compute Template -#### Create compute templates +#### Create compute templates in a given namespace + ``` -POST {{baseUrl}}/apis/v1alpha1/compute_templates +POST {{baseUrl}}/apis/v1alpha2/namespaces//compute_templates ``` ``` { "name": "default-template", + "namespace": "", "cpu": 2, "memory": 4, "gpu": 1, @@ -21,18 +23,18 @@ POST {{baseUrl}}/apis/v1alpha1/compute_templates } ``` -#### List all compute templates +#### List all compute templates in a given namespace ``` -GET {{baseUrl}}/apis/v1alpha1/compute_templates +GET {{baseUrl}}/apis/v1alpha2/namespaces//compute_templates ``` ``` { "compute_templates": [ { - "id": "", "name": "default-template", + "namespace": "", "cpu": 2, "memory": 4, "gpu": 1, @@ -42,16 +44,22 @@ GET {{baseUrl}}/apis/v1alpha1/compute_templates } ``` +#### List all compute templates in all namespaces + +``` +GET {{baseUrl}}/apis/v1alpha2/compute_templates +``` + #### Get compute template by name ``` -GET {{baseUrl}}/apis/v1alpha1/compute_templates/?name= +GET {{baseUrl}}/apis/v1alpha2/namespaces//compute_templates/ ``` #### Delete compute template by name ``` -DELETE {{baseUrl}}/apis/v1alpha1/compute_templates/?name= +DELETE {{baseUrl}}/apis/v1alpha2/namespaces//compute_templates/ ``` ### Clusters diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index b9d3ac8dd1..40cc9e7206 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -26,9 +26,9 @@ type ResourceManagerInterface interface { ListAllClusters(ctx context.Context) ([]*v1alpha1.RayCluster, error) DeleteCluster(ctx context.Context, clusterName string, namespace string) error CreateComputeTemplate(ctx context.Context, runtime *api.ComputeTemplate) (*v1.ConfigMap, error) - GetComputeTemplate(ctx context.Context, name string) (*v1.ConfigMap, error) - ListComputeTemplates(ctx context.Context) ([]*v1.ConfigMap, error) - DeleteComputeTemplate(ctx context.Context, name string) error + 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 } type ResourceManager struct { @@ -82,7 +82,7 @@ func (r *ResourceManager) populateComputeTemplate(ctx context.Context, cluster * dict := map[string]*api.ComputeTemplate{} // populate head compute template name := cluster.ClusterSpec.HeadGroupSpec.ComputeTemplate - configMap, err := r.GetComputeTemplate(ctx, name) + configMap, err := r.GetComputeTemplate(ctx, name, cluster.Namespace) if err != nil { return nil, err } @@ -93,7 +93,7 @@ func (r *ResourceManager) populateComputeTemplate(ctx context.Context, cluster * for _, spec := range cluster.ClusterSpec.WorkerGroupSepc { name := spec.ComputeTemplate if _, exist := dict[name]; !exist { - configMap, err := r.GetComputeTemplate(ctx, name) + configMap, err := r.GetComputeTemplate(ctx, name, cluster.Namespace) if err != nil { return nil, err } @@ -164,32 +164,32 @@ func (r *ResourceManager) DeleteCluster(ctx context.Context, clusterName string, // Compute Runtimes func (r *ResourceManager) CreateComputeTemplate(ctx context.Context, runtime *api.ComputeTemplate) (*v1.ConfigMap, error) { - _, err := r.GetComputeTemplate(ctx, runtime.Name) + _, err := r.GetComputeTemplate(ctx, runtime.Name, runtime.Namespace) if err == nil { - return nil, util.NewAlreadyExistError("Compute template with name %s already exists in namespace %s", runtime.Name, DefaultNamespace) + return nil, util.NewAlreadyExistError("Compute template with name %s already exists in namespace %s", runtime.Name, runtime.Namespace) } - computeTemplate, err := util.NewComputeTemplate(runtime, DefaultNamespace) + computeTemplate, err := util.NewComputeTemplate(runtime) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to convert compute runtime (%s/%s)", DefaultNamespace, runtime.Name) + return nil, util.NewInternalServerError(err, "Failed to convert compute runtime (%s/%s)", runtime.Namespace, runtime.Name) } - client := r.getKubernetesConfigMapClient(DefaultNamespace) + client := r.getKubernetesConfigMapClient(runtime.Namespace) newRuntime, err := client.Create(ctx, computeTemplate, metav1.CreateOptions{}) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a compute runtime for (%s/%s)", DefaultNamespace, runtime.Name) + return nil, util.NewInternalServerError(err, "Failed to create a compute runtime for (%s/%s)", runtime.Namespace, runtime.Name) } return newRuntime, nil } -func (r *ResourceManager) GetComputeTemplate(ctx context.Context, name string) (*v1.ConfigMap, error) { - client := r.getKubernetesConfigMapClient(DefaultNamespace) +func (r *ResourceManager) GetComputeTemplate(ctx context.Context, name string, namespace string) (*v1.ConfigMap, error) { + client := r.getKubernetesConfigMapClient(namespace) return getComputeTemplateByName(ctx, client, name) } -func (r *ResourceManager) ListComputeTemplates(ctx context.Context) ([]*v1.ConfigMap, error) { - client := r.getKubernetesConfigMapClient(DefaultNamespace) +func (r *ResourceManager) ListComputeTemplates(ctx context.Context, namespace string) ([]*v1.ConfigMap, error) { + client := r.getKubernetesConfigMapClient(namespace) configMapList, err := client.List(ctx, metav1.ListOptions{LabelSelector: "ray.io/config-type=compute-template"}) if err != nil { return nil, util.Wrap(err, "List compute runtimes failed") @@ -204,8 +204,30 @@ func (r *ResourceManager) ListComputeTemplates(ctx context.Context) ([]*v1.Confi return result, nil } -func (r *ResourceManager) DeleteComputeTemplate(ctx context.Context, name string) error { - client := r.getKubernetesConfigMapClient(DefaultNamespace) +func (r *ResourceManager) ListAllComputeTemplates(ctx context.Context) ([]*v1.ConfigMap, 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 []*v1.ConfigMap + for _, namespace := range namespaces.Items { + client := r.getKubernetesConfigMapClient(namespace.Name) + configMapList, err := client.List(ctx, metav1.ListOptions{LabelSelector: "ray.io/config-type=compute-template"}) + if err != nil { + return nil, util.Wrap(err, fmt.Sprintf("List compute templates failed in %s", namespace.Name)) + } + + length := len(configMapList.Items) + for i := 0; i < length; i++ { + result = append(result, &configMapList.Items[i]) + } + } + return result, nil +} + +func (r *ResourceManager) DeleteComputeTemplate(ctx context.Context, name string, namespace string) error { + client := r.getKubernetesConfigMapClient(namespace) configMap, err := getComputeTemplateByName(ctx, client, name) if err != nil { diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 085bd5b2ce..bd33d7c852 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -74,6 +74,7 @@ func FromKubeToAPIComputeTemplate(configMap *v1.ConfigMap) *api.ComputeTemplate runtime := &api.ComputeTemplate{} runtime.Name = configMap.Name + runtime.Namespace = configMap.Namespace runtime.Cpu = uint32(cpu) runtime.Memory = uint32(memory) runtime.Gpu = uint32(gpu) diff --git a/apiserver/pkg/server/compute_template_server.go b/apiserver/pkg/server/compute_template_server.go index 00ec5980e3..27f920f305 100644 --- a/apiserver/pkg/server/compute_template_server.go +++ b/apiserver/pkg/server/compute_template_server.go @@ -2,6 +2,7 @@ package server import ( "context" + "fmt" "github.com/ray-project/kuberay/apiserver/pkg/manager" "github.com/ray-project/kuberay/apiserver/pkg/model" @@ -24,30 +25,45 @@ type ComputeTemplateServer struct { func (s *ComputeTemplateServer) CreateComputeTemplate(ctx context.Context, request *api.CreateComputeTemplateRequest) (*api.ComputeTemplate, error) { if err := ValidateCreateComputeTemplateRequest(request); err != nil { - return nil, util.Wrap(err, "Validate compute runtime request failed.") + return nil, util.Wrap(err, "Validate compute template runtime request failed.") } + // use the namespace in the request to override the namespace in the compute template definition + request.ComputeTemplate.Namespace = request.Namespace + runtime, err := s.resourceManager.CreateComputeTemplate(ctx, request.ComputeTemplate) if err != nil { - return nil, util.Wrap(err, "Create Compute Runtime failed.") + return nil, util.Wrap(err, "Create compute template failed.") } return model.FromKubeToAPIComputeTemplate(runtime), nil } func (s *ComputeTemplateServer) GetComputeTemplate(ctx context.Context, request *api.GetComputeTemplateRequest) (*api.ComputeTemplate, error) { - runtime, err := s.resourceManager.GetComputeTemplate(ctx, request.Name) + if request.Name == "" { + return nil, util.NewInvalidInputError("Compute template name is empty. Please specify a valid value.") + } + + if request.Namespace == "" { + return nil, util.NewInvalidInputError("Namespace is empty. Please specify a valid value.") + } + + runtime, err := s.resourceManager.GetComputeTemplate(ctx, request.Name, request.Namespace) if err != nil { - return nil, util.Wrap(err, "Get cluster runtime failed.") + return nil, util.Wrap(err, "Get compute template failed.") } return model.FromKubeToAPIComputeTemplate(runtime), nil } func (s *ComputeTemplateServer) ListComputeTemplates(ctx context.Context, request *api.ListComputeTemplatesRequest) (*api.ListComputeTemplatesResponse, error) { - runtimes, err := s.resourceManager.ListComputeTemplates(ctx) + if request.Namespace == "" { + return nil, util.NewInvalidInputError("Namespace is empty. Please specify a valid value.") + } + + runtimes, err := s.resourceManager.ListComputeTemplates(ctx, request.Namespace) if err != nil { - return nil, util.Wrap(err, "List cluster runtime failed.") + return nil, util.Wrap(err, fmt.Sprintf("List compute templates in namespace %s failed.", request.Namespace)) } return &api.ListComputeTemplatesResponse{ @@ -55,8 +71,27 @@ func (s *ComputeTemplateServer) ListComputeTemplates(ctx context.Context, reques }, nil } +func (s *ComputeTemplateServer) ListAllComputeTemplates(ctx context.Context, request *api.ListAllComputeTemplatesRequest) (*api.ListAllComputeTemplatesResponse, error) { + runtimes, err := s.resourceManager.ListAllComputeTemplates(ctx) + if err != nil { + return nil, util.Wrap(err, "List all compute templates from all namespaces failed.") + } + + return &api.ListAllComputeTemplatesResponse{ + ComputeTemplates: model.FromKubeToAPIComputeTemplates(runtimes), + }, nil +} + func (s *ComputeTemplateServer) DeleteComputeTemplate(ctx context.Context, request *api.DeleteComputeTemplateRequest) (*emptypb.Empty, error) { - if err := s.resourceManager.DeleteComputeTemplate(ctx, request.Name); err != nil { + if request.Name == "" { + return nil, util.NewInvalidInputError("Compute template name is empty. Please specify a valid value.") + } + + if request.Namespace == "" { + return nil, util.NewInvalidInputError("Namespace is empty. Please specify a valid value.") + } + + if err := s.resourceManager.DeleteComputeTemplate(ctx, request.Name, request.Namespace); err != nil { return nil, err } @@ -64,8 +99,12 @@ func (s *ComputeTemplateServer) DeleteComputeTemplate(ctx context.Context, reque } func ValidateCreateComputeTemplateRequest(request *api.CreateComputeTemplateRequest) error { + if request.Namespace == "" { + return util.NewInvalidInputError("Namespace is empty. Please specify a valid value.") + } + if request.ComputeTemplate.Name == "" { - return util.NewInvalidInputError("Cluster name is empty. Please specify a valid value.") + return util.NewInvalidInputError("Compute template name is empty. Please specify a valid value.") } if request.ComputeTemplate.Cpu == 0 { diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index 4a8f14e393..aef46b810c 100644 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -320,11 +320,11 @@ func (c *RayCluster) SetAnnotationsToAllTemplates(key string, value string) { // TODO: reserved for common parameters. } -func NewComputeTemplate(runtime *api.ComputeTemplate, namespace string) (*v1.ConfigMap, error) { +func NewComputeTemplate(runtime *api.ComputeTemplate) (*v1.ConfigMap, error) { config := &v1.ConfigMap{ ObjectMeta: metav1.ObjectMeta{ Name: runtime.Name, - Namespace: namespace, + Namespace: runtime.Namespace, Labels: map[string]string{ "ray.io/config-type": "compute-template", "ray.io/compute-template": runtime.Name, @@ -332,6 +332,7 @@ func NewComputeTemplate(runtime *api.ComputeTemplate, namespace string) (*v1.Con }, Data: map[string]string{ "name": runtime.Name, + "namespace": runtime.Namespace, "cpu": strconv.FormatUint(uint64(runtime.Cpu), 10), "memory": strconv.FormatUint(uint64(runtime.Memory), 10), "gpu": strconv.FormatUint(uint64(runtime.Gpu), 10), diff --git a/cli/README.md b/cli/README.md index e0d5584c4f..4d9220e1ac 100644 --- a/cli/README.md +++ b/cli/README.md @@ -95,17 +95,18 @@ Flags: --gpu-accelerator string GPU Accelerator type --memory uint32 ray pod memory in GB (default 1) --name string name of the compute template + -n, --namespace string kubernetes namespace where the compute template will be stored ``` #### Get a Ray Compute Template -`./kuberay template compute get ` +`./kuberay template compute get -n ` #### List Ray Compute Templates -`./kuberay template compute list` +`./kuberay template compute list -n ` #### Delete a Ray Compute Template -`./kuberay template compute delete ` +`./kuberay template compute delete -n ` ## End to end example @@ -119,8 +120,8 @@ kubectl port-forward svc/kuberay-apiserver-service 8887:8887 -n ray-system Create compute templates ``` -./kuberay template compute create --cpu 2 --memory 4 --name "worker-template" -./kuberay template compute create --cpu 1 --memory 2 --name "head-template" +./kuberay template compute create -n --cpu 2 --memory 4 --name "worker-template" +./kuberay template compute create -n --cpu 1 --memory 2 --name "head-template" ``` List compute templates created @@ -132,7 +133,7 @@ List compute templates created Create the cluster ``` -./kuberay cluster create --name test-cluster --user jiaxin.shan \ +./kuberay cluster create -n --name test-cluster --user jiaxin.shan \ --head-compute-template head-template \ --head-image rayproject/ray:1.9.2 \ --worker-group-name small-wg \ diff --git a/cli/pkg/cmd/template/compute/create.go b/cli/pkg/cmd/template/compute/create.go index d58d9a6549..fb73e3912d 100644 --- a/cli/pkg/cmd/template/compute/create.go +++ b/cli/pkg/cmd/template/compute/create.go @@ -13,6 +13,7 @@ import ( type CreateOptions struct { name string + namespace string cpu uint32 memory uint32 gpu uint32 @@ -31,12 +32,15 @@ func newCmdCreate() *cobra.Command { return createComputeTemplate(opts) }, } - + cmd.Flags().StringVarP(&opts.namespace, "namespace", "n", "", "kubernetes namespace where the compute template will be stored") cmd.Flags().StringVar(&opts.name, "name", "", "name of the compute template") cmd.Flags().Uint32Var(&opts.cpu, "cpu", 1, "ray pod CPU") cmd.Flags().Uint32Var(&opts.memory, "memory", 1, "ray pod memory in GB") cmd.Flags().Uint32Var(&opts.gpu, "gpu", 0, "ray head GPU") cmd.Flags().StringVar(&opts.gpuAccelerator, "gpu-accelerator", "", "GPU Accelerator type") + if err := cmd.MarkFlagRequired("namespace"); err != nil { + klog.Warning(err) + } if err := cmd.MarkFlagRequired("name"); err != nil { klog.Warning(err) } @@ -58,6 +62,7 @@ func createComputeTemplate(opts CreateOptions) error { computeTemplate := &go_client.ComputeTemplate{ Name: opts.name, + Namespace: opts.namespace, Cpu: opts.cpu, Memory: opts.memory, Gpu: opts.gpu, @@ -65,12 +70,13 @@ func createComputeTemplate(opts CreateOptions) error { } r, err := client.CreateComputeTemplate(ctx, &go_client.CreateComputeTemplateRequest{ + Namespace: opts.namespace, ComputeTemplate: computeTemplate, }) if err != nil { log.Fatalf("could not create compute template %v", err) } - log.Printf("compute template %v is created", r.Id) + log.Printf("compute template %v has been created in %v", r.Name, r.Namespace) return nil } diff --git a/cli/pkg/cmd/template/compute/delete.go b/cli/pkg/cmd/template/compute/delete.go index 3c51e46267..7778194e9f 100644 --- a/cli/pkg/cmd/template/compute/delete.go +++ b/cli/pkg/cmd/template/compute/delete.go @@ -8,22 +8,35 @@ import ( "github.com/ray-project/kuberay/cli/pkg/cmdutil" "github.com/ray-project/kuberay/proto/go_client" "github.com/spf13/cobra" + "k8s.io/klog/v2" ) +type DeleteOptions struct { + namespace string +} + func newCmdDelete() *cobra.Command { + opts := DeleteOptions{} + cmd := &cobra.Command{ Use: "delete ", Short: "Delete a compute template by name", Args: cobra.MaximumNArgs(1), RunE: func(cmd *cobra.Command, args []string) error { - return deleteComputeTemplate(args[0]) + return deleteComputeTemplate(args[0], opts) }, } + cmd.Flags().StringVarP(&opts.namespace, "namespace", "n", "", + "kubernetes namespace where the compute template is stored") + if err := cmd.MarkFlagRequired("namespace"); err != nil { + klog.Warning(err) + } + return cmd } -func deleteComputeTemplate(name string) error { +func deleteComputeTemplate(name string, opts DeleteOptions) error { // Get gRPC connection conn, err := cmdutil.GetGrpcConn() if err != nil { @@ -37,7 +50,8 @@ func deleteComputeTemplate(name string) error { defer cancel() request := &go_client.DeleteComputeTemplateRequest{ - Name: name, + Name: name, + Namespace: opts.namespace, } if _, err := client.DeleteComputeTemplate(ctx, request); err != nil { log.Fatalf("could not delete compute template %v", err) diff --git a/cli/pkg/cmd/template/compute/get.go b/cli/pkg/cmd/template/compute/get.go index 8acff083f4..62007a62e3 100644 --- a/cli/pkg/cmd/template/compute/get.go +++ b/cli/pkg/cmd/template/compute/get.go @@ -10,22 +10,35 @@ import ( "github.com/ray-project/kuberay/cli/pkg/cmdutil" "github.com/ray-project/kuberay/proto/go_client" "github.com/spf13/cobra" + "k8s.io/klog/v2" ) +type GetOptions struct { + namespace string +} + func newCmdGet() *cobra.Command { + opts := GetOptions{} + cmd := &cobra.Command{ Use: "get ", Short: "Get a compute template by name", Args: cobra.MaximumNArgs(1), RunE: func(cmd *cobra.Command, args []string) error { - return getComputeTemplate(args[0]) + return getComputeTemplate(args[0], opts) }, } + cmd.Flags().StringVarP(&opts.namespace, "namespace", "n", "", + "kubernetes namespace where the compute template is stored") + if err := cmd.MarkFlagRequired("namespace"); err != nil { + klog.Warning(err) + } + return cmd } -func getComputeTemplate(name string) error { +func getComputeTemplate(name string, opts GetOptions) error { // Get gRPC connection conn, err := cmdutil.GetGrpcConn() if err != nil { @@ -39,7 +52,8 @@ func getComputeTemplate(name string) error { defer cancel() r, err := client.GetComputeTemplate(ctx, &go_client.GetComputeTemplateRequest{ - Name: name, + Name: name, + Namespace: opts.namespace, }) if err != nil { log.Fatalf("could not list compute template %v", err) diff --git a/cli/pkg/cmd/template/compute/list.go b/cli/pkg/cmd/template/compute/list.go index f0415a2b48..58a7e64558 100644 --- a/cli/pkg/cmd/template/compute/list.go +++ b/cli/pkg/cmd/template/compute/list.go @@ -11,22 +11,35 @@ import ( "github.com/ray-project/kuberay/cli/pkg/cmdutil" "github.com/ray-project/kuberay/proto/go_client" "github.com/spf13/cobra" + "k8s.io/klog/v2" ) +type ListOptions struct { + namespace string +} + func newCmdList() *cobra.Command { + opts := ListOptions{} + cmd := &cobra.Command{ Use: "list", Short: "List all compute templates", Args: cobra.NoArgs, RunE: func(cmd *cobra.Command, args []string) error { - return listComputeTemplates() + return listComputeTemplates(opts) }, } + cmd.Flags().StringVarP(&opts.namespace, "namespace", "n", "", + "kubernetes namespace where the compute template is stored") + if err := cmd.MarkFlagRequired("namespace"); err != nil { + klog.Warning(err) + } + return cmd } -func listComputeTemplates() error { +func listComputeTemplates(opts ListOptions) error { // Get gRPC connection conn, err := cmdutil.GetGrpcConn() if err != nil { @@ -39,7 +52,9 @@ func listComputeTemplates() error { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() - r, err := client.ListComputeTemplates(ctx, &go_client.ListComputeTemplatesRequest{}) + r, err := client.ListComputeTemplates(ctx, &go_client.ListComputeTemplatesRequest{ + Namespace: opts.namespace, + }) if err != nil { log.Fatalf("could not list compute template %v", err) } diff --git a/proto/config.proto b/proto/config.proto index dd911a034b..a88b9568de 100644 --- a/proto/config.proto +++ b/proto/config.proto @@ -23,51 +23,60 @@ option (grpc.gateway.protoc_gen_openapiv2.options.openapiv2_swagger) = { }; service ComputeTemplateService { - // Creates a new Cluster. + // Creates a new compute template. rpc CreateComputeTemplate(CreateComputeTemplateRequest) returns (ComputeTemplate) { option (google.api.http) = { - post: "/apis/v1alpha1/compute_templates" + post: "/apis/v1alpha2/compute_templates" body: "compute_template" }; } - // Finds a specific Cluster by ID. + // Finds a specific compute template by its name and namespace. rpc GetComputeTemplate(GetComputeTemplateRequest) returns (ComputeTemplate) { option (google.api.http) = { - get: "/apis/v1alpha1/compute_templates/{id}" + get: "/apis/v1alpha2/namespaces/{namespace}/compute_templates/{name}" }; } - // Finds all Clusters. Supports pagination, and sorting on certain fields. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. rpc ListComputeTemplates(ListComputeTemplatesRequest) returns (ListComputeTemplatesResponse) { option (google.api.http) = { - get: "/apis/v1alpha1/compute_templates" + get: "/apis/v1alpha2/namespaces/{namespace}/compute_templates" }; } - // Deletes an Cluster without deleting the Cluster's runs and jobs. To - // avoid unexpected behaviors, delete an Cluster's runs and jobs before - // deleting the Cluster. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + rpc ListAllComputeTemplates(ListAllComputeTemplatesRequest) returns (ListAllComputeTemplatesResponse) { + option (google.api.http) = { + get: "/apis/v1alpha2/compute_templates" + }; + } + + // Deletes a compuate template by its name and namespace rpc DeleteComputeTemplate(DeleteComputeTemplateRequest) returns (google.protobuf.Empty) { option (google.api.http) = { - delete: "/apis/v1alpha1/compute_templates/{id}" + delete: "/apis/v1alpha2/namespaces/{namespace}/compute_templates/{name}" }; } } message CreateComputeTemplateRequest { - // The ComputeTemplate to be created. + // The compute template to be created. ComputeTemplate compute_template = 1; + // The namespace of the compute template to be created + string namespace = 2; } message GetComputeTemplateRequest { - // The ID of the ComputeTemplate to be retrieved. reserved field for database layer query later - string id = 1; // The name of the ComputeTemplate to be retrieved. - string name = 2; + string name = 1; + // The namespace of the compute template to be retrieved. + string namespace = 2; } message ListComputeTemplatesRequest { + // The namespace of the compute templates to be retrieved. + string namespace = 1; // TODO: support paganation later } @@ -75,19 +84,27 @@ message ListComputeTemplatesResponse { repeated ComputeTemplate compute_templates = 1; } +message ListAllComputeTemplatesRequest { + // TODO: support paganation later +} + +message ListAllComputeTemplatesResponse { + repeated ComputeTemplate compute_templates = 1; +} + message DeleteComputeTemplateRequest { - // The ID of the ComputeTemplate to be deleted. - string id = 1; - // The name of the ComputeTemplate to be deleted. - string name = 2; + // The name of the compute template to be deleted. + string name = 1; + // The namespace of the compute template to be deleted. + string namespace = 2; } // ComputeTemplate can be reused by any compute units like worker group, workspace, image build job, etc message ComputeTemplate { - // The ID of the compute template - string id = 1; // The name of the compute template - string name = 2; + string name = 1; + // The namespace of the compute template + string namespace = 2; // Number of cpus uint32 cpu = 3; // Number of memory @@ -103,7 +120,7 @@ service ImageTemplateService { // Creates a new ImageTemplate. rpc CreateImageTemplate(CreateImageTemplateRequest) returns (ImageTemplate) { option (google.api.http) = { - post: "/apis/v1alpha1/image_templates" + post: "/apis/v1alpha2/image_templates" body: "image_template" }; } @@ -111,38 +128,42 @@ service ImageTemplateService { // Finds a specific ImageTemplate by ID. rpc GetImageTemplate(GetImageTemplateRequest) returns (ImageTemplate) { option (google.api.http) = { - get: "/apis/v1alpha1/image_templates/{id}" + get: "/apis/v1alpha2/namespaces/{namespace}/image_templates/{name}" }; } // Finds all ImageTemplates. Supports pagination, and sorting on certain fields. rpc ListImageTemplates(ListImageTemplatesRequest) returns (ListImageTemplatesResponse) { option (google.api.http) = { - get: "/apis/v1alpha1/image_templates" + get: "/apis/v1alpha2/namespaces/{namespace}/image_templates" }; } // Deletes an ImageTemplate. rpc DeleteImageTemplate(DeleteImageTemplateRequest) returns (google.protobuf.Empty) { option (google.api.http) = { - delete: "/apis/v1alpha1/image_templates/{id}" + delete: "/apis/v1alpha2/namespaces/{namespace}/image_templates/{name}" }; } } message CreateImageTemplateRequest { - // The Compute to be created. + // The image template to be created. ImageTemplate image_template = 1; + // The namespace of the image template to be created. + string namespace = 2; } message GetImageTemplateRequest { - // The ID of the ImageTemplate to be retrieved. reserved field for database layer query later - string id = 1; - // The name of the ImageTemplate to be retrieved. - string name = 2; + // The name of the image template to be retrieved. + string name = 1; + // The namespace of the image template to be retrieved. + string namespace = 2; } message ListImageTemplatesRequest { + // The namespace of the image templates to be retrieved. + string namespace = 1; // TODO: support pagingation later } @@ -151,11 +172,20 @@ message ListImageTemplatesResponse { repeated ImageTemplate image_templates = 1; } +message ListAllImageTemplatesRequest { + // TODO: support pagingation later +} + +message ListAllImageTemplatesResponse { + // A list of Compute returned. + repeated ImageTemplate image_templates = 1; +} + message DeleteImageTemplateRequest { - // The ID of the ImageTemplate to be deleted. - string id = 1; - // The name of the ImageTemplate to be delete. - string name = 2; + // The name of the image template to be deleted. + string name = 1; + // The namespace of the image template to be deleted. + string namespace = 2; } // ImageTemplate can be used by worker group and workspce. @@ -163,18 +193,20 @@ message DeleteImageTemplateRequest { message ImageTemplate { // The ID of the image template string name = 1; + // The namespace of the image template + string namespace = 2; // The base container image to be used for image building - string base_image = 2; + string base_image = 3; // The pip packages to install - repeated string pip_packages = 3; + repeated string pip_packages = 4; // The conda packages to install - repeated string conda_packages = 4; + repeated string conda_packages = 5; // The system packages to install - repeated string system_packages = 5; + repeated string system_packages = 6; // The environment variables to set - map environment_variables = 6; + map environment_variables = 7; // The post install commands to execute - string custom_commands = 7; + string custom_commands = 8; // Output. The result image generated string image = 9; } diff --git a/proto/go_client/config.pb.go b/proto/go_client/config.pb.go index c847836484..c5cb8a0cc2 100644 --- a/proto/go_client/config.pb.go +++ b/proto/go_client/config.pb.go @@ -28,8 +28,10 @@ type CreateComputeTemplateRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ComputeTemplate to be created. + // The compute template to be created. ComputeTemplate *ComputeTemplate `protobuf:"bytes,1,opt,name=compute_template,json=computeTemplate,proto3" json:"compute_template,omitempty"` + // The namespace of the compute template to be created + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` } func (x *CreateComputeTemplateRequest) Reset() { @@ -71,15 +73,22 @@ func (x *CreateComputeTemplateRequest) GetComputeTemplate() *ComputeTemplate { return nil } +func (x *CreateComputeTemplateRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + type GetComputeTemplateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ID of the ComputeTemplate to be retrieved. reserved field for database layer query later - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // The name of the ComputeTemplate to be retrieved. - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + 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 *GetComputeTemplateRequest) Reset() { @@ -114,16 +123,16 @@ func (*GetComputeTemplateRequest) Descriptor() ([]byte, []int) { return file_config_proto_rawDescGZIP(), []int{1} } -func (x *GetComputeTemplateRequest) GetId() string { +func (x *GetComputeTemplateRequest) GetName() string { if x != nil { - return x.Id + return x.Name } return "" } -func (x *GetComputeTemplateRequest) GetName() string { +func (x *GetComputeTemplateRequest) GetNamespace() string { if x != nil { - return x.Name + return x.Namespace } return "" } @@ -132,6 +141,9 @@ type ListComputeTemplatesRequest 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 *ListComputeTemplatesRequest) Reset() { @@ -166,6 +178,13 @@ func (*ListComputeTemplatesRequest) Descriptor() ([]byte, []int) { return file_config_proto_rawDescGZIP(), []int{2} } +func (x *ListComputeTemplatesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + type ListComputeTemplatesResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -213,21 +232,106 @@ func (x *ListComputeTemplatesResponse) GetComputeTemplates() []*ComputeTemplate return nil } +type ListAllComputeTemplatesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ListAllComputeTemplatesRequest) Reset() { + *x = ListAllComputeTemplatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_config_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllComputeTemplatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllComputeTemplatesRequest) ProtoMessage() {} + +func (x *ListAllComputeTemplatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_config_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 ListAllComputeTemplatesRequest.ProtoReflect.Descriptor instead. +func (*ListAllComputeTemplatesRequest) Descriptor() ([]byte, []int) { + return file_config_proto_rawDescGZIP(), []int{4} +} + +type ListAllComputeTemplatesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ComputeTemplates []*ComputeTemplate `protobuf:"bytes,1,rep,name=compute_templates,json=computeTemplates,proto3" json:"compute_templates,omitempty"` +} + +func (x *ListAllComputeTemplatesResponse) Reset() { + *x = ListAllComputeTemplatesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_config_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllComputeTemplatesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllComputeTemplatesResponse) ProtoMessage() {} + +func (x *ListAllComputeTemplatesResponse) ProtoReflect() protoreflect.Message { + mi := &file_config_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 ListAllComputeTemplatesResponse.ProtoReflect.Descriptor instead. +func (*ListAllComputeTemplatesResponse) Descriptor() ([]byte, []int) { + return file_config_proto_rawDescGZIP(), []int{5} +} + +func (x *ListAllComputeTemplatesResponse) GetComputeTemplates() []*ComputeTemplate { + if x != nil { + return x.ComputeTemplates + } + return nil +} + type DeleteComputeTemplateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ID of the ComputeTemplate to be deleted. - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - // The name of the ComputeTemplate to be deleted. - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + // 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 *DeleteComputeTemplateRequest) Reset() { *x = DeleteComputeTemplateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[4] + mi := &file_config_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -240,7 +344,7 @@ func (x *DeleteComputeTemplateRequest) String() string { func (*DeleteComputeTemplateRequest) ProtoMessage() {} func (x *DeleteComputeTemplateRequest) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[4] + mi := &file_config_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -253,19 +357,19 @@ func (x *DeleteComputeTemplateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteComputeTemplateRequest.ProtoReflect.Descriptor instead. func (*DeleteComputeTemplateRequest) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{4} + return file_config_proto_rawDescGZIP(), []int{6} } -func (x *DeleteComputeTemplateRequest) GetId() string { +func (x *DeleteComputeTemplateRequest) GetName() string { if x != nil { - return x.Id + return x.Name } return "" } -func (x *DeleteComputeTemplateRequest) GetName() string { +func (x *DeleteComputeTemplateRequest) GetNamespace() string { if x != nil { - return x.Name + return x.Namespace } return "" } @@ -276,10 +380,10 @@ type ComputeTemplate struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ID of the compute template - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // The name of the compute template - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the compute template + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` // Number of cpus Cpu uint32 `protobuf:"varint,3,opt,name=cpu,proto3" json:"cpu,omitempty"` // Number of memory @@ -293,7 +397,7 @@ type ComputeTemplate struct { func (x *ComputeTemplate) Reset() { *x = ComputeTemplate{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[5] + mi := &file_config_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -306,7 +410,7 @@ func (x *ComputeTemplate) String() string { func (*ComputeTemplate) ProtoMessage() {} func (x *ComputeTemplate) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[5] + mi := &file_config_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -319,19 +423,19 @@ func (x *ComputeTemplate) ProtoReflect() protoreflect.Message { // Deprecated: Use ComputeTemplate.ProtoReflect.Descriptor instead. func (*ComputeTemplate) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{5} + return file_config_proto_rawDescGZIP(), []int{7} } -func (x *ComputeTemplate) GetId() string { +func (x *ComputeTemplate) GetName() string { if x != nil { - return x.Id + return x.Name } return "" } -func (x *ComputeTemplate) GetName() string { +func (x *ComputeTemplate) GetNamespace() string { if x != nil { - return x.Name + return x.Namespace } return "" } @@ -369,14 +473,16 @@ type CreateImageTemplateRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The Compute to be created. + // The image template to be created. ImageTemplate *ImageTemplate `protobuf:"bytes,1,opt,name=image_template,json=imageTemplate,proto3" json:"image_template,omitempty"` + // The namespace of the image template to be created. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` } func (x *CreateImageTemplateRequest) Reset() { *x = CreateImageTemplateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[6] + mi := &file_config_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -389,7 +495,7 @@ func (x *CreateImageTemplateRequest) String() string { func (*CreateImageTemplateRequest) ProtoMessage() {} func (x *CreateImageTemplateRequest) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[6] + mi := &file_config_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -402,7 +508,7 @@ func (x *CreateImageTemplateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateImageTemplateRequest.ProtoReflect.Descriptor instead. func (*CreateImageTemplateRequest) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{6} + return file_config_proto_rawDescGZIP(), []int{8} } func (x *CreateImageTemplateRequest) GetImageTemplate() *ImageTemplate { @@ -412,21 +518,28 @@ func (x *CreateImageTemplateRequest) GetImageTemplate() *ImageTemplate { return nil } +func (x *CreateImageTemplateRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + type GetImageTemplateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ID of the ImageTemplate to be retrieved. reserved field for database layer query later - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - // The name of the ImageTemplate to be retrieved. - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + // The name of the image template to be retrieved. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the image template to be retrieved. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` } func (x *GetImageTemplateRequest) Reset() { *x = GetImageTemplateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[7] + mi := &file_config_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -439,7 +552,7 @@ func (x *GetImageTemplateRequest) String() string { func (*GetImageTemplateRequest) ProtoMessage() {} func (x *GetImageTemplateRequest) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[7] + mi := &file_config_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -452,19 +565,19 @@ func (x *GetImageTemplateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetImageTemplateRequest.ProtoReflect.Descriptor instead. func (*GetImageTemplateRequest) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{7} + return file_config_proto_rawDescGZIP(), []int{9} } -func (x *GetImageTemplateRequest) GetId() string { +func (x *GetImageTemplateRequest) GetName() string { if x != nil { - return x.Id + return x.Name } return "" } -func (x *GetImageTemplateRequest) GetName() string { +func (x *GetImageTemplateRequest) GetNamespace() string { if x != nil { - return x.Name + return x.Namespace } return "" } @@ -473,12 +586,15 @@ type ListImageTemplatesRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + + // The namespace of the image templates to be retrieved. + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` // TODO: support pagingation later } func (x *ListImageTemplatesRequest) Reset() { *x = ListImageTemplatesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[8] + mi := &file_config_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -491,7 +607,7 @@ func (x *ListImageTemplatesRequest) String() string { func (*ListImageTemplatesRequest) ProtoMessage() {} func (x *ListImageTemplatesRequest) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[8] + mi := &file_config_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -504,7 +620,14 @@ func (x *ListImageTemplatesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListImageTemplatesRequest.ProtoReflect.Descriptor instead. func (*ListImageTemplatesRequest) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{8} + return file_config_proto_rawDescGZIP(), []int{10} +} + +func (x *ListImageTemplatesRequest) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" } type ListImageTemplatesResponse struct { @@ -519,7 +642,7 @@ type ListImageTemplatesResponse struct { func (x *ListImageTemplatesResponse) Reset() { *x = ListImageTemplatesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[9] + mi := &file_config_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -532,7 +655,7 @@ func (x *ListImageTemplatesResponse) String() string { func (*ListImageTemplatesResponse) ProtoMessage() {} func (x *ListImageTemplatesResponse) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[9] + mi := &file_config_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -545,7 +668,7 @@ func (x *ListImageTemplatesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListImageTemplatesResponse.ProtoReflect.Descriptor instead. func (*ListImageTemplatesResponse) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{9} + return file_config_proto_rawDescGZIP(), []int{11} } func (x *ListImageTemplatesResponse) GetImageTemplates() []*ImageTemplate { @@ -555,21 +678,107 @@ func (x *ListImageTemplatesResponse) GetImageTemplates() []*ImageTemplate { return nil } +type ListAllImageTemplatesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ListAllImageTemplatesRequest) Reset() { + *x = ListAllImageTemplatesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_config_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllImageTemplatesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllImageTemplatesRequest) ProtoMessage() {} + +func (x *ListAllImageTemplatesRequest) ProtoReflect() protoreflect.Message { + mi := &file_config_proto_msgTypes[12] + 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 ListAllImageTemplatesRequest.ProtoReflect.Descriptor instead. +func (*ListAllImageTemplatesRequest) Descriptor() ([]byte, []int) { + return file_config_proto_rawDescGZIP(), []int{12} +} + +type ListAllImageTemplatesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // A list of Compute returned. + ImageTemplates []*ImageTemplate `protobuf:"bytes,1,rep,name=image_templates,json=imageTemplates,proto3" json:"image_templates,omitempty"` +} + +func (x *ListAllImageTemplatesResponse) Reset() { + *x = ListAllImageTemplatesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_config_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListAllImageTemplatesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListAllImageTemplatesResponse) ProtoMessage() {} + +func (x *ListAllImageTemplatesResponse) ProtoReflect() protoreflect.Message { + mi := &file_config_proto_msgTypes[13] + 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 ListAllImageTemplatesResponse.ProtoReflect.Descriptor instead. +func (*ListAllImageTemplatesResponse) Descriptor() ([]byte, []int) { + return file_config_proto_rawDescGZIP(), []int{13} +} + +func (x *ListAllImageTemplatesResponse) GetImageTemplates() []*ImageTemplate { + if x != nil { + return x.ImageTemplates + } + return nil +} + type DeleteImageTemplateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // The ID of the ImageTemplate to be deleted. - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - // The name of the ImageTemplate to be delete. - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + // The name of the image template to be deleted. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the image template to be deleted. + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` } func (x *DeleteImageTemplateRequest) Reset() { *x = DeleteImageTemplateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[10] + mi := &file_config_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -582,7 +791,7 @@ func (x *DeleteImageTemplateRequest) String() string { func (*DeleteImageTemplateRequest) ProtoMessage() {} func (x *DeleteImageTemplateRequest) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[10] + mi := &file_config_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -595,19 +804,19 @@ func (x *DeleteImageTemplateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteImageTemplateRequest.ProtoReflect.Descriptor instead. func (*DeleteImageTemplateRequest) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{10} + return file_config_proto_rawDescGZIP(), []int{14} } -func (x *DeleteImageTemplateRequest) GetId() string { +func (x *DeleteImageTemplateRequest) GetName() string { if x != nil { - return x.Id + return x.Name } return "" } -func (x *DeleteImageTemplateRequest) GetName() string { +func (x *DeleteImageTemplateRequest) GetNamespace() string { if x != nil { - return x.Name + return x.Namespace } return "" } @@ -621,18 +830,20 @@ type ImageTemplate struct { // The ID of the image template Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The namespace of the image template + Namespace string `protobuf:"bytes,2,opt,name=namespace,proto3" json:"namespace,omitempty"` // The base container image to be used for image building - BaseImage string `protobuf:"bytes,2,opt,name=base_image,json=baseImage,proto3" json:"base_image,omitempty"` + BaseImage string `protobuf:"bytes,3,opt,name=base_image,json=baseImage,proto3" json:"base_image,omitempty"` // The pip packages to install - PipPackages []string `protobuf:"bytes,3,rep,name=pip_packages,json=pipPackages,proto3" json:"pip_packages,omitempty"` + PipPackages []string `protobuf:"bytes,4,rep,name=pip_packages,json=pipPackages,proto3" json:"pip_packages,omitempty"` // The conda packages to install - CondaPackages []string `protobuf:"bytes,4,rep,name=conda_packages,json=condaPackages,proto3" json:"conda_packages,omitempty"` + CondaPackages []string `protobuf:"bytes,5,rep,name=conda_packages,json=condaPackages,proto3" json:"conda_packages,omitempty"` // The system packages to install - SystemPackages []string `protobuf:"bytes,5,rep,name=system_packages,json=systemPackages,proto3" json:"system_packages,omitempty"` + SystemPackages []string `protobuf:"bytes,6,rep,name=system_packages,json=systemPackages,proto3" json:"system_packages,omitempty"` // The environment variables to set - EnvironmentVariables map[string]string `protobuf:"bytes,6,rep,name=environment_variables,json=environmentVariables,proto3" json:"environment_variables,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + EnvironmentVariables map[string]string `protobuf:"bytes,7,rep,name=environment_variables,json=environmentVariables,proto3" json:"environment_variables,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // The post install commands to execute - CustomCommands string `protobuf:"bytes,7,opt,name=custom_commands,json=customCommands,proto3" json:"custom_commands,omitempty"` + CustomCommands string `protobuf:"bytes,8,opt,name=custom_commands,json=customCommands,proto3" json:"custom_commands,omitempty"` // Output. The result image generated Image string `protobuf:"bytes,9,opt,name=image,proto3" json:"image,omitempty"` } @@ -640,7 +851,7 @@ type ImageTemplate struct { func (x *ImageTemplate) Reset() { *x = ImageTemplate{} if protoimpl.UnsafeEnabled { - mi := &file_config_proto_msgTypes[11] + mi := &file_config_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -653,7 +864,7 @@ func (x *ImageTemplate) String() string { func (*ImageTemplate) ProtoMessage() {} func (x *ImageTemplate) ProtoReflect() protoreflect.Message { - mi := &file_config_proto_msgTypes[11] + mi := &file_config_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -666,7 +877,7 @@ func (x *ImageTemplate) ProtoReflect() protoreflect.Message { // Deprecated: Use ImageTemplate.ProtoReflect.Descriptor instead. func (*ImageTemplate) Descriptor() ([]byte, []int) { - return file_config_proto_rawDescGZIP(), []int{11} + return file_config_proto_rawDescGZIP(), []int{15} } func (x *ImageTemplate) GetName() string { @@ -676,6 +887,13 @@ func (x *ImageTemplate) GetName() string { return "" } +func (x *ImageTemplate) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + func (x *ImageTemplate) GetBaseImage() string { if x != nil { return x.BaseImage @@ -736,156 +954,205 @@ var file_config_proto_rawDesc = []byte{ 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, - 0x22, 0x61, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, + 0x22, 0x7f, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x41, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, - 0x61, 0x74, 0x65, 0x22, 0x3f, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, - 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x1d, 0x0a, 0x1b, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, - 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x63, 0x0a, 0x1c, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, - 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x11, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, - 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x22, 0x42, 0x0a, 0x1c, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x9a, 0x01, 0x0a, - 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x61, 0x74, 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, 0x4d, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0x3b, 0x0a, 0x1b, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0x63, 0x0a, + 0x1c, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, + 0x11, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, + 0x52, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x22, 0x20, 0x0a, 0x1e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6d, + 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x66, 0x0a, 0x1f, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, + 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x11, 0x63, 0x6f, 0x6d, 0x70, 0x75, + 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, + 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x10, 0x63, 0x6f, 0x6d, 0x70, + 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x22, 0x50, 0x0a, 0x1c, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0xa8, + 0x01, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 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, 0x10, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x70, 0x75, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x67, 0x70, 0x75, 0x12, 0x27, 0x0a, 0x0f, 0x67, 0x70, 0x75, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x70, 0x75, 0x41, 0x63, - 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x22, 0x59, 0x0a, 0x1a, 0x43, 0x72, 0x65, + 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x22, 0x77, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3b, 0x0a, 0x0e, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x22, 0x3d, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, - 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x22, 0x1b, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x22, 0x5b, 0x0a, 0x1a, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, - 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, - 0x0a, 0x0f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x0e, 0x69, - 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x22, 0x40, 0x0a, - 0x1a, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, - 0xa2, 0x03, 0x0a, 0x0d, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, - 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x69, 0x6d, - 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x62, 0x61, 0x73, 0x65, 0x49, - 0x6d, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x70, 0x5f, 0x70, 0x61, 0x63, 0x6b, - 0x61, 0x67, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x69, 0x70, 0x50, - 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x64, 0x61, - 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x0d, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x12, 0x27, - 0x0a, 0x0f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, - 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x50, - 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x12, 0x63, 0x0a, 0x15, 0x65, 0x6e, 0x76, 0x69, 0x72, - 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, - 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x76, - 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x14, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, - 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, - 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x09, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x1a, 0x47, 0x0a, 0x19, 0x45, - 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, - 0x6c, 0x65, 0x73, 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, 0xbc, 0x04, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, - 0x90, 0x01, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, - 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, - 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, 0x3a, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x34, 0x22, 0x20, - 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x63, - 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, - 0x3a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x12, 0x7d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x22, 0x2d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x27, 0x12, 0x25, 0x2f, 0x61, 0x70, 0x69, - 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x75, - 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, - 0x7d, 0x12, 0x89, 0x01, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, + 0x6c, 0x61, 0x74, 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, 0x4b, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, + 0x39, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 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, 0x5b, 0x0a, 0x1a, 0x4c, 0x69, + 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x0f, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x0e, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x22, 0x1e, 0x0a, 0x1c, 0x4c, 0x69, 0x73, 0x74, 0x41, + 0x6c, 0x6c, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x1d, 0x4c, 0x69, 0x73, 0x74, 0x41, + 0x6c, 0x6c, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3d, 0x0a, 0x0f, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x0e, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x22, 0x4e, 0x0a, 0x1a, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0xc0, 0x03, 0x0a, 0x0d, 0x49, 0x6d, 0x61, 0x67, + 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0x1d, 0x0a, 0x0a, 0x62, + 0x61, 0x73, 0x65, 0x5f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x62, 0x61, 0x73, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, + 0x70, 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0b, 0x70, 0x69, 0x70, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x12, 0x25, 0x0a, + 0x0e, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x5f, 0x70, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x50, 0x61, 0x63, 0x6b, + 0x61, 0x67, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x70, + 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x73, + 0x79, 0x73, 0x74, 0x65, 0x6d, 0x50, 0x61, 0x63, 0x6b, 0x61, 0x67, 0x65, 0x73, 0x12, 0x63, 0x0a, + 0x15, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x72, + 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, + 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x14, 0x65, 0x6e, + 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x63, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x73, + 0x74, 0x6f, 0x6d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x69, + 0x6d, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x1a, 0x47, 0x0a, 0x19, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 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, 0x9b, 0x06, 0x0a, 0x16, 0x43, + 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x90, 0x01, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, + 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, + 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, 0x3a, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x34, 0x22, 0x20, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, + 0x70, 0x68, 0x61, 0x32, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x3a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x96, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, + 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, + 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, + 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, 0x46, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x40, 0x12, 0x3e, 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, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, + 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, + 0x7d, 0x12, 0xa0, 0x01, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, 0x22, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x28, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x22, 0x12, 0x20, 0x2f, 0x61, 0x70, - 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x70, - 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, 0x83, 0x01, - 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, - 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0x2d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x27, 0x2a, 0x25, 0x2f, 0x61, - 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x63, 0x6f, 0x6d, - 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, - 0x69, 0x64, 0x7d, 0x32, 0x99, 0x04, 0x0a, 0x14, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, - 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x86, 0x01, 0x0a, - 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, 0x36, 0x82, - 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x22, 0x1e, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, - 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, - 0x6c, 0x61, 0x74, 0x65, 0x73, 0x3a, 0x0e, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, - 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x75, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, - 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, - 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, 0x12, 0x23, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, - 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, - 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0x81, 0x01, 0x0a, - 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, - 0x74, 0x65, 0x73, 0x12, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x26, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x20, - 0x12, 0x1e, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0x2f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, - 0x12, 0x7d, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, - 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, - 0x61, 0x74, 0x65, 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, 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, 0x2a, 0x23, 0x2f, 0x61, 0x70, 0x69, - 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x69, 0x6d, 0x61, 0x67, 0x65, - 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x42, + 0x6e, 0x73, 0x65, 0x22, 0x3f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x39, 0x12, 0x37, 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, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x92, 0x01, 0x0a, 0x17, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, + 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x12, 0x25, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, + 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x28, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x22, 0x12, 0x20, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, + 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, + 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, 0x9c, 0x01, 0x0a, 0x15, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, + 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, + 0x65, 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, 0x46, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x40, 0x2a, 0x3e, 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, + 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, + 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x32, 0xe4, 0x04, 0x0a, 0x14, 0x49, 0x6d, 0x61, + 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x86, 0x01, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, + 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x22, 0x1e, 0x2f, 0x61, 0x70, 0x69, + 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x69, 0x6d, 0x61, 0x67, 0x65, + 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x3a, 0x0e, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x10, 0x47, + 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, + 0x1e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, + 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x22, 0x44, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3e, 0x12, 0x3c, 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, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, + 0x61, 0x74, 0x65, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x98, 0x01, 0x0a, 0x12, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x12, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, + 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x37, 0x12, + 0x35, 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, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, 0x96, 0x01, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x21, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x49, 0x6d, 0x61, + 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, 0x44, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x3e, 0x2a, 0x3c, 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, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x74, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 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, @@ -906,50 +1173,58 @@ func file_config_proto_rawDescGZIP() []byte { return file_config_proto_rawDescData } -var file_config_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_config_proto_msgTypes = make([]protoimpl.MessageInfo, 17) var file_config_proto_goTypes = []interface{}{ - (*CreateComputeTemplateRequest)(nil), // 0: proto.CreateComputeTemplateRequest - (*GetComputeTemplateRequest)(nil), // 1: proto.GetComputeTemplateRequest - (*ListComputeTemplatesRequest)(nil), // 2: proto.ListComputeTemplatesRequest - (*ListComputeTemplatesResponse)(nil), // 3: proto.ListComputeTemplatesResponse - (*DeleteComputeTemplateRequest)(nil), // 4: proto.DeleteComputeTemplateRequest - (*ComputeTemplate)(nil), // 5: proto.ComputeTemplate - (*CreateImageTemplateRequest)(nil), // 6: proto.CreateImageTemplateRequest - (*GetImageTemplateRequest)(nil), // 7: proto.GetImageTemplateRequest - (*ListImageTemplatesRequest)(nil), // 8: proto.ListImageTemplatesRequest - (*ListImageTemplatesResponse)(nil), // 9: proto.ListImageTemplatesResponse - (*DeleteImageTemplateRequest)(nil), // 10: proto.DeleteImageTemplateRequest - (*ImageTemplate)(nil), // 11: proto.ImageTemplate - nil, // 12: proto.ImageTemplate.EnvironmentVariablesEntry - (*emptypb.Empty)(nil), // 13: google.protobuf.Empty + (*CreateComputeTemplateRequest)(nil), // 0: proto.CreateComputeTemplateRequest + (*GetComputeTemplateRequest)(nil), // 1: proto.GetComputeTemplateRequest + (*ListComputeTemplatesRequest)(nil), // 2: proto.ListComputeTemplatesRequest + (*ListComputeTemplatesResponse)(nil), // 3: proto.ListComputeTemplatesResponse + (*ListAllComputeTemplatesRequest)(nil), // 4: proto.ListAllComputeTemplatesRequest + (*ListAllComputeTemplatesResponse)(nil), // 5: proto.ListAllComputeTemplatesResponse + (*DeleteComputeTemplateRequest)(nil), // 6: proto.DeleteComputeTemplateRequest + (*ComputeTemplate)(nil), // 7: proto.ComputeTemplate + (*CreateImageTemplateRequest)(nil), // 8: proto.CreateImageTemplateRequest + (*GetImageTemplateRequest)(nil), // 9: proto.GetImageTemplateRequest + (*ListImageTemplatesRequest)(nil), // 10: proto.ListImageTemplatesRequest + (*ListImageTemplatesResponse)(nil), // 11: proto.ListImageTemplatesResponse + (*ListAllImageTemplatesRequest)(nil), // 12: proto.ListAllImageTemplatesRequest + (*ListAllImageTemplatesResponse)(nil), // 13: proto.ListAllImageTemplatesResponse + (*DeleteImageTemplateRequest)(nil), // 14: proto.DeleteImageTemplateRequest + (*ImageTemplate)(nil), // 15: proto.ImageTemplate + nil, // 16: proto.ImageTemplate.EnvironmentVariablesEntry + (*emptypb.Empty)(nil), // 17: google.protobuf.Empty } var file_config_proto_depIdxs = []int32{ - 5, // 0: proto.CreateComputeTemplateRequest.compute_template:type_name -> proto.ComputeTemplate - 5, // 1: proto.ListComputeTemplatesResponse.compute_templates:type_name -> proto.ComputeTemplate - 11, // 2: proto.CreateImageTemplateRequest.image_template:type_name -> proto.ImageTemplate - 11, // 3: proto.ListImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate - 12, // 4: proto.ImageTemplate.environment_variables:type_name -> proto.ImageTemplate.EnvironmentVariablesEntry - 0, // 5: proto.ComputeTemplateService.CreateComputeTemplate:input_type -> proto.CreateComputeTemplateRequest - 1, // 6: proto.ComputeTemplateService.GetComputeTemplate:input_type -> proto.GetComputeTemplateRequest - 2, // 7: proto.ComputeTemplateService.ListComputeTemplates:input_type -> proto.ListComputeTemplatesRequest - 4, // 8: proto.ComputeTemplateService.DeleteComputeTemplate:input_type -> proto.DeleteComputeTemplateRequest - 6, // 9: proto.ImageTemplateService.CreateImageTemplate:input_type -> proto.CreateImageTemplateRequest - 7, // 10: proto.ImageTemplateService.GetImageTemplate:input_type -> proto.GetImageTemplateRequest - 8, // 11: proto.ImageTemplateService.ListImageTemplates:input_type -> proto.ListImageTemplatesRequest - 10, // 12: proto.ImageTemplateService.DeleteImageTemplate:input_type -> proto.DeleteImageTemplateRequest - 5, // 13: proto.ComputeTemplateService.CreateComputeTemplate:output_type -> proto.ComputeTemplate - 5, // 14: proto.ComputeTemplateService.GetComputeTemplate:output_type -> proto.ComputeTemplate - 3, // 15: proto.ComputeTemplateService.ListComputeTemplates:output_type -> proto.ListComputeTemplatesResponse - 13, // 16: proto.ComputeTemplateService.DeleteComputeTemplate:output_type -> google.protobuf.Empty - 11, // 17: proto.ImageTemplateService.CreateImageTemplate:output_type -> proto.ImageTemplate - 11, // 18: proto.ImageTemplateService.GetImageTemplate:output_type -> proto.ImageTemplate - 9, // 19: proto.ImageTemplateService.ListImageTemplates:output_type -> proto.ListImageTemplatesResponse - 13, // 20: proto.ImageTemplateService.DeleteImageTemplate:output_type -> google.protobuf.Empty - 13, // [13:21] is the sub-list for method output_type - 5, // [5:13] is the sub-list for method input_type - 5, // [5:5] is the sub-list for extension type_name - 5, // [5:5] is the sub-list for extension extendee - 0, // [0:5] is the sub-list for field type_name + 7, // 0: proto.CreateComputeTemplateRequest.compute_template:type_name -> proto.ComputeTemplate + 7, // 1: proto.ListComputeTemplatesResponse.compute_templates:type_name -> proto.ComputeTemplate + 7, // 2: proto.ListAllComputeTemplatesResponse.compute_templates:type_name -> proto.ComputeTemplate + 15, // 3: proto.CreateImageTemplateRequest.image_template:type_name -> proto.ImageTemplate + 15, // 4: proto.ListImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate + 15, // 5: proto.ListAllImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate + 16, // 6: proto.ImageTemplate.environment_variables:type_name -> proto.ImageTemplate.EnvironmentVariablesEntry + 0, // 7: proto.ComputeTemplateService.CreateComputeTemplate:input_type -> proto.CreateComputeTemplateRequest + 1, // 8: proto.ComputeTemplateService.GetComputeTemplate:input_type -> proto.GetComputeTemplateRequest + 2, // 9: proto.ComputeTemplateService.ListComputeTemplates:input_type -> proto.ListComputeTemplatesRequest + 4, // 10: proto.ComputeTemplateService.ListAllComputeTemplates:input_type -> proto.ListAllComputeTemplatesRequest + 6, // 11: proto.ComputeTemplateService.DeleteComputeTemplate:input_type -> proto.DeleteComputeTemplateRequest + 8, // 12: proto.ImageTemplateService.CreateImageTemplate:input_type -> proto.CreateImageTemplateRequest + 9, // 13: proto.ImageTemplateService.GetImageTemplate:input_type -> proto.GetImageTemplateRequest + 10, // 14: proto.ImageTemplateService.ListImageTemplates:input_type -> proto.ListImageTemplatesRequest + 14, // 15: proto.ImageTemplateService.DeleteImageTemplate:input_type -> proto.DeleteImageTemplateRequest + 7, // 16: proto.ComputeTemplateService.CreateComputeTemplate:output_type -> proto.ComputeTemplate + 7, // 17: proto.ComputeTemplateService.GetComputeTemplate:output_type -> proto.ComputeTemplate + 3, // 18: proto.ComputeTemplateService.ListComputeTemplates:output_type -> proto.ListComputeTemplatesResponse + 5, // 19: proto.ComputeTemplateService.ListAllComputeTemplates:output_type -> proto.ListAllComputeTemplatesResponse + 17, // 20: proto.ComputeTemplateService.DeleteComputeTemplate:output_type -> google.protobuf.Empty + 15, // 21: proto.ImageTemplateService.CreateImageTemplate:output_type -> proto.ImageTemplate + 15, // 22: proto.ImageTemplateService.GetImageTemplate:output_type -> proto.ImageTemplate + 11, // 23: proto.ImageTemplateService.ListImageTemplates:output_type -> proto.ListImageTemplatesResponse + 17, // 24: proto.ImageTemplateService.DeleteImageTemplate:output_type -> google.protobuf.Empty + 16, // [16:25] is the sub-list for method output_type + 7, // [7:16] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name } func init() { file_config_proto_init() } @@ -1007,7 +1282,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteComputeTemplateRequest); i { + switch v := v.(*ListAllComputeTemplatesRequest); i { case 0: return &v.state case 1: @@ -1019,7 +1294,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ComputeTemplate); i { + switch v := v.(*ListAllComputeTemplatesResponse); i { case 0: return &v.state case 1: @@ -1031,7 +1306,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateImageTemplateRequest); i { + switch v := v.(*DeleteComputeTemplateRequest); i { case 0: return &v.state case 1: @@ -1043,7 +1318,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetImageTemplateRequest); i { + switch v := v.(*ComputeTemplate); i { case 0: return &v.state case 1: @@ -1055,7 +1330,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListImageTemplatesRequest); i { + switch v := v.(*CreateImageTemplateRequest); i { case 0: return &v.state case 1: @@ -1067,7 +1342,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListImageTemplatesResponse); i { + switch v := v.(*GetImageTemplateRequest); i { case 0: return &v.state case 1: @@ -1079,7 +1354,7 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteImageTemplateRequest); i { + switch v := v.(*ListImageTemplatesRequest); i { case 0: return &v.state case 1: @@ -1091,6 +1366,54 @@ func file_config_proto_init() { } } file_config_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListImageTemplatesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_config_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListAllImageTemplatesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_config_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListAllImageTemplatesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_config_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteImageTemplateRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_config_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ImageTemplate); i { case 0: return &v.state @@ -1109,7 +1432,7 @@ func file_config_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_config_proto_rawDesc, NumEnums: 0, - NumMessages: 13, + NumMessages: 17, NumExtensions: 0, NumServices: 2, }, diff --git a/proto/go_client/config.pb.gw.go b/proto/go_client/config.pb.gw.go index 832bc588d5..6c26e928b9 100644 --- a/proto/go_client/config.pb.gw.go +++ b/proto/go_client/config.pb.gw.go @@ -33,6 +33,10 @@ var _ = utilities.NewDoubleArray var _ = descriptor.ForMessage var _ = metadata.Join +var ( + filter_ComputeTemplateService_CreateComputeTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"compute_template": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + func request_ComputeTemplateService_CreateComputeTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ComputeTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq CreateComputeTemplateRequest var metadata runtime.ServerMetadata @@ -45,6 +49,13 @@ func request_ComputeTemplateService_CreateComputeTemplate_0(ctx context.Context, 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_ComputeTemplateService_CreateComputeTemplate_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := client.CreateComputeTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) return msg, metadata, err @@ -62,15 +73,18 @@ func local_request_ComputeTemplateService_CreateComputeTemplate_0(ctx context.Co 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_ComputeTemplateService_CreateComputeTemplate_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := server.CreateComputeTemplate(ctx, &protoReq) return msg, metadata, err } -var ( - filter_ComputeTemplateService_GetComputeTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} -) - func request_ComputeTemplateService_GetComputeTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ComputeTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq GetComputeTemplateRequest var metadata runtime.ServerMetadata @@ -82,22 +96,26 @@ func request_ComputeTemplateService_GetComputeTemplate_0(ctx context.Context, ma _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ComputeTemplateService_GetComputeTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.GetComputeTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) @@ -116,22 +134,26 @@ func local_request_ComputeTemplateService_GetComputeTemplate_0(ctx context.Conte _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ComputeTemplateService_GetComputeTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.GetComputeTemplate(ctx, &protoReq) @@ -143,6 +165,24 @@ func request_ComputeTemplateService_ListComputeTemplates_0(ctx context.Context, var protoReq ListComputeTemplatesRequest 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.ListComputeTemplates(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) return msg, metadata, err @@ -152,14 +192,46 @@ func local_request_ComputeTemplateService_ListComputeTemplates_0(ctx context.Con var protoReq ListComputeTemplatesRequest 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.ListComputeTemplates(ctx, &protoReq) return msg, metadata, err } -var ( - filter_ComputeTemplateService_DeleteComputeTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} -) +func request_ComputeTemplateService_ListAllComputeTemplates_0(ctx context.Context, marshaler runtime.Marshaler, client ComputeTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListAllComputeTemplatesRequest + var metadata runtime.ServerMetadata + + msg, err := client.ListAllComputeTemplates(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_ComputeTemplateService_ListAllComputeTemplates_0(ctx context.Context, marshaler runtime.Marshaler, server ComputeTemplateServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq ListAllComputeTemplatesRequest + var metadata runtime.ServerMetadata + + msg, err := server.ListAllComputeTemplates(ctx, &protoReq) + return msg, metadata, err + +} func request_ComputeTemplateService_DeleteComputeTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ComputeTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq DeleteComputeTemplateRequest @@ -172,22 +244,26 @@ func request_ComputeTemplateService_DeleteComputeTemplate_0(ctx context.Context, _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ComputeTemplateService_DeleteComputeTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.DeleteComputeTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) @@ -206,22 +282,26 @@ func local_request_ComputeTemplateService_DeleteComputeTemplate_0(ctx context.Co _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ComputeTemplateService_DeleteComputeTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.DeleteComputeTemplate(ctx, &protoReq) @@ -229,6 +309,10 @@ func local_request_ComputeTemplateService_DeleteComputeTemplate_0(ctx context.Co } +var ( + filter_ImageTemplateService_CreateImageTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"image_template": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} +) + func request_ImageTemplateService_CreateImageTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ImageTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq CreateImageTemplateRequest var metadata runtime.ServerMetadata @@ -241,6 +325,13 @@ func request_ImageTemplateService_CreateImageTemplate_0(ctx context.Context, mar 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_ImageTemplateService_CreateImageTemplate_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := client.CreateImageTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) return msg, metadata, err @@ -258,15 +349,18 @@ func local_request_ImageTemplateService_CreateImageTemplate_0(ctx context.Contex 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_ImageTemplateService_CreateImageTemplate_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := server.CreateImageTemplate(ctx, &protoReq) return msg, metadata, err } -var ( - filter_ImageTemplateService_GetImageTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} -) - func request_ImageTemplateService_GetImageTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ImageTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq GetImageTemplateRequest var metadata runtime.ServerMetadata @@ -278,22 +372,26 @@ func request_ImageTemplateService_GetImageTemplate_0(ctx context.Context, marsha _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ImageTemplateService_GetImageTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.GetImageTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) @@ -312,22 +410,26 @@ func local_request_ImageTemplateService_GetImageTemplate_0(ctx context.Context, _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ImageTemplateService_GetImageTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.GetImageTemplate(ctx, &protoReq) @@ -339,6 +441,24 @@ func request_ImageTemplateService_ListImageTemplates_0(ctx context.Context, mars var protoReq ListImageTemplatesRequest 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.ListImageTemplates(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) return msg, metadata, err @@ -348,15 +468,29 @@ func local_request_ImageTemplateService_ListImageTemplates_0(ctx context.Context var protoReq ListImageTemplatesRequest 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.ListImageTemplates(ctx, &protoReq) return msg, metadata, err } -var ( - filter_ImageTemplateService_DeleteImageTemplate_0 = &utilities.DoubleArray{Encoding: map[string]int{"id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}} -) - func request_ImageTemplateService_DeleteImageTemplate_0(ctx context.Context, marshaler runtime.Marshaler, client ImageTemplateServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq DeleteImageTemplateRequest var metadata runtime.ServerMetadata @@ -368,22 +502,26 @@ func request_ImageTemplateService_DeleteImageTemplate_0(ctx context.Context, mar _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ImageTemplateService_DeleteImageTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.DeleteImageTemplate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) @@ -402,22 +540,26 @@ func local_request_ImageTemplateService_DeleteImageTemplate_0(ctx context.Contex _ = err ) - val, ok = pathParams["id"] + val, ok = pathParams["namespace"] if !ok { - return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "id") + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "namespace") } - protoReq.Id, err = runtime.String(val) + protoReq.Namespace, err = runtime.String(val) if err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "id", err) + return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "namespace", err) } - if err := req.ParseForm(); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + val, ok = pathParams["name"] + if !ok { + return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "name") } - if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_ImageTemplateService_DeleteImageTemplate_0); err != nil { - return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + + 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.DeleteImageTemplate(ctx, &protoReq) @@ -500,6 +642,29 @@ func RegisterComputeTemplateServiceHandlerServer(ctx context.Context, mux *runti }) + mux.Handle("GET", pattern_ComputeTemplateService_ListAllComputeTemplates_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) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_ComputeTemplateService_ListAllComputeTemplates_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_ComputeTemplateService_ListAllComputeTemplates_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + mux.Handle("DELETE", pattern_ComputeTemplateService_DeleteComputeTemplate_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { ctx, cancel := context.WithCancel(req.Context()) defer cancel() @@ -725,6 +890,26 @@ func RegisterComputeTemplateServiceHandlerClient(ctx context.Context, mux *runti }) + mux.Handle("GET", pattern_ComputeTemplateService_ListAllComputeTemplates_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) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_ComputeTemplateService_ListAllComputeTemplates_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_ComputeTemplateService_ListAllComputeTemplates_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + mux.Handle("DELETE", pattern_ComputeTemplateService_DeleteComputeTemplate_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { ctx, cancel := context.WithCancel(req.Context()) defer cancel() @@ -749,13 +934,15 @@ func RegisterComputeTemplateServiceHandlerClient(ctx context.Context, mux *runti } var ( - pattern_ComputeTemplateService_CreateComputeTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha1", "compute_templates"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ComputeTemplateService_CreateComputeTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha2", "compute_templates"}, "", runtime.AssumeColonVerbOpt(true))) + + pattern_ComputeTemplateService_GetComputeTemplate_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", "compute_templates", "name"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ComputeTemplateService_GetComputeTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "compute_templates", "id"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ComputeTemplateService_ListComputeTemplates_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", "compute_templates"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ComputeTemplateService_ListComputeTemplates_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha1", "compute_templates"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ComputeTemplateService_ListAllComputeTemplates_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha2", "compute_templates"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ComputeTemplateService_DeleteComputeTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "compute_templates", "id"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ComputeTemplateService_DeleteComputeTemplate_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", "compute_templates", "name"}, "", runtime.AssumeColonVerbOpt(true))) ) var ( @@ -765,6 +952,8 @@ var ( forward_ComputeTemplateService_ListComputeTemplates_0 = runtime.ForwardResponseMessage + forward_ComputeTemplateService_ListAllComputeTemplates_0 = runtime.ForwardResponseMessage + forward_ComputeTemplateService_DeleteComputeTemplate_0 = runtime.ForwardResponseMessage ) @@ -890,13 +1079,13 @@ func RegisterImageTemplateServiceHandlerClient(ctx context.Context, mux *runtime } var ( - pattern_ImageTemplateService_CreateImageTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha1", "image_templates"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ImageTemplateService_CreateImageTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha2", "image_templates"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ImageTemplateService_GetImageTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "image_templates", "id"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ImageTemplateService_GetImageTemplate_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", "image_templates", "name"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ImageTemplateService_ListImageTemplates_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"apis", "v1alpha1", "image_templates"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ImageTemplateService_ListImageTemplates_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", "image_templates"}, "", runtime.AssumeColonVerbOpt(true))) - pattern_ImageTemplateService_DeleteImageTemplate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"apis", "v1alpha1", "image_templates", "id"}, "", runtime.AssumeColonVerbOpt(true))) + pattern_ImageTemplateService_DeleteImageTemplate_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", "image_templates", "name"}, "", runtime.AssumeColonVerbOpt(true))) ) var ( diff --git a/proto/go_client/config_grpc.pb.go b/proto/go_client/config_grpc.pb.go index 19c6ef3a46..dbbbb808d8 100644 --- a/proto/go_client/config_grpc.pb.go +++ b/proto/go_client/config_grpc.pb.go @@ -19,15 +19,15 @@ const _ = grpc.SupportPackageIsVersion7 // // 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 ComputeTemplateServiceClient interface { - // Creates a new Cluster. + // Creates a new compute template. CreateComputeTemplate(ctx context.Context, in *CreateComputeTemplateRequest, opts ...grpc.CallOption) (*ComputeTemplate, error) - // Finds a specific Cluster by ID. + // Finds a specific compute template by its name and namespace. GetComputeTemplate(ctx context.Context, in *GetComputeTemplateRequest, opts ...grpc.CallOption) (*ComputeTemplate, error) - // Finds all Clusters. Supports pagination, and sorting on certain fields. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. ListComputeTemplates(ctx context.Context, in *ListComputeTemplatesRequest, opts ...grpc.CallOption) (*ListComputeTemplatesResponse, error) - // Deletes an Cluster without deleting the Cluster's runs and jobs. To - // avoid unexpected behaviors, delete an Cluster's runs and jobs before - // deleting the Cluster. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListAllComputeTemplates(ctx context.Context, in *ListAllComputeTemplatesRequest, opts ...grpc.CallOption) (*ListAllComputeTemplatesResponse, error) + // Deletes a compuate template by its name and namespace DeleteComputeTemplate(ctx context.Context, in *DeleteComputeTemplateRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) } @@ -66,6 +66,15 @@ func (c *computeTemplateServiceClient) ListComputeTemplates(ctx context.Context, return out, nil } +func (c *computeTemplateServiceClient) ListAllComputeTemplates(ctx context.Context, in *ListAllComputeTemplatesRequest, opts ...grpc.CallOption) (*ListAllComputeTemplatesResponse, error) { + out := new(ListAllComputeTemplatesResponse) + err := c.cc.Invoke(ctx, "/proto.ComputeTemplateService/ListAllComputeTemplates", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *computeTemplateServiceClient) DeleteComputeTemplate(ctx context.Context, in *DeleteComputeTemplateRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { out := new(emptypb.Empty) err := c.cc.Invoke(ctx, "/proto.ComputeTemplateService/DeleteComputeTemplate", in, out, opts...) @@ -79,15 +88,15 @@ func (c *computeTemplateServiceClient) DeleteComputeTemplate(ctx context.Context // All implementations must embed UnimplementedComputeTemplateServiceServer // for forward compatibility type ComputeTemplateServiceServer interface { - // Creates a new Cluster. + // Creates a new compute template. CreateComputeTemplate(context.Context, *CreateComputeTemplateRequest) (*ComputeTemplate, error) - // Finds a specific Cluster by ID. + // Finds a specific compute template by its name and namespace. GetComputeTemplate(context.Context, *GetComputeTemplateRequest) (*ComputeTemplate, error) - // Finds all Clusters. Supports pagination, and sorting on certain fields. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. ListComputeTemplates(context.Context, *ListComputeTemplatesRequest) (*ListComputeTemplatesResponse, error) - // Deletes an Cluster without deleting the Cluster's runs and jobs. To - // avoid unexpected behaviors, delete an Cluster's runs and jobs before - // deleting the Cluster. + // Finds all compute templates in a given namespace. Supports pagination, and sorting on certain fields. + ListAllComputeTemplates(context.Context, *ListAllComputeTemplatesRequest) (*ListAllComputeTemplatesResponse, error) + // Deletes a compuate template by its name and namespace DeleteComputeTemplate(context.Context, *DeleteComputeTemplateRequest) (*emptypb.Empty, error) mustEmbedUnimplementedComputeTemplateServiceServer() } @@ -105,6 +114,9 @@ func (UnimplementedComputeTemplateServiceServer) GetComputeTemplate(context.Cont func (UnimplementedComputeTemplateServiceServer) ListComputeTemplates(context.Context, *ListComputeTemplatesRequest) (*ListComputeTemplatesResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ListComputeTemplates not implemented") } +func (UnimplementedComputeTemplateServiceServer) ListAllComputeTemplates(context.Context, *ListAllComputeTemplatesRequest) (*ListAllComputeTemplatesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListAllComputeTemplates not implemented") +} func (UnimplementedComputeTemplateServiceServer) DeleteComputeTemplate(context.Context, *DeleteComputeTemplateRequest) (*emptypb.Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method DeleteComputeTemplate not implemented") } @@ -176,6 +188,24 @@ func _ComputeTemplateService_ListComputeTemplates_Handler(srv interface{}, ctx c return interceptor(ctx, in, info, handler) } +func _ComputeTemplateService_ListAllComputeTemplates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListAllComputeTemplatesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ComputeTemplateServiceServer).ListAllComputeTemplates(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/proto.ComputeTemplateService/ListAllComputeTemplates", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ComputeTemplateServiceServer).ListAllComputeTemplates(ctx, req.(*ListAllComputeTemplatesRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ComputeTemplateService_DeleteComputeTemplate_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(DeleteComputeTemplateRequest) if err := dec(in); err != nil { @@ -213,6 +243,10 @@ var ComputeTemplateService_ServiceDesc = grpc.ServiceDesc{ MethodName: "ListComputeTemplates", Handler: _ComputeTemplateService_ListComputeTemplates_Handler, }, + { + MethodName: "ListAllComputeTemplates", + Handler: _ComputeTemplateService_ListAllComputeTemplates_Handler, + }, { MethodName: "DeleteComputeTemplate", Handler: _ComputeTemplateService_DeleteComputeTemplate_Handler, diff --git a/proto/swagger/config.swagger.json b/proto/swagger/config.swagger.json index 8625e4cd64..e6b37dce3e 100644 --- a/proto/swagger/config.swagger.json +++ b/proto/swagger/config.swagger.json @@ -22,15 +22,15 @@ "application/json" ], "paths": { - "/apis/v1alpha1/compute_templates": { + "/apis/v1alpha2/compute_templates": { "get": { - "summary": "Finds all Clusters. Supports pagination, and sorting on certain fields.", - "operationId": "ComputeTemplateService_ListComputeTemplates", + "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/protoListComputeTemplatesResponse" + "$ref": "#/definitions/protoListAllComputeTemplatesResponse" } }, "default": { @@ -45,7 +45,7 @@ ] }, "post": { - "summary": "Creates a new Cluster.", + "summary": "Creates a new compute template.", "operationId": "ComputeTemplateService_CreateComputeTemplate", "responses": { "200": { @@ -64,12 +64,19 @@ "parameters": [ { "name": "body", - "description": "The ComputeTemplate to be created.", + "description": "The compute template to be created.", "in": "body", "required": true, "schema": { "$ref": "#/definitions/protoComputeTemplate" } + }, + { + "name": "namespace", + "description": "The namespace of the compute template to be created.", + "in": "query", + "required": false, + "type": "string" } ], "tags": [ @@ -77,15 +84,15 @@ ] } }, - "/apis/v1alpha1/compute_templates/{id}": { - "get": { - "summary": "Finds a specific Cluster by ID.", - "operationId": "ComputeTemplateService_GetComputeTemplate", + "/apis/v1alpha2/image_templates": { + "post": { + "summary": "Creates a new ImageTemplate.", + "operationId": "ImageTemplateService_CreateImageTemplate", "responses": { "200": { "description": "A successful response.", "schema": { - "$ref": "#/definitions/protoComputeTemplate" + "$ref": "#/definitions/protoImageTemplate" } }, "default": { @@ -97,32 +104,36 @@ }, "parameters": [ { - "name": "id", - "description": "The ID of the ComputeTemplate to be retrieved. reserved field for database layer query later", - "in": "path", + "name": "body", + "description": "The image template to be created.", + "in": "body", "required": true, - "type": "string" + "schema": { + "$ref": "#/definitions/protoImageTemplate" + } }, { - "name": "name", - "description": "The name of the ComputeTemplate to be retrieved.", + "name": "namespace", + "description": "The namespace of the image template to be created.", "in": "query", "required": false, "type": "string" } ], "tags": [ - "ComputeTemplateService" + "ImageTemplateService" ] - }, - "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": "ComputeTemplateService_DeleteComputeTemplate", + } + }, + "/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": { - "properties": {} + "$ref": "#/definitions/protoListComputeTemplatesResponse" } }, "default": { @@ -134,18 +145,11 @@ }, "parameters": [ { - "name": "id", - "description": "The ID of the ComputeTemplate to be deleted.", + "name": "namespace", + "description": "The namespace of the compute templates to be retrieved.", "in": "path", "required": true, "type": "string" - }, - { - "name": "name", - "description": "The name of the ComputeTemplate to be deleted.", - "in": "query", - "required": false, - "type": "string" } ], "tags": [ @@ -153,15 +157,15 @@ ] } }, - "/apis/v1alpha1/image_templates": { + "/apis/v1alpha2/namespaces/{namespace}/compute_templates/{name}": { "get": { - "summary": "Finds all ImageTemplates. Supports pagination, and sorting on certain fields.", - "operationId": "ImageTemplateService_ListImageTemplates", + "summary": "Finds a specific compute template by its name and namespace.", + "operationId": "ComputeTemplateService_GetComputeTemplate", "responses": { "200": { "description": "A successful response.", "schema": { - "$ref": "#/definitions/protoListImageTemplatesResponse" + "$ref": "#/definitions/protoComputeTemplate" } }, "default": { @@ -171,18 +175,34 @@ } } }, + "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": [ - "ImageTemplateService" + "ComputeTemplateService" ] }, - "post": { - "summary": "Creates a new ImageTemplate.", - "operationId": "ImageTemplateService_CreateImageTemplate", + "delete": { + "summary": "Deletes a compuate template by its name and namespace", + "operationId": "ComputeTemplateService_DeleteComputeTemplate", "responses": { "200": { "description": "A successful response.", "schema": { - "$ref": "#/definitions/protoImageTemplate" + "properties": {} } }, "default": { @@ -194,13 +214,50 @@ }, "parameters": [ { - "name": "body", - "description": "The Compute to be created.", - "in": "body", + "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/protoImageTemplate" + "$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": [ @@ -208,7 +265,7 @@ ] } }, - "/apis/v1alpha1/image_templates/{id}": { + "/apis/v1alpha2/namespaces/{namespace}/image_templates/{name}": { "get": { "summary": "Finds a specific ImageTemplate by ID.", "operationId": "ImageTemplateService_GetImageTemplate", @@ -228,17 +285,17 @@ }, "parameters": [ { - "name": "id", - "description": "The ID of the ImageTemplate to be retrieved. reserved field for database layer query later", + "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 ImageTemplate to be retrieved.", - "in": "query", - "required": false, + "description": "The name of the image template to be retrieved.", + "in": "path", + "required": true, "type": "string" } ], @@ -265,17 +322,17 @@ }, "parameters": [ { - "name": "id", - "description": "The ID of the ImageTemplate to be deleted.", + "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 ImageTemplate to be delete.", - "in": "query", - "required": false, + "description": "The name of the image template to be deleted.", + "in": "path", + "required": true, "type": "string" } ], @@ -307,14 +364,14 @@ "protoComputeTemplate": { "type": "object", "properties": { - "id": { - "type": "string", - "title": "The ID of the compute template" - }, "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", @@ -344,6 +401,10 @@ "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" @@ -387,6 +448,17 @@ }, "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": {