diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 73461c03fd..0ce19872f2 100644 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -385,7 +385,17 @@ func FromKubeToAPIComputeTemplate(configMap *corev1.ConfigMap) *api.ComputeTempl runtime.Memory = uint32(memory) runtime.Gpu = uint32(gpu) runtime.GpuAccelerator = configMap.Data["gpu_accelerator"] - val, ok := configMap.Data["tolerations"] + + val, ok := configMap.Data["extended_resources"] + if ok { + err := json.Unmarshal([]byte(val), &runtime.ExtendedResources) + if err != nil { + klog.Error("failed to unmarshall extended resources for compute template ", runtime.Name, " value ", + runtime.ExtendedResources, " error ", err) + } + } + + val, ok = configMap.Data["tolerations"] if ok { err := json.Unmarshal([]byte(val), &runtime.Tolerations) if err != nil { diff --git a/apiserver/pkg/model/converter_test.go b/apiserver/pkg/model/converter_test.go index ba70e92f5a..19dfe5e75e 100644 --- a/apiserver/pkg/model/converter_test.go +++ b/apiserver/pkg/model/converter_test.go @@ -126,24 +126,26 @@ var headSpecTest = rayv1api.HeadGroupSpec{ var configMapWithoutTolerations = corev1.ConfigMap{ Data: map[string]string{ - "cpu": "4", - "gpu": "0", - "gpu_accelerator": "", - "memory": "8", - "name": "head-node-template", - "namespace": "max", + "cpu": "4", + "gpu": "0", + "gpu_accelerator": "", + "memory": "8", + "extended_resources": "{\"vpc.amazonaws.com/efa\": 32}", + "name": "head-node-template", + "namespace": "max", }, } var configMapWithTolerations = corev1.ConfigMap{ Data: map[string]string{ - "cpu": "4", - "gpu": "0", - "gpu_accelerator": "", - "memory": "8", - "name": "head-node-template", - "namespace": "max", - "tolerations": "[{\"key\":\"blah1\",\"operator\":\"Exists\",\"effect\":\"NoExecute\"}]", + "cpu": "4", + "gpu": "0", + "gpu_accelerator": "", + "memory": "8", + "extended_resources": "{\"vpc.amazonaws.com/efa\": 32}", + "name": "head-node-template", + "namespace": "max", + "tolerations": "[{\"key\":\"blah1\",\"operator\":\"Exists\",\"effect\":\"NoExecute\"}]", }, } @@ -578,6 +580,11 @@ func TestPopulateTemplate(t *testing.T) { t.Errorf("failed to convert config map, got %v, expected %v", tolerationToString(template.Tolerations[0]), tolerationToString(&expectedTolerations)) } + + assert.Equal(t, uint32(4), template.Cpu, "CPU mismatch") + assert.Equal(t, uint32(8), template.Memory, "Memory mismatch") + assert.Equal(t, uint32(0), template.Gpu, "GPU mismatch") + assert.Equal(t, map[string]uint32{"vpc.amazonaws.com/efa": 32}, template.ExtendedResources, "Extended resources mismatch") } func tolerationToString(toleration *api.PodToleration) string { diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index 5e740bf546..78c90a2c66 100644 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -145,6 +145,25 @@ func buildNodeGroupAnnotations(computeTemplate *api.ComputeTemplate, image strin return annotations } +// Add resource to container +func addResourceToContainer(container *corev1.Container, resourceName string, quantity uint32) { + if quantity == 0 { + return + } + quantityStr := fmt.Sprint(quantity) + resourceQuantity := resource.MustParse(quantityStr) + + if container.Resources.Requests == nil { + container.Resources.Requests = make(corev1.ResourceList) + } + if container.Resources.Limits == nil { + container.Resources.Limits = make(corev1.ResourceList) + } + + container.Resources.Requests[corev1.ResourceName(resourceName)] = resourceQuantity + container.Resources.Limits[corev1.ResourceName(resourceName)] = resourceQuantity +} + // Build head node template func buildHeadPodTemplate(imageVersion string, envs *api.EnvironmentVariables, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate, enableServeService bool) (*corev1.PodTemplateSpec, error) { image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) @@ -232,15 +251,18 @@ func buildHeadPodTemplate(imageVersion string, envs *api.EnvironmentVariables, s // We are filtering container by name `ray-head`. If container with this name does not exist // (should never happen) we are not adding container specific parameters if container, index, ok := GetContainerByName(podTemplateSpec.Spec.Containers, "ray-head"); ok { - if computeRuntime.GetGpu() != 0 { - gpu := computeRuntime.GetGpu() + if gpu := computeRuntime.GetGpu(); gpu != 0 { accelerator := "nvidia.com/gpu" if len(computeRuntime.GetGpuAccelerator()) != 0 { accelerator = computeRuntime.GetGpuAccelerator() } - container.Resources.Requests[corev1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) - container.Resources.Limits[corev1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) + addResourceToContainer(&container, accelerator, gpu) } + + for k, v := range computeRuntime.GetExtendedResources() { + addResourceToContainer(&container, k, v) + } + globalEnv := convertEnvironmentVariables(envs) if len(globalEnv) > 0 { container.Env = append(container.Env, globalEnv...) @@ -528,16 +550,16 @@ func buildWorkerPodTemplate(imageVersion string, envs *api.EnvironmentVariables, // We are filtering container by name `ray-worker`. If container with this name does not exist // (should never happen) we are not adding container specific parameters if container, index, ok := GetContainerByName(podTemplateSpec.Spec.Containers, "ray-worker"); ok { - if computeRuntime.GetGpu() != 0 { - gpu := computeRuntime.GetGpu() + if gpu := computeRuntime.GetGpu(); gpu != 0 { accelerator := "nvidia.com/gpu" if len(computeRuntime.GetGpuAccelerator()) != 0 { accelerator = computeRuntime.GetGpuAccelerator() } + addResourceToContainer(&container, accelerator, gpu) + } - // need smarter algorithm to filter main container. for example filter by name `ray-worker` - container.Resources.Requests[corev1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) - container.Resources.Limits[corev1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) + for k, v := range computeRuntime.GetExtendedResources() { + addResourceToContainer(&container, k, v) } globalEnv := convertEnvironmentVariables(envs) @@ -800,14 +822,20 @@ func (c *RayCluster) SetAnnotationsToAllTemplates(key string, value string) { // Build compute template func NewComputeTemplate(runtime *api.ComputeTemplate) (*corev1.ConfigMap, error) { + extendedResourcesJSON, err := json.Marshal(runtime.ExtendedResources) + if err != nil { + return nil, fmt.Errorf("failed to marshal extended resources: %v", err) + } + // Create data map dmap := 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), - "gpu_accelerator": runtime.GpuAccelerator, + "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), + "gpu_accelerator": runtime.GpuAccelerator, + "extended_resources": string(extendedResourcesJSON), } // Add tolerations in defined if runtime.Tolerations != nil && len(runtime.Tolerations) > 0 { diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index 03c5748961..1da78dc5b7 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -243,6 +243,22 @@ var template = api.ComputeTemplate{ }, } +var templateWorker = api.ComputeTemplate{ + Name: "", + Namespace: "", + Cpu: 2, + Memory: 8, + Gpu: 4, + ExtendedResources: map[string]uint32{"vpc.amazonaws.com/efa": 32}, + Tolerations: []*api.PodToleration{ + { + Key: "blah1", + Operator: "Exists", + Effect: "NoExecute", + }, + }, +} + var expectedToleration = corev1.Toleration{ Key: "blah1", Operator: "Exists", @@ -591,34 +607,33 @@ func TestBuildRayCluster(t *testing.T) { } func TestBuilWorkerPodTemplate(t *testing.T) { - podSpec, err := buildWorkerPodTemplate("2.4", &api.EnvironmentVariables{}, &workerGroup, &template) + podSpec, err := buildWorkerPodTemplate("2.4", &api.EnvironmentVariables{}, &workerGroup, &templateWorker) assert.Nil(t, err) - if podSpec.Spec.ServiceAccountName != "account" { - t.Errorf("failed to propagate service account") - } - if podSpec.Spec.ImagePullSecrets[0].Name != "foo" { - t.Errorf("failed to propagate image pull secret") - } - if (string)(podSpec.Spec.Containers[0].ImagePullPolicy) != "Always" { - t.Errorf("failed to propagate image pull policy") - } - if !containsEnv(podSpec.Spec.Containers[0].Env, "foo", "bar") { - t.Errorf("failed to propagate environment") - } - if len(podSpec.Spec.Tolerations) != 1 { - t.Errorf("failed to propagate tolerations, expected 1, got %d", len(podSpec.Spec.Tolerations)) - } - if !reflect.DeepEqual(podSpec.Spec.Tolerations[0], expectedToleration) { - t.Errorf("failed to propagate annotations, got %v, expected %v", tolerationToString(&podSpec.Spec.Tolerations[0]), - tolerationToString(&expectedToleration)) - } - if val, exists := podSpec.Annotations["foo"]; !exists || val != "bar" { - t.Errorf("failed to convert annotations") - } - if !reflect.DeepEqual(podSpec.Labels, expectedLabels) { - t.Errorf("failed to convert labels, got %v, expected %v", podSpec.Labels, expectedLabels) - } + assert.Equal(t, "account", podSpec.Spec.ServiceAccountName, "failed to propagate service account") + assert.Equal(t, "foo", podSpec.Spec.ImagePullSecrets[0].Name, "failed to propagate image pull secret") + assert.Equal(t, corev1.PullAlways, podSpec.Spec.Containers[0].ImagePullPolicy, "failed to propagate image pull policy") + assert.True(t, containsEnv(podSpec.Spec.Containers[0].Env, "foo", "bar"), "failed to propagate environment") + assert.Len(t, podSpec.Spec.Tolerations, 1, "failed to propagate tolerations") + assert.Equal(t, expectedToleration, podSpec.Spec.Tolerations[0], "failed to propagate tolerations") + assert.Equal(t, "bar", podSpec.Annotations["foo"], "failed to convert annotations") + assert.Equal(t, expectedLabels, podSpec.Labels, "failed to convert labels") + + // Check Resources + container := podSpec.Spec.Containers[0] + resources := container.Resources + + assert.Equal(t, resource.MustParse("2"), resources.Limits[corev1.ResourceCPU], "CPU limit doesn't match") + assert.Equal(t, resource.MustParse("2"), resources.Requests[corev1.ResourceCPU], "CPU request doesn't match") + + assert.Equal(t, resource.MustParse("8Gi"), resources.Limits[corev1.ResourceMemory], "Memory limit doesn't match") + assert.Equal(t, resource.MustParse("8Gi"), resources.Requests[corev1.ResourceMemory], "Memory request doesn't match") + + assert.Equal(t, resource.MustParse("4"), resources.Limits["nvidia.com/gpu"], "GPU limit doesn't match") + assert.Equal(t, resource.MustParse("4"), resources.Requests["nvidia.com/gpu"], "GPU request doesn't match") + + assert.Equal(t, resource.MustParse("32"), resources.Limits["vpc.amazonaws.com/efa"], "EFA limit doesn't match") + assert.Equal(t, resource.MustParse("32"), resources.Requests["vpc.amazonaws.com/efa"], "EFA request doesn't match") } func containsEnv(envs []corev1.EnvVar, key string, val string) bool { diff --git a/clients/python-apiserver-client/src/python_apiserver_client/params/templates.py b/clients/python-apiserver-client/src/python_apiserver_client/params/templates.py index fb267275ce..0112491369 100644 --- a/clients/python-apiserver-client/src/python_apiserver_client/params/templates.py +++ b/clients/python-apiserver-client/src/python_apiserver_client/params/templates.py @@ -92,6 +92,7 @@ class Template: memory - required, template memory (GB) gpus - optional, number of GPUs, default 0 gpu_accelerator - optional, if not defined nvidia.com/gpu is assumed + extended_resources - optional, name and number of the extended resources tolerations - optional, tolerations for pod placing, default none - to_string() -> str: convert toleration to string for printing - to_dict() -> dict[str, Any] convert to dict @@ -106,6 +107,7 @@ def __init__( memory: int, gpu: int = 0, gpu_accelerator: str = None, + extended_resources: dict[str, int] = None, tolerations: list[Toleration] = None, ): """ @@ -116,6 +118,7 @@ def __init__( :param memory: memory :param gpu: gpu :param gpu_accelerator: accelerator type + :param extended_resources: extended resources :param tolerations: tolerations """ self.name = name @@ -124,6 +127,7 @@ def __init__( self.memory = memory self.gpu = gpu self.gpu_accelerator = gpu_accelerator + self.extended_resources = extended_resources self.tolerations = tolerations def to_string(self) -> str: @@ -136,6 +140,8 @@ def to_string(self) -> str: val = val + f", gpu {self.gpu}" if self.gpu_accelerator is not None: val = val + f", gpu accelerator {self.gpu_accelerator}" + if self.extended_resources is not None: + val = val + f", extended resources {self.extended_resources}" if self.tolerations is None: return val val = val + ", tolerations [" @@ -158,6 +164,8 @@ def to_dict(self) -> dict[str, Any]: dct["gpu"] = self.gpu if self.gpu_accelerator is not None: dct["gpu accelerator"] = self.gpu_accelerator + if self.extended_resources is not None: + dct["extended resources"] = self.extended_resources if self.tolerations is not None: dct["tolerations"] = [tl.to_dict() for tl in self.tolerations] return dct @@ -199,6 +207,7 @@ def template_decoder(dct: dict[str, Any]) -> Template: memory=int(dct.get("memory", "0")), gpu=int(dct.get("gpu", "0")), gpu_accelerator=dct.get("gpu_accelerator"), + extended_resources=dct.get("extended_resources"), tolerations=tolerations, ) diff --git a/clients/python-apiserver-client/test/api_params_test.py b/clients/python-apiserver-client/test/api_params_test.py index ad6a01569d..37b0e3c45d 100644 --- a/clients/python-apiserver-client/test/api_params_test.py +++ b/clients/python-apiserver-client/test/api_params_test.py @@ -77,8 +77,14 @@ def test_templates(): tm2_json = json.dumps(temp2.to_dict()) print(f"template 2 JSON: {tm2_json}") + temp3 = Template(name="template3", namespace="namespace", cpu=2, memory=8, gpu=1, extended_resources={"vpc.amazonaws.com/efa": 32}) + print(f"template 3: {temp3.to_string()}") + tm3_json = json.dumps(temp3.to_dict()) + print(f"template 3 JSON: {tm3_json}") + assert temp1.to_string() == template_decoder(json.loads(tm1_json)).to_string() assert temp2.to_string() == template_decoder(json.loads(tm2_json)).to_string() + assert temp3.to_string() == template_decoder(json.loads(tm3_json)).to_string() def test_volumes(): diff --git a/clients/python-apiserver-client/test/kuberay_api_test.py b/clients/python-apiserver-client/test/kuberay_api_test.py index c875caa983..bb522d5ffb 100644 --- a/clients/python-apiserver-client/test/kuberay_api_test.py +++ b/clients/python-apiserver-client/test/kuberay_api_test.py @@ -43,7 +43,7 @@ def test_templates(): _, _ = apis.delete_compute_template(ns="default", name="default-template") # create toleration = Toleration(key="blah1", operator=TolerationOperation.Exists, effect=TolerationEffect.NoExecute) - template = Template(name="default-template", namespace="default", cpu=2, memory=8, tolerations=[toleration]) + template = Template(name="default-template", namespace="default", cpu=2, memory=8, gpu=1, extended_resources={"vpc.amazonaws.com/efa": 32}, tolerations=[toleration]) status, error = apis.create_compute_template(template) assert status == 200 assert error is None diff --git a/proto/config.proto b/proto/config.proto index 84d3a5c0d7..314f6ddeea 100644 --- a/proto/config.proto +++ b/proto/config.proto @@ -128,6 +128,8 @@ message ComputeTemplate { string gpu_accelerator = 6; // Optional pod tolerations repeated PodToleration tolerations = 7; + // Optional. Name and number of the extended resources + map extended_resources = 8; } // This service is not implemented. diff --git a/proto/go_client/config.pb.go b/proto/go_client/config.pb.go index 4a1de23e94..0484e4a7f8 100644 --- a/proto/go_client/config.pb.go +++ b/proto/go_client/config.pb.go @@ -470,6 +470,8 @@ type ComputeTemplate struct { GpuAccelerator string `protobuf:"bytes,6,opt,name=gpu_accelerator,json=gpuAccelerator,proto3" json:"gpu_accelerator,omitempty"` // Optional pod tolerations Tolerations []*PodToleration `protobuf:"bytes,7,rep,name=tolerations,proto3" json:"tolerations,omitempty"` + // Optional. Name and number of the extended resources + ExtendedResources map[string]uint32 `protobuf:"bytes,8,rep,name=extended_resources,json=extendedResources,proto3" json:"extended_resources,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` } func (x *ComputeTemplate) Reset() { @@ -553,6 +555,13 @@ func (x *ComputeTemplate) GetTolerations() []*PodToleration { return nil } +func (x *ComputeTemplate) GetExtendedResources() map[string]uint32 { + if x != nil { + return x.ExtendedResources + } + return nil +} + type CreateImageTemplateRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1088,7 +1097,7 @@ var file_config_proto_rawDesc = []byte{ 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1b, 0x0a, 0x06, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x03, 0xe0, 0x41, - 0x02, 0x52, 0x06, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x22, 0xf4, 0x01, 0x0a, 0x0f, 0x43, 0x6f, + 0x02, 0x52, 0x06, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x22, 0x98, 0x03, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x17, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x03, 0xe0, 0x41, 0x02, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x21, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, @@ -1104,162 +1113,172 @@ var file_config_proto_rawDesc = []byte{ 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x50, 0x6f, 0x64, 0x54, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x74, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 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, 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, 0x5c, 0x0a, 0x12, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x65, 0x78, 0x74, + 0x65, 0x6e, 0x64, 0x65, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x44, + 0x0a, 0x16, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 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, 0x0d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 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, 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, 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, 0x94, 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, 0xa1, 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, 0x4b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x45, 0x22, 0x31, 0x2f, 0x61, 0x70, - 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x3a, 0x10, - 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, - 0x12, 0x90, 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, 0x40, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3a, 0x12, 0x38, 0x2f, 0x61, 0x70, 0x69, 0x73, - 0x2f, 0x76, 0x31, 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, 0x9a, 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, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x33, 0x12, 0x31, 0x2f, + 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, 0x94, 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, 0xa1, 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, 0x4b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x45, 0x22, + 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x3a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, + 0x6c, 0x61, 0x74, 0x65, 0x12, 0x90, 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, 0x40, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3a, 0x12, 0x38, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x8c, 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, 0x22, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x1c, 0x12, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, - 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, - 0x96, 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, 0x40, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x3a, 0x2a, 0x38, - 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0xcc, 0x04, 0x0a, 0x14, 0x49, 0x6d, 0x61, - 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x12, 0x80, 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, 0x30, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x22, 0x18, 0x2f, 0x61, 0x70, 0x69, - 0x73, 0x2f, 0x76, 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, 0x88, 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, - 0x3e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x38, 0x12, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, + 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x9a, 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, 0x39, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x33, 0x12, 0x31, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x8c, 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, + 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x12, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, + 0x31, 0x2f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, + 0x74, 0x65, 0x73, 0x12, 0x96, 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, 0x40, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x3a, 0x2a, 0x38, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0xcc, 0x04, 0x0a, + 0x14, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x80, 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, 0x30, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x22, 0x18, + 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 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, 0x88, 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, 0x3e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x38, 0x12, 0x36, 0x2f, 0x61, 0x70, + 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x92, 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, + 0x37, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, - 0x92, 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, 0x37, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x31, 0x12, 0x2f, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x6e, 0x61, + 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x12, 0x90, 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, 0x3e, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x38, 0x2a, 0x36, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x90, 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, 0x3e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x38, 0x2a, - 0x36, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 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, 0x74, 0x92, 0x41, 0x21, 0x2a, 0x01, 0x01, 0x52, - 0x1c, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x11, 0x12, 0x0f, 0x0a, 0x0d, - 0x1a, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 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, 0x74, 0x92, 0x41, 0x21, + 0x2a, 0x01, 0x01, 0x52, 0x1c, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x11, + 0x12, 0x0f, 0x0a, 0x0d, 0x1a, 0x0b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1274,7 +1293,7 @@ func file_config_proto_rawDescGZIP() []byte { return file_config_proto_rawDescData } -var file_config_proto_msgTypes = make([]protoimpl.MessageInfo, 18) +var file_config_proto_msgTypes = make([]protoimpl.MessageInfo, 19) var file_config_proto_goTypes = []interface{}{ (*CreateComputeTemplateRequest)(nil), // 0: proto.CreateComputeTemplateRequest (*GetComputeTemplateRequest)(nil), // 1: proto.GetComputeTemplateRequest @@ -1293,41 +1312,43 @@ var file_config_proto_goTypes = []interface{}{ (*ListAllImageTemplatesResponse)(nil), // 14: proto.ListAllImageTemplatesResponse (*DeleteImageTemplateRequest)(nil), // 15: proto.DeleteImageTemplateRequest (*ImageTemplate)(nil), // 16: proto.ImageTemplate - nil, // 17: proto.ImageTemplate.EnvironmentVariablesEntry - (*emptypb.Empty)(nil), // 18: google.protobuf.Empty + nil, // 17: proto.ComputeTemplate.ExtendedResourcesEntry + nil, // 18: proto.ImageTemplate.EnvironmentVariablesEntry + (*emptypb.Empty)(nil), // 19: google.protobuf.Empty } var file_config_proto_depIdxs = []int32{ 8, // 0: proto.CreateComputeTemplateRequest.compute_template:type_name -> proto.ComputeTemplate 8, // 1: proto.ListComputeTemplatesResponse.compute_templates:type_name -> proto.ComputeTemplate 8, // 2: proto.ListAllComputeTemplatesResponse.compute_templates:type_name -> proto.ComputeTemplate 7, // 3: proto.ComputeTemplate.tolerations:type_name -> proto.PodToleration - 16, // 4: proto.CreateImageTemplateRequest.image_template:type_name -> proto.ImageTemplate - 16, // 5: proto.ListImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate - 16, // 6: proto.ListAllImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate - 17, // 7: proto.ImageTemplate.environment_variables:type_name -> proto.ImageTemplate.EnvironmentVariablesEntry - 0, // 8: proto.ComputeTemplateService.CreateComputeTemplate:input_type -> proto.CreateComputeTemplateRequest - 1, // 9: proto.ComputeTemplateService.GetComputeTemplate:input_type -> proto.GetComputeTemplateRequest - 2, // 10: proto.ComputeTemplateService.ListComputeTemplates:input_type -> proto.ListComputeTemplatesRequest - 4, // 11: proto.ComputeTemplateService.ListAllComputeTemplates:input_type -> proto.ListAllComputeTemplatesRequest - 6, // 12: proto.ComputeTemplateService.DeleteComputeTemplate:input_type -> proto.DeleteComputeTemplateRequest - 9, // 13: proto.ImageTemplateService.CreateImageTemplate:input_type -> proto.CreateImageTemplateRequest - 10, // 14: proto.ImageTemplateService.GetImageTemplate:input_type -> proto.GetImageTemplateRequest - 11, // 15: proto.ImageTemplateService.ListImageTemplates:input_type -> proto.ListImageTemplatesRequest - 15, // 16: proto.ImageTemplateService.DeleteImageTemplate:input_type -> proto.DeleteImageTemplateRequest - 8, // 17: proto.ComputeTemplateService.CreateComputeTemplate:output_type -> proto.ComputeTemplate - 8, // 18: proto.ComputeTemplateService.GetComputeTemplate:output_type -> proto.ComputeTemplate - 3, // 19: proto.ComputeTemplateService.ListComputeTemplates:output_type -> proto.ListComputeTemplatesResponse - 5, // 20: proto.ComputeTemplateService.ListAllComputeTemplates:output_type -> proto.ListAllComputeTemplatesResponse - 18, // 21: proto.ComputeTemplateService.DeleteComputeTemplate:output_type -> google.protobuf.Empty - 16, // 22: proto.ImageTemplateService.CreateImageTemplate:output_type -> proto.ImageTemplate - 16, // 23: proto.ImageTemplateService.GetImageTemplate:output_type -> proto.ImageTemplate - 12, // 24: proto.ImageTemplateService.ListImageTemplates:output_type -> proto.ListImageTemplatesResponse - 18, // 25: proto.ImageTemplateService.DeleteImageTemplate:output_type -> google.protobuf.Empty - 17, // [17:26] is the sub-list for method output_type - 8, // [8:17] is the sub-list for method input_type - 8, // [8:8] is the sub-list for extension type_name - 8, // [8:8] is the sub-list for extension extendee - 0, // [0:8] is the sub-list for field type_name + 17, // 4: proto.ComputeTemplate.extended_resources:type_name -> proto.ComputeTemplate.ExtendedResourcesEntry + 16, // 5: proto.CreateImageTemplateRequest.image_template:type_name -> proto.ImageTemplate + 16, // 6: proto.ListImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate + 16, // 7: proto.ListAllImageTemplatesResponse.image_templates:type_name -> proto.ImageTemplate + 18, // 8: proto.ImageTemplate.environment_variables:type_name -> proto.ImageTemplate.EnvironmentVariablesEntry + 0, // 9: proto.ComputeTemplateService.CreateComputeTemplate:input_type -> proto.CreateComputeTemplateRequest + 1, // 10: proto.ComputeTemplateService.GetComputeTemplate:input_type -> proto.GetComputeTemplateRequest + 2, // 11: proto.ComputeTemplateService.ListComputeTemplates:input_type -> proto.ListComputeTemplatesRequest + 4, // 12: proto.ComputeTemplateService.ListAllComputeTemplates:input_type -> proto.ListAllComputeTemplatesRequest + 6, // 13: proto.ComputeTemplateService.DeleteComputeTemplate:input_type -> proto.DeleteComputeTemplateRequest + 9, // 14: proto.ImageTemplateService.CreateImageTemplate:input_type -> proto.CreateImageTemplateRequest + 10, // 15: proto.ImageTemplateService.GetImageTemplate:input_type -> proto.GetImageTemplateRequest + 11, // 16: proto.ImageTemplateService.ListImageTemplates:input_type -> proto.ListImageTemplatesRequest + 15, // 17: proto.ImageTemplateService.DeleteImageTemplate:input_type -> proto.DeleteImageTemplateRequest + 8, // 18: proto.ComputeTemplateService.CreateComputeTemplate:output_type -> proto.ComputeTemplate + 8, // 19: proto.ComputeTemplateService.GetComputeTemplate:output_type -> proto.ComputeTemplate + 3, // 20: proto.ComputeTemplateService.ListComputeTemplates:output_type -> proto.ListComputeTemplatesResponse + 5, // 21: proto.ComputeTemplateService.ListAllComputeTemplates:output_type -> proto.ListAllComputeTemplatesResponse + 19, // 22: proto.ComputeTemplateService.DeleteComputeTemplate:output_type -> google.protobuf.Empty + 16, // 23: proto.ImageTemplateService.CreateImageTemplate:output_type -> proto.ImageTemplate + 16, // 24: proto.ImageTemplateService.GetImageTemplate:output_type -> proto.ImageTemplate + 12, // 25: proto.ImageTemplateService.ListImageTemplates:output_type -> proto.ListImageTemplatesResponse + 19, // 26: proto.ImageTemplateService.DeleteImageTemplate:output_type -> google.protobuf.Empty + 18, // [18:27] is the sub-list for method output_type + 9, // [9:18] is the sub-list for method input_type + 9, // [9:9] is the sub-list for extension type_name + 9, // [9:9] is the sub-list for extension extendee + 0, // [0:9] is the sub-list for field type_name } func init() { file_config_proto_init() } @@ -1547,7 +1568,7 @@ func file_config_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_config_proto_rawDesc, NumEnums: 0, - NumMessages: 18, + NumMessages: 19, NumExtensions: 0, NumServices: 2, }, diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index 268085c540..8e0e3eafb5 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -1548,6 +1548,14 @@ "$ref": "#/definitions/protoPodToleration" }, "title": "Optional pod tolerations" + }, + "extendedResources": { + "type": "object", + "additionalProperties": { + "type": "integer", + "format": "int64" + }, + "title": "Optional. Name and number of the extended resources" } }, "title": "ComputeTemplate can be reused by any compute units like worker group, workspace, image build job, etc", diff --git a/proto/swagger/config.swagger.json b/proto/swagger/config.swagger.json index b359e2288e..7548300d10 100644 --- a/proto/swagger/config.swagger.json +++ b/proto/swagger/config.swagger.json @@ -409,6 +409,14 @@ "$ref": "#/definitions/protoPodToleration" }, "title": "Optional pod tolerations" + }, + "extendedResources": { + "type": "object", + "additionalProperties": { + "type": "integer", + "format": "int64" + }, + "title": "Optional. Name and number of the extended resources" } }, "title": "ComputeTemplate can be reused by any compute units like worker group, workspace, image build job, etc", diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index 5411dc7873..46e23f9ffb 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -362,6 +362,17 @@ func TestBuildPod(t *testing.T) { podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379") pod = BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", nil, utils.GetCRDType(""), fqdnRayIP) + // Check resources + rayContainer = pod.Spec.Containers[utils.RayContainerIndex] + expectedResources := corev1.ResourceRequirements{ + Limits: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: testMemoryLimit, + "nvidia.com/gpu": resource.MustParse("3"), + }, + } + assert.Equal(t, expectedResources.Limits, rayContainer.Resources.Limits, "Resource limits do not match") + // Check environment variables rayContainer = pod.Spec.Containers[utils.RayContainerIndex] checkContainerEnv(t, rayContainer, utils.RAY_ADDRESS, "raycluster-sample-head-svc.default.svc.cluster.local:6379")