diff --git a/apiserver/HACluster.md b/apiserver/HACluster.md new file mode 100644 index 0000000000..48942310ce --- /dev/null +++ b/apiserver/HACluster.md @@ -0,0 +1,217 @@ +# Creating HA cluster with API Server + +One of the issue for long-running Ray applications, for example, Ray Serve is that Ray Head node is a single +point of failure, which means that if the Head node dies, complete cluster has to be restarted. Fortunately, +KubeRay cluster provides an option to create +[fault tolerance Ray cluster](https://docs.ray.io/en/master/cluster/kubernetes/user-guides/kuberay-gcs-ft.html). +The similar type of highly available Ray cluster can also be created using API server. The foundation of this +approach is ensuring high availability Global Control Service (GCS) data. GCS manages cluster-level +metadata. By default, the GCS lacks fault tolerance as it stores all data in-memory, and a failure can cause the +entire Ray cluster to fail. To make the GCS fault tolerant, you must have a high-availability Redis. This way, +in the event of a GCS restart, it retrieves all the data from the Redis instance and resumes its regular +functioning. + +## Creating external Redis cluster + +A comprehensive documentation on creating Redis cluster on Kubernetes can be found +[here]( https://www.dragonflydb.io/guides/redis-kubernetes). For this example we will use a rather simple +[yaml file](test/cluster/redis/redis.yaml). To create Redis run: + +```sh +kubectl create ns redis +kubectl apply -f /kuberay/apiserver/test/cluster/redis/redis.yaml -n redis +``` + +Note that here we are deploying redis to the `redis` namespace, that we are creating here. + +Alternatively, if you run on the cloud you can use managed version of HA Redis, which will not require +you to stand up, run, manage and monitor your own version of redis. + +## Creating Redis password secret + +Before creating your cluster, you need to create [secret](test/cluster/redis/redis_passwrd.yaml) in the +namespace where you are planning to create your Ray cluster (remember, that secret is visible only within a given +namespace). To create a secret for using external redis, run: + +```sh +kubectl apply -f /kuberay/apiserver/test/cluster/redis/redis_passwrd.yaml +``` + +## Ray Code for testing + +For both Ray Jobs and Ray Serve we recommend packaging user code in the image. For a simple testing here +we will create a [config map](test/cluster/code_configmap.yaml), containing simple code, that we will use for +testing. To deploy it run the following: + +```sh +kubectl apply -f /kuberay/apiserver/test/cluster/code_configmap.yaml +``` + +## API server request + +To create a Ray cluster we can use the following curl command: + +```sh +curl -X POST 'localhost:8888/apis/v1alpha2/namespaces/default/clusters' \ +--header 'Content-Type: application/json' \ +--data '{ + "name": "ha-cluster", + "namespace": "default", + "user": "boris", + "version": "2.7.0", + "environment": "DEV", + "annotations" : { + "ray.io/ft-enabled": "true" + }, + "clusterSpec": { + "headGroupSpec": { + "computeTemplate": "default-template", + "image": "rayproject/ray:2.7.0-py310", + "serviceType": "NodePort", + "rayStartParams": { + "dashboard-host": "0.0.0.0", + "metrics-export-port": "8080", + "num-cpus": "0", + "redis-password": "$REDIS_PASSWORD" + }, + "environment": { + "values": { + "RAY_REDIS_ADDRESS": "redis.redis.svc.cluster.local:6379" + }, + "valuesFrom": { + "REDIS_PASSWORD": { + "source": 1, + "name": "redis-password-secret", + "key": "password" + } + } + }, + "volumes": [ + { + "name": "code-sample", + "mountPath": "/home/ray/samples", + "volumeType": "CONFIGMAP", + "source": "ray-example", + "items": { + "detached_actor.py" : "detached_actor.py", + "increment_counter.py" : "increment_counter.py" + } + } + ] + }, + "workerGroupSpec": [ + { + "groupName": "small-wg", + "computeTemplate": "default-template", + "image": "rayproject/ray:2.7.0-py310", + "replicas": 1, + "minReplicas": 0, + "maxReplicas": 5, + "rayStartParams": { + "node-ip-address": "$MY_POD_IP", + "metrics-export-port": "8080" + }, + "environment": { + "values": { + "RAY_gcs_rpc_server_reconnect_timeout_s": "300" + } + }, + "volumes": [ + { + "name": "code-sample", + "mountPath": "/home/ray/samples", + "volumeType": "CONFIGMAP", + "source": "ray-example", + "items": { + "detached_actor.py" : "detached_actor.py", + "increment_counter.py" : "increment_counter.py" + } + } + ] + } + ] + } +}' +``` + +Note that computeTemplate here has to be created using this [command](test/cluster//template/simple) + +Lets discuss the important pieces here: +You need to specify annotation, that tells Ray that this is cluster with GCS fault tolerance + +```sh +ray.io/ft-enabled: "true" +``` + +For the `headGroupSpec` you need the following. In the `rayStartParams` you need to add information about Redis +password. + +```sh +"redis-password:: "$REDIS_PASSWORD" +"num-cpu": "0" +``` + +Where the value of `REDIS_PASSWORD` comes from environment variable (below). Additionally `num-cpus: 0` ensures +that that no application code runs on a head node. + +The following environment variable have to be added here: + +```sh + "environment": { + "values": { + "RAY_REDIS_ADDRESS": "redis.redis.svc.cluster.local:6379" + }, + "valuesFrom": { + "REDIS_PASSWORD": { + "source": 1, + "name": "redis-password-secret", + "key": "password" + } + } + }, +``` + +For the `workerGroupSpecs` you might want to increase `gcs_rpc_server_reconnect_timeout` by specifying the following +environment variable: + +```sh + "environment": { + "values": { + "RAY_gcs_rpc_server_reconnect_timeout_s": "300" + } + }, +``` + +This environment variable allows to increase GCS heartbeat timeout, which is 60 sec by default. The reason for +increasing it is because restart of the head node can take some time, and we want to make sure that the worker node +will not be killed during this time. + +## Testing resulting cluster + +Once the cluster is created, we can validate that it is working correctly. To do this first create a detached actor. +To do this, note the name of the head node and create a detached actor using the following command: + +```sh +kubectl exec -it -- python3 /home/ray/samples/detached_actor.py +``` + +Once this is done, open Ray dashboard (using port-forward). In the cluster tab you should see 2 nodes and in the +Actor's pane you should see created actor. + +Now you can delete head node pod: + +```sh +kubectl delete pods +``` + +The operator will recreate it. Make sure that only head node is recreated (note that it now has a different name), +while worker node stays as is. Now you can go to the dashboard and make sure that in the Cluster tab you still see +2 nodes and in the Actor's pane you still see created actor. + +For additional test run the following command: + +```sh +kubectl exec -it -- python3 /home/ray/samples/increment_counter.py +``` + +and make sure that it executes correctly. Note that the name of the head node here is different diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index ae48755659..1dce5f34cb 100755 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -53,7 +53,6 @@ func getHeadNodeEnv() []string { "RAY_PORT", "RAY_ADDRESS", "RAY_USAGE_STATS_KUBERAY_IN_USE", - "REDIS_PASSWORD", } } @@ -173,13 +172,7 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { // Here we update environment only for a container named 'ray-head' if container, _, ok := util.GetContainerByName(spec.Template.Spec.Containers, "ray-head"); ok && len(container.Env) > 0 { - env := make(map[string]string) - for _, kv := range container.Env { - if !contains(getHeadNodeEnv(), kv.Name) { - env[kv.Name] = kv.Value - } - } - headNodeSpec.Environment = env + headNodeSpec.Environment = convert_env_variables(container.Env, true) } if len(spec.Template.Spec.ServiceAccountName) > 1 { @@ -224,13 +217,7 @@ func PopulateWorkerNodeSpec(specs []v1alpha1.WorkerGroupSpec) []*api.WorkerGroup // Here we update environment only for a container named 'ray-worker' if container, _, ok := util.GetContainerByName(spec.Template.Spec.Containers, "ray-worker"); ok && len(container.Env) > 0 { - env := make(map[string]string) - for _, kv := range container.Env { - if !contains(getWorkNodeEnv(), kv.Name) { - env[kv.Name] = kv.Value - } - } - workerNodeSpec.Environment = env + workerNodeSpec.Environment = convert_env_variables(container.Env, false) } if len(spec.Template.Spec.ServiceAccountName) > 1 { @@ -247,6 +234,67 @@ func PopulateWorkerNodeSpec(specs []v1alpha1.WorkerGroupSpec) []*api.WorkerGroup return workerNodeSpecs } +func convert_env_variables(cenv []v1.EnvVar, header bool) *api.EnvironmentVariables { + env := api.EnvironmentVariables{ + Values: make(map[string]string), + ValuesFrom: make(map[string]*api.EnvValueFrom), + } + for _, kv := range cenv { + if header { + if contains(getHeadNodeEnv(), kv.Name) { + continue + } + } else { + if contains(getWorkNodeEnv(), kv.Name) { + // Skip reserved names + continue + } + } + if kv.ValueFrom != nil { + // this is value from + if kv.ValueFrom.ConfigMapKeyRef != nil { + // This is config map + env.ValuesFrom[kv.Name] = &api.EnvValueFrom{ + Source: api.EnvValueFrom_CONFIGMAP, + Name: kv.ValueFrom.ConfigMapKeyRef.Name, + Key: kv.ValueFrom.ConfigMapKeyRef.Key, + } + continue + } + if kv.ValueFrom.SecretKeyRef != nil { + // This is Secret + env.ValuesFrom[kv.Name] = &api.EnvValueFrom{ + Source: api.EnvValueFrom_SECRET, + Name: kv.ValueFrom.SecretKeyRef.Name, + Key: kv.ValueFrom.SecretKeyRef.Key, + } + continue + } + if kv.ValueFrom.ResourceFieldRef != nil { + // This resource ref + env.ValuesFrom[kv.Name] = &api.EnvValueFrom{ + Source: api.EnvValueFrom_RESOURCEFIELD, + Name: kv.ValueFrom.ResourceFieldRef.ContainerName, + Key: kv.ValueFrom.ResourceFieldRef.Resource, + } + continue + } + if kv.ValueFrom.FieldRef != nil { + // This resource ref + env.ValuesFrom[kv.Name] = &api.EnvValueFrom{ + Source: api.EnvValueFrom_FIELD, + Key: kv.ValueFrom.FieldRef.FieldPath, + } + continue + } + } else { + // This is value + env.Values[kv.Name] = kv.Value + } + } + return &env +} + func FromKubeToAPIComputeTemplate(configMap *v1.ConfigMap) *api.ComputeTemplate { cpu, _ := strconv.ParseUint(configMap.Data["cpu"], 10, 32) memory, _ := strconv.ParseUint(configMap.Data["memory"], 10, 32) diff --git a/apiserver/pkg/model/converter_test.go b/apiserver/pkg/model/converter_test.go index 685fbc8e46..3e8c8fac9e 100644 --- a/apiserver/pkg/model/converter_test.go +++ b/apiserver/pkg/model/converter_test.go @@ -76,20 +76,43 @@ var headSpecTest = v1alpha1.HeadGroupSpec{ Value: "123", }, { - Name: "AWS_SECRET", - Value: "1234", + Name: "REDIS_PASSWORD", + ValueFrom: &v1.EnvVarSource{ + SecretKeyRef: &v1.SecretKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: "redis-password-secret", + }, + Key: "password", + }, + }, }, { - Name: "RAY_PORT", - Value: "6379", + Name: "CONFIGMAP", + ValueFrom: &v1.EnvVarSource{ + ConfigMapKeyRef: &v1.ConfigMapKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: "special-config", + }, + Key: "special.how", + }, + }, }, { - Name: "RAY_ADDRESS", - Value: "127.0.0.1:6379", + Name: "ResourceFieldRef", + ValueFrom: &v1.EnvVarSource{ + ResourceFieldRef: &v1.ResourceFieldSelector{ + ContainerName: "my-container", + Resource: "resource", + }, + }, }, { - Name: "RAY_USAGE_STATS_KUBERAY_IN_USE", - Value: "1", + Name: "FieldRef", + ValueFrom: &v1.EnvVarSource{ + FieldRef: &v1.ObjectFieldSelector{ + FieldPath: "path", + }, + }, }, }, }, @@ -318,9 +341,39 @@ var expectedLabels = map[string]string{ "test": "value", } -var expectedEnv = map[string]string{ - "AWS_KEY": "123", - "AWS_SECRET": "1234", +var expectedHeadEnv = &api.EnvironmentVariables{ + Values: map[string]string{ + "AWS_KEY": "123", + }, + ValuesFrom: map[string]*api.EnvValueFrom{ + "REDIS_PASSWORD": { + Source: api.EnvValueFrom_SECRET, + Name: "redis-password-secret", + Key: "password", + }, + "CONFIGMAP": { + Source: api.EnvValueFrom_CONFIGMAP, + Name: "special-config", + Key: "special.how", + }, + "ResourceFieldRef": { + Source: api.EnvValueFrom_RESOURCEFIELD, + Name: "my-container", + Key: "resource", + }, + "FieldRef": { + Source: api.EnvValueFrom_FIELD, + Key: "path", + }, + }, +} + +var expectedEnv = &api.EnvironmentVariables{ + Values: map[string]string{ + "AWS_KEY": "123", + "AWS_SECRET": "1234", + }, + ValuesFrom: map[string]*api.EnvValueFrom{}, } var expectedTolerations = api.PodToleration{ @@ -347,8 +400,8 @@ func TestPopulateHeadNodeSpec(t *testing.T) { if !reflect.DeepEqual(groupSpec.Labels, expectedLabels) { t.Errorf("failed to convert labels, got %v, expected %v", groupSpec.Labels, expectedLabels) } - if !reflect.DeepEqual(groupSpec.Environment, expectedEnv) { - t.Errorf("failed to convert annotations, got %v, expected %v", groupSpec.Environment, expectedEnv) + if !reflect.DeepEqual(groupSpec.Environment, expectedHeadEnv) { + t.Errorf("failed to convert environment, got %v, expected %v", groupSpec.Environment, expectedHeadEnv) } } @@ -368,7 +421,7 @@ func TestPopulateWorkerNodeSpec(t *testing.T) { t.Errorf("failed to convert labels, got %v, expected %v", groupSpec.Labels, expectedLabels) } if !reflect.DeepEqual(groupSpec.Environment, expectedEnv) { - t.Errorf("failed to convert annotations, got %v, expected %v", groupSpec.Environment, expectedEnv) + t.Errorf("failed to convert environment, got %v, expected %v", groupSpec.Environment, expectedEnv) } } diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index f8b97c4daa..5de61b2900 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -102,8 +102,12 @@ var podTemplateTest = v1.PodTemplateSpec{ }, Items: []v1.KeyToPath{ { - Key: "key", - Path: "path", + Key: "key1", + Path: "path1", + }, + { + Key: "key2", + Path: "path2", }, }, }, @@ -159,7 +163,8 @@ var expectedVolumes = []*api.Volume{ VolumeType: api.Volume_CONFIGMAP, Source: "my-config-map", Items: map[string]string{ - "key": "path", + "key1": "path1", + "key2": "path2", }, }, { diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index c99fae9d5b..ccec827259 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -64,7 +64,7 @@ func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { // TODO(Basasuya & MissionToMars): The job spec depends on ClusterSpec which not all cluster-related configs are included, // such as `metadata` and `envs`. We just put `imageVersion` and `envs` in the arguments list, and should be refactored later. -func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayClusterSpec, error) { +func buildRayClusterSpec(imageVersion string, envs *api.EnvironmentVariables, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayClusterSpec, error) { computeTemplate := computeTemplateMap[clusterSpec.HeadGroupSpec.ComputeTemplate] headPodTemplate, err := buildHeadPodTemplate(imageVersion, envs, clusterSpec.HeadGroupSpec, computeTemplate) if err != nil { @@ -128,7 +128,7 @@ func buildNodeGroupAnnotations(computeTemplate *api.ComputeTemplate, image strin } // Build head node template -func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { +func buildHeadPodTemplate(imageVersion string, envs *api.EnvironmentVariables, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) if len(spec.Image) != 0 { image = spec.Image @@ -215,20 +215,17 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api container.Resources.Requests[v1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) container.Resources.Limits[v1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) } - for k, v := range envs { - container.Env = append(container.Env, v1.EnvVar{ - Name: k, Value: v, - }) + globalEnv := convertEnvironmentVariables(envs) + if len(globalEnv) > 0 { + container.Env = append(container.Env, globalEnv...) } // Add specific environments - if spec.Environment != nil { - for key, value := range spec.Environment { - container.Env = append(container.Env, v1.EnvVar{ - Name: key, Value: value, - }) - } + specEnv := convertEnvironmentVariables(spec.Environment) + if len(specEnv) > 0 { + container.Env = append(container.Env, specEnv...) } + // Replace container podTemplateSpec.Spec.Containers[index] = container } @@ -273,6 +270,73 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api return &podTemplateSpec, nil } +// Convert environment variables +func convertEnvironmentVariables(envs *api.EnvironmentVariables) []v1.EnvVar { + converted := []v1.EnvVar{} + if envs == nil { + return converted + } + if envs.Values != nil && len(envs.Values) > 0 { + // Add values + for key, value := range envs.Values { + converted = append(converted, v1.EnvVar{ + Name: key, Value: value, + }) + } + } + if envs.ValuesFrom != nil && len(envs.ValuesFrom) > 0 { + // Add values ref + for key, value := range envs.ValuesFrom { + switch value.Source { + case api.EnvValueFrom_CONFIGMAP: + converted = append(converted, v1.EnvVar{ + Name: key, + ValueFrom: &v1.EnvVarSource{ + ConfigMapKeyRef: &v1.ConfigMapKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: value.Name, + }, + Key: value.Key, + }, + }, + }) + case api.EnvValueFrom_SECRET: + converted = append(converted, v1.EnvVar{ + Name: key, + ValueFrom: &v1.EnvVarSource{ + SecretKeyRef: &v1.SecretKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: value.Name, + }, + Key: value.Key, + }, + }, + }) + case api.EnvValueFrom_RESOURCEFIELD: + converted = append(converted, v1.EnvVar{ + Name: key, + ValueFrom: &v1.EnvVarSource{ + ResourceFieldRef: &v1.ResourceFieldSelector{ + ContainerName: value.Name, + Resource: value.Key, + }, + }, + }) + default: + converted = append(converted, v1.EnvVar{ + Name: key, + ValueFrom: &v1.EnvVarSource{ + FieldRef: &v1.ObjectFieldSelector{ + FieldPath: value.Key, + }, + }, + }) + } + } + } + return converted +} + // Convert Toleration operator from string func convertTolerationOperator(val string) v1.TolerationOperator { if val == "Exists" { @@ -298,7 +362,7 @@ func constructRayImage(containerImage string, version string) string { } // Build worker pod template -func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *api.WorkerGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { +func buildWorkerPodTemplate(imageVersion string, envs *api.EnvironmentVariables, spec *api.WorkerGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { // If user doesn't provide the image, let's use the default image instead. // TODO: verify the versions in the range image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) @@ -436,20 +500,17 @@ func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *a container.Resources.Limits[v1.ResourceName(accelerator)] = resource.MustParse(fmt.Sprint(gpu)) } - for k, v := range envs { - container.Env = append(container.Env, v1.EnvVar{ - Name: k, Value: v, - }) + globalEnv := convertEnvironmentVariables(envs) + if len(globalEnv) > 0 { + container.Env = append(container.Env, globalEnv...) } // Add specific environments - if spec.Environment != nil { - for key, value := range spec.Environment { - container.Env = append(container.Env, v1.EnvVar{ - Name: key, Value: value, - }) - } + specEnv := convertEnvironmentVariables(spec.Environment) + if len(specEnv) > 0 { + container.Env = append(container.Env, specEnv...) } + // Replace container podTemplateSpec.Spec.Containers[index] = container } @@ -544,7 +605,7 @@ func buildVols(apiVolumes []*api.Volume) ([]v1.Volume, error) { // Add items items := []v1.KeyToPath{} for key, value := range rayVol.Items { - items = append(vol.ConfigMap.Items, v1.KeyToPath{Key: key, Path: value}) + items = append(items, v1.KeyToPath{Key: key, Path: value}) } vol.ConfigMap.Items = items } @@ -563,7 +624,7 @@ func buildVols(apiVolumes []*api.Volume) ([]v1.Volume, error) { // Add items items := []v1.KeyToPath{} for key, value := range rayVol.Items { - items = append(vol.ConfigMap.Items, v1.KeyToPath{Key: key, Path: value}) + items = append(items, v1.KeyToPath{Key: key, Path: value}) } vol.Secret.Items = items } diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index 11f0c7c172..6c6c4d04db 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -2,6 +2,7 @@ package util import ( "reflect" + "sort" "testing" api "github.com/ray-project/kuberay/proto/go_client" @@ -53,7 +54,8 @@ var testConfigMapVolume = &api.Volume{ VolumeType: api.Volume_CONFIGMAP, Source: "my-config-map", Items: map[string]string{ - "key": "path", + "key1": "path1", + "key2": "path2", }, } @@ -84,8 +86,31 @@ var headGroup = api.HeadGroupSpec{ ServiceAccount: "account", ImagePullSecret: "foo", EnableIngress: true, - Environment: map[string]string{ - "foo": "bar", + Environment: &api.EnvironmentVariables{ + Values: map[string]string{ + "AWS_KEY": "123", + }, + ValuesFrom: map[string]*api.EnvValueFrom{ + "REDIS_PASSWORD": { + Source: api.EnvValueFrom_SECRET, + Name: "redis-password-secret", + Key: "password", + }, + "CONFIGMAP": { + Source: api.EnvValueFrom_CONFIGMAP, + Name: "special-config", + Key: "special.how", + }, + "ResourceFieldRef": { + Source: api.EnvValueFrom_RESOURCEFIELD, + Name: "my-container", + Key: "resource", + }, + "FieldRef": { + Source: api.EnvValueFrom_FIELD, + Key: "path", + }, + }, }, Annotations: map[string]string{ "foo": "bar", @@ -107,8 +132,10 @@ var workerGroup = api.WorkerGroupSpec{ }, ServiceAccount: "account", ImagePullSecret: "foo", - Environment: map[string]string{ - "foo": "bar", + Environment: &api.EnvironmentVariables{ + Values: map[string]string{ + "foo": "bar", + }, }, Annotations: map[string]string{ "foo": "bar", @@ -156,6 +183,60 @@ var expectedLabels = map[string]string{ "foo": "bar", } +var expectedHeadNodeEnv = []v1.EnvVar{ + { + Name: "MY_POD_IP", + ValueFrom: &v1.EnvVarSource{ + FieldRef: &v1.ObjectFieldSelector{ + FieldPath: "status.podIP", + }, + }, + }, + { + Name: "AWS_KEY", + Value: "123", + }, + { + Name: "REDIS_PASSWORD", + ValueFrom: &v1.EnvVarSource{ + SecretKeyRef: &v1.SecretKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: "redis-password-secret", + }, + Key: "password", + }, + }, + }, + { + Name: "CONFIGMAP", + ValueFrom: &v1.EnvVarSource{ + ConfigMapKeyRef: &v1.ConfigMapKeySelector{ + LocalObjectReference: v1.LocalObjectReference{ + Name: "special-config", + }, + Key: "special.how", + }, + }, + }, + { + Name: "ResourceFieldRef", + ValueFrom: &v1.EnvVarSource{ + ResourceFieldRef: &v1.ResourceFieldSelector{ + ContainerName: "my-container", + Resource: "resource", + }, + }, + }, + { + Name: "FieldRef", + ValueFrom: &v1.EnvVarSource{ + FieldRef: &v1.ObjectFieldSelector{ + FieldPath: "path", + }, + }, + }, +} + func TestBuildVolumes(t *testing.T) { targetVolume := v1.Volume{ Name: testVolume.Name, @@ -220,8 +301,12 @@ func TestBuildVolumes(t *testing.T) { }, Items: []v1.KeyToPath{ { - Key: "key", - Path: "path", + Key: "key1", + Path: "path1", + }, + { + Key: "key2", + Path: "path2", }, }, }, @@ -346,7 +431,7 @@ func TestBuildVolumeMounts(t *testing.T) { } func TestBuildHeadPodTemplate(t *testing.T) { - podSpec, err := buildHeadPodTemplate("2.4", make(map[string]string), &headGroup, &template) + podSpec, err := buildHeadPodTemplate("2.4", &api.EnvironmentVariables{}, &headGroup, &template) assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { @@ -355,9 +440,21 @@ func TestBuildHeadPodTemplate(t *testing.T) { if podSpec.Spec.ImagePullSecrets[0].Name != "foo" { t.Errorf("failed to propagate image pull secret") } - if !containsEnv(podSpec.Spec.Containers[0].Env, "foo", "bar") { + if len(podSpec.Spec.Containers[0].Env) != 6 { t.Errorf("failed to propagate environment") } + // Sort values for comparison + sort.SliceStable(podSpec.Spec.Containers[0].Env, func(i, j int) bool { + return podSpec.Spec.Containers[0].Env[i].Name < podSpec.Spec.Containers[0].Env[j].Name + }) + sort.SliceStable(expectedHeadNodeEnv, func(i, j int) bool { + return expectedHeadNodeEnv[i].Name < expectedHeadNodeEnv[j].Name + }) + + if !reflect.DeepEqual(podSpec.Spec.Containers[0].Env, expectedHeadNodeEnv) { + t.Errorf("failed to convert environment, got %v, expected %v", podSpec.Spec.Containers[0].Env, expectedHeadNodeEnv) + } + if len(podSpec.Spec.Tolerations) != 1 { t.Errorf("failed to propagate tolerations, expected 1, got %d", len(podSpec.Spec.Tolerations)) } @@ -385,7 +482,7 @@ func TestBuildRayCluster(t *testing.T) { } func TestBuilWorkerPodTemplate(t *testing.T) { - podSpec, err := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) + podSpec, err := buildWorkerPodTemplate("2.4", &api.EnvironmentVariables{}, &workerGroup, &template) assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { diff --git a/apiserver/test/cluster/cluster_external_redis.yaml b/apiserver/test/cluster/cluster_external_redis.yaml new file mode 100644 index 0000000000..c464b00e86 --- /dev/null +++ b/apiserver/test/cluster/cluster_external_redis.yaml @@ -0,0 +1,101 @@ +apiVersion: ray.io/v1alpha1 +kind: RayCluster +metadata: + annotations: + ray.io/ft-enabled: "true" # enable Ray GCS FT + # In most cases, you don't need to set `ray.io/external-storage-namespace` because KubeRay will + # automatically set it to the UID of RayCluster. Only modify this annotation if you fully understand + # the behaviors of the Ray GCS FT and RayService to avoid misconfiguration. + # [Example]: + # ray.io/external-storage-namespace: "my-raycluster-storage" + name: raycluster-external-redis +spec: + rayVersion: '2.7.0' + headGroupSpec: + # The `rayStartParams` are used to configure the `ray start` command. + # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. + # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. + rayStartParams: + # Setting "num-cpus: 0" to avoid any Ray actors or tasks being scheduled on the Ray head Pod. + num-cpus: "0" + # redis-password should match "requirepass" in redis.conf in the ConfigMap above. + # Ray 2.3.0 changes the default redis password from "5241590000000000" to "". + redis-password: $REDIS_PASSWORD + # Pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:2.7.0 + resources: + limits: + cpu: "1" + requests: + cpu: "1" + env: + # Ray will read the RAY_REDIS_ADDRESS environment variable to establish + # a connection with the Redis server. In this instance, we use the "redis" + # Kubernetes ClusterIP service name, also created by this YAML, as the + # connection point to the Redis server. + - name: RAY_REDIS_ADDRESS + value: redis.redis.svc.cluster.local:6379 + # This environment variable is used in the `rayStartParams` above. + - name: REDIS_PASSWORD + valueFrom: + secretKeyRef: + name: redis-password-secret + key: password + ports: + - containerPort: 6379 + name: redis + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + - mountPath: /home/ray/samples + name: ray-example-configmap + volumes: + - name: ray-logs + emptyDir: {} + - name: ray-example-configmap + configMap: + name: ray-example + defaultMode: 0777 + items: + - key: detached_actor.py + path: detached_actor.py + - key: increment_counter.py + path: increment_counter.py + workerGroupSpecs: + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 10 + groupName: small-group + # The `rayStartParams` are used to configure the `ray start` command. + # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. + # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. + rayStartParams: {} + # Pod template + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray:2.7.0 + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + env: + - name: RAY_gcs_rpc_server_reconnect_timeout_s + value: "300" + resources: + limits: + cpu: "1" + requests: + cpu: "1" + volumes: + - name: ray-logs + emptyDir: {} diff --git a/apiserver/test/cluster/code_configmap.yaml b/apiserver/test/cluster/code_configmap.yaml new file mode 100644 index 0000000000..58999fb35c --- /dev/null +++ b/apiserver/test/cluster/code_configmap.yaml @@ -0,0 +1,27 @@ +apiVersion: v1 +kind: ConfigMap +metadata: + name: ray-example +data: + detached_actor.py: | + import ray + + @ray.remote(num_cpus=1) + class Counter: + def __init__(self): + self.value = 0 + + def increment(self): + self.value += 1 + return self.value + + ray.init(namespace="default_namespace") + print("detached actor. Connected to Ray") + Counter.options(name="counter_actor", lifetime="detached").remote() + increment_counter.py: | + import ray + + ray.init(namespace="default_namespace") + print("increment counter. Connected to Ray") + counter = ray.get_actor("counter_actor") + print(ray.get(counter.increment.remote())) diff --git a/apiserver/test/cluster/redis/redis.yaml b/apiserver/test/cluster/redis/redis.yaml new file mode 100644 index 0000000000..f4a31296b6 --- /dev/null +++ b/apiserver/test/cluster/redis/redis.yaml @@ -0,0 +1,63 @@ +kind: ConfigMap +apiVersion: v1 +metadata: + name: redis-config + labels: + app: redis +data: + redis.conf: |- + dir /data + port 6379 + bind 0.0.0.0 + appendonly yes + protected-mode no + requirepass 5241590000000000 + pidfile /data/redis-6379.pid +--- +apiVersion: v1 +kind: Service +metadata: + name: redis + labels: + app: redis +spec: + type: ClusterIP + ports: + - name: redis + port: 6379 + selector: + app: redis +--- +apiVersion: apps/v1 +kind: Deployment +metadata: + name: redis + labels: + app: redis +spec: + replicas: 1 + selector: + matchLabels: + app: redis + template: + metadata: + labels: + app: redis + spec: + containers: + - name: redis + image: redis:5.0.8 + command: + - "sh" + - "-c" + - "redis-server /usr/local/etc/redis/redis.conf" + ports: + - containerPort: 6379 + volumeMounts: + - name: config + mountPath: /usr/local/etc/redis/redis.conf + subPath: redis.conf + volumes: + - name: config + configMap: + name: redis-config diff --git a/apiserver/test/cluster/redis/redis_passwrd.yaml b/apiserver/test/cluster/redis/redis_passwrd.yaml new file mode 100644 index 0000000000..f65c7d9c6f --- /dev/null +++ b/apiserver/test/cluster/redis/redis_passwrd.yaml @@ -0,0 +1,9 @@ +# Redis password +apiVersion: v1 +kind: Secret +metadata: + name: redis-password-secret +type: Opaque +data: + # echo -n "5241590000000000" | base64 + password: NTI0MTU5MDAwMDAwMDAwMA== diff --git a/proto/cluster.proto b/proto/cluster.proto index d956f9b340..37baa8a1e2 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -122,6 +122,27 @@ message DeleteClusterRequest { string namespace = 2; } +message EnvValueFrom { + // Source of environment variable + enum Source{ + CONFIGMAP = 0; + SECRET = 1; + RESOURCEFIELD = 2; + FIELD = 3; + } + Source source = 1; + // Name for config map or secret, container name for resource, path for field + string name = 2; + // Key for config map or secret, resource name for resource + string key = 3; +} + +// This allows to specify both - environment variables containing values and environment values containing valueFrom +message EnvironmentVariables { + map values = 1; + map valuesFrom = 2; +} + message Cluster { // Required input field. Unique cluster name provided by user. string name = 1; @@ -150,7 +171,7 @@ message Cluster { map annotations = 7; // Optional input field. Container environment variables from user. - map envs = 8; + EnvironmentVariables envs = 8; // Output. The time that the cluster created. google.protobuf.Timestamp created_at = 9; @@ -236,7 +257,7 @@ message HeadGroupSpec { // Optional. image pull secret used by head pod string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 9; + EnvironmentVariables environment = 9; // Optional. Annotations for the head pod map annotations = 10; // Optional. Labels for the head pod @@ -266,7 +287,7 @@ message WorkerGroupSpec { // Optional. image pull secret used by worker pod string image_pull_secret = 10; // Optional. Environment variables for worker pod - map environment = 11; + EnvironmentVariables environment = 11; // Optional. Annotations for the worker pod map annotations = 12; // Optional. Labels for the worker pod diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index c8e3e951c2..094bb8a3f1 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -24,6 +24,59 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// Source of environment variable +type EnvValueFrom_Source int32 + +const ( + EnvValueFrom_CONFIGMAP EnvValueFrom_Source = 0 + EnvValueFrom_SECRET EnvValueFrom_Source = 1 + EnvValueFrom_RESOURCEFIELD EnvValueFrom_Source = 2 + EnvValueFrom_FIELD EnvValueFrom_Source = 3 +) + +// Enum value maps for EnvValueFrom_Source. +var ( + EnvValueFrom_Source_name = map[int32]string{ + 0: "CONFIGMAP", + 1: "SECRET", + 2: "RESOURCEFIELD", + 3: "FIELD", + } + EnvValueFrom_Source_value = map[string]int32{ + "CONFIGMAP": 0, + "SECRET": 1, + "RESOURCEFIELD": 2, + "FIELD": 3, + } +) + +func (x EnvValueFrom_Source) Enum() *EnvValueFrom_Source { + p := new(EnvValueFrom_Source) + *p = x + return p +} + +func (x EnvValueFrom_Source) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (EnvValueFrom_Source) Descriptor() protoreflect.EnumDescriptor { + return file_cluster_proto_enumTypes[0].Descriptor() +} + +func (EnvValueFrom_Source) Type() protoreflect.EnumType { + return &file_cluster_proto_enumTypes[0] +} + +func (x EnvValueFrom_Source) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use EnvValueFrom_Source.Descriptor instead. +func (EnvValueFrom_Source) EnumDescriptor() ([]byte, []int) { + return file_cluster_proto_rawDescGZIP(), []int{7, 0} +} + // Optional field. type Cluster_Environment int32 @@ -61,11 +114,11 @@ func (x Cluster_Environment) String() string { } func (Cluster_Environment) Descriptor() protoreflect.EnumDescriptor { - return file_cluster_proto_enumTypes[0].Descriptor() + return file_cluster_proto_enumTypes[1].Descriptor() } func (Cluster_Environment) Type() protoreflect.EnumType { - return &file_cluster_proto_enumTypes[0] + return &file_cluster_proto_enumTypes[1] } func (x Cluster_Environment) Number() protoreflect.EnumNumber { @@ -74,7 +127,7 @@ func (x Cluster_Environment) Number() protoreflect.EnumNumber { // Deprecated: Use Cluster_Environment.Descriptor instead. func (Cluster_Environment) EnumDescriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{7, 0} + return file_cluster_proto_rawDescGZIP(), []int{9, 0} } type Volume_VolumeType int32 @@ -119,11 +172,11 @@ func (x Volume_VolumeType) String() string { } func (Volume_VolumeType) Descriptor() protoreflect.EnumDescriptor { - return file_cluster_proto_enumTypes[1].Descriptor() + return file_cluster_proto_enumTypes[2].Descriptor() } func (Volume_VolumeType) Type() protoreflect.EnumType { - return &file_cluster_proto_enumTypes[1] + return &file_cluster_proto_enumTypes[2] } func (x Volume_VolumeType) Number() protoreflect.EnumNumber { @@ -132,7 +185,7 @@ func (x Volume_VolumeType) Number() protoreflect.EnumNumber { // Deprecated: Use Volume_VolumeType.Descriptor instead. func (Volume_VolumeType) EnumDescriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{9, 0} + return file_cluster_proto_rawDescGZIP(), []int{11, 0} } // If indicate hostpath, we need to let user indicate which type @@ -167,11 +220,11 @@ func (x Volume_HostPathType) String() string { } func (Volume_HostPathType) Descriptor() protoreflect.EnumDescriptor { - return file_cluster_proto_enumTypes[2].Descriptor() + return file_cluster_proto_enumTypes[3].Descriptor() } func (Volume_HostPathType) Type() protoreflect.EnumType { - return &file_cluster_proto_enumTypes[2] + return &file_cluster_proto_enumTypes[3] } func (x Volume_HostPathType) Number() protoreflect.EnumNumber { @@ -180,7 +233,7 @@ func (x Volume_HostPathType) Number() protoreflect.EnumNumber { // Deprecated: Use Volume_HostPathType.Descriptor instead. func (Volume_HostPathType) EnumDescriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{9, 1} + return file_cluster_proto_rawDescGZIP(), []int{11, 1} } type Volume_MountPropagationMode int32 @@ -216,11 +269,11 @@ func (x Volume_MountPropagationMode) String() string { } func (Volume_MountPropagationMode) Descriptor() protoreflect.EnumDescriptor { - return file_cluster_proto_enumTypes[3].Descriptor() + return file_cluster_proto_enumTypes[4].Descriptor() } func (Volume_MountPropagationMode) Type() protoreflect.EnumType { - return &file_cluster_proto_enumTypes[3] + return &file_cluster_proto_enumTypes[4] } func (x Volume_MountPropagationMode) Number() protoreflect.EnumNumber { @@ -229,7 +282,7 @@ func (x Volume_MountPropagationMode) Number() protoreflect.EnumNumber { // Deprecated: Use Volume_MountPropagationMode.Descriptor instead. func (Volume_MountPropagationMode) EnumDescriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{9, 2} + return file_cluster_proto_rawDescGZIP(), []int{11, 2} } type Volume_AccessMode int32 @@ -265,11 +318,11 @@ func (x Volume_AccessMode) String() string { } func (Volume_AccessMode) Descriptor() protoreflect.EnumDescriptor { - return file_cluster_proto_enumTypes[4].Descriptor() + return file_cluster_proto_enumTypes[5].Descriptor() } func (Volume_AccessMode) Type() protoreflect.EnumType { - return &file_cluster_proto_enumTypes[4] + return &file_cluster_proto_enumTypes[5] } func (x Volume_AccessMode) Number() protoreflect.EnumNumber { @@ -278,7 +331,7 @@ func (x Volume_AccessMode) Number() protoreflect.EnumNumber { // Deprecated: Use Volume_AccessMode.Descriptor instead. func (Volume_AccessMode) EnumDescriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{9, 3} + return file_cluster_proto_rawDescGZIP(), []int{11, 3} } type CreateClusterRequest struct { @@ -634,6 +687,127 @@ func (x *DeleteClusterRequest) GetNamespace() string { return "" } +type EnvValueFrom struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Source EnvValueFrom_Source `protobuf:"varint,1,opt,name=source,proto3,enum=proto.EnvValueFrom_Source" json:"source,omitempty"` + // Name for config map or secret, container name for resource, path for field + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + // Key for config map or secret, resource name for resource + Key string `protobuf:"bytes,3,opt,name=key,proto3" json:"key,omitempty"` +} + +func (x *EnvValueFrom) Reset() { + *x = EnvValueFrom{} + if protoimpl.UnsafeEnabled { + mi := &file_cluster_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EnvValueFrom) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnvValueFrom) ProtoMessage() {} + +func (x *EnvValueFrom) ProtoReflect() protoreflect.Message { + mi := &file_cluster_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EnvValueFrom.ProtoReflect.Descriptor instead. +func (*EnvValueFrom) Descriptor() ([]byte, []int) { + return file_cluster_proto_rawDescGZIP(), []int{7} +} + +func (x *EnvValueFrom) GetSource() EnvValueFrom_Source { + if x != nil { + return x.Source + } + return EnvValueFrom_CONFIGMAP +} + +func (x *EnvValueFrom) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *EnvValueFrom) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +// This allows to specify both - environment variables containing values and environment values containing valueFrom +type EnvironmentVariables struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Values map[string]string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ValuesFrom map[string]*EnvValueFrom `protobuf:"bytes,2,rep,name=valuesFrom,proto3" json:"valuesFrom,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *EnvironmentVariables) Reset() { + *x = EnvironmentVariables{} + if protoimpl.UnsafeEnabled { + mi := &file_cluster_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EnvironmentVariables) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnvironmentVariables) ProtoMessage() {} + +func (x *EnvironmentVariables) ProtoReflect() protoreflect.Message { + mi := &file_cluster_proto_msgTypes[8] + 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 EnvironmentVariables.ProtoReflect.Descriptor instead. +func (*EnvironmentVariables) Descriptor() ([]byte, []int) { + return file_cluster_proto_rawDescGZIP(), []int{8} +} + +func (x *EnvironmentVariables) GetValues() map[string]string { + if x != nil { + return x.Values + } + return nil +} + +func (x *EnvironmentVariables) GetValuesFrom() map[string]*EnvValueFrom { + if x != nil { + return x.ValuesFrom + } + return nil +} + type Cluster struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -653,7 +827,7 @@ type Cluster struct { // Optional. Annotations, for example, "kubernetes.io/ingress.class" to define Ingress class Annotations map[string]string `protobuf:"bytes,7,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional input field. Container environment variables from user. - Envs map[string]string `protobuf:"bytes,8,rep,name=envs,proto3" json:"envs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Envs *EnvironmentVariables `protobuf:"bytes,8,opt,name=envs,proto3" json:"envs,omitempty"` // Output. The time that the cluster created. CreatedAt *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` // Output. The time that the cluster deleted. @@ -669,7 +843,7 @@ type Cluster struct { func (x *Cluster) Reset() { *x = Cluster{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[7] + mi := &file_cluster_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -682,7 +856,7 @@ func (x *Cluster) String() string { func (*Cluster) ProtoMessage() {} func (x *Cluster) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[7] + mi := &file_cluster_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -695,7 +869,7 @@ func (x *Cluster) ProtoReflect() protoreflect.Message { // Deprecated: Use Cluster.ProtoReflect.Descriptor instead. func (*Cluster) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{7} + return file_cluster_proto_rawDescGZIP(), []int{9} } func (x *Cluster) GetName() string { @@ -747,7 +921,7 @@ func (x *Cluster) GetAnnotations() map[string]string { return nil } -func (x *Cluster) GetEnvs() map[string]string { +func (x *Cluster) GetEnvs() *EnvironmentVariables { if x != nil { return x.Envs } @@ -803,7 +977,7 @@ type ClusterSpec struct { func (x *ClusterSpec) Reset() { *x = ClusterSpec{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[8] + mi := &file_cluster_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -816,7 +990,7 @@ func (x *ClusterSpec) String() string { func (*ClusterSpec) ProtoMessage() {} func (x *ClusterSpec) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[8] + mi := &file_cluster_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -829,7 +1003,7 @@ func (x *ClusterSpec) ProtoReflect() protoreflect.Message { // Deprecated: Use ClusterSpec.ProtoReflect.Descriptor instead. func (*ClusterSpec) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{8} + return file_cluster_proto_rawDescGZIP(), []int{10} } func (x *ClusterSpec) GetHeadGroupSpec() *HeadGroupSpec { @@ -868,7 +1042,7 @@ type Volume struct { func (x *Volume) Reset() { *x = Volume{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[9] + mi := &file_cluster_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -881,7 +1055,7 @@ func (x *Volume) String() string { func (*Volume) ProtoMessage() {} func (x *Volume) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[9] + mi := &file_cluster_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -894,7 +1068,7 @@ func (x *Volume) ProtoReflect() protoreflect.Message { // Deprecated: Use Volume.ProtoReflect.Descriptor instead. func (*Volume) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{9} + return file_cluster_proto_rawDescGZIP(), []int{11} } func (x *Volume) GetMountPath() string { @@ -998,7 +1172,7 @@ type HeadGroupSpec struct { // Optional. image pull secret used by head pod ImagePullSecret string `protobuf:"bytes,8,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` // Optional. Environment variables for head pod - Environment map[string]string `protobuf:"bytes,9,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Environment *EnvironmentVariables `protobuf:"bytes,9,opt,name=environment,proto3" json:"environment,omitempty"` // Optional. Annotations for the head pod Annotations map[string]string `protobuf:"bytes,10,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. Labels for the head pod @@ -1008,7 +1182,7 @@ type HeadGroupSpec struct { func (x *HeadGroupSpec) Reset() { *x = HeadGroupSpec{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[10] + mi := &file_cluster_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1021,7 +1195,7 @@ func (x *HeadGroupSpec) String() string { func (*HeadGroupSpec) ProtoMessage() {} func (x *HeadGroupSpec) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[10] + mi := &file_cluster_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1034,7 +1208,7 @@ func (x *HeadGroupSpec) ProtoReflect() protoreflect.Message { // Deprecated: Use HeadGroupSpec.ProtoReflect.Descriptor instead. func (*HeadGroupSpec) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{10} + return file_cluster_proto_rawDescGZIP(), []int{12} } func (x *HeadGroupSpec) GetComputeTemplate() string { @@ -1093,7 +1267,7 @@ func (x *HeadGroupSpec) GetImagePullSecret() string { return "" } -func (x *HeadGroupSpec) GetEnvironment() map[string]string { +func (x *HeadGroupSpec) GetEnvironment() *EnvironmentVariables { if x != nil { return x.Environment } @@ -1141,7 +1315,7 @@ type WorkerGroupSpec struct { // Optional. image pull secret used by worker pod ImagePullSecret string `protobuf:"bytes,10,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` // Optional. Environment variables for worker pod - Environment map[string]string `protobuf:"bytes,11,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Environment *EnvironmentVariables `protobuf:"bytes,11,opt,name=environment,proto3" json:"environment,omitempty"` // Optional. Annotations for the worker pod Annotations map[string]string `protobuf:"bytes,12,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. Labels for the worker pod @@ -1151,7 +1325,7 @@ type WorkerGroupSpec struct { func (x *WorkerGroupSpec) Reset() { *x = WorkerGroupSpec{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[11] + mi := &file_cluster_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1164,7 +1338,7 @@ func (x *WorkerGroupSpec) String() string { func (*WorkerGroupSpec) ProtoMessage() {} func (x *WorkerGroupSpec) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[11] + mi := &file_cluster_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1177,7 +1351,7 @@ func (x *WorkerGroupSpec) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkerGroupSpec.ProtoReflect.Descriptor instead. func (*WorkerGroupSpec) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{11} + return file_cluster_proto_rawDescGZIP(), []int{13} } func (x *WorkerGroupSpec) GetGroupName() string { @@ -1250,7 +1424,7 @@ func (x *WorkerGroupSpec) GetImagePullSecret() string { return "" } -func (x *WorkerGroupSpec) GetEnvironment() map[string]string { +func (x *WorkerGroupSpec) GetEnvironment() *EnvironmentVariables { if x != nil { return x.Environment } @@ -1299,7 +1473,7 @@ type ClusterEvent struct { func (x *ClusterEvent) Reset() { *x = ClusterEvent{} if protoimpl.UnsafeEnabled { - mi := &file_cluster_proto_msgTypes[12] + mi := &file_cluster_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1312,7 +1486,7 @@ func (x *ClusterEvent) String() string { func (*ClusterEvent) ProtoMessage() {} func (x *ClusterEvent) ProtoReflect() protoreflect.Message { - mi := &file_cluster_proto_msgTypes[12] + mi := &file_cluster_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1325,7 +1499,7 @@ func (x *ClusterEvent) ProtoReflect() protoreflect.Message { // Deprecated: Use ClusterEvent.ProtoReflect.Descriptor instead. func (*ClusterEvent) Descriptor() ([]byte, []int) { - return file_cluster_proto_rawDescGZIP(), []int{12} + return file_cluster_proto_rawDescGZIP(), []int{14} } func (x *ClusterEvent) GetId() string { @@ -1432,298 +1606,315 @@ var file_cluster_proto_rawDesc = []byte{ 0x73, 0x74, 0x65, 0x72, 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, 0xe6, 0x06, - 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, - 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, - 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, - 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3c, 0x0a, 0x0b, 0x65, 0x6e, 0x76, - 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x45, - 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, - 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, - 0x63, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x41, - 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x12, 0x2c, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, - 0x45, 0x6e, 0x76, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x12, - 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, - 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x64, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6c, - 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, - 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4e, 0x0a, 0x10, 0x73, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x45, - 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x1a, 0x3e, 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x37, 0x0a, 0x09, 0x45, 0x6e, 0x76, 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, - 0x1a, 0x42, 0x0a, 0x14, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x45, 0x6e, 0x64, 0x70, 0x6f, - 0x69, 0x6e, 0x74, 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, 0x22, 0x40, 0x0a, 0x0b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, - 0x65, 0x6e, 0x74, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x45, 0x56, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, - 0x54, 0x45, 0x53, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x54, 0x41, - 0x47, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, - 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x22, 0x8f, 0x01, 0x0a, 0x0b, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x3c, 0x0a, 0x0f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x14, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, - 0x70, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0d, 0x68, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x53, 0x70, 0x65, 0x63, 0x12, 0x42, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x16, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x22, 0xc0, 0x06, 0x0a, 0x06, 0x56, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x70, 0x61, 0x74, - 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x50, 0x61, - 0x74, 0x68, 0x12, 0x39, 0x0a, 0x0b, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x0a, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x61, - 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x72, 0x65, - 0x61, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x40, 0x0a, 0x0e, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x70, - 0x61, 0x74, 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x48, 0x6f, - 0x73, 0x74, 0x50, 0x61, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x68, 0x6f, 0x73, 0x74, - 0x50, 0x61, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x58, 0x0a, 0x16, 0x6d, 0x6f, 0x75, 0x6e, - 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x6f, - 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x50, 0x72, 0x6f, - 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x14, 0x6d, 0x6f, - 0x75, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, - 0x64, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6c, 0x61, - 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x74, - 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, - 0x0a, 0x0a, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, - 0x65, 0x2e, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0a, 0x61, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x6f, 0x72, - 0x61, 0x67, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, - 0x67, 0x65, 0x12, 0x2e, 0x0a, 0x05, 0x69, 0x74, 0x65, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, - 0x2e, 0x49, 0x74, 0x65, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x69, 0x74, 0x65, - 0x6d, 0x73, 0x1a, 0x38, 0x0a, 0x0a, 0x49, 0x74, 0x65, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xab, 0x01, + 0x0a, 0x0c, 0x45, 0x6e, 0x76, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x32, + 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6e, 0x76, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x46, + 0x72, 0x6f, 0x6d, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x41, 0x0a, 0x06, 0x53, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x4d, 0x41, 0x50, 0x10, + 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x45, 0x43, 0x52, 0x45, 0x54, 0x10, 0x01, 0x12, 0x11, 0x0a, + 0x0d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x02, + 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x03, 0x22, 0xb3, 0x02, 0x0a, 0x14, + 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, + 0x62, 0x6c, 0x65, 0x73, 0x12, 0x3f, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6e, 0x76, + 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x4b, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x46, + 0x72, 0x6f, 0x6d, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, + 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x46, 0x72, 0x6f, + 0x6d, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x46, 0x72, + 0x6f, 0x6d, 0x1a, 0x39, 0x0a, 0x0b, 0x56, 0x61, 0x6c, 0x75, 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, 0x1a, 0x52, 0x0a, + 0x0f, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 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, 0x22, 0x71, 0x0a, 0x0a, - 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x50, 0x45, - 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4d, 0x45, 0x5f, - 0x43, 0x4c, 0x41, 0x49, 0x4d, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x4f, 0x53, 0x54, 0x5f, - 0x50, 0x41, 0x54, 0x48, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x50, 0x48, 0x45, 0x4d, 0x45, - 0x52, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x4d, - 0x41, 0x50, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x45, 0x43, 0x52, 0x45, 0x54, 0x10, 0x04, - 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x4d, 0x50, 0x54, 0x59, 0x5f, 0x44, 0x49, 0x52, 0x10, 0x05, 0x22, - 0x27, 0x0a, 0x0c, 0x48, 0x6f, 0x73, 0x74, 0x50, 0x61, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x0d, 0x0a, 0x09, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x4f, 0x52, 0x59, 0x10, 0x00, 0x12, 0x08, - 0x0a, 0x04, 0x46, 0x49, 0x4c, 0x45, 0x10, 0x01, 0x22, 0x48, 0x0a, 0x14, 0x4d, 0x6f, 0x75, 0x6e, - 0x74, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, - 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x48, 0x4f, - 0x53, 0x54, 0x54, 0x4f, 0x43, 0x4f, 0x4e, 0x54, 0x41, 0x49, 0x4e, 0x45, 0x52, 0x10, 0x01, 0x12, - 0x11, 0x0a, 0x0d, 0x42, 0x49, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, - 0x10, 0x02, 0x22, 0x27, 0x0a, 0x0a, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, - 0x12, 0x07, 0x0a, 0x03, 0x52, 0x57, 0x4f, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x58, - 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x57, 0x58, 0x10, 0x02, 0x22, 0xb5, 0x06, 0x0a, 0x0d, - 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x29, 0x0a, - 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, - 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x21, - 0x0a, 0x0c, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x67, 0x72, 0x65, - 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, - 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x52, 0x0a, 0x10, 0x72, 0x61, 0x79, 0x5f, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x52, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, - 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x61, 0x79, - 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x27, 0x0a, 0x07, 0x76, - 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x07, 0x76, 0x6f, 0x6c, - 0x75, 0x6d, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, - 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2a, 0x0a, - 0x11, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x72, - 0x65, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x50, - 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x47, 0x0a, 0x0b, 0x65, 0x6e, 0x76, - 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x12, 0x47, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x6e, - 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, - 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x38, 0x0a, 0x06, 0x6c, - 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, - 0x63, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, - 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x52, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, - 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, 0x1a, 0x3e, 0x0a, 0x10, 0x45, 0x6e, 0x76, 0x69, - 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3e, 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, - 0x6c, 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, 0x22, 0xf7, 0x06, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x72, 0x6f, - 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, - 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x6d, 0x69, 0x6e, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x6d, 0x61, - 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x54, 0x0a, 0x10, 0x72, 0x61, 0x79, - 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x07, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x52, 0x61, 0x79, 0x53, - 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0e, 0x72, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, - 0x27, 0x0a, 0x07, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, - 0x07, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, - 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x6d, - 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x49, 0x0a, - 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, - 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, - 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x65, 0x6e, 0x76, - 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x49, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, - 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x3a, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x0d, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x4c, 0x61, 0x62, 0x65, - 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, - 0x41, 0x0a, 0x13, 0x52, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, - 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, 0x1a, 0x3e, 0x0a, 0x10, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, + 0x65, 0x79, 0x12, 0x29, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6e, 0x76, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xb0, 0x06, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, + 0x73, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3c, 0x0a, + 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, + 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x0c, 0x63, + 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x70, + 0x65, 0x63, 0x12, 0x41, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2f, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6e, 0x76, 0x69, + 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x52, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x64, 0x5f, 0x61, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, + 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x23, 0x0a, 0x0d, + 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4e, + 0x0a, 0x10, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x1a, 0x3e, + 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x42, + 0x0a, 0x14, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x1a, 0x3e, 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 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, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 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, 0x22, 0xd1, 0x02, - 0x0a, 0x0c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 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, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x43, 0x0a, - 0x0f, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x52, 0x0e, 0x66, 0x69, 0x72, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x18, 0x0a, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, - 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x32, 0xff, 0x04, 0x0a, 0x0e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x12, 0x7d, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6c, - 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x22, 0x3f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x39, 0x22, 0x2e, 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, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x3a, 0x07, 0x63, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x12, 0x75, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x12, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 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, 0x63, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, 0x65, 0x7d, 0x12, 0x7e, 0x0a, 0x0b, 0x4c, 0x69, - 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x12, 0x2e, 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, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x71, 0x0a, 0x0f, 0x4c, 0x69, - 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x1d, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1f, 0x82, 0xd3, - 0xe4, 0x93, 0x02, 0x19, 0x12, 0x17, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x32, 0x2f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x83, 0x01, - 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, - 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6c, - 0x75, 0x73, 0x74, 0x65, 0x72, 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, 0x3d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x37, 0x2a, 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, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 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, + 0x38, 0x01, 0x22, 0x40, 0x0a, 0x0b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x45, 0x56, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x45, + 0x53, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x54, 0x41, 0x47, 0x49, + 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x54, 0x49, + 0x4f, 0x4e, 0x10, 0x03, 0x22, 0x8f, 0x01, 0x0a, 0x0b, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, + 0x53, 0x70, 0x65, 0x63, 0x12, 0x3c, 0x0a, 0x0f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, + 0x70, 0x65, 0x63, 0x52, 0x0d, 0x68, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, + 0x65, 0x63, 0x12, 0x42, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x22, 0xc0, 0x06, 0x0a, 0x06, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x50, 0x61, 0x74, 0x68, + 0x12, 0x39, 0x0a, 0x0b, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x0a, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, + 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, + 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x72, 0x65, 0x61, 0x64, + 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x40, 0x0a, 0x0e, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x48, 0x6f, 0x73, 0x74, + 0x50, 0x61, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x68, 0x6f, 0x73, 0x74, 0x50, 0x61, + 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x58, 0x0a, 0x16, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, + 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x6f, 0x64, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x14, 0x6d, 0x6f, 0x75, 0x6e, + 0x74, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, + 0x12, 0x2a, 0x0a, 0x10, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6c, 0x61, 0x73, 0x73, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x0a, + 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, + 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0a, 0x61, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, + 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x12, 0x2e, 0x0a, 0x05, 0x69, 0x74, 0x65, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x49, + 0x74, 0x65, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x69, 0x74, 0x65, 0x6d, 0x73, + 0x1a, 0x38, 0x0a, 0x0a, 0x49, 0x74, 0x65, 0x6d, 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, 0x22, 0x71, 0x0a, 0x0a, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x50, 0x45, 0x52, 0x53, + 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4d, 0x45, 0x5f, 0x43, 0x4c, + 0x41, 0x49, 0x4d, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x4f, 0x53, 0x54, 0x5f, 0x50, 0x41, + 0x54, 0x48, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x50, 0x48, 0x45, 0x4d, 0x45, 0x52, 0x41, + 0x4c, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x4d, 0x41, 0x50, + 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x45, 0x43, 0x52, 0x45, 0x54, 0x10, 0x04, 0x12, 0x0d, + 0x0a, 0x09, 0x45, 0x4d, 0x50, 0x54, 0x59, 0x5f, 0x44, 0x49, 0x52, 0x10, 0x05, 0x22, 0x27, 0x0a, + 0x0c, 0x48, 0x6f, 0x73, 0x74, 0x50, 0x61, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0d, 0x0a, + 0x09, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x4f, 0x52, 0x59, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, + 0x46, 0x49, 0x4c, 0x45, 0x10, 0x01, 0x22, 0x48, 0x0a, 0x14, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x50, + 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x08, + 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x48, 0x4f, 0x53, 0x54, + 0x54, 0x4f, 0x43, 0x4f, 0x4e, 0x54, 0x41, 0x49, 0x4e, 0x45, 0x52, 0x10, 0x01, 0x12, 0x11, 0x0a, + 0x0d, 0x42, 0x49, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x10, 0x02, + 0x22, 0x27, 0x0a, 0x0a, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x07, + 0x0a, 0x03, 0x52, 0x57, 0x4f, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x58, 0x10, 0x01, + 0x12, 0x07, 0x0a, 0x03, 0x52, 0x57, 0x58, 0x10, 0x02, 0x22, 0xeb, 0x05, 0x0a, 0x0d, 0x48, 0x65, + 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x29, 0x0a, 0x10, 0x63, + 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, + 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x24, 0x0a, 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x52, 0x0a, 0x10, 0x72, 0x61, 0x79, 0x5f, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x52, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x61, 0x79, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x27, 0x0a, 0x07, 0x76, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x07, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x11, 0x69, + 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, + 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, + 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, + 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x47, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, + 0x63, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x38, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x52, 0x61, 0x79, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, 0x1a, 0x3e, 0x0a, 0x10, + 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x39, 0x0a, 0x0b, + 0x4c, 0x61, 0x62, 0x65, 0x6c, 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, 0x22, 0xab, 0x06, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1d, 0x0a, 0x0a, 0x67, + 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, + 0x6d, 0x70, 0x75, 0x74, 0x65, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x6c, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x54, 0x65, 0x6d, + 0x70, 0x6c, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x69, 0x6e, 0x5f, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x6d, + 0x69, 0x6e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x61, + 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0b, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x54, 0x0a, + 0x10, 0x72, 0x61, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, + 0x52, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x61, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x12, 0x27, 0x0a, 0x07, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x18, 0x08, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x56, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x52, 0x07, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x41, 0x63, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x5f, 0x70, + 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x53, 0x65, 0x63, 0x72, 0x65, + 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x45, + 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x12, 0x49, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x6e, + 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, + 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3a, 0x0a, 0x06, 0x6c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x53, + 0x70, 0x65, 0x63, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x52, 0x61, 0x79, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, 0x1a, 0x3e, 0x0a, 0x10, 0x41, 0x6e, + 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, + 0x62, 0x65, 0x6c, 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, 0x22, 0xd1, 0x02, 0x0a, 0x0c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x45, 0x76, 0x65, 0x6e, 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, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x43, 0x0a, 0x0f, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x66, 0x69, 0x72, 0x73, + 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, + 0x6c, 0x61, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x16, 0x0a, + 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, + 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x32, 0xff, 0x04, 0x0a, 0x0e, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7d, 0x0a, 0x0d, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1b, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x22, 0x3f, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x39, 0x22, 0x2e, 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, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x73, 0x3a, 0x07, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x75, 0x0a, 0x0a, 0x47, + 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x18, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 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, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x2f, 0x7b, 0x6e, 0x61, 0x6d, + 0x65, 0x7d, 0x12, 0x7e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x12, 0x1a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x30, 0x12, 0x2e, 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, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x73, 0x12, 0x71, 0x0a, 0x0f, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x1d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x19, 0x12, 0x17, 0x2f, 0x61, + 0x70, 0x69, 0x73, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x32, 0x2f, 0x63, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x83, 0x01, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 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, 0x3d, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x37, 0x2a, 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, 0x63, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 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 ( @@ -1738,89 +1929,95 @@ func file_cluster_proto_rawDescGZIP() []byte { return file_cluster_proto_rawDescData } -var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 5) -var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 25) +var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 6) +var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 26) var file_cluster_proto_goTypes = []interface{}{ - (Cluster_Environment)(0), // 0: proto.Cluster.Environment - (Volume_VolumeType)(0), // 1: proto.Volume.VolumeType - (Volume_HostPathType)(0), // 2: proto.Volume.HostPathType - (Volume_MountPropagationMode)(0), // 3: proto.Volume.MountPropagationMode - (Volume_AccessMode)(0), // 4: proto.Volume.AccessMode - (*CreateClusterRequest)(nil), // 5: proto.CreateClusterRequest - (*GetClusterRequest)(nil), // 6: proto.GetClusterRequest - (*ListClustersRequest)(nil), // 7: proto.ListClustersRequest - (*ListClustersResponse)(nil), // 8: proto.ListClustersResponse - (*ListAllClustersRequest)(nil), // 9: proto.ListAllClustersRequest - (*ListAllClustersResponse)(nil), // 10: proto.ListAllClustersResponse - (*DeleteClusterRequest)(nil), // 11: proto.DeleteClusterRequest - (*Cluster)(nil), // 12: proto.Cluster - (*ClusterSpec)(nil), // 13: proto.ClusterSpec - (*Volume)(nil), // 14: proto.Volume - (*HeadGroupSpec)(nil), // 15: proto.HeadGroupSpec - (*WorkerGroupSpec)(nil), // 16: proto.WorkerGroupSpec - (*ClusterEvent)(nil), // 17: proto.ClusterEvent - nil, // 18: proto.Cluster.AnnotationsEntry - nil, // 19: proto.Cluster.EnvsEntry - nil, // 20: proto.Cluster.ServiceEndpointEntry - nil, // 21: proto.Volume.ItemsEntry - nil, // 22: proto.HeadGroupSpec.RayStartParamsEntry - nil, // 23: proto.HeadGroupSpec.EnvironmentEntry - nil, // 24: proto.HeadGroupSpec.AnnotationsEntry - nil, // 25: proto.HeadGroupSpec.LabelsEntry - nil, // 26: proto.WorkerGroupSpec.RayStartParamsEntry - nil, // 27: proto.WorkerGroupSpec.EnvironmentEntry - nil, // 28: proto.WorkerGroupSpec.AnnotationsEntry - nil, // 29: proto.WorkerGroupSpec.LabelsEntry - (*timestamppb.Timestamp)(nil), // 30: google.protobuf.Timestamp - (*emptypb.Empty)(nil), // 31: google.protobuf.Empty + (EnvValueFrom_Source)(0), // 0: proto.EnvValueFrom.Source + (Cluster_Environment)(0), // 1: proto.Cluster.Environment + (Volume_VolumeType)(0), // 2: proto.Volume.VolumeType + (Volume_HostPathType)(0), // 3: proto.Volume.HostPathType + (Volume_MountPropagationMode)(0), // 4: proto.Volume.MountPropagationMode + (Volume_AccessMode)(0), // 5: proto.Volume.AccessMode + (*CreateClusterRequest)(nil), // 6: proto.CreateClusterRequest + (*GetClusterRequest)(nil), // 7: proto.GetClusterRequest + (*ListClustersRequest)(nil), // 8: proto.ListClustersRequest + (*ListClustersResponse)(nil), // 9: proto.ListClustersResponse + (*ListAllClustersRequest)(nil), // 10: proto.ListAllClustersRequest + (*ListAllClustersResponse)(nil), // 11: proto.ListAllClustersResponse + (*DeleteClusterRequest)(nil), // 12: proto.DeleteClusterRequest + (*EnvValueFrom)(nil), // 13: proto.EnvValueFrom + (*EnvironmentVariables)(nil), // 14: proto.EnvironmentVariables + (*Cluster)(nil), // 15: proto.Cluster + (*ClusterSpec)(nil), // 16: proto.ClusterSpec + (*Volume)(nil), // 17: proto.Volume + (*HeadGroupSpec)(nil), // 18: proto.HeadGroupSpec + (*WorkerGroupSpec)(nil), // 19: proto.WorkerGroupSpec + (*ClusterEvent)(nil), // 20: proto.ClusterEvent + nil, // 21: proto.EnvironmentVariables.ValuesEntry + nil, // 22: proto.EnvironmentVariables.ValuesFromEntry + nil, // 23: proto.Cluster.AnnotationsEntry + nil, // 24: proto.Cluster.ServiceEndpointEntry + nil, // 25: proto.Volume.ItemsEntry + nil, // 26: proto.HeadGroupSpec.RayStartParamsEntry + nil, // 27: proto.HeadGroupSpec.AnnotationsEntry + nil, // 28: proto.HeadGroupSpec.LabelsEntry + nil, // 29: proto.WorkerGroupSpec.RayStartParamsEntry + nil, // 30: proto.WorkerGroupSpec.AnnotationsEntry + nil, // 31: proto.WorkerGroupSpec.LabelsEntry + (*timestamppb.Timestamp)(nil), // 32: google.protobuf.Timestamp + (*emptypb.Empty)(nil), // 33: google.protobuf.Empty } var file_cluster_proto_depIdxs = []int32{ - 12, // 0: proto.CreateClusterRequest.cluster:type_name -> proto.Cluster - 12, // 1: proto.ListClustersResponse.clusters:type_name -> proto.Cluster - 12, // 2: proto.ListAllClustersResponse.clusters:type_name -> proto.Cluster - 0, // 3: proto.Cluster.environment:type_name -> proto.Cluster.Environment - 13, // 4: proto.Cluster.cluster_spec:type_name -> proto.ClusterSpec - 18, // 5: proto.Cluster.annotations:type_name -> proto.Cluster.AnnotationsEntry - 19, // 6: proto.Cluster.envs:type_name -> proto.Cluster.EnvsEntry - 30, // 7: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp - 30, // 8: proto.Cluster.deleted_at:type_name -> google.protobuf.Timestamp - 17, // 9: proto.Cluster.events:type_name -> proto.ClusterEvent - 20, // 10: proto.Cluster.service_endpoint:type_name -> proto.Cluster.ServiceEndpointEntry - 15, // 11: proto.ClusterSpec.head_group_spec:type_name -> proto.HeadGroupSpec - 16, // 12: proto.ClusterSpec.worker_group_spec:type_name -> proto.WorkerGroupSpec - 1, // 13: proto.Volume.volume_type:type_name -> proto.Volume.VolumeType - 2, // 14: proto.Volume.host_path_type:type_name -> proto.Volume.HostPathType - 3, // 15: proto.Volume.mount_propagation_mode:type_name -> proto.Volume.MountPropagationMode - 4, // 16: proto.Volume.accessMode:type_name -> proto.Volume.AccessMode - 21, // 17: proto.Volume.items:type_name -> proto.Volume.ItemsEntry - 22, // 18: proto.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry - 14, // 19: proto.HeadGroupSpec.volumes:type_name -> proto.Volume - 23, // 20: proto.HeadGroupSpec.environment:type_name -> proto.HeadGroupSpec.EnvironmentEntry - 24, // 21: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry - 25, // 22: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry - 26, // 23: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry - 14, // 24: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume - 27, // 25: proto.WorkerGroupSpec.environment:type_name -> proto.WorkerGroupSpec.EnvironmentEntry - 28, // 26: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry - 29, // 27: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry - 30, // 28: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp - 30, // 29: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp - 30, // 30: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp - 5, // 31: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest - 6, // 32: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest - 7, // 33: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest - 9, // 34: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest - 11, // 35: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest - 12, // 36: proto.ClusterService.CreateCluster:output_type -> proto.Cluster - 12, // 37: proto.ClusterService.GetCluster:output_type -> proto.Cluster - 8, // 38: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse - 10, // 39: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse - 31, // 40: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty - 36, // [36:41] is the sub-list for method output_type - 31, // [31:36] is the sub-list for method input_type - 31, // [31:31] is the sub-list for extension type_name - 31, // [31:31] is the sub-list for extension extendee - 0, // [0:31] is the sub-list for field type_name + 15, // 0: proto.CreateClusterRequest.cluster:type_name -> proto.Cluster + 15, // 1: proto.ListClustersResponse.clusters:type_name -> proto.Cluster + 15, // 2: proto.ListAllClustersResponse.clusters:type_name -> proto.Cluster + 0, // 3: proto.EnvValueFrom.source:type_name -> proto.EnvValueFrom.Source + 21, // 4: proto.EnvironmentVariables.values:type_name -> proto.EnvironmentVariables.ValuesEntry + 22, // 5: proto.EnvironmentVariables.valuesFrom:type_name -> proto.EnvironmentVariables.ValuesFromEntry + 1, // 6: proto.Cluster.environment:type_name -> proto.Cluster.Environment + 16, // 7: proto.Cluster.cluster_spec:type_name -> proto.ClusterSpec + 23, // 8: proto.Cluster.annotations:type_name -> proto.Cluster.AnnotationsEntry + 14, // 9: proto.Cluster.envs:type_name -> proto.EnvironmentVariables + 32, // 10: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp + 32, // 11: proto.Cluster.deleted_at:type_name -> google.protobuf.Timestamp + 20, // 12: proto.Cluster.events:type_name -> proto.ClusterEvent + 24, // 13: proto.Cluster.service_endpoint:type_name -> proto.Cluster.ServiceEndpointEntry + 18, // 14: proto.ClusterSpec.head_group_spec:type_name -> proto.HeadGroupSpec + 19, // 15: proto.ClusterSpec.worker_group_spec:type_name -> proto.WorkerGroupSpec + 2, // 16: proto.Volume.volume_type:type_name -> proto.Volume.VolumeType + 3, // 17: proto.Volume.host_path_type:type_name -> proto.Volume.HostPathType + 4, // 18: proto.Volume.mount_propagation_mode:type_name -> proto.Volume.MountPropagationMode + 5, // 19: proto.Volume.accessMode:type_name -> proto.Volume.AccessMode + 25, // 20: proto.Volume.items:type_name -> proto.Volume.ItemsEntry + 26, // 21: proto.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry + 17, // 22: proto.HeadGroupSpec.volumes:type_name -> proto.Volume + 14, // 23: proto.HeadGroupSpec.environment:type_name -> proto.EnvironmentVariables + 27, // 24: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry + 28, // 25: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry + 29, // 26: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry + 17, // 27: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume + 14, // 28: proto.WorkerGroupSpec.environment:type_name -> proto.EnvironmentVariables + 30, // 29: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry + 31, // 30: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry + 32, // 31: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp + 32, // 32: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp + 32, // 33: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp + 13, // 34: proto.EnvironmentVariables.ValuesFromEntry.value:type_name -> proto.EnvValueFrom + 6, // 35: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest + 7, // 36: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest + 8, // 37: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest + 10, // 38: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest + 12, // 39: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest + 15, // 40: proto.ClusterService.CreateCluster:output_type -> proto.Cluster + 15, // 41: proto.ClusterService.GetCluster:output_type -> proto.Cluster + 9, // 42: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse + 11, // 43: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse + 33, // 44: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty + 40, // [40:45] is the sub-list for method output_type + 35, // [35:40] is the sub-list for method input_type + 35, // [35:35] is the sub-list for extension type_name + 35, // [35:35] is the sub-list for extension extendee + 0, // [0:35] is the sub-list for field type_name } func init() { file_cluster_proto_init() } @@ -1914,7 +2111,7 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Cluster); i { + switch v := v.(*EnvValueFrom); i { case 0: return &v.state case 1: @@ -1926,7 +2123,7 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClusterSpec); i { + switch v := v.(*EnvironmentVariables); i { case 0: return &v.state case 1: @@ -1938,7 +2135,7 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Volume); i { + switch v := v.(*Cluster); i { case 0: return &v.state case 1: @@ -1950,7 +2147,7 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HeadGroupSpec); i { + switch v := v.(*ClusterSpec); i { case 0: return &v.state case 1: @@ -1962,7 +2159,7 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkerGroupSpec); i { + switch v := v.(*Volume); i { case 0: return &v.state case 1: @@ -1974,6 +2171,30 @@ func file_cluster_proto_init() { } } file_cluster_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HeadGroupSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_cluster_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkerGroupSpec); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_cluster_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClusterEvent); i { case 0: return &v.state @@ -1991,8 +2212,8 @@ func file_cluster_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_cluster_proto_rawDesc, - NumEnums: 5, - NumMessages: 25, + NumEnums: 6, + NumMessages: 26, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index 26351062ae..d1d803bf62 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -992,6 +992,17 @@ "default": "DEV", "description": "Optional field." }, + "EnvValueFromSource": { + "type": "string", + "enum": [ + "CONFIGMAP", + "SECRET", + "RESOURCEFIELD", + "FIELD" + ], + "default": "CONFIGMAP", + "title": "Source of environment variable" + }, "VolumeAccessMode": { "type": "string", "enum": [ @@ -1083,10 +1094,7 @@ "title": "Optional. Annotations, for example, \"kubernetes.io/ingress.class\" to define Ingress class" }, "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "description": "Optional input field. Container environment variables from user." }, "createdAt": { @@ -1180,6 +1188,40 @@ } } }, + "protoEnvValueFrom": { + "type": "object", + "properties": { + "source": { + "$ref": "#/definitions/EnvValueFromSource" + }, + "name": { + "type": "string", + "title": "Name for config map or secret, container name for resource, path for field" + }, + "key": { + "type": "string", + "title": "Key for config map or secret, resource name for resource" + } + } + }, + "protoEnvironmentVariables": { + "type": "object", + "properties": { + "values": { + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "valuesFrom": { + "type": "object", + "additionalProperties": { + "$ref": "#/definitions/protoEnvValueFrom" + } + } + }, + "title": "This allows to specify both - environment variables containing values and environment values containing valueFrom" + }, "protoHeadGroupSpec": { "type": "object", "properties": { @@ -1222,10 +1264,7 @@ "title": "Optional. image pull secret used by head pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for head pod" }, "annotations": { @@ -1363,10 +1402,7 @@ "title": "Optional. image pull secret used by worker pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for worker pod" }, "annotations": { diff --git a/proto/swagger/cluster.swagger.json b/proto/swagger/cluster.swagger.json index 2b5862eace..a3aa3b427e 100644 --- a/proto/swagger/cluster.swagger.json +++ b/proto/swagger/cluster.swagger.json @@ -202,6 +202,17 @@ "default": "DEV", "description": "Optional field." }, + "EnvValueFromSource": { + "type": "string", + "enum": [ + "CONFIGMAP", + "SECRET", + "RESOURCEFIELD", + "FIELD" + ], + "default": "CONFIGMAP", + "title": "Source of environment variable" + }, "VolumeAccessMode": { "type": "string", "enum": [ @@ -293,10 +304,7 @@ "title": "Optional. Annotations, for example, \"kubernetes.io/ingress.class\" to define Ingress class" }, "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "description": "Optional input field. Container environment variables from user." }, "createdAt": { @@ -390,6 +398,40 @@ } } }, + "protoEnvValueFrom": { + "type": "object", + "properties": { + "source": { + "$ref": "#/definitions/EnvValueFromSource" + }, + "name": { + "type": "string", + "title": "Name for config map or secret, container name for resource, path for field" + }, + "key": { + "type": "string", + "title": "Key for config map or secret, resource name for resource" + } + } + }, + "protoEnvironmentVariables": { + "type": "object", + "properties": { + "values": { + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "valuesFrom": { + "type": "object", + "additionalProperties": { + "$ref": "#/definitions/protoEnvValueFrom" + } + } + }, + "title": "This allows to specify both - environment variables containing values and environment values containing valueFrom" + }, "protoHeadGroupSpec": { "type": "object", "properties": { @@ -432,10 +474,7 @@ "title": "Optional. image pull secret used by head pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for head pod" }, "annotations": { @@ -573,10 +612,7 @@ "title": "Optional. image pull secret used by worker pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for worker pod" }, "annotations": { diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json index 7d3206e597..8e0df69b6b 100644 --- a/proto/swagger/job.swagger.json +++ b/proto/swagger/job.swagger.json @@ -191,6 +191,17 @@ } }, "definitions": { + "EnvValueFromSource": { + "type": "string", + "enum": [ + "CONFIGMAP", + "SECRET", + "RESOURCEFIELD", + "FIELD" + ], + "default": "CONFIGMAP", + "title": "Source of environment variable" + }, "VolumeAccessMode": { "type": "string", "enum": [ @@ -264,6 +275,40 @@ } } }, + "protoEnvValueFrom": { + "type": "object", + "properties": { + "source": { + "$ref": "#/definitions/EnvValueFromSource" + }, + "name": { + "type": "string", + "title": "Name for config map or secret, container name for resource, path for field" + }, + "key": { + "type": "string", + "title": "Key for config map or secret, resource name for resource" + } + } + }, + "protoEnvironmentVariables": { + "type": "object", + "properties": { + "values": { + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "valuesFrom": { + "type": "object", + "additionalProperties": { + "$ref": "#/definitions/protoEnvValueFrom" + } + } + }, + "title": "This allows to specify both - environment variables containing values and environment values containing valueFrom" + }, "protoHeadGroupSpec": { "type": "object", "properties": { @@ -306,10 +351,7 @@ "title": "Optional. image pull secret used by head pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for head pod" }, "annotations": { @@ -524,10 +566,7 @@ "title": "Optional. image pull secret used by worker pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for worker pod" }, "annotations": { diff --git a/proto/swagger/serve.swagger.json b/proto/swagger/serve.swagger.json index 7315594d3b..0be19e0614 100644 --- a/proto/swagger/serve.swagger.json +++ b/proto/swagger/serve.swagger.json @@ -317,6 +317,17 @@ } }, "definitions": { + "EnvValueFromSource": { + "type": "string", + "enum": [ + "CONFIGMAP", + "SECRET", + "RESOURCEFIELD", + "FIELD" + ], + "default": "CONFIGMAP", + "title": "Source of environment variable" + }, "VolumeAccessMode": { "type": "string", "enum": [ @@ -428,6 +439,40 @@ } } }, + "protoEnvValueFrom": { + "type": "object", + "properties": { + "source": { + "$ref": "#/definitions/EnvValueFromSource" + }, + "name": { + "type": "string", + "title": "Name for config map or secret, container name for resource, path for field" + }, + "key": { + "type": "string", + "title": "Key for config map or secret, resource name for resource" + } + } + }, + "protoEnvironmentVariables": { + "type": "object", + "properties": { + "values": { + "type": "object", + "additionalProperties": { + "type": "string" + } + }, + "valuesFrom": { + "type": "object", + "additionalProperties": { + "$ref": "#/definitions/protoEnvValueFrom" + } + } + }, + "title": "This allows to specify both - environment variables containing values and environment values containing valueFrom" + }, "protoHeadGroupSpec": { "type": "object", "properties": { @@ -470,10 +515,7 @@ "title": "Optional. image pull secret used by head pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for head pod" }, "annotations": { @@ -886,10 +928,7 @@ "title": "Optional. image pull secret used by worker pod" }, "environment": { - "type": "object", - "additionalProperties": { - "type": "string" - }, + "$ref": "#/definitions/protoEnvironmentVariables", "title": "Optional. Environment variables for worker pod" }, "annotations": {