From 8bc85a9a989ab92e8240fee01d41baef78c5ed58 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 01/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/converter.go | 10 + apiserver/pkg/model/converter_test.go | 28 +- apiserver/pkg/model/volumes.go | 103 ++++ apiserver/pkg/model/volumes_test.go | 127 ++++ apiserver/pkg/util/cluster.go | 60 +- apiserver/pkg/util/cluster_test.go | 65 +++ proto/cluster.proto | 49 +- proto/go_client/cluster.pb.go | 796 +++++++++++++++----------- proto/kuberay_api.swagger.json | 51 +- proto/swagger/cluster.swagger.json | 49 +- proto/swagger/job.swagger.json | 28 +- proto/swagger/serve.swagger.json | 28 +- 12 files changed, 1013 insertions(+), 381 deletions(-) create mode 100644 apiserver/pkg/model/volumes.go create mode 100644 apiserver/pkg/model/volumes_test.go diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 3f5b12ede86..b9c86c158df 100755 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -107,6 +107,10 @@ func FromCrdToApiCluster(cluster *v1alpha1.RayCluster, events []v1.Event) *api.C ClusterState: string(cluster.Status.State), } + if len(cluster.ObjectMeta.Annotations) > 0 { + pbCluster.Annotations = cluster.ObjectMeta.Annotations + } + // loop container and find the resource pbCluster.ClusterSpec = PopulateRayClusterSpec(cluster.Spec) @@ -146,6 +150,7 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { ServiceType: string(spec.ServiceType), Image: spec.Template.Annotations[util.RayClusterImageAnnotationKey], ComputeTemplate: spec.Template.Annotations[util.RayClusterComputeTemplateAnnotationKey], + Volumes: PopulateVolumes(&spec.Template), } for _, annotation := range getNodeDefaultAnnotations() { @@ -162,6 +167,10 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { headNodeSpec.Labels = spec.Template.Labels } + if *spec.EnableIngress { + headNodeSpec.EnableIngress = true + } + // 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) @@ -196,6 +205,7 @@ func PopulateWorkerNodeSpec(specs []v1alpha1.WorkerGroupSpec) []*api.WorkerGroup GroupName: spec.GroupName, Image: spec.Template.Annotations[util.RayClusterImageAnnotationKey], ComputeTemplate: spec.Template.Annotations[util.RayClusterComputeTemplateAnnotationKey], + Volumes: PopulateVolumes(&spec.Template), } for _, annotation := range getNodeDefaultAnnotations() { diff --git a/apiserver/pkg/model/converter_test.go b/apiserver/pkg/model/converter_test.go index e4e7805bfb5..63250f08bd3 100644 --- a/apiserver/pkg/model/converter_test.go +++ b/apiserver/pkg/model/converter_test.go @@ -11,7 +11,7 @@ import ( ) var ( - enableIngress = false + enableIngress = true headNodeReplicas int32 = 1 workerReplicas int32 = 5 ) @@ -195,6 +195,22 @@ var workerSpecTest = v1alpha1.WorkerGroupSpec{ }, } +var ClusterSpecTest = v1alpha1.RayCluster{ + ObjectMeta: metav1.ObjectMeta{ + Name: "raycluster-sample", + Namespace: "default", + Annotations: map[string]string{ + "kubernetes.io/ingress.class": "nginx", + }, + }, + Spec: v1alpha1.RayClusterSpec{ + HeadGroupSpec: headSpecTest, + WorkerGroupSpecs: []v1alpha1.WorkerGroupSpec{ + workerSpecTest, + }, + }, +} + var expectedAnnotations = map[string]string{ "custom": "value", } @@ -220,6 +236,9 @@ func TestPopulateHeadNodeSpec(t *testing.T) { if groupSpec.ServiceAccount != "account" { t.Errorf("failed to convert service account") } + if groupSpec.EnableIngress != *headSpecTest.EnableIngress { + t.Errorf("failed to convert enableIngress") + } if groupSpec.ImagePullSecret != "foo" { t.Errorf("failed to convert image pull secret") } @@ -254,6 +273,13 @@ func TestPopulateWorkerNodeSpec(t *testing.T) { } } +func TestPopulateRayClusterSpec(t *testing.T) { + cluster := FromCrdToApiCluster(&ClusterSpecTest, []v1.Event{}) + if len(cluster.Annotations) != 1 { + t.Errorf("failed to convert cluster's annotations") + } +} + func TestPopulateTemplate(t *testing.T) { template := FromKubeToAPIComputeTemplate(&configMapWithoutTolerations) if len(template.Tolerations) != 0 { diff --git a/apiserver/pkg/model/volumes.go b/apiserver/pkg/model/volumes.go new file mode 100644 index 00000000000..344e11b698b --- /dev/null +++ b/apiserver/pkg/model/volumes.go @@ -0,0 +1,103 @@ +package model + +import ( + api "github.com/ray-project/kuberay/proto/go_client" + v1 "k8s.io/api/core/v1" +) + +func PopulateVolumes(podTemplate *v1.PodTemplateSpec) []*api.Volume { + if len(podTemplate.Spec.Volumes) == 0 { + return nil + } + var volumes []*api.Volume + for _, vol := range podTemplate.Spec.Volumes { + mount := GetVolumeMount(podTemplate, vol.Name) + if vol.VolumeSource.HostPath != nil { + // Host Path + volumes = append(volumes, &api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + Source: vol.VolumeSource.HostPath.Path, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_VolumeType(api.Volume_HOSTTOCONTAINER), + HostPathType: GetVolumeHostPathType(&vol), + }) + continue + + } + if vol.VolumeSource.PersistentVolumeClaim != nil { + // PVC + volumes = append(volumes, &api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_PERSISTENT_VOLUME_CLAIM, + ReadOnly: vol.VolumeSource.PersistentVolumeClaim.ReadOnly, + }) + continue + } + if vol.VolumeSource.Ephemeral != nil { + // Ephimeral + request := vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.Resources.Requests[v1.ResourceStorage] + volume := api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_EPHEMERAL, + AccessMode: GetAccessMode(&vol), + Storage: request.String(), + } + if vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName != nil { + volume.StorageClassName = *vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName + } + volumes = append(volumes, &volume) + continue + } + } + return volumes +} + +func GetVolumeMount(podTemplate *v1.PodTemplateSpec, vol string) *v1.VolumeMount { + for _, container := range podTemplate.Spec.Containers { + for _, mount := range container.VolumeMounts { + if mount.Name == vol { + return &mount + } + } + } + return nil +} + +func GetVolumeMountPropagation(mount *v1.VolumeMount) api.Volume_MountPropagationMode { + if mount.MountPropagation == nil { + return api.Volume_NONE + } + if *mount.MountPropagation == v1.MountPropagationHostToContainer { + return api.Volume_HOSTTOCONTAINER + } + if *mount.MountPropagation == v1.MountPropagationBidirectional { + return api.Volume_BIDIRECTIONAL + } + return api.Volume_NONE +} + +func GetVolumeHostPathType(vol *v1.Volume) api.Volume_HostPathType { + if *vol.VolumeSource.HostPath.Type == v1.HostPathFile { + return api.Volume_FILE + } + return api.Volume_DIRECTORY +} + +func GetAccessMode(vol *v1.Volume) api.Volume_AccessMode { + modes := vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes + if len(modes) == 0 { + return api.Volume_RWO + } + if modes[0] == v1.ReadOnlyMany { + return api.Volume_ROX + } + if modes[0] == v1.ReadWriteMany { + return api.Volume_RWX + } + return api.Volume_RWO +} diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go new file mode 100644 index 00000000000..42efea7b787 --- /dev/null +++ b/apiserver/pkg/model/volumes_test.go @@ -0,0 +1,127 @@ +package model + +import ( + "fmt" + "reflect" + "testing" + + api "github.com/ray-project/kuberay/proto/go_client" + v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +var ( + hostToContainer = v1.MountPropagationHostToContainer + bidirectonal = v1.MountPropagationBidirectional +) +var podTemplateTest = v1.PodTemplateSpec{ + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "ray-head", + Image: "blah", + VolumeMounts: []v1.VolumeMount{ + { + Name: "hostPath", + MountPath: "/tmp/hostPath", + MountPropagation: &hostToContainer, + }, + { + Name: "pvc", + MountPath: "/tmp/pvc", + MountPropagation: &bidirectonal, + }, + { + Name: "ephemeral", + MountPath: "/tmp/ephemeral", + }, + }, + }, + }, + Volumes: []v1.Volume{ + { + Name: "hostPath", + VolumeSource: v1.VolumeSource{ + HostPath: &v1.HostPathVolumeSource{ + Path: "/tmp", + Type: newHostPathType(string(v1.HostPathDirectory)), + }, + }, + }, + { + Name: "pvc", + VolumeSource: v1.VolumeSource{ + PersistentVolumeClaim: &v1.PersistentVolumeClaimVolumeSource{ + ClaimName: "pvc", + ReadOnly: false, + }, + }, + }, + { + Name: "ephemeral", + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse("5Gi"), + }, + }, + }, + }, + }, + }, + }, + }, + }, +} + +var expectedVolumes = []*api.Volume{ + { + Name: "hostPath", + Source: "/tmp", + MountPath: "/tmp/hostPath", + VolumeType: api.Volume_HOST_PATH, + HostPathType: api.Volume_DIRECTORY, + MountPropagationMode: api.Volume_HOSTTOCONTAINER, + }, + { + Name: "pvc", + MountPath: "/tmp/pvc", + VolumeType: api.Volume_PERSISTENT_VOLUME_CLAIM, + MountPropagationMode: api.Volume_BIDIRECTIONAL, + ReadOnly: false, + }, + { + Name: "ephemeral", + MountPath: "/tmp/ephemeral", + MountPropagationMode: api.Volume_NONE, + VolumeType: api.Volume_EPHEMERAL, + Storage: "5Gi", + AccessMode: api.Volume_RWO, + }, +} + +// Build host path +func newHostPathType(pathType string) *v1.HostPathType { + hostPathType := new(v1.HostPathType) + *hostPathType = v1.HostPathType(pathType) + return hostPathType +} + +func TestPopulateVolumes(t *testing.T) { + volumes := PopulateVolumes(&podTemplateTest) + for i, vol := range volumes { + fmt.Printf("volume = %#v\n", vol) + if !reflect.DeepEqual(vol, expectedVolumes[i]) { + t.Errorf("failed volumes conversion, got %v, expected %v", volumes, expectedVolumes) + } + } +} diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index b251b9b647e..db5b237cbb7 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -49,9 +49,10 @@ func buildRayClusterLabels(cluster *api.Cluster) map[string]string { // Build cluster annotations func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { - annotations := map[string]string{} - // TODO: Add optional annotations - return annotations + if cluster.Annotations == nil { + return map[string]string{} + } + return cluster.Annotations } // TODO(Basasuya & MissionToMars): The job spec depends on ClusterSpec which not all cluster-related configs are included, @@ -71,6 +72,11 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe WorkerGroupSpecs: []rayalphaapi.WorkerGroupSpec{}, } + // If enable ingress is specified, add it to the head node spec. + if clusterSpec.HeadGroupSpec.EnableIngress { + rayClusterSpec.HeadGroupSpec.EnableIngress = &clusterSpec.HeadGroupSpec.EnableIngress + } + for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] workerPodTemplate := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) @@ -534,6 +540,54 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { } vols = append(vols, vol) } + if rayVol.VolumeType == api.Volume_EPHEMERAL { + vol := v1.Volume{ + Name: rayVol.Name, + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse(rayVol.Storage), + }, + }, + }, + }, + }, + }, + } + if len(rayVol.StorageClassName) > 0 { + // Populate storage class, if defined + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName = &rayVol.StorageClassName + } + + // Populate access mode if defined + switch rayVol.AccessMode { + case api.Volume_RWO: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + } + case api.Volume_RWX: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteMany, + } + case api.Volume_ROX: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadOnlyMany, + } + default: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + } + } + vols = append(vols, vol) + } } return vols diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index 1a88a42a537..de321065b97 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -6,6 +6,8 @@ import ( api "github.com/ray-project/kuberay/proto/go_client" v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) var testVolume = &api.Volume{ @@ -35,6 +37,13 @@ var testPVCVolume = &api.Volume{ ReadOnly: true, } +var testEphemeralVolume = &api.Volume{ + Name: "test-ephemeral", + VolumeType: api.Volume_EPHEMERAL, + MountPath: "/ephimeral/dir", + Storage: "10Gi", +} + // Spec for testing var headGroup = api.HeadGroupSpec{ ComputeTemplate: "foo", @@ -47,6 +56,7 @@ var headGroup = api.HeadGroupSpec{ }, ServiceAccount: "account", ImagePullSecret: "foo", + EnableIngress: true, Environment: map[string]string{ "foo": "bar", }, @@ -81,6 +91,20 @@ var workerGroup = api.WorkerGroupSpec{ }, } +var rayCluster = api.Cluster{ + Name: "test_cluster", + Namespace: "foo", + Annotations: map[string]string{ + "kubernetes.io/ingress.class": "nginx", + }, + ClusterSpec: &api.ClusterSpec{ + HeadGroupSpec: &headGroup, + WorkerGroupSpec: []*api.WorkerGroupSpec{ + &workerGroup, + }, + }, +} + var template = api.ComputeTemplate{ Name: "", Namespace: "", @@ -134,6 +158,32 @@ func TestBuildVolumes(t *testing.T) { }, }, } + + targetEphemeralVolume := v1.Volume{ + Name: testEphemeralVolume.Name, + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + AccessModes: []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + }, + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse(testEphemeralVolume.Storage), + }, + }, + }, + }, + }, + }, + } + tests := []struct { name string apiVolume []*api.Volume @@ -151,6 +201,11 @@ func TestBuildVolumes(t *testing.T) { []*api.Volume{testPVCVolume}, []v1.Volume{targetPVCVolume}, }, + { + "ephemeral test", + []*api.Volume{testEphemeralVolume}, + []v1.Volume{targetEphemeralVolume}, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -239,6 +294,16 @@ func TestBuildHeadPodTemplate(t *testing.T) { } } +func TestBuildRayCluster(t *testing.T) { + cluster := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + if len(cluster.ObjectMeta.Annotations) != 1 { + t.Errorf("failed to propagate annotations") + } + if !(*cluster.Spec.HeadGroupSpec.EnableIngress) { + t.Errorf("failed to propagate create Ingress") + } +} + func TestBuilWorkerPodTemplate(t *testing.T) { podSpec := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) diff --git a/proto/cluster.proto b/proto/cluster.proto index e28bd809eb0..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -146,24 +146,26 @@ message Cluster { // Required field. This field indicates ray cluster configuration ClusterSpec cluster_spec = 6; + // Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class + map annotations = 7; + + // Optional input field. Container environment variables from user. + map envs = 8; // Output. The time that the cluster created. - google.protobuf.Timestamp created_at = 7; + google.protobuf.Timestamp created_at = 9; // Output. The time that the cluster deleted. - google.protobuf.Timestamp deleted_at = 8; + google.protobuf.Timestamp deleted_at = 10; // Output. The status to show the cluster status.state - string cluster_state = 9; + string cluster_state = 11; // Output. The list related to the cluster. - repeated ClusterEvent events = 10; + repeated ClusterEvent events = 12; // Output. The service endpoint of the cluster - map service_endpoint = 11; - - // Optional input field. Container environment variables from user. - map envs = 12; + map service_endpoint = 13; } message ClusterSpec { @@ -178,6 +180,7 @@ message Volume { enum VolumeType { PERSISTENT_VOLUME_CLAIM = 0; HOST_PATH = 1; + EPHEMERAL = 2; // Ephemeral (https://kubernetes.io/docs/concepts/storage/ephemeral-volumes/#generic-ephemeral-volumes) } VolumeType volume_type = 2; string name = 3; @@ -197,7 +200,16 @@ message Volume { HOSTTOCONTAINER = 1; BIDIRECTIONAL = 2; } - MountPropagationMode mount_propagation_mode = 7; + MountPropagationMode mount_propagation_mode = 7; + // If indicate ephemeral, we need to let user specify volumeClaimTemplate + string storageClassName = 8; // If not defined, default is used + enum AccessMode { + RWO = 0; // ReadWriteOnce + ROX = 1; // ReadOnlyMany + RWX = 2; // ReadWriteMany + } + AccessMode accessMode = 9; + string storage = 10; // For ephemeral - required storage, GB } message HeadGroupSpec { @@ -207,21 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. The ray start params of head node group - map ray_start_params = 4; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 4; + // Optional. The ray start params of head node group. + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; + map labels = 11; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 9f5ed4618c5..9f5952e72f8 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -82,6 +82,7 @@ type Volume_VolumeType int32 const ( Volume_PERSISTENT_VOLUME_CLAIM Volume_VolumeType = 0 Volume_HOST_PATH Volume_VolumeType = 1 + Volume_EPHEMERAL Volume_VolumeType = 2 // Ephemeral (https://kubernetes.io/docs/concepts/storage/ephemeral-volumes/#generic-ephemeral-volumes) ) // Enum value maps for Volume_VolumeType. @@ -89,10 +90,12 @@ var ( Volume_VolumeType_name = map[int32]string{ 0: "PERSISTENT_VOLUME_CLAIM", 1: "HOST_PATH", + 2: "EPHEMERAL", } Volume_VolumeType_value = map[string]int32{ "PERSISTENT_VOLUME_CLAIM": 0, "HOST_PATH": 1, + "EPHEMERAL": 2, } ) @@ -220,6 +223,55 @@ func (Volume_MountPropagationMode) EnumDescriptor() ([]byte, []int) { return file_cluster_proto_rawDescGZIP(), []int{9, 2} } +type Volume_AccessMode int32 + +const ( + Volume_RWO Volume_AccessMode = 0 // ReadWriteOnce + Volume_ROX Volume_AccessMode = 1 // ReadOnlyMany + Volume_RWX Volume_AccessMode = 2 // ReadWriteMany +) + +// Enum value maps for Volume_AccessMode. +var ( + Volume_AccessMode_name = map[int32]string{ + 0: "RWO", + 1: "ROX", + 2: "RWX", + } + Volume_AccessMode_value = map[string]int32{ + "RWO": 0, + "ROX": 1, + "RWX": 2, + } +) + +func (x Volume_AccessMode) Enum() *Volume_AccessMode { + p := new(Volume_AccessMode) + *p = x + return p +} + +func (x Volume_AccessMode) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Volume_AccessMode) Descriptor() protoreflect.EnumDescriptor { + return file_cluster_proto_enumTypes[4].Descriptor() +} + +func (Volume_AccessMode) Type() protoreflect.EnumType { + return &file_cluster_proto_enumTypes[4] +} + +func (x Volume_AccessMode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Volume_AccessMode.Descriptor instead. +func (Volume_AccessMode) EnumDescriptor() ([]byte, []int) { + return file_cluster_proto_rawDescGZIP(), []int{9, 3} +} + type CreateClusterRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -589,18 +641,20 @@ type Cluster struct { Environment Cluster_Environment `protobuf:"varint,5,opt,name=environment,proto3,enum=proto.Cluster_Environment" json:"environment,omitempty"` // Required field. This field indicates ray cluster configuration ClusterSpec *ClusterSpec `protobuf:"bytes,6,opt,name=cluster_spec,json=clusterSpec,proto3" json:"cluster_spec,omitempty"` + // Optional. Annotations, for example, IngressClassAnnotationKey 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"` // Output. The time that the cluster created. - CreatedAt *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` + CreatedAt *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` // Output. The time that the cluster deleted. - DeletedAt *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=deleted_at,json=deletedAt,proto3" json:"deleted_at,omitempty"` + DeletedAt *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=deleted_at,json=deletedAt,proto3" json:"deleted_at,omitempty"` // Output. The status to show the cluster status.state - ClusterState string `protobuf:"bytes,9,opt,name=cluster_state,json=clusterState,proto3" json:"cluster_state,omitempty"` + ClusterState string `protobuf:"bytes,11,opt,name=cluster_state,json=clusterState,proto3" json:"cluster_state,omitempty"` // Output. The list related to the cluster. - Events []*ClusterEvent `protobuf:"bytes,10,rep,name=events,proto3" json:"events,omitempty"` + Events []*ClusterEvent `protobuf:"bytes,12,rep,name=events,proto3" json:"events,omitempty"` // Output. The service endpoint of the cluster - ServiceEndpoint map[string]string `protobuf:"bytes,11,rep,name=service_endpoint,json=serviceEndpoint,proto3" json:"service_endpoint,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,12,rep,name=envs,proto3" json:"envs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ServiceEndpoint map[string]string `protobuf:"bytes,13,rep,name=service_endpoint,json=serviceEndpoint,proto3" json:"service_endpoint,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *Cluster) Reset() { @@ -677,6 +731,20 @@ func (x *Cluster) GetClusterSpec() *ClusterSpec { return nil } +func (x *Cluster) GetAnnotations() map[string]string { + if x != nil { + return x.Annotations + } + return nil +} + +func (x *Cluster) GetEnvs() map[string]string { + if x != nil { + return x.Envs + } + return nil +} + func (x *Cluster) GetCreatedAt() *timestamppb.Timestamp { if x != nil { return x.CreatedAt @@ -712,13 +780,6 @@ func (x *Cluster) GetServiceEndpoint() map[string]string { return nil } -func (x *Cluster) GetEnvs() map[string]string { - if x != nil { - return x.Envs - } - return nil -} - type ClusterSpec struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -788,6 +849,10 @@ type Volume struct { ReadOnly bool `protobuf:"varint,5,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` HostPathType Volume_HostPathType `protobuf:"varint,6,opt,name=host_path_type,json=hostPathType,proto3,enum=proto.Volume_HostPathType" json:"host_path_type,omitempty"` MountPropagationMode Volume_MountPropagationMode `protobuf:"varint,7,opt,name=mount_propagation_mode,json=mountPropagationMode,proto3,enum=proto.Volume_MountPropagationMode" json:"mount_propagation_mode,omitempty"` + // If indicate ephemeral, we need to let user specify volumeClaimTemplate + StorageClassName string `protobuf:"bytes,8,opt,name=storageClassName,proto3" json:"storageClassName,omitempty"` // If not defined, default is used + AccessMode Volume_AccessMode `protobuf:"varint,9,opt,name=accessMode,proto3,enum=proto.Volume_AccessMode" json:"accessMode,omitempty"` + Storage string `protobuf:"bytes,10,opt,name=storage,proto3" json:"storage,omitempty"` // For ephemeral - required storage, GB } func (x *Volume) Reset() { @@ -871,6 +936,27 @@ func (x *Volume) GetMountPropagationMode() Volume_MountPropagationMode { return Volume_NONE } +func (x *Volume) GetStorageClassName() string { + if x != nil { + return x.StorageClassName + } + return "" +} + +func (x *Volume) GetAccessMode() Volume_AccessMode { + if x != nil { + return x.AccessMode + } + return Volume_RWO +} + +func (x *Volume) GetStorage() string { + if x != nil { + return x.Storage + } + return "" +} + type HeadGroupSpec struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -882,21 +968,24 @@ type HeadGroupSpec struct { Image string `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"` // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` - // Optional. The ray start params of head node group - RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + // Optional. The ray start params of head node group. + RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // Optional. image pull secret used by head pod - ImagePullSecret string `protobuf:"bytes,7,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` + 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,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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"` // Optional. Annotations for the head pod - Annotations map[string]string `protobuf:"bytes,9,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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 - Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -952,6 +1041,13 @@ func (x *HeadGroupSpec) GetServiceType() string { return "" } +func (x *HeadGroupSpec) GetEnableIngress() bool { + if x != nil { + return x.EnableIngress + } + return false +} + func (x *HeadGroupSpec) GetRayStartParams() map[string]string { if x != nil { return x.RayStartParams @@ -1319,7 +1415,7 @@ 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, 0xe3, 0x05, + 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, @@ -1333,252 +1429,274 @@ var file_cluster_proto_rawDesc = []byte{ 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, 0x39, - 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x07, 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, 0x08, 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, 0x09, 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, 0x0a, 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, 0x0b, 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, 0x12, 0x2c, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x0c, - 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, 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, 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, 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, 0xf4, 0x03, 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, 0x22, - 0x38, 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, 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, 0x8f, 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, 0x52, 0x0a, 0x10, 0x72, 0x61, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 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, 0x05, 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, 0x06, 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, 0x07, 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, 0x08, 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, 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, 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, 0x0a, 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, + 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, 0xac, 0x05, 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, 0x22, 0x47, 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, 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, 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, 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, + 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, 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, 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, + 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 ( @@ -1593,83 +1711,87 @@ func file_cluster_proto_rawDescGZIP() []byte { return file_cluster_proto_rawDescData } -var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 23) +var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 5) +var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 24) 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 - (*CreateClusterRequest)(nil), // 4: proto.CreateClusterRequest - (*GetClusterRequest)(nil), // 5: proto.GetClusterRequest - (*ListClustersRequest)(nil), // 6: proto.ListClustersRequest - (*ListClustersResponse)(nil), // 7: proto.ListClustersResponse - (*ListAllClustersRequest)(nil), // 8: proto.ListAllClustersRequest - (*ListAllClustersResponse)(nil), // 9: proto.ListAllClustersResponse - (*DeleteClusterRequest)(nil), // 10: proto.DeleteClusterRequest - (*Cluster)(nil), // 11: proto.Cluster - (*ClusterSpec)(nil), // 12: proto.ClusterSpec - (*Volume)(nil), // 13: proto.Volume - (*HeadGroupSpec)(nil), // 14: proto.HeadGroupSpec - (*WorkerGroupSpec)(nil), // 15: proto.WorkerGroupSpec - (*ClusterEvent)(nil), // 16: proto.ClusterEvent - nil, // 17: proto.Cluster.ServiceEndpointEntry - nil, // 18: proto.Cluster.EnvsEntry - nil, // 19: proto.HeadGroupSpec.RayStartParamsEntry - nil, // 20: proto.HeadGroupSpec.EnvironmentEntry - nil, // 21: proto.HeadGroupSpec.AnnotationsEntry - nil, // 22: proto.HeadGroupSpec.LabelsEntry - nil, // 23: proto.WorkerGroupSpec.RayStartParamsEntry - nil, // 24: proto.WorkerGroupSpec.EnvironmentEntry - nil, // 25: proto.WorkerGroupSpec.AnnotationsEntry - nil, // 26: proto.WorkerGroupSpec.LabelsEntry - (*timestamppb.Timestamp)(nil), // 27: google.protobuf.Timestamp - (*emptypb.Empty)(nil), // 28: google.protobuf.Empty + (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.HeadGroupSpec.RayStartParamsEntry + nil, // 22: proto.HeadGroupSpec.EnvironmentEntry + nil, // 23: proto.HeadGroupSpec.AnnotationsEntry + nil, // 24: proto.HeadGroupSpec.LabelsEntry + nil, // 25: proto.WorkerGroupSpec.RayStartParamsEntry + nil, // 26: proto.WorkerGroupSpec.EnvironmentEntry + nil, // 27: proto.WorkerGroupSpec.AnnotationsEntry + nil, // 28: proto.WorkerGroupSpec.LabelsEntry + (*timestamppb.Timestamp)(nil), // 29: google.protobuf.Timestamp + (*emptypb.Empty)(nil), // 30: google.protobuf.Empty } var file_cluster_proto_depIdxs = []int32{ - 11, // 0: proto.CreateClusterRequest.cluster:type_name -> proto.Cluster - 11, // 1: proto.ListClustersResponse.clusters:type_name -> proto.Cluster - 11, // 2: proto.ListAllClustersResponse.clusters:type_name -> proto.Cluster + 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 - 12, // 4: proto.Cluster.cluster_spec:type_name -> proto.ClusterSpec - 27, // 5: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp - 27, // 6: proto.Cluster.deleted_at:type_name -> google.protobuf.Timestamp - 16, // 7: proto.Cluster.events:type_name -> proto.ClusterEvent - 17, // 8: proto.Cluster.service_endpoint:type_name -> proto.Cluster.ServiceEndpointEntry - 18, // 9: proto.Cluster.envs:type_name -> proto.Cluster.EnvsEntry - 14, // 10: proto.ClusterSpec.head_group_spec:type_name -> proto.HeadGroupSpec - 15, // 11: proto.ClusterSpec.worker_group_spec:type_name -> proto.WorkerGroupSpec - 1, // 12: proto.Volume.volume_type:type_name -> proto.Volume.VolumeType - 2, // 13: proto.Volume.host_path_type:type_name -> proto.Volume.HostPathType - 3, // 14: proto.Volume.mount_propagation_mode:type_name -> proto.Volume.MountPropagationMode - 19, // 15: proto.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry - 13, // 16: proto.HeadGroupSpec.volumes:type_name -> proto.Volume - 20, // 17: proto.HeadGroupSpec.environment:type_name -> proto.HeadGroupSpec.EnvironmentEntry - 21, // 18: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry - 22, // 19: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry - 23, // 20: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry - 13, // 21: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume - 24, // 22: proto.WorkerGroupSpec.environment:type_name -> proto.WorkerGroupSpec.EnvironmentEntry - 25, // 23: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry - 26, // 24: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry - 27, // 25: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp - 27, // 26: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp - 27, // 27: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp - 4, // 28: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest - 5, // 29: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest - 6, // 30: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest - 8, // 31: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest - 10, // 32: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest - 11, // 33: proto.ClusterService.CreateCluster:output_type -> proto.Cluster - 11, // 34: proto.ClusterService.GetCluster:output_type -> proto.Cluster - 7, // 35: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse - 9, // 36: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse - 28, // 37: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty - 33, // [33:38] is the sub-list for method output_type - 28, // [28:33] is the sub-list for method input_type - 28, // [28:28] is the sub-list for extension type_name - 28, // [28:28] is the sub-list for extension extendee - 0, // [0:28] is the sub-list for field type_name + 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 + 29, // 7: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp + 29, // 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.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry + 14, // 18: proto.HeadGroupSpec.volumes:type_name -> proto.Volume + 22, // 19: proto.HeadGroupSpec.environment:type_name -> proto.HeadGroupSpec.EnvironmentEntry + 23, // 20: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry + 24, // 21: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry + 25, // 22: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry + 14, // 23: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume + 26, // 24: proto.WorkerGroupSpec.environment:type_name -> proto.WorkerGroupSpec.EnvironmentEntry + 27, // 25: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry + 28, // 26: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry + 29, // 27: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp + 29, // 28: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp + 29, // 29: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp + 5, // 30: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest + 6, // 31: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest + 7, // 32: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest + 9, // 33: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest + 11, // 34: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest + 12, // 35: proto.ClusterService.CreateCluster:output_type -> proto.Cluster + 12, // 36: proto.ClusterService.GetCluster:output_type -> proto.Cluster + 8, // 37: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse + 10, // 38: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse + 30, // 39: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty + 35, // [35:40] is the sub-list for method output_type + 30, // [30:35] is the sub-list for method input_type + 30, // [30:30] is the sub-list for extension type_name + 30, // [30:30] is the sub-list for extension extendee + 0, // [0:30] is the sub-list for field type_name } func init() { file_cluster_proto_init() } @@ -1840,8 +1962,8 @@ func file_cluster_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_cluster_proto_rawDesc, - NumEnums: 4, - NumMessages: 23, + NumEnums: 5, + NumMessages: 24, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index 834b8f07dba..bd6e85a685c 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -992,6 +992,15 @@ "default": "DEV", "description": "Optional field." }, + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -1014,7 +1023,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -1062,6 +1072,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "Required field. This field indicates ray cluster configuration" }, + "annotations": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class" + }, + "envs": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Optional input field. Container environment variables from user." + }, "createdAt": { "type": "string", "format": "date-time", @@ -1089,13 +1113,6 @@ "type": "string" }, "title": "Output. The service endpoint of the cluster" - }, - "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Optional input field. Container environment variables from user." } } }, @@ -1175,12 +1192,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -1267,6 +1288,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, @@ -1959,4 +1990,4 @@ } } } -} \ No newline at end of file +} diff --git a/proto/swagger/cluster.swagger.json b/proto/swagger/cluster.swagger.json index 6c9b5475c3c..9409bb30677 100644 --- a/proto/swagger/cluster.swagger.json +++ b/proto/swagger/cluster.swagger.json @@ -202,6 +202,15 @@ "default": "DEV", "description": "Optional field." }, + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -224,7 +233,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -272,6 +282,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "Required field. This field indicates ray cluster configuration" }, + "annotations": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class" + }, + "envs": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Optional input field. Container environment variables from user." + }, "createdAt": { "type": "string", "format": "date-time", @@ -299,13 +323,6 @@ "type": "string" }, "title": "Output. The service endpoint of the cluster" - }, - "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Optional input field. Container environment variables from user." } } }, @@ -385,12 +402,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -477,6 +498,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json index c3d67eeac5b..394f7362a70 100644 --- a/proto/swagger/job.swagger.json +++ b/proto/swagger/job.swagger.json @@ -191,6 +191,15 @@ } }, "definitions": { + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -213,7 +222,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -266,12 +276,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -435,6 +449,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, diff --git a/proto/swagger/serve.swagger.json b/proto/swagger/serve.swagger.json index 1428d52b417..854a3cd2e22 100644 --- a/proto/swagger/serve.swagger.json +++ b/proto/swagger/serve.swagger.json @@ -317,6 +317,15 @@ } }, "definitions": { + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -339,7 +348,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -430,12 +440,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -783,6 +797,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, From 949b9278371cd5ef92d335e9d34782f1cddb8c6b Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 11 Aug 2023 08:35:37 +0100 Subject: [PATCH 02/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From 2170b9649fe0776776ef7c072e433b721801334e Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Thu, 10 Aug 2023 15:54:11 -0700 Subject: [PATCH 03/56] Downgrade `kind` from to `v0.20.0` to `v0.11.1` (#1313) Closes #1309. I don't know why, after but downgrading the kind version to 0.11.1, I never observed the issue again. Whereas with 0.20.0, the issue is consistently reproducible. I haven't investigated which kind version between 0.11.1 and 0.20.0 is the first one that caused the issue. The reason for choosing 0.11.1 is that this is the version used in Ray CI, and we haven't observed the issue in Ray CI. https://github.com/architkulkarni/ray/blob/5cb837dbaf1e5875f4f365e67cec6b09d90bf710/ci/k8s/prep-k8s-environment.sh#L8 Signed-off-by: Archit Kulkarni --- .buildkite/kind-in-docker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.buildkite/kind-in-docker.yml b/.buildkite/kind-in-docker.yml index 80766b16eab..1f5e8c1be81 100644 --- a/.buildkite/kind-in-docker.yml +++ b/.buildkite/kind-in-docker.yml @@ -6,7 +6,7 @@ - export PATH=$PATH:/usr/local/go/bin # Install kind - - curl -Lo ./kind https://kind.sigs.k8s.io/dl/v0.20.0/kind-linux-amd64 + - curl -Lo ./kind https://kind.sigs.k8s.io/dl/v0.11.1/kind-linux-amd64 - chmod +x ./kind - mv ./kind /usr/local/bin/kind From 44c0cecf49e4846a45aa22fc5aa3c771ab964fb9 Mon Sep 17 00:00:00 2001 From: Josh Karpel Date: Thu, 10 Aug 2023 19:14:13 -0500 Subject: [PATCH 04/56] Do not update pod labels if they haven't changed (#1304) Do not update pod labels if they haven't changed --- .../controllers/ray/rayservice_controller.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 68158ad7eeb..04b12744300 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -1162,14 +1162,24 @@ func (r *RayServiceReconciler) labelHealthyServePods(ctx context.Context, rayClu if pod.Labels == nil { pod.Labels = make(map[string]string) } + + // Make a copy of the labels for comparison later, to decide whether we need to push an update. + originalLabels := make(map[string]string, len(pod.Labels)) + for key, value := range pod.Labels { + originalLabels[key] = value + } + if httpProxyClient.CheckHealth() == nil { pod.Labels[common.RayClusterServingServiceLabelKey] = common.EnableRayClusterServingServiceTrue } else { pod.Labels[common.RayClusterServingServiceLabelKey] = common.EnableRayClusterServingServiceFalse } - if updateErr := r.Update(ctx, &pod); updateErr != nil { - r.Log.Error(updateErr, "Pod label Update error!", "Pod.Error", updateErr) - return updateErr + + if !reflect.DeepEqual(originalLabels, pod.Labels) { + if updateErr := r.Update(ctx, &pod); updateErr != nil { + r.Log.Error(updateErr, "Pod label Update error!", "Pod.Error", updateErr) + return updateErr + } } } From c3a02851fc9b151bb2dc8bee6530544d9f0ce130 Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Fri, 11 Aug 2023 12:26:03 -0700 Subject: [PATCH 05/56] [CI] Run sample job YAML tests in buildkite (#1315) Sets up the Buildkite CI pipeline to test the RayJob sample YAML files using kind. Related issue number Closes #1246 --------- Signed-off-by: Archit Kulkarni --- ...ocker.yml => test-rayjob-sample-yamls.yml} | 15 +++++++++++- .../config/kind-config-buildkite.yml | 24 ++++++++++++++++++- tests/framework/config/kind-config.yaml | 4 ---- tests/framework/utils.py | 20 ++++++++++++++-- 4 files changed, 55 insertions(+), 8 deletions(-) rename .buildkite/{kind-in-docker.yml => test-rayjob-sample-yamls.yml} (83%) diff --git a/.buildkite/kind-in-docker.yml b/.buildkite/test-rayjob-sample-yamls.yml similarity index 83% rename from .buildkite/kind-in-docker.yml rename to .buildkite/test-rayjob-sample-yamls.yml index 1f5e8c1be81..0f14960c1aa 100644 --- a/.buildkite/kind-in-docker.yml +++ b/.buildkite/test-rayjob-sample-yamls.yml @@ -1,4 +1,4 @@ -- label: KubeRay operator on kind +- label: Test RayJob Sample YAMLs instance_size: large image: golang:1.17 commands: @@ -54,3 +54,16 @@ - popd - echo "Kuberay operator successfully installed." + + # Delete kind clusters + - kind delete clusters --all + + # Install python 3.10 and pip + - apt-get update + - apt-get install -y python3.10 python3-pip + + # Install requirements + - pip install -r tests/framework/config/requirements.txt + + # Run test + - BUILDKITE_ENV=true python3 tests/test_sample_rayjob_yamls.py diff --git a/tests/framework/config/kind-config-buildkite.yml b/tests/framework/config/kind-config-buildkite.yml index 57fd5e55c11..04abf43850f 100644 --- a/tests/framework/config/kind-config-buildkite.yml +++ b/tests/framework/config/kind-config-buildkite.yml @@ -14,4 +14,26 @@ nodes: kind: ClusterConfiguration apiServer: certSANs: - - "docker" \ No newline at end of file + - "docker" + - | + kind: InitConfiguration + nodeRegistration: + kubeletExtraArgs: + node-labels: "ingress-ready=true" + extraPortMappings: + - containerPort: 30265 + hostPort: 8265 + listenAddress: "0.0.0.0" + protocol: tcp + - containerPort: 30001 + hostPort: 10001 + listenAddress: "0.0.0.0" + protocol: tcp + - containerPort: 32365 + hostPort: 52365 + listenAddress: "0.0.0.0" + protocol: tcp + - containerPort: 30800 + hostPort: 8000 + listenAddress: "0.0.0.0" + protocol: tcp \ No newline at end of file diff --git a/tests/framework/config/kind-config.yaml b/tests/framework/config/kind-config.yaml index 0cd2ec5d07c..861ffd46063 100644 --- a/tests/framework/config/kind-config.yaml +++ b/tests/framework/config/kind-config.yaml @@ -9,10 +9,6 @@ nodes: kubeletExtraArgs: node-labels: "ingress-ready=true" extraPortMappings: - - containerPort: 30379 - hostPort: 6379 - listenAddress: "0.0.0.0" - protocol: tcp - containerPort: 30265 hostPort: 8265 listenAddress: "0.0.0.0" diff --git a/tests/framework/utils.py b/tests/framework/utils.py index 813dabe00a3..079903ecf30 100644 --- a/tests/framework/utils.py +++ b/tests/framework/utils.py @@ -1,5 +1,6 @@ """Utilities for integration tests of KubeRay.""" +import os import subprocess import logging from pathlib import Path @@ -34,7 +35,13 @@ class CONST: # Paths REPO_ROOT = Path(__file__).absolute().parent.parent.parent HELM_CHART_ROOT = REPO_ROOT.joinpath("helm-chart") - DEFAULT_KIND_CONFIG = REPO_ROOT.joinpath("tests/framework/config/kind-config.yaml") + + # Decide the config based on the environment + BUILDKITE_ENV = "BUILDKITE_ENV" + if os.getenv(BUILDKITE_ENV, default="") == "true": + DEFAULT_KIND_CONFIG = REPO_ROOT.joinpath("tests/framework/config/kind-config-buildkite.yml") + else: + DEFAULT_KIND_CONFIG = REPO_ROOT.joinpath("tests/framework/config/kind-config.yaml") # Ray features RAY_FT = "RAY_FT" @@ -52,7 +59,7 @@ class CONST: class KubernetesClusterManager: """ - KubernetesClusterManager controlls the lifecycle of KinD cluster and Kubernetes API client. + KubernetesClusterManager controls the lifecycle of KinD cluster and Kubernetes API client. """ def __init__(self) -> None: @@ -66,11 +73,20 @@ def delete_kind_cluster(self) -> None: k8s_client.api_client.close() self.k8s_client_dict = {} + def _adjust_kubeconfig_server_address(self) -> None: + """Modify the server address in kubeconfig to https://docker:6443""" + if os.getenv(CONST.BUILDKITE_ENV, default="") == "true": + shell_subprocess_run("kubectl config set clusters.kind-kind.server https://docker:6443") + def create_kind_cluster(self, kind_config=None) -> None: """Create a KinD cluster""" # To use NodePort service, `kind_config` needs to set `extraPortMappings` properly. kind_config = CONST.DEFAULT_KIND_CONFIG if not kind_config else kind_config shell_subprocess_run(f"kind create cluster --wait 900s --config {kind_config}") + + # Adjust the kubeconfig server address if necessary + self._adjust_kubeconfig_server_address() + config.load_kube_config() self.k8s_client_dict.update( { From 9bd1a500701eb6f6a8b89e18c101634a9ee22f09 Mon Sep 17 00:00:00 2001 From: Laurentiu Bradin <109964136+z103cb@users.noreply.github.com> Date: Sun, 13 Aug 2023 06:35:53 +0300 Subject: [PATCH 06/56] Api server makefile (#1301) Api server makefile --- apiserver/DEVELOPMENT.md | 106 ++++++++-- apiserver/Makefile | 199 ++++++++++++++++++ apiserver/deploy/base/apiserver.yaml | 5 + apiserver/deploy/base/kustomization.yaml | 4 + apiserver/deploy/local/kustomization.yaml | 18 ++ apiserver/hack/kind-cluster-config.yaml | 33 +++ .../kuberay-apiserver/templates/service.yaml | 2 +- 7 files changed, 343 insertions(+), 24 deletions(-) create mode 100644 apiserver/Makefile create mode 100644 apiserver/deploy/local/kustomization.yaml create mode 100644 apiserver/hack/kind-cluster-config.yaml diff --git a/apiserver/DEVELOPMENT.md b/apiserver/DEVELOPMENT.md index 9a30da26d39..8e0304e51a1 100644 --- a/apiserver/DEVELOPMENT.md +++ b/apiserver/DEVELOPMENT.md @@ -4,11 +4,27 @@ This guide covers the purpose, requirements, and deployment of the Kuberay API S ## Requirements -| Software | Version | Link | -| :------- | :------: | ------------------------------------------------------------: | +| Software | Version | Link | +| :------- | :------: | ------------------------------------------------------------------: | | kubectl | v1.18.3+ | [Download](https://kubernetes.io/docs/tasks/tools/install-kubectl/) | -| Go | v1.13+ | [Download](https://golang.org/dl/) | -| Docker | 19.03+ | [Download](https://docs.docker.com/install/) | +| Go | v1.17 | [Download](https://golang.org/dl/) | +| Docker | 19.03+ | [Download](https://docs.docker.com/install/) | +| GNU Make | 3.81+ | | +| curl | 7.88+ | | +| helm | v3.12.1 | [Install](https://helm.sh/docs/intro/install/) | + +### Optional Development Tools + +These tools are downloaded and installed when they are needed. The directory of the download is `../bin`. +Typing `make dev-tools` will download and install all of them. The `make clean-dev-tools` command can be used to remove all the tools from the filesystem. + +| Software | Version | Link | +| :------- | :------: | -----------------------------------------------------------------------:| +| kind | v0.19.0 | [Install](https://kind.sigs.k8s.io/docs/user/quick-start/#installation) | +| golangci-lint | v1.50.1 | [Install](https://golangci-lint.run/usage/install/) | +| kustomize | v3.8.7 | [install](https://kubectl.docs.kubernetes.io/installation/kustomize/) | +| gofumpt | v0.3.1 | To install `go install mvdan.cc/gofumpt@v0.3.1` | +| goimports | latest | To install `go install golang.org/x/tools/cmd/goimports@latest` | ## Purpose @@ -16,58 +32,102 @@ The Kuberay API Server is designed to simplify the lifecycle management of Ray c ## Build and Deployment -The backend service can be deployed locally or within a Kubernetes cluster. The HTTP service listens on port 8888. +The backend service can be deployed locally or within a Kubernetes cluster. The HTTP service listens on port 8888, the RPC port on 8887. ### Pre-requisites -Ensure that the admin Kubernetes configuration file is located at `~/.kube/config`. +Ensure that the admin Kubernetes configuration file is located at `~/.kube/config`. As a convenience, there are two makefile targets provided to help you manage a local kind cluster: -### Local Deployment +* `make cluster` -- creates a 3 node cluster (1 control plane 2 worker) named ray-api-server-cluster +* `make clean-cluster` -- deletes the cluster created with the `cluster` target + +### Local Development #### Build ```bash -go build -a -o raymgr cmd/main.go +#To build the executable +make build + +#To start the executable build above +../bin/kuberay-apiserver ``` -#### Start Service +#### Test ```bash -./raymgr +#To run the unit tests +make test +``` + +#### Start Local Service + +This will start the api server on your development machine. The golang race detector is turned on when starting the api server this way. It will use Kubernetes configuration file located at `~/.kube/config`. The service will not start if you do not have a connection to a Kubernetes cluster. + +```bash +make run ``` #### Access -Access the service at `localhost:8888`. +Access the service at `localhost:8888` for http, and `locahost:8887` for the RPC port. ### Kubernetes Deployment -#### Build +#### Build Image ```bash -./docker-image-builder.sh +#creates an image with the tag kuberay/apiserver:latest +make docker-image ``` -This script will build and optionally push the image to the remote Docker Hub (hub.byted.org). +#### Start Kubernetes Deployment -#### Start Service +Note that you should make your KubeRay API server image available by either pushing it to an image registry, such as DockerHub or Quay, or by loading the image into the Kubernetes cluster. If you are using a Kind cluster for development, you can run `make load-image` to load the newly built API server image into the Kind cluster. The operator image will also be needed to be loaded on your cluster. You can use `make operator-image` to build a fresh image from sources, and, if you are using a Kind cluster for development, you can run `make load-operator-image`. ```bash -kubectl apply -f deploy/ -``` +#Optionally, to load the api server image into the local kind cluster created with make cluster +make load-image -#### Access +#To use the helm charts +make deploy -To obtain the port, run the following command: +#To use the configuration from deploy/base +make install +``` + +#### Stop Kubernetes Deployment ```bash -NODE_PORT=$(kubectl get -o jsonpath="{.spec.ports[0].nodePort}" services backend-service -n ray-system) +#To use the helm charts +make undeploy + +#To use the configuration +make uninstall ``` -To obtain the node, run the following command: +#### Local Kind Cluster Deployment + +As a convenience for local development the following `make` targets are provided: + +* `make cluster` -- creates a local kind cluster, using the configuration from `hack/kind-cluster-config.yaml`. It creates a port mapping allowing for the service running in the kind cluster to be accessed on `localhost:318888` for HTTP and `localhost:318887` for RPC. +* `make clean-cluster` -- deletes the local kind cluster created with `make cluster` +* `load-image` -- loads the docker image defined by the `IMG` make variable into the kind cluster. The default value for variable is: `kuberay/apiserver:latest`. The name of the image can be changed by using `make load-image -e IMG=` +* `operator-image` -- Build the operator image to be loaded in your kind cluster. The tag for the operator image is `kuberay/operator:latest`. This step is optional. +* `load-operator-image` -- Load the operator image to the kind cluster created with `create-kind-cluster`. The tag for the operator image is `kuberay/operator:latest`, and the tag can be overridden using `make load-operator-image -E OPERATOR_IMAGE_TAG=`. To use the nightly operator tag, set `OPERATOR_IMAGE_TAG` to `nightly`. +* `deploy-operator` -- Deploy operator into your cluster. The tag for the operator image is `kuberay/operator:latest`. +* `undeploy-operator` -- Undeploy operator from your cluster + +When developing and testing with kind you might want to execute these targets together: ```bash -NODE_IP=$(kubectl get nodes -o jsonpath='{ $.items[*].status.addresses[?(@.type=="InternalIP")].address }') +#To create a new API server image and to deploy it on a new cluster +make docker-image cluster load-image deploy + +#To create a new API server image, operator image and deploy them on a new cluster +make docker-image operator-image cluster load-image load-operator-image deploy deploy-operator ``` -Select any IP address from the output, and use `NODE_IP:NODE_PORT` to access the service. +#### Access API Server in the Cluster + +Access the service at `localhost:318888` for http and `locahost:318887` for the RPC port. diff --git a/apiserver/Makefile b/apiserver/Makefile new file mode 100644 index 00000000000..0462d9257cd --- /dev/null +++ b/apiserver/Makefile @@ -0,0 +1,199 @@ +BUILD_TIME := $(shell date "+%F %T") +COMMIT_SHA1 := $(shell git rev-parse HEAD ) +REPO_ROOT_BIN := $(shell dirname ${PWD})/bin + +# Image URL to use all building/pushing image targets +IMG_TAG ?=latest +IMG ?= kuberay/apiserver:$(IMG_TAG) + +# Get the currently used golang install path (in GOPATH/bin, unless GOBIN is set) +ifeq (,$(shell go env GOBIN)) +GOBIN=$(shell go env GOPATH)/bin +else +GOBIN=$(shell go env GOBIN) +endif + +# Setting SHELL to bash allows bash commands to be executed by recipes. +# This is a requirement for 'setup-envtest.sh' in the test target. +# Options are set to exit when a recipe line exits non-zero or a piped command fails. +SHELL = /usr/bin/env bash -o pipefail +.SHELLFLAGS = -ec + +# Container Engine to be used for building images +ENGINE ?= docker + +all: build + +##@ General + +# The help target prints out all targets with their descriptions organized +# beneath their categories. The categories are represented by '##@' and the +# target descriptions by '##'. The awk commands is responsible for reading the +# entire set of makefiles included in this invocation, looking for lines of the +# file as xyz: ## something, and then pretty-format the target and help. Then, +# if there's a line with ##@ something, that gets pretty-printed as a category. +# More info on the usage of ANSI control characters for terminal formatting: +# https://en.wikipedia.org/wiki/ANSI_escape_code#SGR_parameters +# More info on the awk command: +# http://linuxcommand.org/lc3_adv_awk.php + +help: ## Display this help. + @awk 'BEGIN {FS = ":.*##"; printf "\nUsage:\n make \033[36m\033[0m\n"} /^[a-zA-Z_0-9-]+:.*?##/ { printf " \033[36m%-20s\033[0m %s\n", $$1, $$2 } /^##@/ { printf "\n\033[1m%s\033[0m\n", substr($$0, 5) } ' $(MAKEFILE_LIST) + +##@ Development + +fmt: ## Run go fmt against code. + go fmt ./... + +vet: ## Run go vet against code. + go vet ./... + +fumpt: gofumpt ## Run gofmtumpt against code. + $(GOFUMPT) -l -w . + +imports: goimports ## Run goimports against code. + $(GOIMPORTS) -l -w . + +test: fmt vet fumpt imports lint ## Run unit tests. + go test ./... -race -coverprofile ray-kube-api-server-coverage.out + +lint: golangci-lint fmt vet fumpt imports ## Run the linter. + $(GOLANGCI_LINT) run --timeout=3m + +##@ Build + +COMMIT := $(shell git rev-parse --short HEAD) +VERSION := $(shell git describe --tags $(shell git rev-list --tags --max-count=1)) +DATE=$(shell date -u +'%Y-%m-%dT%H:%M:%SZ') +REPO=github.com/ray-project/kuberay + +BUILD_FLAGS ?= -ldflags="-X '${REPO}/apiserver/cmd/version.Version=$(VERSION)' \ + -X '${REPO}/apiserver/cmd/version.gitCommit=$(COMMIT)' \ + -X '${REPO}/apiserver/cmd/version.buildDate=$(DATE)'" + +build: fmt vet fumpt imports lint ## Build api server binary. + go build $(BUILD_FLAGS) -o ${REPO_ROOT_BIN}/kuberay-apiserver cmd/main.go + +run: fmt vet fumpt imports lint ## Run the api server from your host. + go run -race cmd/main.go + +docker-image: test ## Build image with the api server. + ${ENGINE} build -t ${IMG} -f Dockerfile .. + +docker-push: ## Push image with the api server. + ${ENGINE} push ${IMG} + +##@ Deployment +.PHONY: install +install: kustomize ## Install the kuberay api server to the K8s cluster specified in ~/.kube/config. + cd deploy/local && $(KUSTOMIZE) edit set image kuberay/apiserver=$(IMG) + $(KUSTOMIZE) build deploy/local | kubectl create -f - + +.PHONY: uninstall +uninstall: ## Install the kuberay api server to the K8s cluster specified in ~/.kube/config. + $(KUSTOMIZE) build deploy/local | kubectl delete -f - + +.PHONY: deploy +deploy: ## Deploy via helm the kuberay api server to the K8s cluster specified in ~/.kube/config. +# Note that you should make your KubeRay APIServer image available by either pushing it to an image registry, such as DockerHub or Quay, or by loading the image into the Kubernetes cluster. +# If you are using a Kind cluster for development, you can run `make load-image` to load the newly built image into the Kind cluster. + helm upgrade --install kuberay-apiserver ../helm-chart/kuberay-apiserver --wait \ + --set image.tag=${IMG_TAG} --set image.pullPolicy=IfNotPresent + +.PHONY: undeploy +undeploy: ## Undeploy via helm the kuberay api server to the K8s cluster specified in ~/.kube/config. + helm uninstall kuberay-apiserver --wait + +##@ Development Tools + +## Location to install dependencies to +$(REPO_ROOT_BIN): + mkdir -p $(REPO_ROOT_BIN) + +## Tool Binaries +KUSTOMIZE ?= $(REPO_ROOT_BIN)/kustomize +GOIMPORTS ?= $(REPO_ROOT_BIN)/goimports +GOFUMPT ?= $(REPO_ROOT_BIN)/gofumpt +GOLANGCI_LINT ?= $(REPO_ROOT_BIN)/golangci-lint +KIND ?= $(REPO_ROOT_BIN)/kind + + +## Tool Versions +KUSTOMIZE_VERSION ?= v3.8.7 +GOFUMPT_VERSION ?= v0.3.1 +GOIMPORTS_VERSION ?= latest +GOLANGCI_LINT_VERSION ?= v1.50.1 +KIND_VERSION ?= v0.19.0 + +KUSTOMIZE_INSTALL_SCRIPT ?= "https://raw.githubusercontent.com/kubernetes-sigs/kustomize/master/hack/install_kustomize.sh" +.PHONY: kustomize +kustomize: $(KUSTOMIZE) ## Download kustomize locally if necessary. +$(KUSTOMIZE): $(REPO_ROOT_BIN) + test -s $(KUSTOMIZE) || (curl -Ss $(KUSTOMIZE_INSTALL_SCRIPT) | bash -s -- $(subst v,,$(KUSTOMIZE_VERSION)) $(REPO_ROOT_BIN);) + +.PHONY: goimports +goimports: $(GOIMPORTS) ## Download goimports locally if necessary +$(GOIMPORTS): $(REPO_ROOT_BIN) + test -s $(GOIMPORTS) || GOBIN=$(REPO_ROOT_BIN) go install golang.org/x/tools/cmd/goimports@$(GOIMPORTS_VERSION) + +.PHONY: gofumpt +gofumpt: $(GOFUMPT) ## Download gofumpt locally if necessary. +$(GOFUMPT): $(REPO_ROOT_BIN) + test -s $(GOFUMPT) || GOBIN=$(REPO_ROOT_BIN) go install mvdan.cc/gofumpt@$(GOFUMPT_VERSION) + +.PHONY: golangci-lint +golangci-lint: $(GOLANGCI_LINT) ## Download golangci_lint locally if necessary. +$(GOLANGCI_LINT): $(REPO_ROOT_BIN) + test -s $(GOLANGCI_LINT) || (curl -sSfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh | bash -s -- -b $(REPO_ROOT_BIN)/ $(GOLANGCI_LINT_VERSION)) + +.PHONY: kind +kind: $(KIND) ## Download kind locally if necessary. +$(KIND): $(REPO_ROOT_BIN) + test -s $(KIND) || GOBIN=$(REPO_ROOT_BIN) go install sigs.k8s.io/kind@$(KIND_VERSION) + +.PHONY: dev-tools +dev-tools: kind golangci-lint gofumpt kustomize goimports ## Install all development tools + +.PHONY: clean-dev-tools +clean-dev-tools: ## Remove all development tools + rm -f $(REPO_ROOT_BIN)/golangci-lint + rm -f $(REPO_ROOT_BIN)/gofumpt + rm -f $(REPO_ROOT_BIN)/kustomize + rm -f $(REPO_ROOT_BIN)/goimports + rm -f $(REPO_ROOT_BIN)/kind + + +##@ Testing Setup and Tools +KIND_CONFIG ?= hack/kind-cluster-config.yaml +KIND_CLUSTER_NAME ?= ray-api-server-cluster +OPERATOR_IMAGE_TAG ?= latest +.PHONY: cluster +cluster: kind ## Start kind development cluster. + $(KIND) create cluster -n $(KIND_CLUSTER_NAME) --config $(KIND_CONFIG) + +.PHONY: clean-cluster +clean-cluster: kind ## Delete kind development cluster. + $(KIND) delete cluster -n $(KIND_CLUSTER_NAME) + +.PHONY: load-image +load-image: ## Load the api server image to the kind cluster created with create-kind-cluster. + $(KIND) load docker-image $(IMG) -n $(KIND_CLUSTER_NAME) + +.PHONY: operator-image +operator-image: ## Build the operator image to be loaded in your kind cluster. + cd ../ray-operator && $(MAKE) docker-image -e IMG=kuberay/operator:$(OPERATOR_IMAGE_TAG) + +.PHONY: deploy-operator +deploy-operator: ## Deploy operator via helm into the K8s cluster specified in ~/.kube/config. +# Note that you should make your operatorimage available by either pushing it to an image registry, such as DockerHub or Quay, or by loading the image into the Kubernetes cluster. +# If you are using a Kind cluster for development, you can run `make load-image` to load the newly built image into the Kind cluster. + helm upgrade --install raycluster ../helm-chart/kuberay-operator --wait \ + --set image.tag=${OPERATOR_IMAGE_TAG} --set image.pullPolicy=IfNotPresent + +.PHONY: undeploy-operator +undeploy-operator: ## Undeploy operator via helm from the K8s cluster specified in ~/.kube/config. + helm uninstall raycluster --wait + +.PHONY: load-operator-image +load-operator-image: ## Load the operator image to the kind cluster created with create-kind-cluster. + $(KIND) load docker-image kuberay/operator:$(OPERATOR_IMAGE_TAG) -n $(KIND_CLUSTER_NAME) diff --git a/apiserver/deploy/base/apiserver.yaml b/apiserver/deploy/base/apiserver.yaml index 7652425c7ad..e2a52123e29 100644 --- a/apiserver/deploy/base/apiserver.yaml +++ b/apiserver/deploy/base/apiserver.yaml @@ -118,3 +118,8 @@ rules: verbs: - get - list +--- +apiVersion: v1 +kind: Namespace +metadata: + name: ray-system diff --git a/apiserver/deploy/base/kustomization.yaml b/apiserver/deploy/base/kustomization.yaml index 0f1a49666e9..e9dff844f71 100644 --- a/apiserver/deploy/base/kustomization.yaml +++ b/apiserver/deploy/base/kustomization.yaml @@ -9,3 +9,7 @@ resources: commonLabels: app.kubernetes.io/name: kuberay app.kubernetes.io/component: kuberay-apiserver +images: +- name: kuberay/apiserver + newName: kuberay/apiserver + newTag: nightly diff --git a/apiserver/deploy/local/kustomization.yaml b/apiserver/deploy/local/kustomization.yaml new file mode 100644 index 00000000000..1ae18223f8f --- /dev/null +++ b/apiserver/deploy/local/kustomization.yaml @@ -0,0 +1,18 @@ +apiVersion: kustomize.config.k8s.io/v1beta1 +kind: Kustomization +resources: +- ../base +namespace: ray-system +patchesJson6902: +- patch: |- + - op: replace + path: /spec/template/spec/containers/0/imagePullPolicy + value: IfNotPresent + target: + kind: Deployment + name: kuberay-apiserver + version: v1 +images: +- name: kuberay/apiserver + newName: kuberay/apiserver + newTag: latest diff --git a/apiserver/hack/kind-cluster-config.yaml b/apiserver/hack/kind-cluster-config.yaml new file mode 100644 index 00000000000..e0c4aeaa160 --- /dev/null +++ b/apiserver/hack/kind-cluster-config.yaml @@ -0,0 +1,33 @@ +kind: Cluster +apiVersion: kind.x-k8s.io/v1alpha4 +nodes: +- role: control-plane + image: kindest/node:v1.23.17@sha256:59c989ff8a517a93127d4a536e7014d28e235fb3529d9fba91b3951d461edfdb + kubeadmConfigPatches: + - | + kind: InitConfiguration + nodeRegistration: + kubeletExtraArgs: + node-labels: "ingress-ready=true" + extraPortMappings: + - containerPort: 30265 + hostPort: 8265 + listenAddress: "0.0.0.0" + protocol: tcp + - containerPort: 30001 + hostPort: 10001 + listenAddress: "0.0.0.0" + protocol: tcp + - containerPort: 8000 + hostPort: 8000 + listenAddress: "0.0.0.0" + - containerPort: 31888 + hostPort: 31888 + listenAddress: "0.0.0.0" + - containerPort: 31887 + hostPort: 31887 + listenAddress: "0.0.0.0" +- role: worker + image: kindest/node:v1.23.17@sha256:59c989ff8a517a93127d4a536e7014d28e235fb3529d9fba91b3951d461edfdb +- role: worker + image: kindest/node:v1.23.17@sha256:59c989ff8a517a93127d4a536e7014d28e235fb3529d9fba91b3951d461edfdb diff --git a/helm-chart/kuberay-apiserver/templates/service.yaml b/helm-chart/kuberay-apiserver/templates/service.yaml index a202a00ea47..e0fa881b5db 100644 --- a/helm-chart/kuberay-apiserver/templates/service.yaml +++ b/helm-chart/kuberay-apiserver/templates/service.yaml @@ -5,7 +5,7 @@ metadata: annotations: prometheus.io/path: /metrics prometheus.io/scrape: "true" - prometheus.io/port: "8080" + prometheus.io/port: "8888" spec: type: {{ .Values.service.type }} selector: From c8045cc560f0be9d22b97c6377ee3b8444063a0a Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 14 Aug 2023 12:39:14 -0700 Subject: [PATCH 07/56] Upgrade to Go 1.19 (#1325) Upgrade to Go 1.19 --- .buildkite/example.yml | 8 - .buildkite/test-rayjob-sample-yamls.yml | 6 +- .github/workflows/consistency-check.yaml | 8 +- .github/workflows/image-release.yaml | 8 +- .github/workflows/test-job.yaml | 12 +- apiserver/Dockerfile | 7 +- apiserver/go.mod | 2 +- apiserver/go.sum | 242 +----------------- cli/go.mod | 5 +- cli/go.sum | 7 +- proto/Dockerfile | 2 +- proto/go.mod | 2 +- proto/go.sum | 1 - ray-operator/DEVELOPMENT.md | 11 +- ray-operator/Dockerfile | 2 +- .../apis/ray/v1alpha1/rayjob_types.go | 6 +- .../controllers/ray/rayservice_controller.go | 4 +- .../ray/utils/dashboard_httpclient.go | 16 +- .../ray/utils/httpproxy_httpclient.go | 4 +- ray-operator/go.mod | 4 +- ray-operator/go.sum | 33 +-- .../clientset/versioned/fake/register.go | 14 +- .../clientset/versioned/scheme/register.go | 14 +- scripts/install-docker.sh | 3 - 24 files changed, 71 insertions(+), 350 deletions(-) delete mode 100644 .buildkite/example.yml diff --git a/.buildkite/example.yml b/.buildkite/example.yml deleted file mode 100644 index 40b29a6923d..00000000000 --- a/.buildkite/example.yml +++ /dev/null @@ -1,8 +0,0 @@ -- label: Buildkite example - instance_size: large - image: golang:1.17 - commands: - - ls -alp /workdir - - export PATH=$PATH:/usr/local/go/bin - - cd /workdir/ray-operator - - make test diff --git a/.buildkite/test-rayjob-sample-yamls.yml b/.buildkite/test-rayjob-sample-yamls.yml index 0f14960c1aa..b349564e1a5 100644 --- a/.buildkite/test-rayjob-sample-yamls.yml +++ b/.buildkite/test-rayjob-sample-yamls.yml @@ -1,6 +1,6 @@ - label: Test RayJob Sample YAMLs instance_size: large - image: golang:1.17 + image: golang:1.19 commands: # Install Go - export PATH=$PATH:/usr/local/go/bin @@ -60,7 +60,9 @@ # Install python 3.10 and pip - apt-get update - - apt-get install -y python3.10 python3-pip + - apt-get install -y python3.11 python3.11-venv + - python3 -m venv .venv + - source .venv/bin/activate # Install requirements - pip install -r tests/framework/config/requirements.txt diff --git a/.github/workflows/consistency-check.yaml b/.github/workflows/consistency-check.yaml index bfd66b94ed5..e342a921845 100644 --- a/.github/workflows/consistency-check.yaml +++ b/.github/workflows/consistency-check.yaml @@ -17,11 +17,11 @@ jobs: with: fetch-depth: 0 - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: # Use the same go version with build job - go-version: '1.17' + go-version: '1.19' - name: Check golang version working-directory: ./ray-operator @@ -44,11 +44,11 @@ jobs: with: fetch-depth: 0 - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: # Use the same go version with build job - go-version: '1.17' + go-version: '1.19' - name: Update CRD/RBAC YAML files working-directory: ./ray-operator diff --git a/.github/workflows/image-release.yaml b/.github/workflows/image-release.yaml index 558170eb3e9..416fb79cfc1 100644 --- a/.github/workflows/image-release.yaml +++ b/.github/workflows/image-release.yaml @@ -18,10 +18,10 @@ jobs: runs-on: ubuntu-latest steps: - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: - go-version: '1.17' + go-version: '1.19' - name: Check out code into the Go module directory uses: actions/checkout@v2 @@ -90,10 +90,10 @@ jobs: runs-on: ubuntu-latest steps: - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: - go-version: '1.17' + go-version: '1.19' - name: Check out code into the Go module directory uses: actions/checkout@v2 diff --git a/.github/workflows/test-job.yaml b/.github/workflows/test-job.yaml index e469ef8b26d..75d0e43fe53 100644 --- a/.github/workflows/test-job.yaml +++ b/.github/workflows/test-job.yaml @@ -12,11 +12,11 @@ jobs: name: Lint runs-on: ubuntu-latest steps: - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: # Use the same go version with build job - go-version: '1.17' + go-version: '1.19' - name: Check out code into the Go module directory uses: actions/checkout@v2 @@ -107,10 +107,10 @@ jobs: name: Build Apiserver, CLI Binaries and Docker Images runs-on: ubuntu-latest steps: - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: - go-version: '1.17' + go-version: '1.19' - name: Check out code into the Go module directory uses: actions/checkout@v2 @@ -202,10 +202,10 @@ jobs: runs-on: ubuntu-latest steps: - - name: Set up Go 1.17.x + - name: Set up Go 1.19.x uses: actions/setup-go@v2 with: - go-version: '1.17' + go-version: '1.19' - name: Check out code into the Go module directory uses: actions/checkout@v2 diff --git a/apiserver/Dockerfile b/apiserver/Dockerfile index 92c6c28defb..ec9250d8fcb 100644 --- a/apiserver/Dockerfile +++ b/apiserver/Dockerfile @@ -1,5 +1,5 @@ # Build the backend service -FROM golang:1.17 as builder +FROM registry.access.redhat.com/ubi8/go-toolset:1.19.10-10 as builder WORKDIR /workspace # Copy the Go Modules manifests @@ -17,12 +17,11 @@ WORKDIR /workspace/apiserver RUN go mod download # Build +USER root WORKDIR /workspace/apiserver RUN CGO_ENABLED=0 GOOS=linux GOARCH=amd64 go build -a -o kuberay-apiserver cmd/main.go -# Use distroless as minimal base image to package the manager binary -# Refer to https://github.com/GoogleContainerTools/distroless for more details -FROM gcr.io/distroless/static:nonroot +FROM scratch WORKDIR /workspace COPY --from=builder /workspace/apiserver/kuberay-apiserver apiserver/ # Support serving swagger files diff --git a/apiserver/go.mod b/apiserver/go.mod index 52dbf8c1278..e644fc0a1d6 100644 --- a/apiserver/go.mod +++ b/apiserver/go.mod @@ -1,6 +1,6 @@ module github.com/ray-project/kuberay/apiserver -go 1.17 +go 1.19 require ( github.com/go-openapi/runtime v0.19.31 diff --git a/apiserver/go.sum b/apiserver/go.sum index 34c10eab678..758563e20a1 100644 --- a/apiserver/go.sum +++ b/apiserver/go.sum @@ -26,7 +26,6 @@ cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4g cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -37,8 +36,6 @@ cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohl cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/Azure/go-ansiterm v0.0.0-20210608223527-2377c96fe795/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8= -github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= github.com/Azure/go-autorest v14.2.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= github.com/Azure/go-autorest/autorest v0.11.18/go.mod h1:dSiJPy22c3u0OtOKDNttNgqpNFY/GeWa7GH/Pz56QRA= github.com/Azure/go-autorest/autorest/adal v0.9.13/go.mod h1:W/MM4U6nLxnIskrw4UwWzlHfGjwUS50aOsc/I3yuU8M= @@ -48,9 +45,7 @@ github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZ github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/Masterminds/semver/v3 v3.2.0/go.mod h1:qvl/7zhW3nngYb5+80sSMF+FG2BjYrf8m9wsX0PNOMQ= github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= -github.com/NYTimes/gziphandler v1.1.1/go.mod h1:n/CVRwUEOgIxrgPvAQhUUr9oeUtvrhMomdKFjzJNB0c= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= @@ -61,29 +56,16 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= -github.com/antlr/antlr4/runtime/Go/antlr v0.0.0-20210826220005-b48c857c3a0e/go.mod h1:F7bn7fEU90QkQ3tnmaTx3LTKLEDqnwWODIYppRQ5hnY= -github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/asaskevich/govalidator v0.0.0-20180720115003-f9ffefc3facf/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY= github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY= github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:oGkLhpf+kjZl6xBf758TQhh5XrAeiJv/7FRz/2spLIg= github.com/asaskevich/govalidator v0.0.0-20200428143746-21a406dcc535 h1:4daAzAu0S6Vi7/lbWECcX0j45yZReDZ56BQsrVBOEEY= github.com/asaskevich/govalidator v0.0.0-20200428143746-21a406dcc535/go.mod h1:oGkLhpf+kjZl6xBf758TQhh5XrAeiJv/7FRz/2spLIg= -github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= -github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM= -github.com/blang/semver v3.5.1+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/certifi/gocertifi v0.0.0-20191021191039-0944d244cd40/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= -github.com/certifi/gocertifi v0.0.0-20200922220541-2c3bb06c6054/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -95,53 +77,31 @@ github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGX github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cockroachdb/datadriven v0.0.0-20200714090401-bf6692d28da5/go.mod h1:h6jFvWxBdQXxjopDMZyH2UVceIRfR84bdzbkoKrsWNo= -github.com/cockroachdb/errors v1.2.4/go.mod h1:rQD95gz6FARkaKkQXUksEje/d9a6wBJoCr5oaCLELYA= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-oidc v2.1.0+incompatible/go.mod h1:CgnwVTmzoESiwO9qyAFEMiHoZ1nMCKZlZ9V6mm3/LKc= -github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/elazarl/go-bindata-assetfs v1.0.1 h1:m0kkaHRKEu7tUIUFVwhGGGYClXvyl4RE03qmvRTNfbw= github.com/elazarl/go-bindata-assetfs v1.0.1/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/elazarl/goproxy v0.0.0-20180725130230-947c36da3153/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc= github.com/emicklei/go-restful v0.0.0-20170410110728-ff4f55a20633/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs= -github.com/emicklei/go-restful v2.9.5+incompatible/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/evanphx/json-patch v0.5.2/go.mod h1:ZWS5hhDbVDyob71nXKNL0+PWn6ToqBHMikGIFbs31qQ= github.com/evanphx/json-patch v4.12.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= -github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= github.com/getkin/kin-openapi v0.76.0/go.mod h1:660oXbgy5JFMKreazJaQTw7o+X00qeSyhcnluiMv+Xg= -github.com/getsentry/raven-go v0.2.0/go.mod h1:KungGk8q33+aIAZUIVWZDr2OfAEBsO49PX4NzFV5kcQ= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/globalsign/mgo v0.0.0-20180905125535-1ca0a4f7cbcb/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q= github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q= @@ -159,7 +119,6 @@ github.com/go-logr/logr v0.2.0/go.mod h1:z6/tIYblkpsD+a4lm/fGIIU9mZ+XfAiaFtq7xTg github.com/go-logr/logr v1.2.0 h1:QK40JKJyMdUDz+h+xvCsru/bJhvG0UxvePV0ufL/AcE= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/zapr v1.2.0 h1:n4JnPI1T3Qq1SFEi/F8rwLrZERp2bso19PJZDB9dayk= -github.com/go-logr/zapr v1.2.0/go.mod h1:Qa4Bsj2Vb+FAVeAKsLD8RLQ+YRJB8YDmOAKxaBQf7Ro= github.com/go-openapi/analysis v0.0.0-20180825180245-b006789cd277/go.mod h1:k70tL6pCuVxPJOHXQ+wIac1FUrvNkHolPie/cLEU6hI= github.com/go-openapi/analysis v0.17.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik= github.com/go-openapi/analysis v0.18.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik= @@ -182,7 +141,6 @@ github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3Hfo github.com/go-openapi/jsonreference v0.18.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= -github.com/go-openapi/jsonreference v0.19.5/go.mod h1:RdybgQwPxbL4UEjuAruzK1x3nE69AqPYEJeo/TWfEeg= github.com/go-openapi/loads v0.17.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU= github.com/go-openapi/loads v0.18.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU= github.com/go-openapi/loads v0.19.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU= @@ -223,7 +181,6 @@ github.com/go-openapi/validate v0.19.3/go.mod h1:90Vh6jjkTn+OT1Eefm0ZixWNFjhtOH7 github.com/go-openapi/validate v0.19.10/go.mod h1:RKEZTUWDkxKQxN2jDT7ZnZi2bhZlbNMAuKvKB+IaGx8= github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/gobuffalo/attrs v0.0.0-20190224210810-a9411de4debd/go.mod h1:4duuawTqi2wkkpB4ePgWMaai6/Kc6WEz83bhFwpHzj0= github.com/gobuffalo/depgen v0.0.0-20190329151759-d478694a28d3/go.mod h1:3STtPUQYuzV0gBVOY3vy6CfMm/ljR4pABfrTeHNLHUY= github.com/gobuffalo/depgen v0.1.0/go.mod h1:+ifsuy7fhi15RWncXQQKjWS9JPkdah5sZvtHc2RXGlg= @@ -248,16 +205,12 @@ github.com/gobuffalo/packd v0.1.0/go.mod h1:M2Juc+hhDXf/PnmBANFCqx4DM3wRbgDvnVWe github.com/gobuffalo/packr/v2 v2.0.9/go.mod h1:emmyGweYTm6Kdper+iywB6YK5YzuKchGtJQZ0Odn4pQ= github.com/gobuffalo/packr/v2 v2.2.0/go.mod h1:CaAwI0GPIAv+5wKLtv8Afwl+Cm78K/I/VCm/3ptBN+0= github.com/gobuffalo/syncx v0.0.0-20190224160051-33c29581e754/go.mod h1:HhnNqWY95UYwwW3uSASeV7vtgYkT2t16hJgV3AEPUpw= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= -github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0 h1:nfP3RFugxnNRyKgeWd4oI1nYvXpxrx8ck8ZrcizshdQ= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= -github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -292,8 +245,6 @@ github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA= -github.com/google/cel-go v0.9.0/go.mod h1:U7ayypeSkw23szu4GaQTPJGx66c20mx8JklMSxrmI1w= -github.com/google/cel-spec v0.6.0/go.mod h1:Nwjgxy5CbjlPrtCWjeDjUyKMl8w41YBYGjsyDdqk0xA= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -325,7 +276,6 @@ github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= @@ -335,40 +285,18 @@ github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5m github.com/googleapis/gnostic v0.5.1/go.mod h1:6U4PtQXGIEt/Z3h5MAT7FNofLnw9vXk2cUuW7uA/OeU= github.com/googleapis/gnostic v0.5.5 h1:9fHAtK0uDfpveeqqo1hkEZJcFvYXAiCN3UutL8F9xHw= github.com/googleapis/gnostic v0.5.5/go.mod h1:7+EbHbldMins07ALC74bsA81Ovc97DwqyJO1AENw9kA= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= -github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 h1:rgxjzoDmDXw5q8HONgyHhBas4to0/XWRo/gPpJhsUNQ= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0/go.mod h1:qrJPVzv9YlhsrxJc3P/Q85nr0w1lIRikTl4JlhdDH5w= -github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= -github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= -github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= -github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= -github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= @@ -376,11 +304,7 @@ github.com/imdario/mergo v0.3.5/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJ github.com/imdario/mergo v0.3.12 h1:b6R2BslTbIEToALKP7LxUvijTsNI9TAe80pLWN2g/HU= github.com/imdario/mergo v0.3.12/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/jarcoal/httpmock v1.2.0/go.mod h1:oCoTsnAz4+UoOUIf5lJOWV2QQIW5UoeUI6aM2YnWAZk= -github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= -github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= -github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= @@ -391,12 +315,10 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/karrick/godirwalk v1.8.0/go.mod h1:H5KPZjojv4lE+QYImBI8xVtrBRgYrIVsaRPx4tDPEn4= github.com/karrick/godirwalk v1.10.3/go.mod h1:RoGL9dQei4vP9ilrpETWE8CLOZ1kiN0LhBygSwrAsHA= -github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -404,7 +326,6 @@ github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0 github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= @@ -413,8 +334,6 @@ github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -424,26 +343,14 @@ github.com/mailru/easyjson v0.7.6 h1:8yTIVnZgCoiM1TgqoeTl+LfU5Jg6/xL3QhGQnimLYnA github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/markbates/oncer v0.0.0-20181203154359-bf2de49a0be2/go.mod h1:Ld9puTsIW75CHf65OeIOkyKbteujpZVXDpWK6YGZbxE= github.com/markbates/safe v1.0.1/go.mod h1:nAqgmRi7cY2nqMc92/bSEeQA+R4OheNU2T1kNSCBdG0= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= -github.com/maxatome/go-testdeep v1.11.0/go.mod h1:011SgQ6efzZYAen6fDn4BqQ+lUR72ysdyKe7Dyogw70= -github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= -github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= -github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.3.2/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/mitchellh/mapstructure v1.4.1 h1:CpVNEelQCZBooIPDn+AR3NpivK/TIKU8bDxdASFVQag= github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/moby/spdystream v0.2.0/go.mod h1:f7i0iNDQJ059oMTcWxx8MA/zKFIuD/lY+0GqbN2Wy8c= -github.com/moby/term v0.0.0-20210610120745-9d4ed1856297/go.mod h1:vgPCkQMyxTZ7IDy8SXRufE172gr8+K/JE/7hHFxHW3A= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -453,7 +360,6 @@ github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9G github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= -github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw= @@ -461,40 +367,27 @@ github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWb github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= -github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= -github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= -github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/onsi/gomega v1.17.0 h1:9Luw4uT5HTjHTN8+aNcSThgH1vdXnmdJ8xIfZ4wyTRE= -github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/orcaman/concurrent-map v1.0.0/go.mod h1:Lu3tH6HLW3feq74c2GC+jIMS/K2CFcDWnWD9XkenwhI= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.4.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/pelletier/go-toml v1.9.3/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= -github.com/pquerna/cachecontrol v0.0.0-20171018203845-0dec1b30a021/go.mod h1:prYjPmNq4d1NPVmpShWobRqXY3q7Vp+80DqgxxUrUIA= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= @@ -505,56 +398,31 @@ github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1: github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/common v0.28.0 h1:vGVfV9KrDTvWt5boZO0I19g2E3CsWfpPPKZM9dt3mEw= github.com/prometheus/common v0.28.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= -github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= -github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= -github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= -github.com/spf13/cobra v1.2.1/go.mod h1:ExllRjgxM/piMAM+3tAZvg8fsklGAf3tPfi+i8t68Nk= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= -github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns= github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -565,32 +433,16 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.2 h1:4jaiDzPyXQvSd7D0EjG45355tLlV3VOECpq10pLC+8s= -github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= -github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= +github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/tidwall/pretty v1.0.0 h1:HsD+QiTn7sK6flMKIvNmpqz1qrpP3Ps6jOKIKMooyg4= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v0.0.0-20180714160509-73f8eece6fdc/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= -github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= -go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= -go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= -go.etcd.io/etcd/client/v2 v2.305.0/go.mod h1:h9puh54ZTgAKtEbut2oe9P4L/oqKCVB6xsXlzd7alYQ= -go.etcd.io/etcd/client/v3 v3.5.0/go.mod h1:AIKXXVX/DQXtfTEqBryiLTUXwON+GuvO6Z7lLS/oTh0= -go.etcd.io/etcd/pkg/v3 v3.5.0/go.mod h1:UzJGatBQ1lXChBkQF0AuAtkRQMYnHubxAEYIrC3MSsE= -go.etcd.io/etcd/raft/v3 v3.5.0/go.mod h1:UFOHSIvO/nKwd4lhkwabrTD3cqW5yVyYYf/KlD00Szc= -go.etcd.io/etcd/server/v3 v3.5.0/go.mod h1:3Ah5ruV+M+7RZr0+Y/5mNLwC+eQlni+mQmOVdCRJoS4= go.mongodb.org/mongo-driver v1.0.3/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM= go.mongodb.org/mongo-driver v1.1.1/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM= go.mongodb.org/mongo-driver v1.3.0/go.mod h1:MSWZXKOynuguX+JSvwP8i+58jYCXxbia8HS3gZBapIE= @@ -603,34 +455,14 @@ go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= -go.opentelemetry.io/contrib v0.20.0/go.mod h1:G/EtFaa6qaN7+LxqfIAT3GiZa7Wv5DTBUzl5H4LY0Kc= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0/go.mod h1:oVGt1LRbBOBq1A5BQLlUg9UaU/54aiHw8cgjV3aWZ/E= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.20.0/go.mod h1:2AboqHi0CiIZU0qwhtUfCYD1GeUzvvIXWNkhDt7ZMG4= -go.opentelemetry.io/otel v0.20.0/go.mod h1:Y3ugLH2oa81t5QO+Lty+zXf8zC9L26ax4Nzoxm/dooo= -go.opentelemetry.io/otel/exporters/otlp v0.20.0/go.mod h1:YIieizyaN77rtLJra0buKiNBOm9XQfkPEKBeuhoMwAM= -go.opentelemetry.io/otel/metric v0.20.0/go.mod h1:598I5tYlH1vzBjn+BTuhzTCSb/9debfNp6R3s7Pr1eU= -go.opentelemetry.io/otel/oteltest v0.20.0/go.mod h1:L7bgKf9ZB7qCwT9Up7i9/pn0PWIa9FqQ2IQ8LoxiGnw= -go.opentelemetry.io/otel/sdk v0.20.0/go.mod h1:g/IcepuwNsoiX5Byy2nNV0ySUF1em498m7hBWC279Yc= -go.opentelemetry.io/otel/sdk/export/metric v0.20.0/go.mod h1:h7RBNMsDJ5pmI1zExLi+bJK+Dr8NQCh0qGhm1KDnNlE= -go.opentelemetry.io/otel/sdk/metric v0.20.0/go.mod h1:knxiS8Xd4E/N+ZqKmUPf3gTTZ4/0TjTXukfxjzSTpHE= -go.opentelemetry.io/otel/trace v0.20.0/go.mod h1:6GjCW8zgDjwGHGa6GkyeB8+/5vjT16gUEi0Nf1iBdgw= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= -go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190320223903-b7391e95e576/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190422162423-af44ce270edf/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE= @@ -639,13 +471,10 @@ golang.org/x/crypto v0.0.0-20190530122614-20be4c3c3ed5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190617133340-57b3e21c3d56/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -681,16 +510,11 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -723,19 +547,13 @@ golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -749,7 +567,6 @@ golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210402161424-2e8d93401602/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f h1:Qmd2pbz05z7z6lm0DrgQVVPuBm92jqujBKMHMOlOQEw= golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= @@ -765,14 +582,9 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -791,7 +603,6 @@ golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -812,14 +623,11 @@ golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200831180312-196b9ba8737a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -827,27 +635,15 @@ golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211029165221-6e7872819dc8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.10.0 h1:3R7pNqamzBraeqj/Tj8qt1aQ2HpmlC+Cx/qL/7hn4/c= golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -859,17 +655,13 @@ golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs= golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -877,7 +669,6 @@ golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3 golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190329151228-23e29df326fe/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190416151739-9c9e1878f421/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190420181800-aa740d480789/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= @@ -889,13 +680,10 @@ golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgw golang.org/x/tools v0.0.0-20190614205625-5aca471b1d59/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190617190820-da514acc4774/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190624222133-a101b041ded4/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -927,21 +715,15 @@ golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82u golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.6-0.20210820212750-d4cc65f0b2ff/go.mod h1:YD9qOF0M9xpSpdWTBbzEl5e/RnCefISl8E5Noe10jFM= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gomodules.xyz/jsonpatch/v2 v2.2.0/go.mod h1:WXp+iVDkoLQqPudfQ9GBlwB2eZ5DKOnjQZCYdOS8GPY= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= @@ -963,7 +745,6 @@ google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34q google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= -google.golang.org/api v0.44.0/go.mod h1:EBOGZqzyhtvMDoxwS97ctnh0zUmYY6CxqXsc1AvkYD8= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1005,7 +786,6 @@ google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201019141844-1ed22bb0c154/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201102152239-715cce707fb0/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= @@ -1015,8 +795,6 @@ google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/genproto v0.0.0-20210831024726-fe130286e0e2/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20210909211513-a8c4777a87af h1:aLMMXFYqw01RA6XJim5uaN+afqNNjc9P8HPAbnpnc5s= google.golang.org/genproto v0.0.0-20210909211513-a8c4777a87af/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= @@ -1039,8 +817,6 @@ google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA5 google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.37.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= google.golang.org/grpc v1.40.0 h1:AGJ0Ih4mHjSeibYkFGh1dD9KJ/eOtZ93I6hoHhukQ5Q= google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= @@ -1067,14 +843,8 @@ gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/ini.v1 v1.62.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= -gopkg.in/square/go-jose.v2 v2.2.2/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= @@ -1090,8 +860,6 @@ gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gotest.tools/v3 v3.0.2/go.mod h1:3SzNCllyD9/Y+b5r9JIKQ474KzkZyqLqEfYqMsX94Bk= -gotest.tools/v3 v3.0.3/go.mod h1:Z7Lb0S5l+klDB31fvDQX8ss/FlKDxtlFlw3Oa8Ymbl8= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= @@ -1101,17 +869,11 @@ honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9 honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= k8s.io/api v0.23.0 h1:WrL1gb73VSC8obi8cuYETJGXEoFNEh3LU0Pt+Sokgro= k8s.io/api v0.23.0/go.mod h1:8wmDdLBHBNxtOIytwLstXt5E9PddnZb0GaMcqsvDBpg= -k8s.io/apiextensions-apiserver v0.23.0/go.mod h1:xIFAEEDlAZgpVBl/1VSjGDmLoXAWRG40+GsWhKhAxY4= k8s.io/apimachinery v0.23.0 h1:mIfWRMjBuMdolAWJ3Fd+aPTMv3X9z+waiARMpvvb0HQ= k8s.io/apimachinery v0.23.0/go.mod h1:fFCTTBKvKcwTPFzjlcxp91uPFZr+JA0FubU4fLzzFYc= -k8s.io/apiserver v0.23.0/go.mod h1:Cec35u/9zAepDPPFyT+UMrgqOCjgJ5qtfVJDxjZYmt4= k8s.io/client-go v0.23.0 h1:vcsOqyPq7XV3QmQRCBH/t9BICJM9Q1M18qahjv+rebY= k8s.io/client-go v0.23.0/go.mod h1:hrDnpnK1mSr65lHHcUuIZIXDgEbzc7/683c6hyG4jTA= -k8s.io/code-generator v0.23.0/go.mod h1:vQvOhDXhuzqiVfM/YHp+dmg10WDZCchJVObc9MvowsE= -k8s.io/component-base v0.23.0/go.mod h1:DHH5uiFvLC1edCpvcTDV++NKULdYYU6pR9Tt3HIKMKI= k8s.io/gengo v0.0.0-20210813121822-485abfe95c7c/go.mod h1:FiNAH4ZV3gBg2Kwh89tzAEV2be7d5xI0vBa/VySYy3E= -k8s.io/klog v1.0.0 h1:Pt+yjF5aB1xDSVbau4VsWe+dQNzA0qv1LlXdC2dF6Q8= -k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= k8s.io/klog/v2 v2.0.0/go.mod h1:PBfzABfn139FHAV07az/IF9Wp1bkk3vpT2XSJ76fSDE= k8s.io/klog/v2 v2.2.0/go.mod h1:Od+F08eJP+W3HUb4pSrPpgp9DGU4GzlpG/TmITuYh/Y= k8s.io/klog/v2 v2.30.0 h1:bUO6drIvCIsvZ/XFgfxoGFQU/a4Qkh0iAlvUR7vlHJw= @@ -1124,7 +886,6 @@ k8s.io/utils v0.0.0-20210930125809-cb0fa318a74b/go.mod h1:jPW/WVKK9YHAvNhRxK0md/ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.0.25/go.mod h1:Mlj9PNLmG9bZ6BHFwFKDo5afkpWyUISkb9Me0GnK66I= sigs.k8s.io/controller-runtime v0.11.1 h1:7YIHT2QnHJArj/dk9aUkYhfqfK5cIxPOX5gPECfdZLU= sigs.k8s.io/controller-runtime v0.11.1/go.mod h1:KKwLiTooNGu+JmLZGn9Sl3Gjmfj66eMbCQznLP5zcqA= sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 h1:fD1pz4yfdADVNfFmcP2aBEtudwUQ1AlLnRBALr33v3s= @@ -1136,4 +897,3 @@ sigs.k8s.io/structured-merge-diff/v4 v4.2.0/go.mod h1:j/nl6xW8vLS49O8YvXW1ocPhZa sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= -volcano.sh/apis v1.6.0-alpha.0.0.20221012070524-685db38b4fae/go.mod h1:drNMGuHPn1ew7oBSDQb5KRey6tXOQksbUtw3gPxF3Vo= diff --git a/cli/go.mod b/cli/go.mod index 93955b5e337..14e40fcbb1b 100644 --- a/cli/go.mod +++ b/cli/go.mod @@ -1,6 +1,6 @@ module github.com/ray-project/kuberay/cli -go 1.17 +go 1.19 require ( github.com/fatih/color v1.13.0 @@ -18,7 +18,6 @@ require ( github.com/fsnotify/fsnotify v1.5.1 // indirect github.com/go-logr/logr v1.0.0 // indirect github.com/golang/protobuf v1.5.2 // indirect - github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 // indirect github.com/hashicorp/hcl v1.0.0 // indirect github.com/inconshreveable/mousetrap v1.0.0 // indirect @@ -43,4 +42,4 @@ require ( gopkg.in/yaml.v2 v2.4.0 // indirect ) -replace github.com/ray-project/kuberay/proto => ../proto +replace github.com/ray-project/kuberay/proto => ../proto diff --git a/cli/go.sum b/cli/go.sum index 577bd12908c..2e3cb0c3459 100644 --- a/cli/go.sum +++ b/cli/go.sum @@ -124,6 +124,7 @@ github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5x github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.0.0 h1:nfP3RFugxnNRyKgeWd4oI1nYvXpxrx8ck8ZrcizshdQ= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -197,12 +198,10 @@ github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+ github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0eJc8R6ouapiM= -github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 h1:rgxjzoDmDXw5q8HONgyHhBas4to0/XWRo/gPpJhsUNQ= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0/go.mod h1:qrJPVzv9YlhsrxJc3P/Q85nr0w1lIRikTl4JlhdDH5w= github.com/hashicorp/consul/api v1.11.0/go.mod h1:XjsvQN+RJGWI2TWy1/kqaE16HrR2J/FWgkYjdZQsX9M= -github.com/hashicorp/consul/api v1.12.0/go.mod h1:6pVBMo0ebnYdt2S3H87XhekM/HHrUoTD2XXb/VrZVy0= github.com/hashicorp/consul/sdk v0.8.0/go.mod h1:GBvyrGALthsZObzUGsfgHZQDXjg4lOjagTIwIR1vPms= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= @@ -322,8 +321,6 @@ github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8b github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/ray-project/kuberay/proto v0.0.0-20220119062608-4054f1bf1765 h1:ReNv3bgRn4N1YjRFDfvbbdguo97YP9wez3ZFS6obD4Q= -github.com/ray-project/kuberay/proto v0.0.0-20220119062608-4054f1bf1765/go.mod h1:0ZKkYhwLTi84toH7lFzyEzKzCmAEGKLpDxpmutk5RA4= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= @@ -331,7 +328,6 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/sagikazarmark/crypt v0.3.0/go.mod h1:uD/D+6UF4SrIR1uGEv7bBNkNqLGqUr43MRiaGWX1Nig= -github.com/sagikazarmark/crypt v0.4.0/go.mod h1:ALv2SRj7GxYV4HO9elxH9nS6M9gW+xDNxqmyJ6RfDFM= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -669,7 +665,6 @@ google.golang.org/api v0.57.0/go.mod h1:dVPlbZyBo2/OjBpmvNdpn2GRm6rPy75jyU7bmhdr google.golang.org/api v0.59.0/go.mod h1:sT2boj7M9YJxZzgeZqXogmhfmRWDtPzT31xkieUbuZU= google.golang.org/api v0.61.0/go.mod h1:xQRti5UdCmoCEqFxcz93fTl338AVqDgyaDRuOZ3hg9I= google.golang.org/api v0.62.0/go.mod h1:dKmwPCydfsad4qCH08MSdgWjfHOyfpd4VtDGgRFdavw= -google.golang.org/api v0.63.0/go.mod h1:gs4ij2ffTRXwuzzgJl/56BdwJaA194ijkfn++9tDuPo= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= diff --git a/proto/Dockerfile b/proto/Dockerfile index bda584758d2..5b5382dffd8 100644 --- a/proto/Dockerfile +++ b/proto/Dockerfile @@ -1,5 +1,5 @@ # Generate client code (go & json) from API protocol buffers -FROM golang:1.17.8 as generator +FROM golang:1.19.10 as generator ENV PROTOC_VERSION 3.17.3 ENV GOLANG_PROTOBUF_VERSION v1.5.2 diff --git a/proto/go.mod b/proto/go.mod index cf7d73fa54d..1ff1ff1038a 100644 --- a/proto/go.mod +++ b/proto/go.mod @@ -1,6 +1,6 @@ module github.com/ray-project/kuberay/proto -go 1.17 +go 1.19 require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 diff --git a/proto/go.sum b/proto/go.sum index afa275f668e..13b2dac0d7e 100644 --- a/proto/go.sum +++ b/proto/go.sum @@ -110,7 +110,6 @@ github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm4 github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 h1:rgxjzoDmDXw5q8HONgyHhBas4to0/XWRo/gPpJhsUNQ= github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0/go.mod h1:qrJPVzv9YlhsrxJc3P/Q85nr0w1lIRikTl4JlhdDH5w= diff --git a/ray-operator/DEVELOPMENT.md b/ray-operator/DEVELOPMENT.md index 195a5ad35c7..c9c5204cddb 100644 --- a/ray-operator/DEVELOPMENT.md +++ b/ray-operator/DEVELOPMENT.md @@ -18,15 +18,14 @@ The instructions assume you have access to a running Kubernetes cluster via `kub For local development, we recommend using [Kind](https://kind.sigs.k8s.io/) to create a Kubernetes cluster. -### Use go v1.17 +### Use go v1.19 -Currently, Kuberay does not support go v1.16 ([#568](https://github.com/ray-project/kuberay/issues/568)) or go v1.18 ([#518](https://github.com/ray-project/kuberay/issues/518)). -Hence, we strongly recommend you to use go v1.17. The following commands can help you switch to go v1.17.6. +Currently, KubeRay uses go v1.19 for development. ```bash -go install golang.org/dl/go1.17.6@latest -go1.17.6 download -export GOROOT=$(go1.17.6 env GOROOT) +go install golang.org/dl/go1.19.12@latest +go1.19.12 download +export GOROOT=$(go1.19. env GOROOT) export PATH="$GOROOT/bin:$PATH" ``` diff --git a/ray-operator/Dockerfile b/ray-operator/Dockerfile index 9c550d80743..429c7042196 100644 --- a/ray-operator/Dockerfile +++ b/ray-operator/Dockerfile @@ -1,5 +1,5 @@ # Build the manager binary -FROM registry.access.redhat.com/ubi8/go-toolset:1.17.12-11 as builder +FROM registry.access.redhat.com/ubi8/go-toolset:1.19.10-10 as builder WORKDIR /workspace # Copy the Go Modules manifests diff --git a/ray-operator/apis/ray/v1alpha1/rayjob_types.go b/ray-operator/apis/ray/v1alpha1/rayjob_types.go index 758675bdac2..670b1764795 100644 --- a/ray-operator/apis/ray/v1alpha1/rayjob_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayjob_types.go @@ -97,9 +97,9 @@ type RayJobStatus struct { ObservedGeneration int64 `json:"observedGeneration,omitempty"` } -//+kubebuilder:object:root=true -//+kubebuilder:subresource:status -//+genclient +// +kubebuilder:object:root=true +// +kubebuilder:subresource:status +// +genclient // RayJob is the Schema for the rayjobs API type RayJob struct { metav1.TypeMeta `json:",inline"` diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 04b12744300..990b3a826cb 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -734,8 +734,8 @@ func (r *RayServiceReconciler) updateServeDeployment(ctx context.Context, raySer // // (1) `isHealthy` is used to determine whether restart the RayCluster or not. // (2) `isReady` is used to determine whether the Serve applications in the RayCluster are ready to serve incoming traffic or not. -// (3) `err`: If `err` is not nil, it means that KubeRay failed to get Serve application statuses from the dashboard agent. We should take a -// look at dashboard agent rather than Ray Serve applications. +// (3) `err`: If `err` is not nil, it means that KubeRay failed to get Serve application statuses from the dashboard agent. We should take a look at dashboard agent rather than Ray Serve applications. + func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashboardClient utils.RayDashboardClientInterface, rayServiceServeStatus *rayv1alpha1.RayServiceStatus, serveConfigType utils.RayServeConfigType, unhealthySecondThreshold *int32) (bool, bool, error) { // If the `unhealthySecondThreshold` value is non-nil, then we will use that value. Otherwise, we will use the value ServiceUnhealthySecondThreshold // which can be set in a test. This is used for testing purposes. diff --git a/ray-operator/controllers/ray/utils/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboard_httpclient.go index dce2cc3afc7..bcabf92c10c 100644 --- a/ray-operator/controllers/ray/utils/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboard_httpclient.go @@ -5,7 +5,7 @@ import ( "context" "encoding/base64" "fmt" - "io/ioutil" + "io" "net/http" "time" @@ -121,7 +121,7 @@ func (r *RayDashboardClient) GetDeployments(ctx context.Context) (string, error) } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) if resp.StatusCode < 200 || resp.StatusCode > 299 { return "", fmt.Errorf("GetDeployments fail: %s %s", resp.Status, string(body)) } @@ -153,7 +153,7 @@ func (r *RayDashboardClient) UpdateDeployments(ctx context.Context, configJson [ } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) if resp.StatusCode < 200 || resp.StatusCode > 299 { return fmt.Errorf("UpdateDeployments fail: %s %s", resp.Status, string(body)) } @@ -174,7 +174,7 @@ func (r *RayDashboardClient) GetSingleApplicationStatus(ctx context.Context) (*S } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) if resp.StatusCode < 200 || resp.StatusCode > 299 { return nil, fmt.Errorf("GetDeploymentsStatus fail: %s %s", resp.Status, string(body)) @@ -225,7 +225,7 @@ func (r *RayDashboardClient) GetServeDetails(ctx context.Context) (*ServeDetails } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) if resp.StatusCode < 200 || resp.StatusCode > 299 { return nil, fmt.Errorf("GetServeDetails fail: %s %s", resp.Status, string(body)) @@ -350,7 +350,7 @@ func (r *RayDashboardClient) GetJobInfo(ctx context.Context, jobId string) (*Ray return nil, nil } - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) if err != nil { return nil, err } @@ -387,7 +387,7 @@ func (r *RayDashboardClient) SubmitJob(ctx context.Context, rayJob *rayv1alpha1. } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) var jobResp RayJobResponse if err = json.Unmarshal(body, &jobResp); err != nil { @@ -413,7 +413,7 @@ func (r *RayDashboardClient) StopJob(ctx context.Context, jobName string, log *l } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) var jobStopResp RayJobStopResponse if err = json.Unmarshal(body, &jobStopResp); err != nil { diff --git a/ray-operator/controllers/ray/utils/httpproxy_httpclient.go b/ray-operator/controllers/ray/utils/httpproxy_httpclient.go index 589df374e97..08b1ab40ff8 100644 --- a/ray-operator/controllers/ray/utils/httpproxy_httpclient.go +++ b/ray-operator/controllers/ray/utils/httpproxy_httpclient.go @@ -2,7 +2,7 @@ package utils import ( "fmt" - "io/ioutil" + "io" "net/http" "time" ) @@ -49,7 +49,7 @@ func (r *RayHttpProxyClient) CheckHealth() error { } defer resp.Body.Close() - body, _ := ioutil.ReadAll(resp.Body) + body, _ := io.ReadAll(resp.Body) if resp.StatusCode < 200 || resp.StatusCode > 299 { return fmt.Errorf("RayHttpProxyClient CheckHealth fail: %s %s", resp.Status, string(body)) } diff --git a/ray-operator/go.mod b/ray-operator/go.mod index a22f493d983..a5ecf326420 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -1,6 +1,6 @@ module github.com/ray-project/kuberay/ray-operator -go 1.17 +go 1.19 require ( github.com/Masterminds/semver/v3 v3.2.0 @@ -14,7 +14,7 @@ require ( github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.11.1 github.com/sirupsen/logrus v1.8.1 - github.com/stretchr/testify v1.7.2 + github.com/stretchr/testify v1.8.1 go.uber.org/zap v1.19.1 gopkg.in/natefinch/lumberjack.v2 v2.0.0 k8s.io/api v0.23.0 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index 60a4d109cbc..d505520be6d 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -84,7 +84,6 @@ github.com/blang/semver v3.5.1+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnweb github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/certifi/gocertifi v0.0.0-20191021191039-0944d244cd40/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= github.com/certifi/gocertifi v0.0.0-20200922220541-2c3bb06c6054/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -332,7 +331,6 @@ github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5 github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/maxatome/go-testdeep v1.11.0 h1:Tgh5efyCYyJFGUYiT0qxBSIDeXw0F5zSoatlou685kk= -github.com/maxatome/go-testdeep v1.11.0/go.mod h1:011SgQ6efzZYAen6fDn4BqQ+lUR72ysdyKe7Dyogw70= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= @@ -452,14 +450,18 @@ github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.2 h1:4jaiDzPyXQvSd7D0EjG45355tLlV3VOECpq10pLC+8s= -github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= @@ -470,7 +472,6 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= @@ -505,7 +506,6 @@ go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= -go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -524,8 +524,6 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -561,7 +559,6 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0 h1:LUYupSeNrTNCGzR/hVBk2NHZO4hXcVaW1k4Qx7rjPx8= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -611,9 +608,6 @@ golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210825183410-e898025ed96a/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -642,9 +636,7 @@ golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= -golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -709,18 +701,10 @@ golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211029165221-6e7872819dc8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.10.0 h1:3R7pNqamzBraeqj/Tj8qt1aQ2HpmlC+Cx/qL/7hn4/c= golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -732,8 +716,6 @@ golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -801,7 +783,6 @@ golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.6-0.20210820212750-d4cc65f0b2ff/go.mod h1:YD9qOF0M9xpSpdWTBbzEl5e/RnCefISl8E5Noe10jFM= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0 h1:BOw41kyTf3PuCW1pVQf8+Cyg8pMlkYB1oo9iJ6D/lKM= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -980,8 +961,6 @@ k8s.io/component-base v0.23.0 h1:UAnyzjvVZ2ZR1lF35YwtNY6VMN94WtOnArcXBu34es8= k8s.io/component-base v0.23.0/go.mod h1:DHH5uiFvLC1edCpvcTDV++NKULdYYU6pR9Tt3HIKMKI= k8s.io/gengo v0.0.0-20210813121822-485abfe95c7c h1:GohjlNKauSai7gN4wsJkeZ3WAJx4Sh+oT/b5IYn5suA= k8s.io/gengo v0.0.0-20210813121822-485abfe95c7c/go.mod h1:FiNAH4ZV3gBg2Kwh89tzAEV2be7d5xI0vBa/VySYy3E= -k8s.io/klog v1.0.0 h1:Pt+yjF5aB1xDSVbau4VsWe+dQNzA0qv1LlXdC2dF6Q8= -k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= k8s.io/klog/v2 v2.0.0/go.mod h1:PBfzABfn139FHAV07az/IF9Wp1bkk3vpT2XSJ76fSDE= k8s.io/klog/v2 v2.2.0/go.mod h1:Od+F08eJP+W3HUb4pSrPpgp9DGU4GzlpG/TmITuYh/Y= k8s.io/klog/v2 v2.30.0 h1:bUO6drIvCIsvZ/XFgfxoGFQU/a4Qkh0iAlvUR7vlHJw= diff --git a/ray-operator/pkg/client/clientset/versioned/fake/register.go b/ray-operator/pkg/client/clientset/versioned/fake/register.go index c64765715ad..5130e6ba593 100644 --- a/ray-operator/pkg/client/clientset/versioned/fake/register.go +++ b/ray-operator/pkg/client/clientset/versioned/fake/register.go @@ -21,14 +21,14 @@ var localSchemeBuilder = runtime.SchemeBuilder{ // AddToScheme adds all types of this clientset into the given scheme. This allows composition // of clientsets, like in: // -// import ( -// "k8s.io/client-go/kubernetes" -// clientsetscheme "k8s.io/client-go/kubernetes/scheme" -// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" -// ) +// import ( +// "k8s.io/client-go/kubernetes" +// clientsetscheme "k8s.io/client-go/kubernetes/scheme" +// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" +// ) // -// kclientset, _ := kubernetes.NewForConfig(c) -// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) +// kclientset, _ := kubernetes.NewForConfig(c) +// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) // // After this, RawExtensions in Kubernetes types will serialize kube-aggregator types // correctly. diff --git a/ray-operator/pkg/client/clientset/versioned/scheme/register.go b/ray-operator/pkg/client/clientset/versioned/scheme/register.go index 7ea47d1c691..4c376262b39 100644 --- a/ray-operator/pkg/client/clientset/versioned/scheme/register.go +++ b/ray-operator/pkg/client/clientset/versioned/scheme/register.go @@ -21,14 +21,14 @@ var localSchemeBuilder = runtime.SchemeBuilder{ // AddToScheme adds all types of this clientset into the given scheme. This allows composition // of clientsets, like in: // -// import ( -// "k8s.io/client-go/kubernetes" -// clientsetscheme "k8s.io/client-go/kubernetes/scheme" -// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" -// ) +// import ( +// "k8s.io/client-go/kubernetes" +// clientsetscheme "k8s.io/client-go/kubernetes/scheme" +// aggregatorclientsetscheme "k8s.io/kube-aggregator/pkg/client/clientset_generated/clientset/scheme" +// ) // -// kclientset, _ := kubernetes.NewForConfig(c) -// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) +// kclientset, _ := kubernetes.NewForConfig(c) +// _ = aggregatorclientsetscheme.AddToScheme(clientsetscheme.Scheme) // // After this, RawExtensions in Kubernetes types will serialize kube-aggregator types // correctly. diff --git a/scripts/install-docker.sh b/scripts/install-docker.sh index 10f5849baa4..5eab81250ea 100644 --- a/scripts/install-docker.sh +++ b/scripts/install-docker.sh @@ -4,7 +4,4 @@ export DEBIAN_FRONTEND=noninteractive export TZ=America/Los_Angeles -apt-get update -qq && apt-get upgrade -qq -apt-get install -y -qq curl - curl -o- https://get.docker.com | sh From 55341f8118ef1a344ff7387d64dbbb609537f454 Mon Sep 17 00:00:00 2001 From: Anish Asthana Date: Mon, 14 Aug 2023 21:40:13 +0200 Subject: [PATCH 08/56] Fix release actions (#1323) Fix release actions --- .github/workflows/image-release.yaml | 2 +- .github/workflows/test-job.yaml | 2 +- docs/deploy/docker.md | 13 ------------- docs/deploy/images.md | 21 +++++++++++++++++++++ 4 files changed, 23 insertions(+), 15 deletions(-) delete mode 100644 docs/deploy/docker.md create mode 100644 docs/deploy/images.md diff --git a/.github/workflows/image-release.yaml b/.github/workflows/image-release.yaml index 416fb79cfc1..ec7d702b18a 100644 --- a/.github/workflows/image-release.yaml +++ b/.github/workflows/image-release.yaml @@ -76,7 +76,7 @@ jobs: username: ${{ secrets.QUAY_USERNAME }} password: ${{ secrets.QUAY_ROBOT_TOKEN }} - - name: Push Operator to Quay.io + - name: Push Apiserver to Quay.io run: | docker image tag kuberay/apiserver:${{ steps.vars.outputs.sha_short }} quay.io/kuberay/apiserver:${{ steps.vars.outputs.sha_short }}; docker push quay.io/kuberay/apiserver:${{ steps.vars.outputs.sha_short }}; diff --git a/.github/workflows/test-job.yaml b/.github/workflows/test-job.yaml index 75d0e43fe53..deb6d652d0a 100644 --- a/.github/workflows/test-job.yaml +++ b/.github/workflows/test-job.yaml @@ -183,7 +183,7 @@ jobs: password: ${{ secrets.QUAY_ROBOT_TOKEN }} if: contains(fromJson('["refs/heads/master"]'), github.ref) - - name: Push Operator to Quay.io + - name: Push Apiserver to Quay.io run: | docker image tag kuberay/apiserver:${{ steps.vars.outputs.sha_short }} quay.io/kuberay/apiserver:${{ steps.vars.outputs.sha_short }}; docker push quay.io/kuberay/apiserver:${{ steps.vars.outputs.sha_short }}; diff --git a/docs/deploy/docker.md b/docs/deploy/docker.md deleted file mode 100644 index d39da1f6485..00000000000 --- a/docs/deploy/docker.md +++ /dev/null @@ -1,13 +0,0 @@ -## Docker images - -Find the Docker images for various KubeRay components on [Dockerhub](https://hub.docker.com/u/kuberay). - -#### Stable versions -For stable releases, use version tags (e.g. `kuberay/operator:v0.6.0`). - -#### Master commits -The first seven characters of the git SHA specify images built from specific commits -(e.g. `kuberay/operator:944a042`). - -#### Nightly images -The nightly tag specifies images built from the most recent master (e.g. `kuberay/operator:nightly`). diff --git a/docs/deploy/images.md b/docs/deploy/images.md new file mode 100644 index 00000000000..c8d7721d47a --- /dev/null +++ b/docs/deploy/images.md @@ -0,0 +1,21 @@ +# Container Images + +Images for the various KubeRay components are published at the following locations: + +1. [Quay.io](https://quay.io/organization/kuberay) +2. [DockerHub](https://hub.docker.com/u/kuberay) + +We recommend using Quay.io as the primary source for images as there are image-pull restrictions on DockerHub. DockerHub allows you to pull only 100 images per 6 hour window. Refer to [DockerHub rate limiting](https://docs.docker.com/docker-hub/download-rate-limit/) for more details. + +## Stable versions + +For stable releases, use version tags (e.g. `quay.io/kuberay/operator:v0.6.0`). + +## Master commits + +The first seven characters of the git SHA specify images built from specific commits +(e.g. `quay.io/kuberay/operator:4892ac1`). + +## Nightly images + +The nightly tag specifies images built from the most recent master (e.g. `quay.io/kuberay/operator:nightly`). From 3b0e26a2aa2d1010e6bccd9c120705c51ae005d8 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 09/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 62a351c5130..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,7 +15,6 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From c667e8975fc15387555132fa83fd51976c093a35 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 08:37:49 +0100 Subject: [PATCH 10/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..95d8fa2e0e0 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -20,7 +20,7 @@ var podTemplateTest = v1.PodTemplateSpec{ Containers: []v1.Container{ { Name: "ray-head", - Image: "blah", + Image: "my_image", VolumeMounts: []v1.VolumeMount{ { Name: "hostPath", From ff17794d6535064559234e8196b8ad8868c8ca02 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 08:40:53 +0100 Subject: [PATCH 11/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 95d8fa2e0e0..1b421e6f334 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -19,7 +19,7 @@ var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head", + Name: "ray_head", Image: "my_image", VolumeMounts: []v1.VolumeMount{ { From 16d72cb67c2fea3732126bd7cd0b75295e168fbe Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 14 Aug 2023 15:35:23 -0700 Subject: [PATCH 12/56] [Benchmark] KubeRay memory / scalability benchmark (#1324) KubeRay memory / scalability benchmark --- .../images/benchmark_architecture.png | Bin 0 -> 30822 bytes .../images/benchmark_result.png | Bin 0 -> 69259 bytes .../memory_benchmark/memory_benchmark.md | 84 ++++++++++++++++++ .../scripts/experiment_figures.py | 75 ++++++++++++++++ .../scripts/memory_benchmark_utils.py | 60 +++++++++++++ .../ray-cluster.benchmark.yaml.template | 57 ++++++++++++ 6 files changed, 276 insertions(+) create mode 100644 benchmark/memory_benchmark/images/benchmark_architecture.png create mode 100644 benchmark/memory_benchmark/images/benchmark_result.png create mode 100644 benchmark/memory_benchmark/memory_benchmark.md create mode 100644 benchmark/memory_benchmark/scripts/experiment_figures.py create mode 100644 benchmark/memory_benchmark/scripts/memory_benchmark_utils.py create mode 100644 benchmark/memory_benchmark/scripts/ray-cluster.benchmark.yaml.template diff --git a/benchmark/memory_benchmark/images/benchmark_architecture.png b/benchmark/memory_benchmark/images/benchmark_architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..cb1d47237dc307a8b8b57caf4bf05b2accdbd417 GIT binary patch literal 30822 zcmeFZbySvH_ceTj3W6ev0v4cB(u#$Ipdu|O5`u&QQWDY)ii%B_qyo~QbO{DZDIL<% zB_Le_@4P(cJn?(q@1Jjs@1JkH&v?dRoTJ?LzV_a0tvTnK>w0uSQJRK&H#LbwqLG!6 zR3?$uy(f{#_iiM|f5{fVz)K?8^T|q{RAls&B(ABU^Hoar(5u+d zAhnAfMKr1}w#v#RWX-*6NVPSN3pBo{@;GWvx!}ug*$w-jyfT*e>l(DHuKt}$kUNP6KXyv|-(UN`Tk`+UThg6RM@OfbWt^o~PUrHO z>CvM{U2bbG7cO{5Mt1pgT}st?muQYCxq6)ZsFu35b*87M^!#|oEdB8l=AE}{qV61x zSB|a7s6I71JgiaVyjWHy<068PxwI{+JD+)PRH+w>K-M~n%@>rK({v~L`x|p__ZD2i z2L?YS(KOHgAax{b98prL-gCmtkbQQfu`*mFN84}BX5ec`@!IMY^^J&Lv-O6#{MzoP zoaSzgwPr=SE)8FbQ!Kya=H`~rSk)pUCzq&}(#q;3`YM8hgJW)_G0CF)BlB_9BsJ|^ zi&=@t`jvr=iK_Aut4pJ4tzWVDI!`PGydw_lT12fnwk9)pKKCH9r-q+ zr7UjkZEYf=qU}YtRbeM}baVu;oOFD1vPONnzPwR;k&BZm&n2vY@8Cg0{1XQ>w;j)< zx1MZws{95+pCbl}iy~G73MnIa^2O;vx1L9wW}Srz^m1B(H?ge>+2My)er8lBmdEx} zvN*r7?3YVnCy^xbi4#(BpPQVdqNPp44Y)(TAx2Www5v!v@xkU@*Sl8dyWE04o^n|- zWUotBPt&m(X(*2nD^QsoZJt=Wf&?0~2}`J4mA-c+`ba8}Hz8&`c6`S+aga$Nrd}=VtR>5^0~<#u^Ll{L1eMqqm%w$8x3? zno_Qud%`d~Ihp6YnBG)m+Lm2+b+jpED%X0rE`Ue9DV2`TCtvE>?*58^`rX6=Xo)>w zj7rnWQ_jBC;>qH=@HOPf)J(xdQDuafQ>>dr)rH|L-dckIUN1JWJPpFRmExGt}W4 z8%e`w|3`hg#=h!9{eE%Cfuy&UY`5CdLa*2CIns}-GomEHha?)F!%xX9PB}7@c1gNF z2-x|BB*Y>{>%RGb#3#A1M;rh3d;e7c9|SB&bCU+62g$rouQ$B>C~}JQ|L_r;Z;-;g zA}A#&wnbfIuaaEFRa+ zJT}(6p))u@{?FG+soh?j>VLN9#B*k`e1!5?Th7`}sfDE_+mQyN?{BVcV&HxF{CQKF z7Advw_u{np%1phwu1jOem7J)keJg{z=Ic_mMTYw+XqgM`Cwo45(st$B$QNspi6Gq+ z6B$OLXK?U@?a%MTg>TCdIB?(qnN&$>DT|2ht+s5_>A`B+&71cMS^V%}m7d|*?H=pv z>)Us>HOoYu>f3e>g^imwiP-=8Fef6f16fz!m$NLIJ2@RReXX%9PQ@_$*%8&15vf3F9PTOGJ|zdNoUQL$ zxX1|(4ymqV2R>DWi*V{Ho;h=-Dncwtch=p+%CNfESyooIGDyI@Ikf}fGQ*;x_jEH0 zUDUdD>y*NUV|6FWT8UNmK6s13&NOWc>@KNQOVi=|H23yK>C7II#kJ?74q-}l8El0* z?;LO6e9?RF!2^o*>m|>f8*V<8eh6pW7j;d6f=;%Ya?9SD`MVbGd{?r|S~F+sp7VkZ!j$IUz2^Q;Dkh74B>=(Zd(?=0;895QW~5}p!>FxM-D-;A!t^_LmZeef9lrMQ9u9Bx1Qn?? z7|-00aA`L3tWe$3?CA3BXj~UvnJ=*+!)r23f_i1hy|XUNVLez0&T4POozWxj25%RI zo-kipUY5olhIru`{Z50)8 zrj5daePB*BaHuV3qVt{8EQ`2MCpNLbX7u&HmTGVSSOH~$fGydXe4$Tj_ zuVW6p+xG7L8h(j)8$Ep^f@)<>qUo&rgXN9X)X4dnSf3S>h;8!u(Bn&{>)xyIX=Xlr z@}%DF_YeEY$;oe_Cz{P^^~4Z$v{F0Nx9-_9Iy#!vb)2LyJ6XyyD(+;#a6)Lq#$Cj1 z{<!Tmwj*RcmXFKXS(3#C64zUSvFYdZ!@j}^+l ze(C?+ABk=E+4lL|q5SusLvRpx$E(g*-D<1@z*s6Y|lPD$p5kZ8+C9MjqSyJkxh9YRz04wBKiLJAJxv-*mZYz zFHhM1UbyYT>Uq*?;AvqG!eo}d%5vdXH?YZw{4=Znc+9#Juije^n5n(DHhc5rby`7^ zM#gJtYTSb>xb8&>>z*=+sOgvy_k!AhOCwzm2xK<)%fG)q;!$wXwhgau{Jp)BOZg&H zH4zaJ+b4FOwA7Q3kRaeve`Su_nyZUR2JnL-DmcsVo{Lw7i4Cc1?vQn#o0_VluL?eV z*3vYHU)#F>H6AP+k#*|N;-2!mN5`fW6d1U77P}SqwW_6PwlMPR7PVJbr(KMeGHuPI zqM>Q5So=L)J>F5U2E2s`3QV-?bl?R1K`_|*ag>ylC`5?1G&ierOtj}sfAFAG&0~lf z{2Ee^)p4AEF*Dk%bTi(#x2H$1YD!bY_UFcp8@)9{hyYg3vL048>OxMq_UVCkp?&S@ zec~5}<06k6)=21>l|0-Ataee=$Hzx72UooONzw9n!OHS{(3gDDN{3ySY$$MuF}u-U z2cN7zF0NcpZ2D7vi`zs1$>`eIRg+d`n|0+=!;hH+s!g>RM9Rtnu&@UJ9bROf|Jia$ zbp;oDO2yLZ`QDR&k7`M3DZmedRpD2rGL0>IN*)0fiULy!yqoboT8Rx<{k{0<<43yj z^@bW0=H?TOI`*eulwVp*a#geM{#0 zzp%WV#OV7_>z$*GgNQ)!@ATwn_)57H|1#&8x%YS0O=V&yn2!TrKJ)U*p{+rcnV*^Y zUG28&_nhTe-RJ!wBpQ>DHkw#-$veYeJCfap+pjX7*R|vPpfZ)37Ot-2a83>-FD@(O zxakYFDpb~46-$HA<DrzWPBXrJ!n@|=m{mUj(1Ku z>xYnh+1jaUPMtc{df3Q8n=3@b&g^42iBI4t(7I}pN+%b}Y64&1Wpb=miZzkk2% z(3*1PgsJA|>S`r90&`n%O2`-L6x!d`(rO3pqx-4Y9emR2waPK$hJz@1p`v5$SJzH| zA@GP*!ONLv2M>k+lPHEq$HyJ$x0suo%f5?Kic~YzvG2Zr=00@`XXO!tDw$AP@!yBK z72REXpPk6moUA!Z+lboz!B+8pNq@7B)3+kqslIaLY`=Y} zX|6xq^HaP`h)CFGC;W+^JYttY{@Thz`>0EYcEu$j+lkH&RR8i9++SBjdZT=_k! z?U}wmeq7}`Y5Ox+Egd%>e`9{AHd@w?gI&(wyLH!*o5Ug2>6nCE-Bfe#@s4sIR?pIo zxa6Jv6w9=N&w(jJoH(v8N$m)JUb}(6sy(8V4qK~CVc7reM3=t;*X`r{{QS7nzJjB# ze~lMRaNyHWt?2_#V5wb#Plotfh1H# zz*Q0gimB#lYm`JMC7qprgOtT&8`Zy_%QBMwEA7kv{kRsH$kg+jYX++$(^L|Ct#m67qzHSB>0Gv~8 ze*B4W5(|@DjAE{vdlSdx>}&upTv{>XCKIT*p9V^pR|&?K8|Fl zXJ$OWK{2$x?a4rO7 zvfZp~=-~mE)D?+W$y#}W9h_fIp4QWg;q3#+ouN-H`KQ7~cZ6G3Ct2*^R*46y9+@hy z04bwp;Eg}_Yi^GD_^seXxnmNgrKR#IK|w*s9dH|}Bg8_l+PV`F9U$Dlf%D`Z%+1c8mpSXska6>y@A305j=nYVHZGJtCGFADw6pd|vg!BUIn5UpJTf`_cSJQ9+Qr@T^qeSeBY^y~Td$&{8TmBX z!mGnhS}8_|CIpmz`BI^>ZQC|m)9qVWPC3sEUDPh08LqeKn!{gVt#7qUp?eX_CMc*Z z7En4@Di9OvZ+L=n$5g)ZtJHOT&soIz_g~gIcP5Y9zd7Kc04={x;X%GElW8ulpi7q7 zM>E8`+4%k=J?)QXeN0^=Q26VkoCt14vT9PG^yAT^8s{izHG+?uw#LV;H{92YBz&y3 zwaI2P4wLU0=lQhggozWtSL)3>&TB45PXyexct=C?$9>v zy}nd67MxPj{>vK-C+p+|9NJIT^Ruz(eMBnYR7{4ZC8-(Bvk&(J8Dk@+)5)$sBDX8rrEP&UUvU+B~ zf6k~+%q}w*HJEL9G~b#ur_!EUkO+*i?5_}1axmhPP>K-E!#Yt2%!^m%Dgt=2G}ynd z%J12^^I)YK&K{ZZ!Gj0Js`7!nK8x|oJ={}t%IS`?#NUIzoQ&0C%bOfj{@7VMAx9`@Ts@XiKQ!1-+ zW~n$Mf0aZPSvfYHq&ZJomcWyP<$iHCqfG!k(da@dH4#jk~n|2)B zSD9_rA$;tF(WT_eZ#|hEzN zB!f5G*o9 zIxCxmNj>G4VQF#TP>M*tp83lB8Xdu9c8-l-Jy1FpQYQJY`0jUA@80UWJBuYk-ktyQ zo2a=XV-Ow%1O&=+*Th-dLTCj8@|}IDYp|iU#%Ir*i78oCdA$3WF%F6A1H)@EF|p8N zCa+Z=;fQ3HU!L{kXy=r013Z}W#7HyGHykz+h0b1*!VN|FP6jRP08evnSDi0tBvzTM1S|ij1uG;6jIKQWU(R_RT zoWmQ}6~_c_JeRfzPAl+?M)1Am^@bgv&m-Vx0GBJ*Y2c22Fkj9SLr^Mz2RXmjKt2Q3 z5a~32k~XxI*(v#~=a#vb;$wPeGtIkJ<04(_%=f>42y#5~MBzHB2~~6llS1FW%6c2k zE~bm)4%E_ebWvILd?f&p2QJM{bakbcuW@Un$5ga%ODF{&I-R6U<_LDE`G_HkUElG$ zVsO)$LaA^HmugZ<0ktOpdUd2g7Bs}#>dIz22y9gI)HTLNs_u$!!m|%M&HaShSPbEo zTlR!W)7PQfhAmqK%Y$@XxddJW2gi3Qx;GL$6ES?GcCxpWcCS#Bz_D>46ogw-&`pZ} zkqGM;qiC5$4_;baTx>bE*ku)s`thULVAK|eM#52eUm`pg?wux7o(dJ?oxteyT?n09{LuM_)q{ZprkD)pEPS1@)BWe z!g6|hdvP%?y^?$c43MJs&GhH@w^IwC1I)H@$HavsN=r(1@M~RWs9x34g^cbkiwnc< zMX2gtSx+nzP0(u|DJJm>n zC0OVI@}p$CQNVMzN3+W1ugerRz7nSx9_hBKaeWe{KO9mxauSGp>>llp zA3yfp|M=w)y78G#xS7PR+64!#4oR z;u6t3$iM!X`j~;Lh*4e4fpd=y>rXBA`OR@E#9geY?MEX3kiD(YAtQC~n|ps3&|Ukx zTw}MJ)Wyg`akKmR-A7PjZKhsnb#)c=%eS~yJEX7DEluZL!>rp;<%1nTOR^8BYwBp_ z897(pSa0KUB3LA_Uo5#nxbd|CqFOVWSZRh66pB<7T~Io`LRIq-y1{ zv4}7K%;@awq~}(7LtAs_&Yda!E7UG7E=ej0J=g5ERp)d%%5!9ji=XewI=V2Dlty=p zOmjdhU9Vg&fZOHQ2g=Wn>8EFA!UXjnH(j2}Kgtg+4`_{BEqUnL^7)LVbEOc?^{QIe znh0s~NzlQIXGFZaFZ?@*JG|?qc8GpoIX()YOu2dIA;_@kWekmyIP<&sUpx{@DR4rc zgKfZg{^st(*YB;@;FCz2Zuz~k(wODgXO8xj|L%g~Alj8V5^B>BB_P?&qs_;F^moR|nmTpju z#+IdOzv2J3zIDe(p*nM(f6;(ml6(;V@F>Z^;XVHg4WYOK0FNLBirvfw`a>)+e$(F){ci>mGQBFK@9bh6*w39fYS16AerNy}Yq@ z#k9Ny&5XngFYl^Nec5Af@8DoFRP$l=P0se*e`!Y%kA}jwZAg%_u|^**@2LYS^%C`a zE+T7ohmDW7nHLA#5=IMc`6IB45m`QREl5#~wvCQ*I)jymhX<82j zyt^v^Tf+@@G^cv2R7J9`Txv0C%}h8dWH%n>2o3S#Z0NJW2q*qAgIs;BMD4xlPL9q`QX0vdqH1`&;l6+kWBO()s#@|)(Ss`S-YY3e0e}z*6p(1V zPj23Q^xy|uIcs43tV@Dy62(YcBX1ArCdS?dZ|Nh5fQ~~y{jz3lp@WrC)XGoEo2nT% z&wCyH&Csr8G~DxJj?kj2c3t#NJo=oUQz>HemMsLgDy`MQm^9 zlaSx%igG_@a>;QZ#2~_Pgln;|QUz(Wrh;8UJw>xIwR^4OvRwGni)bquHzu|;KSpX4 zlYYKeb3Fvo0?)C;aAw;eI?KC9ZEstc6AE>^Ta2)QHkQ=q(CCHj)Icge#7+uK?-6mP6 zY6MeO&K+N#pAax@h#fk$G!%{6Fgg&12ZV4E9|t@!JvHSxnv$dE6Jg);$a!(9rE7h~ z%ev0bAD;c4I0l~{lnHqG&2eREHm!Koaj51cgtYo*vgg+f8Ewuhj*d5|jx5cM?b*J4 zJ3W1@;;S0+86+S}Fs>|PRU4{7Aj5_vb*0_ttx1QT1-wUwh-o=eEgDVF(Vj`vuxXT`^S5-GMO5ns>eY@V9ELOtQY%3c3&)QE8I+qPleZ@l)L zREfL0JK9zwJfutiXwk#tJwN`auh5bYutv*UTU*P>l-0$`&(f2A)^unbWYyl=P66!U z0To)$!WwM`OR%voUMy~-O?)m)q&*Ox_1`@4aN0)*&wr`add}89L!#(=py~S?8s?I) zd`FRsh_JAxuCYNw{6!?DnxKz=8WV&bzH6fA2gP5Qd@B_2Lnaw=6;xDI*cb6x`X-{| zooQ4*6KlKs_^kyrE=J|MuYYYl$0uoLu>Zt&H}Bhj!p7_&78snX0lAf7>()1fw0!?u zMIfKNyX`uT@mFdu3o^X_LklZ;*mD3fBmfXCH8nM~V)YnxY%^L)@yu&;ZRU{Q#4Ucf zJGMd{Pq-9U;Q=k_gmFVzY3U&NOF+){UAwsM!$M$$JDjMJP?uq#BL8a0cFl%2VoAl=#_OzDCXI{UO!HyM9cCf5P5%R0kVA8m8I59 zu;pE7?9A;rvJK#f4<+5`MZ`Y(A`{CpR}2x(dqw zaor;49J3BEfqIa?SG&vPWi9}jSOLN^yDnTCdzi%#r7sJ!!0T@pSjmKigf2U2BH@>> z*v`72Bu*fTe@h4)2x@}!ED1zMdOs#kh7z)JDPQZf&J}kr* z0^15)$yP81YTNI#ge~-cQ>0h!iyIH&5zpw_=Og`dFk#45QpVl(Jh7oj8~2^pHMq_F zuupH*?;riAlV(?hWV%foH7NNaN3_D2z6i1PQioaD58$bP*1QCa{vDMjlR-_--3K0G`9_pk z+|?QNtJ7hSNtJ%b4wZasLigq5;6S2c!|vawrdL+Q)n4xNQ`)juXrC&umN_u3we<`_ zXQVl84UGj3tM$TuhtO=31i9-s30>`_EcI*T=g}<|^PYvkz;5MvxLl0RapoOUnEdfO zR$HpIpkgRRoQg}!Q8Wv>hC=Ao45Z_kpB8++jNo|?duWle(i2rpa z;~@{hquX=LBZ;PCImd;;b3z}G0!p9lX)bniv!S`U?-_xJ2>A)Z)sVO=v8m}(e>YXf zZ$PSj`SK;v;+(A%C>EOjXbKVq#c2^8sp#nFSsV9dUvn=2eF9#gyDF=en{2DGxHNVr z8J?i|Z43;@K{H%DDF=Pi4&$L4%=b_(Bb5RRL2!;qBegf#0&sol6m8W3p02Jg-Q)LM+k|xN?d`8#{eiMPLoY#%Td9_3JNDw| zG|@_T+tR%=JyHKZKWZj&YV$6^dn`?tWz>r9GloBW^k^8ZGBgU{QJArDXR3E%PP)}u zn1?c3TU)#2b>J%pk48SDM)#7bd-7vK%|G=tT+~7E(4h-={fnkOddrQGZ-F@cU6X`r zp*-}-H$2?ND(G5jo_iP{wuuia3=W--an`TLWgyL!3_DR$@U~W(uzaqwatmV`bH7`Aqomb}Gl;NJhz?CAlcS}$|KIshIPqkF77WAW+ z#Olkr1L%2@f@U@|AuW2Byv{aln`+w~=cCHUBYNP#nPPU{EF^B!S)#b>lI2DtJ=-98 zLoNWtt9O&U7M3if5t1&a|9PbOn1uU-f>v~|B!!=s3f)Q8(`fWnn{9i)0i+_^yvr5r z2Y7*rot=HGJul-*ju|Wg2#3vo93lJW4bHP_Tujf`i!D#|qd%B*QH_H#;EK}^@ z!J!*5cajfkXs6fdHI`F1q+R*}>)hqx8m!4TafzUIcKEmM}z?Gz;_7ifi z)?ne_a^CIUFF>A@G`p?g9&uqxXOo^6CABjnzgkuuf}&TNZUm&%L{pq z!ew%Ps+>F*)S@FJBa`bCX12eJiHU)A1;2m`XN84|RSi3kat{;skFVE$f^j1#_CkpR zJH}y6EHyLLMFA?Nk2Hkm^W`;P{gS}az;e=3Quhw;zzxnhtbvYL!}5`B-NtNUNl7ll zJl+8{f4PR-ll$={;lehWxfc72(c;s`8kUF78CTGNV;Exf(LzfY4ailiAEJ9=XD*!*DcaL*Hco$kte51cA`Hlg3^7@ zpSyqR&)tW!=!fFHfyd9D1x)Ps6+--=65FiERfqNMIpf7F*2i?abRzkEtr#sS2bPJp z$GA`4@M;zY2k&&DkbD4rN3kgE-o1Od-`=hCRlY|Q!|i7|)_M$w{sy%_0z^gEc*Ek0 zAlZp)>vLHUFZI~3)@;)Pqk0aPagPM><_;8atbFu@#DKrQKU-I>u-!PXZvKo-*>$t_ z+}dKdH9B@V`|mrrl=b?`e3qn!>38iS@R5s=ai@LnGsqvUK}h>jf#(ktxpYIOp9as> z^-c6;mxm1!BHAEmKseeRx^646X(YnR1N)F$3<{x)<0$~VQ{x}VP}>few3!NKVnfef zy?Pa0w<7ZK1C0^VQ`aFtJz*3m0?FOP$X~|DEn5w7pK#KgI3ENFtvpUjcS!H^o7hEUJ_tsgSRTdouq&Ni5sMw!S5MIWFMF{r-rwMRqjV7-?4S;uhF#P z5uP6m%3}_w@n|;9Va**yZy>&)O%=E=G&tCPX=WG!fjyTiRLde<4X`+}8_F$YN6J+Y znC^D36rW3l2Y_FhAYhS{2}eO#ro;RFohXPzUpW7x%DT+Ld%W(vF(Y(iq2*H(UB$ZE zZc>a{pT4^7um0%$ zqc4%!M*7{8Jy4`$9XVcI6qHQSd8Zlq^T!UrOuwNYip`#i*(=kIV4^NdGkMm-zC~L? z)Dp#%TGI8QQl$#KJG)_skbA{&ug!6yS(=OcWX4bs*&fLCsjhuMttM=(A>?t>pYwDqsrev+F-nQV*fCW3Sfd~MaJ;&oe;!_z9UXTzW zXM&ng0Z_2ga7igafNqX#sd#NA?cTsz=ncy$$mNiQzYN!w856=ykbv&o^mKv!%PrU-L8NW9k$26Ssxi(z_! zSVuG%9|gnY?Q3ashmlXBQd&wp)>FD-UH%l`IE0FNdXBa&-11Hxs&mkuW?|?^C`Df* zN&p6Ec7b=GpCNK*^3+zL=Hk*t_CTI3zZ7!z?e))Wmx+KWx1$__=%twRo;kZdIxCM% zgA62_m6!Jxk>?aBIejOO9Xwd}ZG1N6R#%bOzI|5-#~gKTG&rExR2ggjCA;Hh?L+*Hk-^O#L|(INRnzfDK{0I9oReh=uIe=W{>;7v>Y|Ida7W9Ls#+&+wn# zf_a7BF4u5FvsioRqQ+=*ieXZsD`~uD{-{>d0f1iB-4$%lq#=G1W{|`*kd1}wGBPq# z9b`)Z>N@Z+JUh`e>+SM4Cg$%A_TS6!s5-!q^O>9 z!kj4GVIXz(mW#_&!wC3Dhk=l|2uC?NYckC_2cx~Hm=W-t=i$N&dIDFY*Dpi=7@*z0 z{bWZEnqBkL)AB`Dw}r=YTmT*%yYDkvi!=D|)s2$4bFU!nZbcy*as7D^`7%ovBR^WswfKN-N zz5Lg2+#oEEuO1*`q%|EQAR1M*wG}`gU^0~pqK3sEzrBr7nh)J@UTJiPI zk5Muz($R8|`#1yom;78p-UAp$G4%LYsCkjk99rok?_|cSA%CMDHm6)+WML671xzL^ zptz9z-$K>3wbz#^6A4rE!P`u~4?nPROA?fW!Y~3ktvSuZ?FwgfP<+Y$oUc63#HwKi zfE-^_aT9&2R$5sUfOZsnlm}xzV2qRc6+4WY@@-5WRdUC0wDDi=Mg{3tcIy2?(_-3S z6Qr;CD1s;qzrt^c81hkFc~Ksu)FD9r8Kk^%SL1w!+uAB(!MM`M?Xb}->lqErgx!$A zc|Fv!tOgZje@z($L<#KI2`}O8z8rXZcV#*hTVf`{w@`H%mq*rVw*IUJ{S=q?O+hrN zE3+KPH7Efd<$4bVO8sp*r4$xYzjZ4)EiKKpEjNfrC1-pk8J0KRS15sJDL~de;_!_J zMLQ%O{E3cg$)l1K=UiTR(5K4?!G(H6l}l*29}Be7NL!V!bTc9kS^nnq&q5# zcn4ho_&>(0k$!K&-kp`OK5nexKQOv}NUv?89H#5^s%_^<66gyy37)Xs8l^26NL|wd zIy_PzR}DDcK>P8>5;!VJkMP--x5u1wx7{{i)}(Oy)ii(-__W>Pl)ej-mJ&nzIkMm< z1X&Y#x-Ci@=GSA}HPCoOiNDhMu=ZO_{*&>pVz;>tS4=z*{mQPYhcHhfwQSjWNTSFP zoK2$ORXeJc{^|&a+pxv*oT-aXMp?m|Vix;E+{lLW+W|8se{7oW@H+1K>(?)A@^BIv zCow1hVDSKpC%)9G1t$*6N4^GymRe##e2ErJic5B)z&k|0P^U20c6vwj7G}}Q5CWj4 z9W!fRfR~b>jG|%=Q+nxRop+d-nbCM`Q1aU?+leZoPvQNNro7MOwJXILOakqH8qZO7#q$mc)OaedCz7hao0j9CF!pK`BOF+hLnRXsH}4kn0k%@7*_dQKq=ERMyPO3 zAuqqGrQGXt+AsvG;CCgPo)Rhz(wL&Rd+*-?a2Gqa>9Vgnu-_=UVWXozrj#j$0M2C` z12+cW5(!<*LtRBv22B>O+bJ+E0)iI$r(J$~fM~lohXG!|33}ie7{M&?^7ra*I(N0* zQ(}J%$nNnfpJ8$*vrf7rzMcKBd(S7v)%05 zzm$~7gyMXPCco@iTbXdnI{5_hn0$c30W!z0!H40-3)+#C$Tm5MELp)Gr-o~U63l(2 zQc&hrz`PD@L9jy^##tmo%LGt_sGPv|lQ6_~W_^Xw%k2Q$F8eITG&I$Mcpc!`)Z>Ou z5C^f4M}T|ca7*@T>}|Bl^*>;CAPAQq*PE@2M?$_*n#Dy$Irl{pVPka z&X@9Cma|K0;2@F;D28PTV-M%E`>SwMM5deM;iC(DNThAmgNL>^a78=b&AC<-W0Z$FkeFdW|u$1(hyoXe!nkO1?5_FZjWll=Z)X=Ji z+!}b7=zplj7ik|g@6=*G@%Car@xjV55BVh~S~8?{R79Tgy}j&GZ$I&FJ~Ja}ZE=uA zKp+jy1!4=NrKC_kwVATjN4si_T8L3K=!||0h`itejUxpZy#gRlihT0n9CDmok26Tz zQkyf>36MN^uIjf~aYNR^5{QfmbJIcD86;TWkNn#CGR_Lw9oMAqhA-7#!_^1u(-%*9 zckk%>Aw-h=zEyOM4Za57M$Gtcr=Xx%T39e_-W&q)U!}Y!f%MK{`le~bZu+RP>T~=T zUzMQ2!Imhu2s)CL2U4rp>59HVC3S5pz+)xr7_d%a07$teQ?YMPJUv#&kRGNGS%DmqMebi@g%haMsnTnuuzVA&>?0J9jZ%(rx zdG6ZyO`COuhn1}JBa72%&PHt&J8I98rduuPEiBuKPAA+h`yJGMZxa}i#NIr|ODrA+uKorW?_|Jq`tdX>%B*^HaK;ZZ;D*HOZ<*)iesYpZu4Yxpev>H3W z;$q1&f8J>oJ;p6zQf#t=PL{n^t}h_xb!b4zklq#dK>a2>vdho!heNz0QRXyK>IzlCqnC!X()$OnE(@#(KeTKDTb8ukrfmw+Ok zdL>8k+Zxm&x^J;Am~vgYN0*SKn)Ldc)@#Je7$EN@nwr-8i)=NOq3~gShIK{n10ZWu zRy*N;%Ygfr?t}hGx8IY}DTO$iI%Q^{&(EttPt4!V2J%bDUlzdKpX)uR5&ZuBd+3`L zl3zgQ8CClCw6?NbEKp`MTE!;30>U3@&cftdNu(d@C zI8m}LLazw_!qMfjFApeG@Pv($y=to0#w4CBZ(4uN*k5C z(NqZJpF>{6ux6hFuN(YM(M2{;pHCud=95ws-vrX0OCReq($($4XhSzVSn0&9mRedJ z(ziq6)Qs4=Ok#$^Mkhg;J!QSc(Z8$k)~Ja=92vNJfW^%=zzq6jlAvz5#0iQ*A} z0Rc>P(DsD*3CGxlkE6idDo3cA=&0F)Ta|dOY09xZk_(V*+k4VdX5i8ZR)sg4Y6v%i zA-}Blls^10kV~|fKdI-? zkR$2f!QwLG&Xw9ouPd)v>nbh%&tZe|eO;)_W{ zvfX(5$;DMN&8CRLAk4y;u)9ZNGKavNcS$QGsOhjzjC_9^J$LC)ZWQ$XgwfZ-GD9lT zFL3d=uFb>OplV0w^JQegBz^3~H(2{?w?wr*dGe&qFq$b$F`Fqeu*KMJE^rt4okqAa zugD|YIJ(8UEP#|p!*g9lY;miYW@-H}fvZ=R2_@KvG5|!Z^5)~7xx3gl{=kg~|U<P12s11G0$(TK{4^RyNH2NLj4>iNfS_bV9Slav+@lzwsf5* zKSLC@X59H8`lJ~nU`w*?$t^;j&f9AgHORsq{3l*-d`CDz3Hc9WeB+x{gmx4h{A&(( zAmS1FkK1ClpCy_kis4I;m=ak6KbMpYU3q^C^n;X+vDtP{zjZU9mjHFgYHyV zJuQIh3DHdd0T4w)vp)0sA?pQMjhP(?|!D0)El#Ah#cF%h98AA7JOd7m>0um1H z?QrPs85p_}^Hk$6oYoC;*cw$oh(iOWzD4vwUFw?2NLo~Xw_U5KYG6RY)M#W*jvasv zX8gehS796X+Iun`_B@#L#DD+taJ_qU84t=XOaKt1M={5l_#)xyeG+li01)hAm8~W% z>Fks*nJU^bILtsyC4W7y34a4&8iiLTW)K65+Kp&Buw6zs@tk^u?TDiSIBmVU-f?gv zUV=XvC>l`mEi!PQotjs*H^tbHtMHz$|t0FaUfK% z8!f8sw;Wc$v=^aF#7q)~c?jkBnlo8mj~@BlM1c+3CD-RmA-G6<7OPBSUG_6Mjpocsn#7x)oVo_dvmDIi!QMH(A6G59Bftjh<)bQ9j{@D*N%6GxBE zz{5k#v%tg3rYyfqC|SlJ&^-NhI;)A9WBvW`RDA>IcXD#7>;<4n-0(f$Ch2Rj#2s?@ z(*usNW(3U9y&em!y$BP=b1QTnG~^KpL20`{#TRxOZG~B1cLyPgfYGd$`LNE->r_de zs$wfRNDBJXwieqyd83rl5SLBVVQNTz;@pNIc$Q4ta}S?7rSnSaFLx1}tZt$G#Vov- zASjRM-j599T?u-=m=H{eBc}1+ga?%j99B}!})+0H6g~<8)_iu2bY+xJc_A79@F6G>D^69udcV)5X zFn$@90O(MaTeu1`UFi#s!-`JQz%+fx6=mhRN834Fe@~V|i^Z8R!q|&yKgSNFF!e4} zm?D|n8ePt;K=g>|j={M&xwf5?O%$e5?5LWT4VfcC1zP1}48iUuz*yZ?+zTo@@);;~ z@r{pBt(ucHE{O%&q=Tm}YX%fImha!vUqME+v+%(tP^_FMg)qA7T6@j=XCpC*j`HXN znLTN#k%naUnDk763xjVx12DPbF|m0ZXBSpimNAG~8Nd@nI1znyT^D12G7**Po-pK~ zdm?0xXh6+9+kR`1UH<7fLKznW8sdIa*f!%6q4ZI0+T>T#jt&;`0g+=`f2+D@(+>5# zEoJs|Fjms%lYNSDp3CUOKokafVE6mko>y;1?P<|MV%oEZr|SUL&-qrVxTee<#7_Y; zMYb6j2^v-xCgsAelB_qxk^tuz(Od+uhE#$MAz(})Z2HBkOY$Z^?vXlR>7d+O5}=p7 zsfJbR=^OKX!+hX}$~rMJ+hjB8R@i;I1SD9%WW(vl@J14d>V%1a{4u`-u=r9{;Tm28 z5;&xleEF~uP*?;+%n_) z`-4hju$UA$==rpA-yoaPF|d1KDg|+`)>ZX|ip&uMG`^?V%SL!^;q4~`=V6DOA?m#F zM(y$kDY2#vL&Q4>Dl6rS4nc~E_r?Aj(co|D)6?@_enZ@%ptezn-hnhVaHyv@B3sAk zss$SZ$xmhd!u~DM{J5@V!MMRY59W*5WttlpOhW11E2#HyeF=`3NM!iS@(KzKIR&BH z$usp@APG7;I<~H5E1?DVs^ zo0sMRElY&Arn_328OscO>LyLeT=$`FEw54S?R5=pKm1PQ&NHmncUvFx)d0Mw=@r8fI2n-5O19rIq;2`WT4}1(2u#Fa~?7~wg?8!Y+R0eJII8O9#vT}zVPk>{JXeWfjBC{ zbw^@p0De0wBp(xN*>?cc1hJ^!IR%!bHS6D;0s&9V4J(4cDXN}vUM*g6{G z{TCM5Xs)0;4#N^VnmazfPdG39!oGk$M=sfZImsu%1Kx}8@UIcjIBi*+s%F`SJJIhl zT!HH^41YUR4QWY9w(pj2juI&U?pTpWzHE$h?W=Qi&X_T7nEi>@oY2}(XE}fK{*wPh zc_IDU*-WQh!q4S5454xv1yww-g5v=%I54PlN+wBYLh51j-Nb%S zUMsMMw^oBZv&4|9e#v&%k3Q0~<=f$;6-YV@i;K7484UVW=Q_WK>xR_6fo%5{w2z9> zcR*GJ(xYB)82po{ zTJz)KnEAuqhCFq|_4fj<-}1_;`rfu*^aGe z#<`<4gFY*ak|~~>ccmkfp$Xewr1r5Mq`a6!AYl=UIbxp5-}R zJu5giYHDgwmt_pcG2jhMM$06uHxzXCUx@MBLrmSy-Z(F$fx$UQsIE@Ao10qzTa?!u z##{$?g3}1~&&zQM70ncn56O`3JTfDQOg35yph=x#7d0_^djR4{uA;CJKw7>dRT%H7 zQ$^U(rLk_of3)QGX86w0>p!q91`iG|XE5pjt_2DN`h6MbbipF9vR;i9uC`{XnxAwG zmf;;Fl7h7Rmv`qRB~c6AZf)PYw;i2A7N91$>_9G-Uh{io_Tg|&zLmdRW@EdKbcl&; z_xu}tH?Di)Omf+!G2SiPLzcZ%>U^JAg!h&3U)$f@LZd90gEz_iHQL~L8Gi(xe{+4o zz57}QCuGV4f@maO!(>Ca++^hc+Va2FvyWX}=Qt^ZoB*J9J6ZYi2m_wfn?K*};s4KV zfNXwbn=F3OrjF{L&-@2jF7LxHHOngg$MZe-KmE5qFPktb$EI3d4|?N0`jdhS=37$P zT@Vi6ytBOcBSMV&*sgtmpFb5>kFeLIik>>P3hgE^5nFUoJ)jQb7_t43A77`*>^YXF z2rdd`3%AF}#Y8@@%FNrBM_`Y;!kxcJK}cwzw;AB<(?fPaxk%NXo~2gRVg|Yghx1T% zlvj}{!$j<=ibf&;_}QPqY!y#@?4lTI$1h@#OX(^TvA34}`s=*8gjgjiA2Qq21e{TYG{FWUqSY~zMMNThwIXfrWuN92zUeR8>mm;)irRY z|Lp6Q%{78ynI_Hr=n{?$!C^g%e4dmD1jec~VHUxRQQTb^S}SU(!tHc+cGi>4^u^0e zMB%H&`&!mkm(#RA6(|HBdWkmwQXOTYR$h8_2i%F_;0vQ|d7>J81Yrphewt@+;03W2 zseX!IJJ^9dw(qmQfs!kjJ`y!p>mXMZ~M&##Xx&ahG7<&w; zk2CoOcnMXTttIdR4}eVMmt4HB03+q<@H8X91YZ%p3$cJ1tr!^})K}u=5aSQ*cHx~a z!1@USF8k2T07o(_?8W<~y!Cl5L`i6Ag%u^3_|=|(fU#7k@UcDfp0!nYpWZnCHI}HX znctmG6L=p>>j{u1e;HHV16|WdB~A$4dm>c!s6GUgu6|+7G-ab*bgL~VXp5xI=lc31 z7=pum95KD2HtSAWK~8Uzwi=#CCqab}ayFjLZ%0eD?7Dm-Vo%>8CN4eF872gRTcDqB zq#L{~HDeFSeRwZ@)Y@{_np~-=+rR@owc!sKb_~457QdJX!zy94=kj_{xjS$QbZNALJ-6}`JZe@_ozN?0W zDyQvUWZD1tD|i2iJ0&HOEql|CTlX^N3*O%qz4N zuN&gsVL#=O`(oZYKc^dK`+=1a5-(je#v^p!T>XIjTFH6w7zQH}`v#EZpTjNTRGkpr)mktKDMSK6qOEK86c;M3zOi17~53IT3lV8 zU>`ukb_U{OpbslIw_E_x8HNfGln8<*hLE7^@Es$EZ9kK33OkHOdofI2-_SR#sNl+!&4nxcTR( zLH*U(xvTrk*|Vy7UlUQyln*|60dx%LmSm~1I5l=oO3P@UJp07C07?pqiX~zMkT>Az z2fQXF-LYTWpYMvA9JmK(CGXbmoR3g9ryH zkT?s% zvl2NsE~*&kc&B&4`wY#oY=&L?)I%@2V%N^5Q&F>m91BB6da2v*-*ihWDSWHV^BXaH^UX z89XeI@zITu>9b~CVFN=0+sO8iAYjM(DOmR21kUpZuCvy=!b~#Qr%OHqu+qo?ANtpm ztf9)$3^-Wk$^agQ!uL~>!_j5($V8q|%JzzO6Y6PNcucm@7b;(Nx5 zEj94-;N>}mj)G1?#}(XN;XrIuJ;-}gfADJW336+#h22UAMSYi1HhwlF_`OD3GU+Oj zP0q8xw2um1(R5RLdFyjXliUjGcstF9W-X5iG^38=^+MOqqtDhXEQXmP&Z8+|w>xZ_ zL--Er@g$~Re*3ZTXy)TNSS>me54zYk2ItG_cU=&YNb)<*$xO_MJa?I?m3}b}9t+ev z!~a4D8j7=X;gs3Iy-_x)f~KuU=@XeDr8a`h5nZYb_$ZN9$!xy9)YZMCcCgv#>8|pW zOFK-Pv@{wh8dZ5xAU?K)mH#&L8M_XP>I_~aBUEr(GH7)@q}TXq^X*&3$uplHTS-z# zZj#*&VrR>1D@?W&8>Evia3MwWSU9uE_5ao0mp{a~_wUcNNr@9>ImnWwMTI2Voi?JP znwBX=v}i|Dr$~bnLW`wnU$jqCR8v}nEG?9x87ir0nbU@}m-=2ekMlgw=MVV)^!$QR zv%T;4`?XxJ>$+~#IMJx>ET`TBM6GmK;IW;t5ggD)3#PrzZQHC617qW&T=-gTp9%=U z0>AD!{l-u^>Xu^+5Rt74G#>yp9dGSx@)tz_E@g$3oTHf`F|M6L^}R0(B8PH>*h@5~zdVVG zigK(*@ZTya`D9<;(M0f>YMw@;d=HDzsZA7f@Zk4I?VR;c-{;AUQ{JL1Ch^n+iWf|y zFIJEzGt=YqBFixOjA##ArI{n;wzfP;@dnC@pbB!cZBegbL~w0^zm@wa~e5CY)ITACQgzBk@fG4p9RlW|Llv352op=>EKS<%t z^Pq=!3c6?X=tc$xc{KndnBwf^U(rc6X}Y5XCL5i=2{wGo3%Jel!Z16V z#FVd>vCBIeJh}bBF{BFX0_To*>L>l)Z4sEnu56GYMr(eV;@FXz5F)|evmZ5Zmfgd0 zrQP1h`_OVtBM%XPnaFsd)ci=ag|;T>?g8xmRY6vfldF}nQ1|B6A4XuDER!kvoK*Z8 z@A{Yf;u)arja_};fcU;pTat>ws$QtX`jy{t8Uus<7R03%Wd1tdv3Bqxx$l zfZw0&spH%=(n*l9>p8LU1N(O|#H3Fb&Ah!C`}GSb$>CWUF4pVzUQCz#>G;I&trjUm zF<1m3@U@N7hS6dbSEvo+Cc*+zSdFUpz4Hg`ZiQx{+0x+7v-ei=!V%BqJ+)C6i8?Q zrgW3k#3bb1MC>)Zx#4*%jKF!*0P>!pc}6eQI6WS;#Map(uB05y-f)hi}$DL&=heTeCCfIrLBtbe=0$gYAf0H?J(NKDqg z34~~c67JfQtQ}jn9M~w+w0zu{zI2=Oe(qbcKbnr?g}H(SRs%`PmL=Ec+H!8T9j{zD z`oJGF0C`XygFXuia4!*_CNuJN@hcjVYm{fN+3Se$6c}~WY`8nv!3^o*fx?zy6S;W5 zOmFLCow@pSkJl-|EdG;=4x=O9_5jl@B?p(W{soa9jTzS#Te4gtSd+oV9rdLQo1PYz z&G{7_O7d!Jc;o$Dgc%sT~x% z3J&Iqozykfr2(}{Z=L-H^e|XpUUZBd7n@H@pf=cCcsN+nH(ZQ{6s}id?xWZD%HIon zzP)t-JcZe!Jaj4Uh)faD(OEbC0)|Ks_q+E<3e`84pBo2JZTPYH*IL0St9(akkv`tu z2ZImBY-u6g+n!4#Qua>X?KL3$uw>Zfe?NUR&wI-!O2J3q(7-dz?t2}6=Iow@L6N`e z$y(J&-eH93kv&IxD&N&fW%+!+Sae~k@|+YOH~UMfO-944bFtIa53m_8+}*dvUsQ)K5Pk(GNxNgeDjGcW zelp*r`Lh)7z8-))L15m}LvlL-N%kFch1!n*ym5jFS2B%9{+r@ncK~un?~yk6RBgEt zXzX)o3mmDfVfJnK&zF0TU}h!ra&qK}GU0Jk0S30KKlHDj!*=*`{VS8^W~M1Odu z6q#Nnc+ZAOAD~=%M%9znpUQT)e&a^O2;10N@-EFIagv}HO2z847}R}xT1#Zj7IDPA z!=}gAQZk?UJIg2UVS5C;Stz-7d7`4CBIM+>epRQV4u5ZM$i|c<>Xynq<*`M5W~S*} zg9p*B@%7o=d*7%uP<~VHP=eGIIDEdRVLDgEI#gH|+#0B1n9KQUdof=sQt))1LKKpM zAYj63F_Q)+wyQAsb)$yLttsXf%<$y0CK2tPYU(5ZZTd%_ouH52=^}gSecht=!e${tuWq6Z!_yavNh>aos~Fy*^TbwIyHMrW&)K&;E}2s z7@Q(Hc5k9pOWZO}d2@!J3xE+JRP^iG@@LX~wty?$esB_&s8r$Dayaz3tU!%kzgPzf+R z=($=I+!fyI`hvH8A8cwoe1If0Ha4PtB~I07&$-8vl(eh@WOZ~SS6Ihi`S5W`SXcp4 z7!1X}Z{4~z{`62a>~e|11D~D{k(RZ^&;(3UGkeX>LMC>=0d8q(Y4Mi6CU~l7BDvEZ zONh3^uzHkZ?Fq%okXhI<)d;|t#}4*w@u`YPp3v=u2iphCFfi&P=$Oim%&|;oou@@_-qK!`=FPu(KzMQJcX=SVJbU zVBo3RGp}>)z4y=7E?KQLM=COG@W*1E3nMpA=fxdIC1>qXoQgg_)J|wxgpaF8Ze4ns zrpM0(b~XLnH1&?cuc!QRlgC?1Ekpb4pTKjHz8LR$6|DpW)^>sVaQOEkUP2!L<{k2r zs*)0{ou6$KsYg*jI1wO@M8(FcK^sNz2D-x6Xcw>^ejo5ehp-!UI~z3jn>WS!Hl)MQ z2h_(@B(J5L4?RMqQHF^Kp;rfgi|Ph(F6WgHQQ~84HULQZPM8Li*#4#&C+Q$+5gKop zsTYU}eBG-ofgHlcQtr z`Mvx=(ezkz}!(2 zt{Od8+n)|Ktb8#guv-6;E!{j87=134@~je5)(smr$W*|y@_X#bx_f8rC*w~IN`uO4 zPUx0VIsEyWM7Tqs>U(7>jZdMcvXxR&D5!YJWzF|mu zB`oZ%o&cN*UNPQ%Z&R+@xinbNE^a58>h#IwuwZ?9aO51izJ>r7+1qkVG&&>QGTSP* zZ0K@YEz9zo$r=jT7c>trstIO*VeyHMOrC-W8^k~>xg%#D@0v^LXqn!TaLy5YpO;$+ zISe8j%Ie>R<~XMs>rxB}-en(Yk)COHSaX2DSoJ1KN?=Xhn$cxek2V?#ssQxpj_^Fs&uKV$NTwpS^sS=Jj^>zKNsgnHb!QJbyv-sbfJ1k5R~6g``|9ea-?LL9Yb~Rcs_9QO+xng z2XN2x9R{=^`N{5z&_N1xO}i+vJT0zK2T^%ny)trXax02+H|XFD7(^bAwSjtE4#xD4TVDL9)4pY2pRljXj`c*9mFr5i&xVVBt41E`cwH z5F!ot?K3-!>rWeW7b1Wh4kkRdU5Fdz6@?N(zeM5DrRXS zn(9q1*#JJA7541jP0YV=ZW<@8g8oFO?k zh}Rp{M0OYZWQOpTHvl73=GFrXweWS2lXQxsMP4tNTR&=NvmB#m`5S2E`I<{LQMBWA?&41aXIw$v6IB)z%ylbclh z?gr}JTM$+!I=)uCd`Lmd^W9zOHxT@9Iquv3%w2yz_HUZ{rvjxRl;{H5Jj<+`vNE- zJjZaLTwhT2tT{t2PGXlvOwzlgrKPPpyn$JFpe}PcC>ID{O_v;M1agA&Ad1Jsg}H|U zzQNlT1Dfstw8FgG`HJ@W&|{<>a@0#Eq3F&-s*YVSl%5p*$~+MBGhg=GdR-rPgBDi^#m zLMY9I{SvF}_oH%O0AV{i%yDL=5;HQE*3)Od@`T`OEd{GQot4HwG5jVT@h-@)o51WH z$YQ`?agPQdm838CL(%)sl#`fqX}Zzojv_&ohX}LH^+|7piS(-86MGW!gq1>?;wi|A zP-Xp%6J!ZSfWF6!)C1(5%bgkX5^CsId|BWIsdrju-gjh4b2IHcFfc%!7f~XR@OzE6V z&yI>98ofn$>~otPSSU)8R!!s*oZeA>h@xTCBmvSv|0$>wntGyp=Btl1n&KCzAxQz` zxKxH=zO9TbV+}jF$nV&KdX)PXiyw|~teHW&8>e0NL?hlttV~2wcCuUj(SCv(+7-YX zF3I;>q65k`p=-To3f>X<9p*l!Ia)TDfdExPuHCSH{r5pqTvU`FU|ZD9I5qy5{97z+ zE%LPE4)YW^zpB0cj6iKrY#_vmIA{i;udH=Q0%fd4SPqP$w?1Ni1XH&z8@<1ypuUSs zmVX|$g3HDcKZsNo3Tbv5Pu)34kphaHJ=y%%UwkLeS{*Che_JEFIiSli|2yol4TG|N zbJt(qsDJl?U!9h+fZc>xB9WL@j>AR9`=;Hg0hWtbULJ1&Rc6X&Wm5Mhs@(YvfGN)D z#O#$XUMH_6AStV=CEML>(FQZM<{}JJ8C{S7S=v~l0IRh*Mc=-lVHJE*185DUHZ(Sc zSlG`WEm!CLvsB~aiEIvoOp9!GK8sAY36T3d$GiPdQHPY4ta7MC2bS=sQ@r^!c3tf- zk@x_;HaRs_Zq7#9ntKWnm!k*n`O$bMCy=y0VaDY+iIWy;)!T_lqr>;e`%SAVjju57 zT!w%8Xpasc`7;2{SsSB2fZUGB&Y>D5X4piX>SxO^$jlOsVraM`&d7NS<%*xGTw*Kd zOmFH;9P|Y$R9Eo_Kdb;5&50jADXS&hjAKOTPwO<^@^OcU zy}tzw*3dYBa@l|nvPRjGO zA3ZHp59Ar)&=#GaEPDAyaOj*R_}zKDKa~09!Z{cma_(uQeL_G}+k5QTF_6_| zP19b213f$W_Tea)_QG#74jciA$!eo;Jub589#oTrKR4P5-#*R&5+X%sd$0}W8h7}; z&wYnKGq(%tHS7b2kj%_XYASlS%ToqyM;OFXB}fvYiJ~c%gZ9^i$Co5_p>Z%UFi=74 z6w?E@>vQR6jW}+!)dnqJbRTF44057#C=3opssjw!XIYzJ&g?h>D1^#@BJeG$xzu$>^#_cM9yFZ$yj|9gx4;1ms?dc~OSve8sMRAW{k%J86fA_7ZydDs z{-?uOttGkJ`^XmxW(vVZ{);-IHvH<+h%VTE-P+T-^!fWYT{Qdo)ybXIUHn|+H1HQZ zzz(Wg=5*b1KK7a@`FE`xaWlH!V&Wfq_!LC(v6g5Qf2MT7dO?#P*;25ceEI)=S{`ru zKcVKgMA3C6TgESwRwi0%lpOil**o(EU%T{0oUEb+koK4#=}!V(as?#O0eGYC=$zTi zX39j}!#SkR#n@iO<|j^^*spms;DRZOF%%cLfK(-i_g`q2Qegpu?ROX(#&yh}`T4`J z+q+)yJ_2v@oY;#y>#hl3U8OmLF6Q_J)p28XPL1!3y1f8VlMpX}T*X2j13uooY_MIE zQYzCnF?MMgaq`#xTu3%bB;+f=_!3U#=M&62vV17z-#&{fI{*04df{kR9;U*~A~kiV zq{MWJkOfP1cHzChh@OS`7*?4AVzas;9OFbE{^nxVo*!cQ&M)E z|M?@;SpiabDY0VZyLH9j-g+9`QzUbmqq)G+rR zSb|O;wL3HXDb-mZa8yeCYincabqoP~0L(0&Kd^|Bon;ZxpFMV4=U;22qkO@+=H*M= zuj!D1fzX(kUbu`YCF^pn=-YUaUoWAzc5My(+A$v(#(_>tP5pY>In3;AmL8ChZKIXy zEe%6N1Q2Pz3ge9rKuZ7=g)QA;5B3}Y-B(prUeC2-ugVy(Qw>bnnLLybwR`g)EhC;V zZv?G8T=Gpszko0HxwUlc;0}>2DatJwLrS`kH%FQjyX@zM=DDQt>#&yIkC>y?`13tO zRyTx@|M?NVEU7{NU*GKMZk>M~;1*l$*#G#;>ntf`^;caPIu}S7vCDUTG?)12_s;1k z3#^{}b!oZJNjj^3t+k{m`PT!y3|;u=Z87<=&7LmFBqfLayx@Gn)_(u-q}HYXexKy! zyuStf+F4h1_;HHkk9k_dzt&*OLeXCji2HKh z|9M|d8zL5*z9#hVFGMWdKTrDK!iB8<|F&?oQHnh|Zk$_LVkW!S1T{$W(c8eQ{jY2F v-}~Ud_ZS}id;k9LX^(L1{)Y6CTj`;swQVd+%xz2zbyJn{i*^yTZkK?um_! zh4nRdcGEwCY6ZRPu~e^UTtd@?O1q zbvN~+iS|hE^>Zx8`+MK7?&!WfFUDYS=wj<*tBTU8yHl<2t)t%$3Xq71_||dYA140e zC91b(d-&ELuht}wOBa5Bxk=-){_(A4V`eQM@84gVayU|d{J?Jb38g=NV6691<>^0u zq9Wl?=pR3@$wQj?#}Cw^lmCC*A|~Rh;prMLq77T_fw{b14c(=0RY&mfNXW^tvr}4G zN)DT&Uamv<#KeJwpME|FQRzxEixMjtE7{2rmsM%s27;P%!ooDZb=9xWa=U!KdHhzH z^G3Z+){^_dhC3UrYQD)?GT)c>!;TGjY_n?lrkHx$UUi8Iw>^jH_%Mg%I&#&ajZrT; zyS)rmRx+3YifcBPEB5BinOIl^U#6206HgC5a~VRx17xtWvcBdUaS6(4zt&>9<9@KC zX=P<~jDTS6LCeKAWj{Kl2JJ^&K zMjp4mbUAM6fD<1YH?pq&bqUK>GEY1e@5){ z9R0Sjl+=td@`_E|#j*|BFg2^=Bd!}XBkl)#k_8gCZpE?OM^?YqE2D7i1RlY` zVL(Aqu|aULv?fGRV|rPD(ChijljK|`eH_)*)pulMrlumy6XN1(O)G}WoVj)D@k7~E z>-(~Fj}s9Y)Or)Ms^;T+Yz#XG5Yyj2MMhTtj#F2u#Lm>$+7$*G>AK#3>-O!_N%dOt9eGr z!ZoLjg&`g1uC4H$-PI0(*|Z#QUtfA==2thKAGJK#+bE$$r>2fq;n5*;Ra;+w`s~@J zeuJ1cMNQZ4u}v=`x~Z8|Gy?;JXJH|KM%lVb4_Y&zPG=MbYKgJO5s+u%P5 z%XC%P>pjejbsJCbQl}KV<@e9$yIcdX1`x-E^Z6L1+$Y3hUWAu~gd%-N7_%+MZ=*CE zMsc?{7G;%`D1sQIbwOuEEV< zfGa(!=0*9gD@}~Sp`j~XG5cS~t@eu|5G;d`ann{U|RdC5YtZ)!&ID+yZ17egiX*8O^6qZa&X z9qPq4Omc}b@gF{Tc+t7N6R@7jblI{(?hsJt6|+0R_()!U`TM5;t1f9iPU9X{m{-Sf zKR%VYoq8&B4#UoiPCvd|K3g1o5W#1zU;p|nOzKL>$eurAjLX~i+y*Q21tZ1lkPgPt z?t3iAD(&xX3GFV`oFOCQvYmOoOEtF-J?{6C~l<$}!x1)92 z)P~K~4B60Oz29I9=Dn8PeM<_N_41tsLAU+Ym;whkNy+0#3OdWJj~^hq&oAt}J`t*F zHoEA3P-0ByRa4dao_lMY?x0beT)mdNL#U>XT(k8V?#aU4l@?X%RoJByuqVe>TG+AE zi81R+Z;V%#YKW|7(+l20*agXoG{LhspJXqsd`YhHeS1iy(Bi!4{&D~gHnXxy4{o7@NT@_H1W+7KF|oD8zjCh+gaj~`QXCCb_hFV3PXcu^1^!xn9mcmoHzY)27D8RwbpU+hPh?-FjB#Y=)df+6W)2FdZQ00nE4@QaHpZu7|^x7Us zh;9dRLT>NUr3sjFEZ*Oq^~=oU82=nA=BZ7Aj7kqx|54mmZKq((K#Rf?haxsMwjQ>= zIwssIAp7N_rh21YUpse4k!Gpg$=s?dq`mf$L!ow}YLL5@GwEG-b~^9ZtLX z0c3U4R14ZsuQm5q!pwHm9VbJMY6<=LES6!Km6e6$#hE-;Bt~93A$VxBPcKaUwyiCX zFFAVxB)K|Y@`7tF7GFM!LnX1Q!X>}|=+Q*6-&tznd+$*E zUhHc#-Kll!u!&G_A|hIfZD$J?if8ZKxnnWUXEkx>O5wLdBKw=;WTrcH>~3FlFg(3# zHL+q|g9B@jflkrVw!>zbSt~R2O>p#v?RikRcFwq6KZp4A=|lX^OJ~T*yFP!eggQIq z`G`$9`^C3!T4u%5486Rm-Ob2I>`KSRbjo*Eq^^|AaW%KJsEc|5qT+_!WVbMAcui;p zvi&FLjfLG=%>ye_7AB_A)ecenS#h|qgp3SfszSQa4prxE1odFb$a`QzE%wwTGbx5? zI5B3^n2odHea{0)QMEUr%t%&Cbce!HO0K@H}oj1z23=-q+zo*%-1N4T~z=xP*ja zPvUFTgu_m=Y;He3(G`(im8pl_+y34lx(V{d9V;s?(YeQ%gc= zA1bz`hX)G~+4k=2eAp|}-`@{c1BgAkKB%i0j>$77_tajhJWQ0kDR8wdz-F%FW6_5X z=dP6fc=$6Mrc9jx!zH27_AFX$QG#JfNtf#b=sG-b4i~QVs1AAFj(%P4x?{E0t++bo z%l8se>%*yAU)UZM8rm~vuhaPEyqRfBdwac6UME^p zv|vwYFB*w!?&1#G^bQdGAM z=^dvLlyo2|85$ldg)M)u36O3RD!Wqlm8bNU_TvKrK^iD?27#`+K(2e4ztGDg=|qhWhm*8OL_f~sGQ6g!N0MccQk=Vy70 zf{kO&i+N_c?%0vBYqUa1Z~(;Q2#@BlRE6g_5h&qbGSsl$X6{yFj4rF+-C8Z$9P?Eb z8t#d?5hNGL2gTHznBHL~IZ;T&%_OT;?uzSZq;-2NY{&RSw$qHEC`Hi!1pIRbNkjHy3dL1K7jFMn9OY5wb|Z)c&RX}srLqU z-$h%wxwue$<*v@Y4M22m1L#xz5o`!(RILUn$$*X;VR$!nbaYb1W2VN&$A=W$jmMH1 zUw0c#0;t9!@8rbju`d#IlYA9h1e&gEq;d)OMx*S9W6ej(O98-`iN8-6ekjew#f40M zQc8-|?8`jk-YW%r+w1cu`57O_aOu73?Hy|pz($H088QD{769v;02Q`ij=FRz8XNCi zso1qb-biwocq_FDe5cZNV@N_tsqc7Hi6pSI!kLs*B-yLgY=;X*$J}t{hZq0^Zq`4L zrHkib{6q)7q*UQ1479~96*r&ci_ogHsHR1RR+0VUFv-3^!P=ly*p>Lw(!q4{9+-{6 zfzZIfr!}KmFe_$d84$XMU=OaeXzss7YQ&i{XWqPj-$b{+gok93IJwSKHv!U$o+2f! zgKUWeb}L|^uQP7QirJM!9c&!zYx*`&Qc?<$QOVqbp?;DGnCe#4v=y96kd8pYa1&~6 z-;}N&wZK~d;VZor`yo&*!(#S-Bp$(aJaB)qx4rf$H#eAoinlO-a%w7HX%t;pSXgH* z5kO1ujWW9T>)|BFfEZF1g{2|;ks@KV;iU%1$xV4}qZUhW{o?ts;NaRg$slB+ga|8G zSXqB;Z>K4+Amk4ocmnwJCn(x=%uc|6>`LfPpRR`L&WT@>$E+bQPYSH(Q&Ljntbp*= zyt%uStE(_TK?mQ4fo<+5>;wb^rAyT(Gj?okZIOQsyH1entGW@Qvj zQM#?AYU1P?Zl2@!>O=A<-N9x9QezN;0I|<)@$GtnPp|t%$;fiW!CnBcub{xd8#iv? zbO>)yBM><)r~Qx?Py}QrAfR1;`^PlE5`Z<6?D#a-f?vHl(VKOpd{YO>g|KFA8R{hz zLJk~2C!eO?=_zgK$mtMiL3l2tgu>~Bi0_yJ_N}cg7sx0R6BC(kdroNdGJ>k1ri{jV z(dq!4%Mb_%SA!)60l)(G`P<0IR^S}SthWA`?ot=FRmf0lnrI4g{2uG&zk@v_;jCS@ zJ(J2tYdqWWp$zm0PWOG6^?6JKRDmrB!aBewi12Zoknrd?*u+ynaHbHQ0NL)qYZ)NA zO)>e)ka$-C+;iE@^^J?z>*!p7holP-S~=F0Agv3)++?IY3RcJ8a=boIpn!17So=!lKV5J2A!plu^nRI7q~ECJQnAky-vrpvOpFYU9* z$;lsc{Xxn(`kY2zFM~*;1f&jt$ygI(%v@$K0Qt^Xm%>9IK_m%VgZZKI$TI3??%IPq z^ct3_pl=@vAEj>nYffNye&4@8L2z(A5H@23&uYY+q$n76O2y=ZHbm*MTuUNbXsTIk zlWYm|Nr}Mud-q;j0)IOTa*-tl3lqHNiop%OpijH3EGl9HUjfBa~LohZ2VTRDbdW_im*ylS-L5)-LBmZ2(^*NwB6EwpI;Dl-GvRCGBI4;bZ&Q+W?pmNizUMnC7QM;2_Tf`)+= zZjRUEE%I^^5izq+I(a+^sPxk`$_8L-eFOnSAE8R_fJ;kU%dkY z_+WSF3avEKn=Se!@;a`<)Az#;$RwtwHbLl1!Jw7zgNF~t@?}-I+)H;t!@{VbN{1oy z1+pL2;iLE*(GFwBYep>!=M~k}O_A++yTEMdHB^mph%hcfYW{DbPk|qgM>{WY-u9!^ z1BHSLD0O|(gHNv^9v2FSPav)f%iWQdo&G5TZn=m)Jd! zlM@5|r)V-%6_Jx%hOWXm1FuN^Ro)Fh{4bJj@x;Xi(DOilBBr771BuPn8Xo)S>rl8h zMQ~VHP-v)Ux6|%ka`9uFS=coYP&>tEVKRQc)SI6k#{1r%gS8YzOOa)M{PYP28$m`w zGCtlDS!z4m!wCdflTU0nc;&o)FcIe`7w@x^VxmoTldDtiA&&85zQn(Gg!5Kz)Jjt;zRE>Wh%M&h4#J+|6Fd zU8wYKwO*z}K9GG>xl+l2zp}EzZ#ChKj_72_%gYN64&E$2Yf1(T>h9eWWWII1($lAb zZ8g0(PU13zzbyCLRbcGHm+61Mti#7cp?K1Dsc~@^VMce$KR;AfmVEp;4jnNXqD=uc z=)wK_sOv^XDFLd;7m!!f(-TS!nV9*EF?rYnE<=&L)AwptAjRQazI^!{6H{_mZV5LP z*JF6Bb8h#3_3&4Hb~wYQN?&<W3ax8ojI_ zqB6+<@u63m|4P(!8Nm^N5bL%iwE;{&X9cFo;nQnmjbWO`d`>9gT*p1n`AIwc*K^1U zl0vyzZL$JHfr$xge0)4xC^juQSsOG+C}Lh9JkaHFWKIi%;EueeRa%WWO=D@5XlfTY zHqAD)_j4A`jm(R@9{OTwTix7z7g2;+$)tQSxrSZW=1NgTMMY{r^6{zpPXQ|iz}g>+ z4H_?iBB9B5WB1*kLHU;;dO})FnE6=lN`On@%fU{{-u$n99JIwxE_Zn*=I?BUU;82* z!Rr^M>DE7=1u~nRsHQb6Jiwq8%*cKy?nyG6??n@}klO$C21$4df~wg}VX7J(RxBDg z6CphoEwjq3{lS<8EIt(9$hwr^;4=lPARwcFazRabfv(seSSxLybB}~5Sy^4hc?a#0 zf+0I91AoU#tVhmc;5TvOUz3om_ts0h?-(6NtBLwAyO!Gf;1ijt zR$XRg#rJ^X1uDOJ@#4y43(n;$R}u;f-!(Qi@z#L>d%vZNb~TXq#ucFvfo?*nDde28SvUiRZO3B$c_ zAiSAo71z8XWf@$6p3eq81;_O=^tG(FZ%_YwhO^u8E5foa!~ zg@#Y%<+L79b0G;1nu&{xd*n3QcFQYrg2aR<_!@3odb}omm5BUjm$tXF0joltZ$8ql zYF<`{sK+$+gW68}P$ja=hOUA50+aF>pb3=Acy-NBk%iUOH~;mao*dH_jNlat!J?Zd z0*MOVsH%()M4zUrhiB6pSo&%wY9J?@en~3yP56x4E;NI(OfkIsOJ1UHqN3Wy>U?pK z!ViRg;uTBAHy{s>za~9EA;Rncu zmw7RyooQvxi95-2QQV8OT=r(kMyZiG(oFdA{R?ROnPMGX8jH{a{Q0sW)X#gD|M^rk zp-zs+1WMzY*8cOJjVaA!4y#6pGAG2;JgK-&m$eqI z$A=H=Tbx%^^V8J*>Gv<4KM(o-lxV`w;*BJBKduYES98Xhr=%!Ap03EQZfa^OH+#0p z`Kt+mJ%xg~-9c`Pbou^PmzQs!CH}p_lKj1wOp1cm+s_d}{A~PJp{X=0_<3=NqyL-% z(93>a2e`%7zs|w`fe)lLZhOXyX5Ec|YQ)mb)@G0ckt+VT&>+GL()EJY81&LOEXQt% zKzoN!IFUMLdA#B0gZ*+R1(tG5(|s>*))DFdfG|3N#U4hwVP+-syvQ49 z$n7Z8_ zP~6w-&UdA;+nnh2NPHH6qomcO_FeFJ8t-DmNq>}Y-K^P-8^i9eVONZ`3ocFoHPr`& z1d_cMlspgUhX(@88(0L5Wzv(%Olz_^1ZYWke~v*{svP|DLkMBKtf77>E^w&h~3{= z#QU~UaUhI{qmU`|fdM1jtu<1h%k%)GPcl{|k2)Rg&mV4>3>F4LDatMKSOcvt5Cj1b z<+FL`*=f)Q9nt@~X}+AQMt#1D*8?Z&9DAMgQGBQ%i)VkrT z8wHYwQux|-I4B*0_6xOU8zRT|BFxL1=mUhRLA8{A`0$}%DVHHp1h47JzmDEX%aMe7 zbnbXJS%1r@pYrg=PQTzA%iEZCSLzG&^wT3171JUuVeEJxpi)71;R1cx#*&dQLi+|C z#{F1Ta=k%j&u!rZ>hkk-T{yEeM=7#qbHHlWp0+j)C%sKo@98_*_PqhCQr=c~-1a66 zMhCRFL9kzxf>+|+JC3J_!o!2(5(ElY;Sf~##j_k*C|=cBYr8Vgwbk7B)=J^;dodzq z(x!oOiI9x{bpjvYqzWsWa+tb}N9p>?`fVh(8DpSu#TtfG{-tiA`sVF2if5QX_ibM5 z)$D-QU@OpD2k21``RBPGxPh!u15$}2G%_84h!u|e3#tfjAhY|2hzbB-4a>&HX4>0( znK2_Fq5fYJCbL#vK`i%Zw)^#aDLd=$u430?xrId+FJ5KiEPmW{`k|cMkeNZ0Eg8xk zbh5_w_9I$YZSuM>w<8h>@krCWylT3Ve)ynjt8z8+>l^_kG{ z6@=QjadLA0M`W;|UU+)`#8}`Vng5S^bm+RgdWbgKZG|x{;DnAQqzC{==7sQw z&+FHZ*XR0aoXiX(x?}{~{$o|U+e7u)_1d!q%<5P?o~0a`VvVTjoQt1#YlkgbshoCK zQqr=duA$*@rb$27&6_uorm=d5GgLvzyLT^$j`a6P1~FK|BOq^LDk>>RCyw8q;@!jc zU7OGCDja*oLcwc7fe<6GKIzPSO*X?4k3he`>p$jvrJ4gjWhQmNh6T-r%?b(7{+#_y z_v${M`J_;@aSO(#G+(BDIP~}egLHT;(8?yLKGL9F8uw;Qf|T?P@q_>x%Gg8Neb1gf zQ!28$BrCdiRaSWIjQifKCT7l-pBlxieuXJ4{I%t)!h;oFA}x`SZ@vQ!%Lw0MML*L5^7oX&T5a=LbsjI~lHivtW5yk>)qh=&7o=X;pXIR>&PPM$P;^5h#B1E!!6 zZPH~N>~C+va4Y;z+2#M>S|miAGOqF&1=6<^fp_#DGVrFh^|q~?CCxl^y-P?jh*5TG z3maqa%nA^4&Gmn1)Z4SxWTG2=8os1^U8up6K_};x}!2^N%ivfmR zMVOB%DJLvsk{0I4T|sn1LAzT{P3<1Q`iS;X=@=1L6a}Zw5m4U|GY_-YmJgz0Y2Vyg zyAWLY@qZC;SNH0ds|D*>T_ikh>3d(}5{n-bOId3<{$ywVPrBw`=W^fizf=FLZZ#|O z&FR4EvxWw_(++LUH}kv@vl8siWpJr*YF8gYR)?C<3nKzmk9-yxJK!j-ft?v8La!R0 ztkT~}K(61Aw!v_-V|sf!0f+mf%N*gNwKg6lY34v+o6$Jz(?hwJb$Eti1l`O^0+b1k zAIAY!I5jz0Gz4lV3VUl&n&pTFQbNJ9W73!T6hy?%d8M6=MT7Q80Ytlp!r_b9*aQ~} zKXqG+?#$)5fZ1|xuqX(cD=P&Rdwz&xM-7PH{5IueZ9F0e{)(p-TZv?!$R6}0plzu< zhh`2)dzL2G$9(YN{EK;$yY*+H!+!)fWTjX)bj-gobCVX%3uRTl2`U#x7ux?i(1QZK z)cu^GU<4qp!*9aFo1v~XB3%L~hmD}=q6$IcfV!c-ySWTf-9q6wp?*j7&`c2xAs*6^ z0G4&>@?~#OxRE{w7|f>Z+@q{?(iFOr<8A-3$J7PrwnEa$(cvs<%yZvDH|cz}DcE-6 zaSOAvxd8Dpv9VoZV5pzZ3J<4TUl@{tC^>fWq~NG^WOMf+ARGgFA)b(g5U~@=qQipEATyj65;Oxh)9NGMF7_l3QWVtTpF9B6 ziXXoOdNwazAepQVw{L9jBvi=Vmq%T{sdT89pu|7U`{KgRwg{@=rh-I!`p{YAaz@6+ zVB{2?9tR)i+yoW9n*oS((AE0d+jdGc{(lEX(zunBhMt(8H(za@QoNkObo3n41)mtA z1A#F7!WT>Duod%Srm=}OX7laWann#*1@>i=-0|~ZPxgv}y z@DSB`p{r24mOyRrKY0$ z?wy+)ztcvnf!U@sH**bowS5A`rNdQdS?hOiQ7}r&*dn!~dvIXkQRTJU7Ma{a^78Ug zQ(*}?=@W|b^0?Hl=y+X%vs|^mENLXiwl1e!8Ka9hs1hwW2%7OULT%Tr>#jHU-JASaTPznW z5NcQythYJJtmOz&8t5jMn3#OFx1BWk8d6J3OJ(FvoggGs=tz}Ig9i04;nDl{;DEVo z6ODnT)4m(5^=AC!N>o8Yp8fO&+?=)_Pbkjc!DS`kjYyX}qYA-$ZAeJTYUag@7Y-wm z#jO7v@0$SiOC=7LUB1r7Gfa z$vSrK^A}T2Lk?3B{HT66?DVbw^TKmx#abL@H**L)nA+{lGrfyuYNZv3I6pMt*j>v; zSBdFGdG4%dt6|q$_P<>6RU)h5z+WuE4gFhY>1`7*gfIgT2K|Gv#6P8izlbg4vRg_~ z1>xkprIi~+BZsx}l-~C^Q}X#0`s^?7VGdpp3bIl!46io()%BCiw);hy@sBq3=;XTT zqWCSodHfyf{G|a=-tMJM!77>?X)Ib4IxAWhy=Yy**m%0OZjXIRRCJqYwd4H=CjhfK zbLm%#sH4F&Htl}hb)deXe(QVuwZZV$%QnMhciFM-rG=y7yk|^N=lUpHk!yMq^L?gI zIjks=CUS_1yo&E6Cf{A$ts$>YcFUG1&2e=KN+;qBQ^x?cMSM)tJ?T^r9z0NUnvS2q z+LzI(S%Y`s?_u_ru0!OE3qL+yjDdT*o-g}&KD)WD*q5U>O=Hz~CYVLaCv<5eH?pXF=9zWh&OmtQP7_}N$2{5 z7+tz@0_40wJB+DPxCZKU83w$wEN;NpXhc+K;g!$bH93VMAEm*`T>z`24QaXky+LQ6%Kk7Z*m6u zF*5%cwKzRZU5}-x;>?0fJ48DoHQOn0Y}Bc~1!6}@4+yQk+NcteXd6%=|b{m6go z^blA4Pqjnoi0KwNHFm3UPMpxWc}?p@sIqx#R&PyX%Q>OpJprJwdIzvihZt=FB(2HN zp6wCZTz#)IMpYBw%)m!Yxr27FG35ZDZoF*x{8 zuff;jy0iWOT3X=o1IDNe!Z8Z`7RV_9=$D`F0-0X-(+1Uy9T5c5LF-l2!1!i;#Nimc9yiQE2YHc=-}>NhqK-2%w?K z3@bf1JU>Tv4SWKGG}mr{`5gMLkM;Gl!N92)4Suw~BJ0H2y(^b5e_I-@0hEV4Dmb>` z1Px5a+c$42;RZMq{1#B$E+LZvogPs)A>*DjND)urga*=*TC@kskerlcp=S?J5$wPI z#l_dg#>Sv&%^1Nez-|GH6`zpcDdrVPjs{<2rsJwUq#kb{pIESrnZ3`0KI=vaz)8f< z2rWir3PvhK!E|&OIboFAJrmjw-14D@hKz#)znCY!W!Z?Z(~on|wu6t0hZhgy5n4<% z5`*93tbkSA>;~_pOD`eLdUu3vzTbz|2{eApmfaXHT>|bp>r{b9M0@YS17ZlUj8@*V z5nu_4$jveNZ6W<&6FqU_gvw*y^XH#JT;!Ydk2Ra<>BZ0P(LxYV@tS8#M8@^}V_&iMb1&mtTxUhq?|zPzrxwWcEB<_B zTe|#}y7T(OEFJO0(%blySG|Kq6|J>s_$@rLbQ@NHX>x-KfkHU>_b;E2#0_!>L$9WL znW{On#Xgq|^z^n?vCcvnaH48@dK!ltPXVSHrucK8+VqcKRoyO)f5o$eg~6F0GJ{0C zA{3le2|PJ_Ym!=2*Yjs?%GeRIR?IxjgRBRu)!5ozb1t~D-^u(Z?D=bLGZAT?X?8H4RV~e5t2(vY=iCmXg0i|Gxk;fnk?3^5n|_Z3o|CFb9&hVmzq5 z@onoa?}&E;&dmAiDp|5+=`PLa6H2C8T2+U@kwO_cszLQq?!&gl6;-)K(ROv}Vvj)r3(-yyoO2(2J8HF2UFXdle zynXTQ``~`JnxSj42^+e$lSXNdJ!Fd#cFMf$rVc)J&SbiLbEwynv^8z?=2Jhuc3~1N z;)fO7T68EpM`v$G8ZnmG{PY`ndp9;Ig_i|8vS0%>Lly@@sRLWf#uxQ~sng}R^0f{J z40Dz_{t|RY?=}$}Ce*JAuPE8b&whE$S=Fyhqbz-(3g9PboF0dIfCj(g(0+!FIPZ4% zPh-jNU@;*Tt82La5JbP{Pj^Hc*!WLgl$CZ$ks{4faTXO4D8LpWXgbh`UUnbBNS4HD zZgpbjsk2%WlQX8>{xbPwpU%JdU3yOR{!JU9_2rSq&}!^+$P>sPQGK4J#Hy5~ljl&V zZ7VX|tgGO=Y8g>y_k+JDG(ee60ge|bn<_nVEKO+e7uJ-hplg$!=6Q*5lQ_(qAi-EC zRa+HiXo-UJ;+M;MPq7sNcfEn?yUx88L!M2J4^-Scr~4thaq+t*S?H^k~DGCZRK*!`(-IDEVGc)_pqL zcR{(!n%eU@n^(coW=ca@J`3lCtarFlNMg_)9{RkTY4ckeRdED3iUC z6Dgj$%X%W^q{>_`V}3xYmaR&s;g8odK6PSM6Nnye&`-^(H28{FI`vdLjVO6UcaJ4| zV?bY8evq@MQM1-K238jrQBXj5E3T!m65#w(cKRdz>_-QLds6a}nc9%1^^j(K$}aCT zjX5WEO00zF5_gkw8fDbg@E4i+q<0e@-y2Pql;s5Zx!`zf5dGWtf^JvjxDs+_XP508 z_S`6rQa3;Ah8u?benUWMF~8hz^&=|}9?FgJx_5`xsH(JTlAfh*`d<4bGSr6hebamV zMUlv*4@)E>*x=Vp_he%3{VmPtIcd-sLq=fO;P*=NLuVOcnLUXpB&NJmvv6Nh4@s7# z7heVC*Q49XsGdOz8~2Ae^at-4Z*igUydfj*Z+!XNz_YwLrXleC$1Pp#y4pg2wc}@0Wmjq$*v+HGqG!l40bOrptqG3L5qH;&@5*=L zQ7X{R(CIdmOe7{J>peeD%ua2m#6^y-OC=V4n#Gq?AWx12AAr|i zve)X=UtlkHp*zbN1< z?r(ta|JLuESOHY-{eTTeyXv;`VUg|P5^T`J;y3!qg0vQmsXL$|+_ji-j@5)4U zAB`Qzyp0)63H<23*%W62|D$%nb6TcVD!p$iKhAt4+gS^kU8Fqm?tP@B%H!}U^a`nS zSq^`9sZvPAS?V>m4ZrV%mS-)V_-%~ezEt3@8ZU=9MG(4ioP^<*Vn~C^A9cz$mr>g4 z7T1P#+NxbsKGQRA3+69{GG)-8@Z^{yE$hkJxEu9nX?9d< z7ytat1~f*$S1ikWPcKw3$CQK))fEYM#1|&KOZO_se^4b~1$`X&8<+ z&B>F5f@kJ!7NIXkOqrs(Kt*}d@DtTZ1wYMkE+N@+v#hGzEx*)LDi-WIJPV?xv`>yb z2{Yit|3onR=xAt=dhwSL=u-r}`7kL~$NfLp@D52yP5W2-#NxG5t|02bxf%74_j8dI z%|=;v7A$psH8^!8L%#AwRNvlpQ{x=-F=Kt`Co)jN9J{!TK2tb(7;E&R4M_9!D60lR zl;dgk+q-p8xREw0n3iS&4jYF2DWCoZWa1Tb^W1Dt zF#p5jyc;n&{i>&KKBQ}lWd!;5x8GI~Xr~CmIi2c$)#_}tdDp%Vu@BAQ$1C+5AAWe` zNXW0odhkf7EEu1B8BDw&oLWqDDB|wXG104#1zMl}_S-VXWXs#P!nabVg49C^u4nSR zbCM`)aAi}1&W?Ulyi}%PVV@5tK6dbdnQ&^)&F<0Aoq?O@9N#&2q|{bvofMgD3(XL> z);bOvE|A;pUkH4iJT#X@E5}o54f>zzp=^j2+ao!xkLA3y+;Cq!YN6`$xwEi&aB$hW zM1Q$i+iPy}ejw3iLH=M0mavzQGpA9mgz?N;N zjsr%qBd~A(+JxxLLplHje(vLw((H2BBzjQ%Iy`q@YrfvbeQ^E(N=VP~6+pov>F?11 z1mVu_-*VO|N^j5vq%Ooms4k=~$e+odQQST_g}FrqmiRd~+55Sko7VAmdPYBswDYnx@YVwwKnMB|g6Y(b~@f^B&LH5RTSFv6&Wg;<5Tpib4`G9=5Z~G|0?ZfALPPvJUVRv0NJB||>sS9C=v5YrABYQu74Shr{;-d`t!NQq&tVM&~;wzpNcy1h3Qnt*zSN$#kBF4D-L@)iw&>tij0Q!^iOY}2YWs#3cB22 z9|O0;KdHdeQRvA;EAQa@+ObwwA_{4Q#Bgi37@DD$%*qANIDbatUKg)y>Oxf7_RJ@- zb(MLZ5WGA+Q&IgVCc4BaTl4R#A3eH@QP}mBv>tA@S{5P-t=n>_KWl2P>G5Sv3~}|M zCMG9AwHuw)_m24=q4o||-@eb$9$T_&I3II4oelRJo~v8sknRG5z{(M)qwh0{?FrP@ zi5A7vJXMk!H-?QtH!j>2F|M!V}?L;b5M-SQi(OUuIT zwD&@A7lNYs76s&O3+J);Cye7MY+J!-;4fm6kn^OskG|~^Lz{A`Q3tKz1)OT2bcv~V zuCuVP(1VT#QV~$?Qqigo0cyfDh16foc$Dk|TjG;Ftlypw(V_3(}j`#!>wV~&$fuwMGtlTUNDe; zGW;eAI3Dtk7DlJzSHT%a>)8mQihW0BDn{0d$CZ}`-3I1uPA=N5FgB|9=I>RHpN~cF zd=@UKew1Pz(+-i#l1i{mjlH$0{`>slfWFo93rRb$eROAV(b1$%7ck}F$|bX!BKPXO z*LSEbt?)7~zmQ(6_H?g~%`I%I zVdhGTvvSwCaB__`x`H`JiYyDsS+3hZmjBk&{w-(KZh%?G!Ejuvw0KJ%WR)Yz&kOax z^9%~D=wEn7KVbJ}z^ZRMHs?tmwxZDUK>4}$w@27hRXF4~??1u#_ZC8CnK*an?zPz7 zOHa|M8lrQA&!4SOy6x;IkkzqR?5`_(NQewpC{V-|LT zMKkv4x8%1_k^STMru-wMV)}T`9GxS`#InF$P3dlkt$k^5l70~wR`uEU{DsjMm?RhP z_t%>x)So}E3CTMX()-}i11W8BHgQskMLzem!W6<9xyh8Xha1__wc$Vk!}_e4ey!YUWXB; z%0CWUbvA``hb-?FV;zk#s(l24vHADSC@c^65|u=QJlYeZ^p=2%T)^uX**t%-@JA~A zm`PT(wMG+H4VFN=uemeB_IO5?Hli-g3ParZ&sPRK$?B4->ldRiQ2{g=N%7h3ZfXrg z*(%@3NN;W=ebz6i-}$nDv7ZQA&8f&(a%zox5;QW4(&NN8r}=tF_HSnU*{R4}XjA_* zr~gy?6W{?+;$Uu@=wuGrLp3@4%3uw-&F(_YFZ3y1SVnoer)-OY&Z_g8ZZ^*MDe$*r ztu<2Juus#+-Q1eJs8h=nhphSW{U7!6kwsIUl*_pJ^^49Uxq#&2qYaicqE{OSlsrzv zJI7(+!$4+UAqr!vla_G0MtQa3B09HzTu6!R+$G-E=2OwV(^Hk#`c+;#0aQy@E#LzE z`d7#0t_@FpY!fOTdw{f3KggQUEzr2L8^lF@vBG*@>su>=r=yXo9KPD{26sDhe$J5R zF~f_f@PjI}%N=V|bz5}R!iU0?b{>OZRm+)o7vwGVXbAkrXVD~Qb*#za!Z+R+X~Y?= zD?NJhc{zQo|2A*xORp=G&1`4`ZJdIQxM>qO`jOj3iXM0lN%KQ-@xYEur{4b%A(HdPj{LY7i3X2Q5*HBgs3 znk?ilukTB1$i(=Cp6wKZD}E}xe?3_A8`d;*2R=$RuT;xz91~lbLG5wY)qumGmHIUO z>3{_XhXTk5qnh`#7r(D3oiV5~`YLJHyc=b zbcG_ky1A?lHqDzeG?ILW=|c`B?KcLYm~! z`-VJ45-9?YRBB@DFUT(h8m+V#XVGEri^xvNP}Qb!PG7v`X)}F6#sChgzJOmfzyTJt ze?*?jxQe*RygoxTZ(7R+1-~+`Jg!jDwxr1L4kwE9JSS4WTPgWZMwAiq9$*B{?qqaS zed+V9)3KGQ7Fn^P5Tdzx!FBV5P!~^}byAY)x6gq_iZ`0KTnUQeB?`LpwO3ZprM?XA zk@P!B&vqp;w~ynYZVgjP(&X?4uaj{7_a6?YKZ#++^&}bNfB%JpxxTuJ#5-Lx zi?T+fSs7`=8XdE$nppPPgPZbVq-3mW*Onfd(b-ZhZ>fH(5!7mN|ByfTJ?NoCRbx!` zlGOQY{q}xgGO_tpG8%?{syAlt>!OnrXyqq0JlA<7UlZDr#&wQWOZpmb4=bn5(2@=q zoyS0MZu>@QC=OALFpk!0rX4wgRDgdtdm(Q&7|k;4j13jwRp^03Z;l0;pN+MFgQDN= zdg`H{RYV@v6Ogd?m-ZqSwOr4?OF0*DGzO(Eb}e*B`@WI}hCdCRg?-eu5#N4D{(-n@ zgg+?>*;l3D6O25(t@FkzCj3oVu`iBBZGL`~Ud(73|5HjTweJtkkov2p-qufZ6pk*& zcgxqZR?^d%jXsNI+LlQ2iCm$MR$4ovNxOO4x<7yRbWFVYJZ(WZA^gN6X|dh0YM&GH zo7sR$g7SYBBIv^9=3f0r427!}74*&1MKy4Fz;Q~lD??T9UOVX2JXbGxx1{yVGzyLC ziZ-G&a&YWw*){DNNy_Whsi_XBwQBKiyCulWZ=Zd%%Hw2|c+e3m zhJQNg8jnNf4qd!r)K%W$Qq(fO?tKdL<(a~}oFv=aO46?%zjg1rDP_8+rSvj;(PW2- ziHqF2BvyYmBLqcI^Q~o1bEP(=K|smzn8z~D+Mwd2TL;@F*i~nLYqE4t^+SO}E^Ewj zQC~4MeycJ(GOvcOp4|q#yek*qO`V)#-j!7>@$wBNru5vT2%fyoqsup<()dk&JIvMd z@jk;3x#7B~I*!*tnjP6e0{!t+d(x(+Y_YK_em8{vbl-vicSlB&bTLDqu|!#5jy=Pu z%i1&U_F=N-lv$?KjKtD>Q<7T|$B$>9CLz0Ll5?{+OFX0JmCl3M8#I`m#~`bQwO&g7 ze{7v)Sd{Df_CW~|1QbD}Mg&1h0i{z>5T(0AN*bga0U1IW5D7`??(XjHlJ1THhUUEo zU2D7d|NX$1bsS3`p68A$&g*w>F@I(!^|j^sK{=T#9@#ThINN!;RKh5*q2|I=7Wc^s zH_Sp?o-OoioG_|20miX-=yPn1v&@gvGi?sTjiT@;l~av2_-qm*>ity4_nz*g z_C7#($jLrU=Ujg4N|&d7p;^g?^(h>W>-hKm2DA-GB*6c!kObrO?1BPat0(>n%el@7 z?UBP)d{SIb1t=R^*2ChX%Whjz+(9inDUAU^WpTcwArW)Ep<7!B<0$l;1Q<^ZH%_Xt zg_NB1#1d9oWPvhMqZc~4?5~?6LeyXq z65`^GC)EKrQtIAQjxv{K7k(i%&U%G&`lG;N$cpT!EjK%$e=M64%*HAV^u|93QFXNT z*gxx6S=*ReKD?Qy3{EVo$QyiY-RGy_xihjm>1T=t9ndHeWYGnPm;bl!kgyKUX11|M zl?61=Cen{-ABK(Bz<6yqTQco{chcT8Ir^_X$51#|R}2Wkl>)ooI_uup4WgtS%=QB@ z5qeI3vxj0D)FTW&EU0Y5XJdos?)45K3(XPPRB8ln@)Pe13~AlwXY72bn`6O-67;$WE`#BqR3RQZk9& z_~BX|#(a#lTU)`eF$kMu2ph5a6*R7Rf}bvgPj)<3Cb7pSSR&Y#(1>Z5t9Da6KzS*PLt#j~UGk4u&uk4E(kC%h2aEqCi5didVN+Ed<+ zeH8go^)9EZWPYt7F$R=jorLy94uA5!5cug?K8a#z*DJ5l0HQ>a<(Ri$!8JAJM9)iY zu^hced-4TK3~(L*Ta^h+`tH~~Yn1#uM!N0D;ZO19&roT)3M@(^lS3hgr@6kgHc1*`m|&G zHDZ=7KQaVFwtacG?n>6hk(ypmK~wxR%Sd^L-dImb7o16K z2e@Py*`~A)SsRxWM!9Xn6H;Qdf+%0`p4BaZ1d@|mbG&NXOjpE}t1PLgyIyXddeu}p zt^fU}##E^i4>==~@B*nJPCR);s#F)xqy9EzOW5&nWA(pglEATVEm3sKwt7^hmdE(x z(RRSgjj!|R^&VMgTK4n>&HK9Vr_DC^@LXc`8mCZ_Fja_BA{?cMLVpMdP?Owo6mo`? zyd>PMqGTsG>2`a$NjKRrH|c+DKcUH~WZ8SL5880a2S1ucc%~9ctH~OKU=;q^fn+2i z`t7Cjj)s7h+}iu#p^)brs~6|<5k+g0ozi?AxQ8O#C4y!7%#!?rFm0Y32j}NVig(e@ga$e8ej0zjy)A2^I zonEw}GFN62-HRU1=N#I1dG)fxcZ8G>=ttV7B$X-+SX6P%Elpt{G4c3<9o}b-G5@Va zyI>sISzBVP!1mWJCCp>=MkT_w>aizQb*g!);e|eN1G_i~#O=R&Omlf2o3m%`ehBJt znLA~bgeeX=ZMfKEV!7PmtTN{qah4T_OBdN3rK8~@|Ge@$Xzm*uCSd6D|8zAj73HKm zPQMDFl`Mu-6sI(LS&Hi8ffG}7x-s>TZ#5JT%)dV7d8IZ`9R56<9pX*VqPysgy68Z$ z6lZi+o!)wOTrB0&V?mU7hXh+re7fbpW2P9Pk)ED2=YOub(-#9fdV=5~^R}u^0-a=o zL(X83^H`XjlzCy6L9DJOFAClNk$J?B{iy%-d$ucbVg}E{L}89f$Op!I;ae28O+VKT za_sWyXju%_Jk1gU!x%K;zcw10k7I>rPBCHb2XS0YQ-+^$mOff<0z~9T5NQO^InYvc*c!f5lC2uFl);Z1zwUdzDLe}Gf zLVO8?Su&HEOjHCStO?sV$%ip-Un+U5T7oXP_Qw-l&~JR4Nl(r4FsL)JQx-^`M`-pp zg^f&z9g^56oZc(DgOs8f!Qv+HI?=d**t;)uDhhQ`H_U9G1Bg1kppZ*q`&E*^8<`h+ z#%A}Chx~aFCYPFnwO+aOs%0n1K=2Q$0iUlQQYbxQI5&_T!5?BC6eQP$^aa~o4f4`A z9x(Z-cPVFfbEWX&0<^1k-sZ_2P?7JLWn!Ev2!|nD#Xv)MJx`;rD!k7QW8hRGMNurr zmd{+Vuka>i>E&SLTR{JL`B*)eU+3Q%>c&I~3~&YiPv^5*QAWANZJ2x^I+Ki@oR^!$ z;l1_au&5=q*A`kIty?$}t1;N0us+rJWVN?%V_t5K9bhtSm0sc$N1uIflsk82ct!9S zm46D(0^YB_<{1CT;Zd(Wa>swEen#Bqf)=W<)by?R$6rK+36pC_;7tPdNvfaXoG|jb zM5h0logHuK`;-2@TAUN=wr1RTu$2qBp-HOnv=W}ve6$HhvHRaTC7TZyqELO&#>O39 zvp2Y>H%JvG>=wgU5Rb;f4-<{dT47o{`w5@ZTeq?#!l!})2FYd%HhPe|6^KsCi=pE$ zxt^Z$>hDL^H5U4!sq$324~K}6#556}2W-8l#}dn)^r%7&(B)u| zV`ET|4=cQ=oOzR)v+sJg%K`d|lGV|)Ka=CUfyh4$cIezVf?1PX*N5Vcj!hs#%M4a2 za@<0ia+5c&sx(iTNo-@}tB;?OuwlKe;b(rGMuILdy=tY!_WU{i-9(i{`+r#qAmv}N zPm(`yE}g~qA7044sJ%UM=+%nbbO?j0db+66>N#XYWdAZ84K&cB^*H@%bO~o?6kbF{r_ANf38} zPaO_s2HJw!QZmKo5{;4Iq-()TwRZp_x8eUFiokIkplwI&T#(^oH@vX3y2QIiLWY6{ z{x*szP(a}O!TZ=Z@FzRY?dLeluU1=dj~T|X7R#~(o<1fkN0=+8zFonjyqzUrK7Q6L ze}lCZOGwLNnbH-$JkZm+zk-*NQkO7K2sM$(w+W+InC)f z-DI%mV1Od>am&Rm_&6U$DC;i{n6sqIUlI_qlMN(%IK{k(lLL3-aNT#6=jPaKbdBc#Wwrx)VR#B1689{6x8xlPAo2W$a%l9{630W?B z*4`q0=*^waUM;LW=6XlRF^Fr<$mB%~t(+YvPW7}^_-ha=0GtuqK{us`0(KM+b5IAX zESEr0MBul{T%ci`Y!)4CcprEA6O@SxUdlV0=SI(k>}RsCSg{AB)&+)-Zb5TOM&O%q z=E`7KcHlB547*y(oOULPA$R5H`}gjUk^pj1=wZUudb(=+)`E)kn+rI5FI|@*miyq-oyfD0Oo!vS&#$p9q&W96cRuF>%+?GgT}6 zPR=&>3&`dyJ19?}=Exl zLq%<(;>)Wxk8d7OS@Zsh8edG+dA1jywTFgh`_uM`}Bbq4GQR{y&>113nd%~N&O z1X$Zj=E|0XwF6B`;H#g63b{{Z<@}ak%8S7WB417RM12s(RfOn+84~BhWh$g(%oGpr zrU7!=XobgfH*Fp(f9XC7>oIPegV^9cNinceD|>`Vi@(KhSc-xV7e{p?=R1Z%!RMqtKA6uUo`&d6-{M{Y~GiMr$!heY+ zj%@gWQYgqxOgi1cat;4GBHKS1oZhd98vIf~ zmrZ9o`PS&I%C8)&LFq}Ut~x(YvdkSR{{bjdu@@ z?q&*?f+vJ4bW_#PL)^n~t6u#_)&$B1rcD#b8!>j{+^c@=<%pHjs{<|J>HEk-6D)V` z%B(q6SC?D2W09}-_dRppJI*cFWx)ro4Qe`6pka-;ojVjdrYfZ zhKUe&e-h}8&!@v7d!|>L&{^Cw|gbUft6I#mMuT`ly4fAAR zgzdo?>bvZ=dLrEmJQgsln3|J>{Cb z-iwI)-cr2rd(Pi2pvW?7*K@lJiLi`>tMD_h#66L#2Qj2^7p;&i0jU|TOh~pj*%G7C7WZN`dO#)@|6@dEcRKf)j*HlDhG_k zyoGduW8Eus!-6O$Im%|*GPZ^Z*yn({exp}o;O*d98#n0e&&upy#&H>|wuy%~a2sAI ztKZkN(F2w35i`>o*U^Jt&-ZBJh2keR!vP$V4cUmf%6R^6#5}>RC|C}GPO){H5*;0x z<$mUbZaYL+r59(KF5e9JFebYO%C)z#D4?6sUqI9KBHAVpgd4d9KkJM5nApDzcyBG9 z=7*(P4fgXSb4bP_p1bIrUSnR&JPmg~nbb9P{#u`ZPTjo!(5i9Bbj`sgU5QBnjEec* z`f57xO9ni$p{Ze{lO1^)-(22u2rZb6&9Hyu4kEIss~7`xj^gA0`z(4=QEfpKr=lnL z8nx1{Q6T|a33to*pddKyY;05W}J^q`3W3JzxmxCbD{j#=k^N5*WiHkfp2>2{_)8-y*$@We%z-aE-p^KyFxOnj2LlZyOm zHBB&Yu%)NX+@>~BAPpX%1qwp37xJH4rj&>nsYR%Re|%r{B=QsJH~50=wEv-Vm0xRc z_j5+n%fa8vi?6oD&bRakfe`tElYGYFHPdzuPa~*Mj7B(FbLxB)dmN^Fs5=}Uv;WTe z0&)pvcT3MRCb*d&ktOrljA+8?FkD7|Sj_4e?kOUN-iVc!Vb6OHKmkfgpd0nOka3qK~GQxUS8UJRa_`cEJIw})u3!iHU z^{wfD6nUz%3;Gp2;Cd+q{L#~0D{>~Vjx4XQUih3@O z5N<94SG_0(OwG2O=5s$S;~hFfr=P#YcTlQ{Iv|LuHVZhS0~H4l?Ov(3(9l;~9%ud2 z2!RRX^5{Ob>xK{_XT1w0B@@=Zg$kWT_{kF3`inrvgRFM-ipTZOaXPH zLzA zh=26qN&uP@uC~x`-gg@JjbYpF~_E zmyyB69Lo2f$n}}e@XHFLG_>AZawBqjIp6ST=Py!a?KDjJ&t+V?sP z5S_k&*C+sx5i%XU%L4>k5+|cTD*4d}SqR6tMY8Z$A^aLQM7#k_u@Z$RbS5I4uNr3c zW@d#!d;5|^krEBj_KVaBRXlP;&q=dxcR3+F`xA^~DeuDps=@}7ttku_doEtu5V@!Z z`sc?@`P1Xq9&TyaasailQm%#5(&jJbGmI`fo15tSY_B9Dek+!d!d@vr2vSpv`b$KK z)#^N`P!2|J6g}>8_CbzIPNdLmuRV+H_v%CFYbCR(<+gM{Bry8GF7r1KwY7 z!MReFxi%efcb_}}3&8BJr3L`0z+YqiulV2^Jp^jkkX}f3Q7M{aGg$U@?BZD$G?rq6^-^~HaNbDDJ0~29q z=&e~}i|4soUDq!179l8618}mXXg=6V;0i8Rto_I;cyDok-rN6&zma^)trVcJ-z-b3 z;;v75ggmXCRTnA2E5&b~#yRii;TDq`Hv!LO=D_Dad|`ouoi8&mV=);!T0#oiE(@4c zGCns}M=uRjR8-sfE~}v1A8W49B;V$+TwhVmvzd+g2-fn&lCCiel3)jh-@AZk(_iDL zVksRWeNU8*A7(lcthGl^ywuU!@6kFF@6C!epBVA|N6#@wC@AjfNnxpdrfb3-rhiDy zH#?ZQUs9k;TjJ)x!$rvDh4~qN*?r~)_R+!jDAi7za$}z;nLqm#WZmwQW`6@zqaQ+W zF*iUv+qB4dwQqA0lN*W^a6;#0bYsjnHQiXtz!iN0-l1HaRm*15+euI$Y$<%XmGaX@ z;$1=5$4d@`?4*=9ER}+n@7NII(T7EPw?4XT%DMkgT(1dUvC03F3TQ4TBoBhsxacoc znPQoMjv?WRRS>tWyDVcjCvWN*&G$KEqxGqQlLi;hUdLFFX5Gac{k9~(u=Z1y0Om@i06@-FXV0w{Qmv#Bjbz}@_B+3 zidBt%5YlZLmlx>3_N4;#PT_lLV<7b0Xb}ILesg)`>?61W-rLzzkd5-ue}6$I{R)bn zrz=}MVg=Z|MRoWd}`~%dF!h$DD(V!Z~~A)-sJSxH0ZkgR@qMigKPb1c8IQ&-owxh)3Z7a zD=A2^#;Y6F0;H1%U@p8=sE3ZW+Z=qFuzwE@WS`V6o&TD#rrlvF_qWYo`g!%WkXobjKib~?Uhu5O!*3h|24)qCSsrk;gW_(0zLp={7)19( z!H9nSq8+eF%YamuaYmuh$Z;4@n49!BOJiGr7HTo(a#F}aoy&LDmocg`5RCGIvl~!Y9oR;q`M5;3UL$RE-8l$?E)aD)K!69tl=*Y%WY` z>4;Ld+^&~3jgJ*!>B%1QJx3Oqv`V@W|LI_qg6Wn25TOaUvpGN<1E^F=;YIk!IA-n* zk%Y>d_pjbVqa?>+ASFafU;iv2DKJy`@@0$dSXlQTc@=Q_MLrBqFOQ3Wh6*%-(lqMKH`K75NUR#lMxuVo<>14l z-xYy>!eEe23{=>e|D~g`db=krk7KOq8RGW2{~RRR3^*~k1g@y}4KNm%>j;#Ob41ii zRfCMQK0ZGWjt4!>==x9|D1pSr$Sp9ZL9@SJDj)xm;bR#x#8IgvjBIMoX*rKzz|hp@ z9+W=w|NGS-J%hOp_Sm_cR85u8ZVh#t&h9cVaT7aTZ%jHZbl6r3U z-c9e60avgBV@%0lI1Y52Vxsn`!MN~oxiWGE2!X?x_wwLwEy(DA<607MJpy=&E6#+xdm=RGX|@aP>??%}e_ZVM7H^+l{{}luHLBjzRMaKL zFgJIL1g*_Jt>?^~+RT!Yt`UW)K#?X6xJnZJe})tQoFwSSkH4$1<4lkcDCN4m(gDZg z4okCwQLWb3-P#^i65G!k&?I#iYjU-Wbu8D86fWR)n=a?61tYsFej>IJAa!5*cavR>UNPYNy6;gU|D#p^ajW_^IBYF%kCM_-!d7gp!?jFc!Dc!l zMzD}e3>uLj3W#Mr!{u?oVWlge9$1bj+T~?ikP|Wf3Ua^{fLy>O)Mp4lHpF!`|2OA5 zO_&?wcHw~hx#FwOdI44|r@2Gbgb>K~ zUm=ymW5}W7=m}+}^X)@7J8P%4$6y0$hY4CJowDVVN3FEGUH_Kw%Tdl|u-)wBnvfQX(lSc|mK7=kV?yEnIv(hG9DX=Bv+yL7M{EmykR&!_CAKIKu~Sy}*$UR55{P zN(gO?y$9gsAiaal$`*-`zJ5qY6)^u^2CPeRRkD!A$EN%uKf6B3OH03yll%Tx$i-cwdyoN!`0m+adkmRicJnc7PVhc!5WzY+|8b1Y1*d5^2WO=fh zQnG6QNugFH?>+ke2y0B+TjGjx=g>#@pNy_YO*|yf?mS>VT*eO`-@V1x+?IVdVoe%I z_FC+o%JH^+bmu`rU0-c*M(n~tR?c3c`g4^{pe2S<Eq zs%I926I+ps0cIxIVM0b@&w^%i_STWMd`}au?FJrYkVnSKOY?kNF|$=2wRR(@pv-1J zONmL{;3x^)3}gZ|j9R7uSPk3_0F78kPVO$!hVDLazy{D8q#Xqy%#@r52M4!xbQmAl z1Kh^oFhz7rd}d~5Lc|uqe~py|5`h8hcS|qFC{M0uV!rR!8h3sZi(;(tX~+%g*W9CS z%#3r&B363eF<>*0rf38#wcvy|zE}f|Gn)&qvLRr8+X7r^R=INzW?#&nU35-cX>Vim z{Ir?F`&}oys+77a*OCAt5_meKZyNA{gaU+gDDt6cohw@?oW1~y+{C6s;737l|Gucc zK8=#! zpNfZv)E#s$^3A9D!I=dhkUFEe+FTp#Z>$54_coD4MeV zW49t%85tixzxTjG5d~?Ed3NUP=kJeo^Cr^l5>&L`6JsFh@hvR{Agdd3QU)o9wl<|g zMhHNw=Q$o20ox(Kcn5B>zF<(`8Z&^cgC}V1>A}g<4j}5+` z4){UYUmL=92h{!R%)oogc6&-3Ag{pTk6eG5nT#W+9 zwgN0F3?#1_fDOU_b&K;JCeox25SoCKQ!p^o^CRrOB#4TN5(9=wfK+Z>H4RKd$@ra# z!ASukV`VUU4;+aAaR``&0M|ku6aY{HCTCtC*#YHgQ1CDSKrQ5>2L=j${FuLfz6Eeh zgoK3IbL&UD^MI314(K`n^3?~JT z@X`zY5WU>uaDM^61CzGBF;z{%$j<0Z?tapXmN{>(>taE35Qnw@;B%mdM{u?9 zAPIZD?8)9Lf|OHApEhUw?O_mifSl74IrIs}L}8WH`#L%nZ*i!e-O2mD73=EApa66nI}|Nxw6)r?0txL0Z4ppcGW| za>PgC<5$k98Y(cpp}zg&B4EXFcbW62J zdqJ>B=K+Qq=}(yg?B@YYNmv*I?0#CLby&-?)7qzyn3!y?+{AakBl9OnL?W&pv=gz; zLL^ax?%nD2GmwCPayR?Tz)8wzq{@tS~ph!d=(h|06;3zFB_oCfW6l}B&QGs$lZUeCwe4=U(CADL#+w;8MZdfOloNtT z`3(Hx*I;fK;o<`6_!xSXbsz`p(3E+aVt*n7D3P;l%wb@W*@EY~a~&A%ymoNlLNc%5 z%RS2jk^+k!*iUAV2Hx{9fh7QQrvyg#m!~zC*R=p95P8jJz5GxzkvFyvE&Q!Wp+XrW zje^Uv5vIPydwyEI>ngtL3%TsCsq;~0l63N5xV$B1n|;RdxJf|qLiHaE!++y~f(wo&$D_%zJDs&QTu9~(tNS4!B2`TXM z_$b*YkB2nyj5Q8Vo>S5LdOng@f4^9hH1_UCfe9|g7JBI)%(!fD%v%3z#)9=04$PX_ zk;`+o#e+ra&$M|KdoP8l12~(Q?rIMFu8|`pDM#w4>wvi!xjL_{t<4L_$jG0R)Ylg| ze*^y;dY*6d*(|t2r)0ULCcLL)hadQ?S{8CHma|Q6qsH~MM3sMvXM+TOAZuRJ4&sNR zI#}zhbr60Zaar6gYyofg0hiwky1wX-ivj@b@TXmfgTts4Uh|P3bZG3xX1!vvAt0AQ zf`C^uQ&^aZLpc4=gNXu(gH}MZCg<-cEZhB2%TsXh{5$8G5Y&J>))Ds*690{I9%scT z{*`!ITW@vn3g6-h_~3H=mL+a_wsbjz>Rs?UY)fLGii{9O;g@k^Tg3I$iu_zX8M&0o z&5f38;EcvDIg}&uIO9Qw>F7)z4$fZs-jUn%rGU2$L|E+27dR2g{0TojC;d-<)fNo_ zKuvL2nSdVYPSc{OanHyg$w8FG1hE^M+#J7){QoFiej0Kq-IlzHq*1P#x|6ibFi}rK zxb{kCF{EV*WwC6@N^&UFK|n%wJFQfsWk1GsoTX)h1FWNvOR(uC3a2Z0rWYrpw;i23 zJw5{xW=Xt4#x%rAfSY>U++R~Xcziif$U%eahv<(5#l@?h0BBaf2z{^=5ZeK!rnjy6 zwi8|cdiTim7F@N1cM`66Zq~=nCRY%H0CF0I-?jMwqPz%1jFC!N&T-ZmK5@*l`$vM9 znK*=8Pw*PwkOGjmY+DdcZ!j*n@%SIPBDC9$Nq-$45390JW|Je%R(g1+#%Ke3-oQ$64FgqDuD4KZ3W zHu;^+qEVX-L?2}z(AMccbe0SrZ-F)g^kzbS+&8aWG-Q?Kei&=EQ1{d5cVU-bZp;2# zPXgwIpvM=@m&4m>xiEe>^>s&H^D)n8^OS5|_i>WS-INHz)Rzw@;PUo1iRl506?pK2 z?w6q7@=8LqXky;-c3sq14UfU$db39o4;_6@)2EPaa-qq?0u(Sg@?|^N;->00DNq~D zBc?B>tRdJ$->AMhY@IFh9)3?R7%97{(&eujQ~flD!EQofND5u>wF^4o0pWte5(UB- z3?Et$7L2PL`C<7|vQvj@MJ~@N8V8Su4Thos_$=mANtYjkse|)oIUwBi;XoGcfr0%$ z81sQ7c;2QCLfbRu`QXdh;RFP1laDZL%T<6CE3G zh)(PP+yk+&HmG)OkV)$WZhp<@^Xa--byKIUA9i&s4AFcU5|-O(#UNjs{2bz3RS;-y zOSC%~(;zogCW$8SfX=S~!+IGgI>9Fr)J5@Ob{f9WDgLgN4;NJGi^$t!n$ zdvkc^!_WBadaK!vj&r~An>aXN{id0vCjU&I0mkE+A(9F&LcQ_)-h~~12_pE7<^K(M zSXk)2vdH{xIeCp86glsjd7gJfXJG5QEk`A+z1kOm+`%k1Kb4RHW3ba3^mCET+Ml}o z#Ze|73UL;=*>zL&MuR^RjNMgswZwpO61I0T(xq!UYqGO(z#D1PReM-Du2%R%q}s_V ztEoH2HsOxQ?o1=*ed6u2AAfk{x`evYO7ft(^bnjEt~_;)fowjdxtw$La{;Gi;HLL; z>}kpgDPvI1H}@UUDYV&hS|z!5_)K{G^FKu>#@nkTWVQyB3BQl(FSwD9Rh?@!@XFRn z{pCAmYY|fFbJZ}m8~v=5IuG}`uR8{OzSnIlF}h``^t}l*ADZu{%n^pCo8}hiYhq~v zHTQb-XA^$8C3Z$;>elwssc!901_#^Sy@SM=w9*#&SDMvZ#?p@oPHc>!iZgEXYxGC( zasfE_eip>IV3CzQgG=b&iuyb>$B#1z(Dc3?hZjk@Uxdc%|s&n}1rH=e1f?;cSMlY)hl!`8SI17o$S~|btQ{Yw6 zaGT#Zc)9lGgB}}OSlqMd`$w0XlMc>u2UVB5h)&&Ug8bVLC^z2W-@t#R#z2@kWOPp@ zM|EgI{#UMoskw2{lhKLtA_b?atf|w2A|utT!{XAp0F&D+8(O1n*RZfeTiLB#-EeTc z$f~dFa?u8lz;{~{boIlJX)1~PVav|2O?RsoImOadHLoEi;|X?nTYBQ-MZOFESiO0Ho@xK;_m(yzg+g_JX#U@_8Zq;^8768lNY>)=Nlk0}8`6>PL^b zs?bW6YPNLNq2#di)y8jYc5gdkzcp(zKEerfbP$|*uDGwQEYoEGg=LSQe3Gk3g3n`o zxq%zHhliWhf3tkMr(-?%X3t8@GPk#2ZGXAnIr72B6CN$H8JTX|tIQl#e(xQ+mU1_d z;mP<|T{^?bO1xycK;NYQ_OSQ(t#65Q8}BE%<|S^uBVF2QKS`$ID7v(7Xo=u8%Fo~^ zf)+fS1rH4U{lFATw{tNStgH&==8m4W=8oi?9_>Hv*UaI&XlbDqT@~_#7@|YnqWR_C zc7(-KZJ(~ZfuPpNm7VNd$WIVq!koFkwJBsx}A(QPKBJe`SP$H*Z% zx=7|5ALt#Qz4-N=Gn${(zNq#G6>G?kyX(DgHxCvOu#;g&K9!Ty7|qK{M&-9(#fO+= z&HBQGozn(rHaWRm%JBAaNj_n@)OzthByiGt5$n9?3sB%&`>ZoE=fyB`hC~>JKy?d^ z?ptxEq+yJ-e8wYfTCSXVDTZ?&N<|roSYF2r#gh-_x3-jv4m@XajVA7-C1ua+IJ5Ns zq{BqyaEp`3JZ!S5z7sLQR#a>(>hN`Hnz&MB#=G4c7ttbeYQ2_=6V1Z<&frVupwC94 zh5*yftNkVfUPu1^TBzbdZ9Znkv|JH6N}M*U+Y#k=KL-T%Jg2oc#jV-r@1JNH;o)D1&#;hD?f&A@TIH$&?l z5ZgNiJa{FzF4TCuWnC>5FG}|NN)arVTt8_)6^X17m&|chsZl-PDLPWU zyZLpvMB+F!o2guZrMmj`+^(X!A2UjbiaGA;NZIeYM4+HRH%^oso$79TGUQmdb0j`< zdvJ@GqLPV=!=A04io;s64I4M7S^e$>8!rW&rQnXv&i8Hv1iq5x3F)$t(UITLY3c8- znZ8HxOQ*xEG(_P+(?P9<3u;Z_fl$+?5-l5yD=%Yfb7IQ!lD;BN-RPn&Ap=*P(n2?u zw@Kz3iOP7-ctP!dvM!G~6UJS=`<8gR6Yrd)-h- z!bJSZ9G;R&k&dPXBtzUu4qPmopBhBA9r*eiDGkAOklalik|ZkBe|OWrQ@n2S>_U1( zCIza%n^e2brI!2y8qGc`!HGLLh7!t2s5%4q!?G_8 zcB}?w*4}8u%GnF&Fy;jx?azozC3Yi@Cz4L0qb1+*?)Mcca!V1MjmnQ7Bb*Wv2PqFv z&6DtMRc{3iS2FUsC2_|{(51QB?HuOtKhneDD|H*f>@S8TCi9U))^|m(OJZ%Q+kDUp z;hgcRVKU3FN~W{+8iGb5(zgU#2Cl)M-LXDt6^rI#m3`)|N6$t%9v4!d&v-sx;IXu!Iy76Oyw9|>@v+#$WY}aZnj|6b zM+-J>GYg9luKKZ}gT$qiZlNrp$d2s^5?;p+>L&*~%nUX@`@5IeG5z*p6gDJgubV_Y zlPUNgS-kPR8|l2K8H_#1QDV5eROM){Q#3VQDx!MGZc{42_2lg{C?*12#Jb5hG4>P0 zk6>V?55{DpYTfhAsZbYjN3?%uaQ0;NOKdu4%IIv!fuz~`^{qcUL_~LOjnSJb_hIc) zS0o!SuT2R<4O@e|Y+O*=r_Y>RV!Lkbz9p!(wXo1Dm~~9_eHZkR7EK_4=*F8<;FGkS z)tn9vS_Ew!Z2rAyA69?w0N$3SO~iK&Wk0{ocMIL^m>ird5OyPEWd-MwF-io zjErXWqz@302CgP1m)feoR?O|6z*2nmQEk(S38X?1U6N1@0h?JqH?^?gt<<2ik=oH& z>Fh?7uv75MWZ3!_?%Ei_BWGg~^&*L~OSY#Uor3k0$IDAZw@ovLok$TM!22Da?w^`v zX<#M5qtFgXl7;F$<_dVx+V^YcA}{m(k1Y-7lHx)0AnLBp$TW!LWuAB9{-n$B!_TBA z?iGDE#8$3@#ms|~`%v)ZOOJtf?*0d5`~ehdZC3UcA#Ck^Ne5gkzB1jVGf--9lTM>B zl$o1_m*u3%L(KS{i)zV>Fu8xlhT>=a00)LWSb(=AwsqV%$YvH!MNF--T+7bF906|p zTOz9Eb!*i&(}-u*xRHewuVPOT=v4v2AS?j~->5cT=8e(o8ChB4eSKtHC{8ZPFh5C8 zCp8;lH5YkqI20$l{9Wx+KDE2Qa?e*>vZ+O5W`=HE_l|n1Ok6*)SIwH5E)>}Y z0hFZO#=va@bo%jSX-n7gZnBUh5bULYkV<#fz<|)!`(9m;{mEhb8H6=5>hGd?_daZ` zl$FOFVI(!~pSu-MiN+oy&;#Obg9uM-g^D!~lSocQC4F@ug{KOq^!xg0>f_)kXrtl@rL-RT_6s61nimDsRsAMHWHGHaeAO#%Dvor&60tMc~B{9rYPF4UMjC z?}XnT6Xl+tXb6uKUotJgKY{qMb+KZWm8wQ5N)p2VRv>y;R=hJ})Y!Oo z*Xq3HsRh|r5NBLPagv89QPR>xyz&;C9hl}jZ(eH#b)QgrX2d)PXX)1ETWs^-H9PWZ zzDG|vD@p8vtc(k!2S(HJ*f+7%nJg65t)QpR?&w&b_%tYDlZBIP_rC2T?Aq*d+#8@h zfaVLN^7UpPrP{l=%^_^YN;~duF4k0ELqTyzRd|I&Nh23XSvKB8-B1`f_4Hwu)`sWJ zrT&R5_ui-q$aNJLwYNe@E60Cg2UHpvB8;1cRjNuXVaVODSFvWj;Jd}bpOZ|O3>FI~ ztL!{Ra%+d-C%$BkDFOap2XR}+D-@>FHR|82mu8+F%&!V4Xvk>5YS_!0Z7!>8ZT{Xw z59s{_gvla}Pg2#$$<(rrPwPl|`;Xow7MST+ETZK|((?+LVkIO-k+pScALD#EYNZ9cuo`n)i(S0@ zPOaBsuzt8&Vytk%$Zohzr3T-|R`dU7-NyOdKw>Z#Igi`~XUzf-(q&6CGiynn9AqSF z=Thwy>%Yn$wEf^uN0gB9GFgDvAeYsC5{ip3plR|x%;R=>m7Fg-=V;@?nMr$cLkacV zW?u84*AH4)V^dQ}k@JMyDs4$TA|)|A(aeQupRwiA<|7j3YmsJ z*T-?XQKyk}AYG*3xO}gF?cR{3#eonVG=bq)dc{d2Q)SfSL{`^B;ICr z6W{-3@3Q10rbp9cd(74cFHeBk*;QtohW<*4zQyxMba^Q@W8$5eqS<>hJI33%fBo*? z=^bdVNjyb;GCc1hm&Ie>I27cW@syl*&aW`L4~1>1&M&$1G4`F zwxaUJ*h%NdBD*A(dP@@gNYIjs2SWbQ5GX* zwKM1k*wUdi%PDl@%bOuZ@!Lv;ra@b(j- z$A9z_=`wl?!moBu!#>V+KLJAb^r__T?rwTonly01nR43Z10Jsq5&SOYuE4z2c%e0D zvdot3IgktN;JvJ|>37)}vj-o)=FMd3)mF3vgPKVR@mNk7;{Il+cPc)bPX%| zW3Cs@{&*|yv-LToE=z11Lim-i+^y~0(fCyLTjC2*&wWVB1yllJ?P$(~hdckDJ&(B& zwU1&vmapdstS#AGE6?@2KtH`4w9WP#2_#q2Qeyg@wyRoDGN9DdxpYqgHoZ%z;=TinP2S@%iORHyCXT2z=2-u-uMT|Nk(-ppJJ~%j)_*n9L zHi=tE^|M_bKbW7LR*1#Tzjxv?cwasIryG`hPuSTNrq8$RBoFm0$k@%NGCMoPkYOkH z7XP6MH~47SP)x;yr-9woU+d%{a&|w(&9gEF>+^3`6~w7@nSR=-S^c@8n|;zheSFY-Vky!_7ZQaum$9^|bJ`y;Ov^onRkn+Nl@+DDH@ngUK+WDnjOstzli!N= z2Uo<%**X^s8+;kYtl*cbz4@4nMj4*cIT9w~s;q%`8jp&C@^jh7!M*51-@_2VO&^q% z90HCv2Nl!j5{N0M99%NapTNOX3C4etIycXi^plo+{kg=x)l~v2zHb)NuA1aAkVut- zQEX>rVNtYJMunlMd1K46X(qjARAV4fQBiSaXoygyx=3O8Hcug}Q=&gAUagBFHe=U; zZqJ^Vp02aEvgR`9y_H4~NBNWeQ9;#o75b-x(f2gyV#64)4;9T#T!i4$F;CZ1vo11l z<B46!CS#P6_9Yjk^@(fcTOarx7t-2cbicgIuRxA9jhOQ95s5+=yF~{+p~JNQrzV0 zs)L;Np3$?Ja}ALSJ&7TS0RewA30*(ps}>m3cnTZb99-Yj7!UXqZ6+$hmRXAxWvLUD zc>-Sw1x|?0%($6-iz?z1~}Xm0kVCJeUqjfPo1 z8n&(G^mIvV$}pRbyrpx$u~uMsp?ui>ZIs-YUfkbb1KxEe^1M7O6)~8!eq*vI)^tN# zFXckrJv{>%xs=M?5!lm)C8A#%*oTc@*WMW_kzl{p{rKc4!7YZ75c)?i8N2*_P25L} zyv!MIQ=V+i)OC7-S+6RaSC~Y)DJW?8F}LhyoXZ*L#I?ZxymcOs5J^eK3@A^p&37@8 zN|wY*@^Ra$GdjByq<**$n#?-R$dpVmt9r}=qVJ@W(=q41l?1ey(Pwb>D6uyR^v6-1#TGK zFjh2vmX6hal?>rFIAFdszPrvhtQD+%l9oF&es785r09}*dU_b9HQpB&bMo_NyzLzr zI54hlYyQRa;jS~C&yv|CDYI>c{YEexKBM(% z2@M_0RBMfA(k8Q=a~br_!cHzW&Q$C)Gi|aRO$GW->bmY#QlnamV zV&iAEcf49J7s2jyBMmn_hrMQQ^U`9Ag<^x^b>aH44^g3aS(x<4==RiN=%71jajs>+eJ z{EFBZkxe-~T!PAQE)6Ot-GW6Ai`R(W4AQIvnf_w)c!kh-S7uOXjYQ-R@d$?kw_jP z+o;z56`N&i9(axvXU%qq!|p+JjGxKuE#E8a@kd~9XRKjnKx-pzT!IdgLPsk@{;WKpbVb14A=~pg9C>HR+zA|`~7u_^MrI+#2@){f7fS8j8E&>-&Was z_(ZaRviDxaft!uSST0Im-#YI0JQV=uTt&r?eL?%yv#r}`^lvNs{I=tSc+sIe_BWo2nnp>d=f)n zOxW2(-z0ufC|zgebaC`l zf{vcqCZ)d=%9y!Xg>v>w?-{j%8OHOyr$M^{KC{0jXZrxVs z>92tIwY(P^RhuXupb~`r3w!DPr}r%O?lM20q}Qxykp0waXEj>Nowtd1*c*(6#rEe9 z-*;;;l&4m|rNc&f4rvQuhokIY7q!4J#d zf05l`Tzq#I&EnMSLzX2yV^m&l+u#)%bhIAA@z4`LN5P^UBeY9H6Vth)$4-5HaX7)I zd@~)EF}iuQ(x;r+(N0y#Oxd#_L4zKJ}&D2Q#dM6A$_c9%3BXQI@B=`A8K zI#de?l!;jE!~5b}NhH$VFQoey0{1E5#1OX37(>ey5-siuusFaX`i66k7vT3yZa2}K z2yi8zz?Hxefb*P&g%n8pQ5CjMLeHpj@^Y9kzXqM}RmU5wf!B%oVMD~nwGwHOHg`A< z3ICyim(G#dv9a)iT>J2x*KN_;@Gao;l-d-N!Cbo7pABY38O486#6J~O^p9!^tuQ9m z#!zA8EDi|2XWiZ1QG{#-qkIpEh~#t0KHRFPr1W0fG;_^lOxZCQF!egJA>+M#UAwo> z$a$$do{U4n;r3PN^tC5&6Fq|O0t#G6mskv)uD-Wv<@+$DM@w@u2uQ+(*#I+CL$%J&)wgpF zXxUG$!J4pRxLGGe%xc2Ui64qL=x+HhUuAI}xerc9dpJ~_( z9zO=HZf=hKYGq|kjUustQEI(K(sL#@l?y1ze_ zz6A52V(Dq2B>yDRAR-grsMMesQzB-6na9*g&ZV3OSAyl)IOkg+Mw~8_{wPB1jX@_j4Uo1AFD1@{}XhVir0Rn zuyytMh(s5pYsm(P#OODkjM8&8!zHJBgeLDk&=J3xIHPh1?$od=4Qqy!*3Z$z+TnA+Wat(TO-!6t|>Q0gLg0>gVo%<$WwM7SM|uzQaVOP z>^eQkr%@a?hgct2J}10w&&Zc{u(o!GwYx;jP-hh^WA_V5`qTxX^GiBE6WkYko%#$n z704T;QrQSa0CYJFEyl04o$S_=t`r?VcU)#0s?Y**`a`sD48sE@Wn>zaR5W8y8eUde z8OXuM&u?SLA|XggI<;rh>}lHRB6gOOe2S77!Pmnd)k#5N0fJ-UR9yfMq`UPL=d9Z4;m(^`=j zu?c|dL_f*J!Hc4WEt_CrO97iI&GWIke}sflV1+T^&Kxw9h>w%;dyo; zr>K={$!-{e?BWdPY;1k=CKmO9j)sw=Y&XrS62;#p&g_!Es&_LvPlcSh{!rR9oq9%$ zGA;!oSsOf`$Kb;ed+Xv47b}jGH6*zj zpp$nTlPfg!Z4NeJvm_71maxn~Z2k^^Vnp7swh-!Zl1JCJ&54}C8Q(C(w~(;uW^j{| zvGU^OpO4gTc}|&nDHyEaj|JWX(~g0Zy<54Mx#o!_G15GTbXI+(*jbm}mm;@4=uk4f zno?H|G2gOfctPUwMljM=U0wplzhhg zQW5*f(Y_}VEcVgvnNm?b7iJqxS=gSkS$6Iu5m&Vv9IDSNR+vEPPa@gt>C>xzk|ov1@SKU9`s?X@&z4 z^72R^U)Iq7{IWE2=zbhNOvU4d>CMJeQh0-3glGaS zd3qlPg25d7vWj4TkP@M$rKa60+d3yw(Dc`xNIw>~Pa?(m^6B-gIg&aw(@*TH zde*jUo2d__l%Io;i0{Ue;}R30Ny(18=OgyRSTM=Xtb@D3A`3#a3n!^YuyoX}RWQvr z8qhYaPcxou8dFa=LvFj;N<@2AS16b^LIP&?zIpm8J3-{f11}TAZypp4okDFE<=i>l zff_lwB5Qi>d^o7oi0IF~E9s@RCc-2eO!5jkiz!vwhf1To#gp#5brW1q?QPh_Xz})M z4wmv)X_2o^uP?tXp`P5P+3|U{bjtAc-sIw5BZJuKiQ3e|QV%nDYNKL&j4a#qF!FGy zQGtbrVfwk#k-nEa%n1utPPoY|s57th`cszW=Zf;c< zM>96Ce5bTaTz>T`50%F*O2wj_n_iGhy`BGwS`EtEW2>D-aT0M1<<+4h+0>b=%iXb1 zvZ>vwpstpPj8{Md*FH?b%LHoU-J3*I3Q^TgN_{wg-8`(Ux7T&=eNkP0;>(Bj`BK}; zi8aRY8`u(*&D7_nMVnyYuY2spr(ej&I51GZ4m3@u!Oq+E7|y;B@rn0m?X7#jYC`_6 zrfcaRzO$=SQnod=S=u3*yg5I=%LF4I$|DoxlY3cZhi0<>&IyLHwtD*3c?#CQKJ&`0 zh8+tz1-}GKH)UwAcb;g%@P;^*&q4zO1KtHLmLb}RY=^vxu1Snh2!l4?F))MxIT$05 zh%GERx~X6yP_afe61l)2fx@I)On+B()$QBMQ!}yOMHZlH86KX)ann$i{xQmN9-8C+ zlW>2px6MuU1h>8W&EIXlasApgGS!$rEsZolQ9=&)90Y((eT;2E;hU1He9CJecnRY_ zsbF5?ocbvzP>41WHpy>E2Y4ksp3Jq@=sD=(Q11@N*1ja_~}F2jukNrS;ePh z_b@w;g>)#oxVYG!+$U4ioNm0}Z-DDiD$3axOlc0DPd9x>nWyMHCa&!k+H*Sfp#{pD z0a*0Iw`AftY5~FM5TI!mYEEe=vBvvycuBIygm~VpCEk%4N>!2kOhc9bA42ua4KGY{ zi2Froc!CdP~!CpeMF|J`>(BT#RkI%=k ze4Gaosu--j0mEr;9uH)jn%o?K50LU5-7>~|bM&OMpz-79nqc5Yyv<_N|QeZ?Y;off3 z>IM~U*3A;vvZTeu>0GnTFCM6q*rQ|E?d|N4e%LO$p)p@Na@tADM{mRKXMg}80Nz6W zOO9iycMFtVef&kAL;aBp0~H56Q&&p0K#8jYBvK5x6vvTS#3}kQ5usN#F)?v~Rx1jC z>_iG)8xQr5MCkZbMnS)i3wdw;NbzoJUBqmC7|&V^bOv%BJZJ_;*)G?j+S*ZO?pZsm zUfKbaFN?kvRYcYy5RWo{&}2LlK;e#@zWO{*$nshsLR%V}Zp7?6TU;X{(bDij$|u?B zBJPIExxm8!PC8rHt5Cue3Zjzk9*zsb@3oslMXv8XYaKUT67)riz>7KnD%(wQu5KGWt9^1*_l!^WS6?Y!RO_-6Tz2K~;g)2mYeltLSNKLt1i41LC79=oy)h3~;Qfo7 z1!ZhR7yzi8;FTWGjIZCG!*hsVGGe!1@~19K>v6*2eZ}Jg4+E+5)s)1X1a-BwYMhlv z0;@=V91Z;VK1c-WBk(je6SvNZ&b-@WR!2EI$ueAcqHcn z1QD9{iCS9?>5_cJW>x(T$lAbuFNggmGUhP}jrQ4Xxad{aKk{0-DQ=^*ia*b*l*EbR zMgd-6FuxM=A`3D59X&6=)Dc}bDnnA4Qyu;J*!}6(6_PdWBFEtXTG|K4hvjJAx(`8@j>y)#s11~k3NT5oV~Dd*_+o__*;BDA0t?m2^ZiA|^TmKw z#h(Mf#8>IhUf!`G_mr0GwT(mG^gn+lV8GWELZDF;;rX105BGb=y=+c`fSf{e*hE4eBji6L%Py%J#kS8rX9x7Yloua;9 zdh|}WP#R637n^i*nH`HTF7rPmdGTVunJY(VY_HxOdPW-n46jovOw?bdE0>oUrvdc= zYg3=l!2b6(7OnRi9b)T-M$R@IGOa#t)NY^sNjw_R+38}(hn%GJ+8hp&YNa{iW-6^Ua05hYPm&KdX`IiH`L!&%Kv z{}~>uPQiI*RMI8vrI)4mkg1sM>X60ezMJ~8Z8sD4gm&ncs4q-AnBq=oUmkXi`m$~4 zAtMbcG!h~%#x!M`(R2Ry3#S4e8E;=o26q0sB%1kvK z)+hwKTUaC(BHq}I6_-k@HvmH=9)1USE`A!vWvm=dbCI z@a8%e2Ko;*KtBM=efhjtCtOGEML3!L6N&D-g86yH>+Z#JasLrpXPFMfLuwx5JL2`uoxv?L>giE3Qsomy#6a}` z(_|%aDMyczcjxVuLL*WP;a#aLU~iXgIrr2q}U?Qg&DPX5^@~= z#F@<5P!Q-Gdd&7NG%Z+F{%r}eH;H|*M%VXUOE{66rOj?{z2>Ft)?CikjR2M)E1Mt74(s6$iJin- zTUW<$?k~$5QgD0*N+os@JV~yn+54$`tQ>95WE2SXVX33{dAJT7m4Hg}9fh77lX(!~ z)*}Y`va`&Fl6V%BrA3kVCXUN2w9YR z2b|%BvX_hrBBTD<-K0R{z9$I)w`=_NDm3B?XRVRfrFQeUCe+CYOZ$>v)3XyAHEuS< z1Zg$1@+_z0&msu|P!3?fy2J1~Hz_c!mK|;~KDz__wh6JmfMAABp-Z-pGR`+tCl#Vq zyMDJ;bq1~FOy&;#9mYWBq&mIsW$Gg3JW?^Nb@tg`O;V4#s%#Ri(l`d8J`{1YjlnJ!Oz8Io)+{o5E&@)g2B+p zs20e~`!FD-p>pQT8CwzSOiw^<5shMU(ogtHT%j@G^L0=Ku`ts~zbchH&Tj2q^yDCJ zKO?4&{HAkO>HQQ@^3edbKl5d>++5QPqe$jwL6fAmojJ}oid8FEMO3s8&fMPeJa^Xi zK0-o&4l=T@GA4ZZD`tsTqjGKz#Z&l5lvZjl*)@U|5!`%d$BFyEgPwUa#%Ei9nd$oW zuxb~z&-1DEZ&HrnkPr$S}0hPr_#u#P&9d#X-0aryzE_(O`Ubj@hI;bnOf@ z;#Z<;yGPy%94N|WAbgI+xlpj2hYk@Nqa-$Ioj&bWCuQcEOP|ma#2kQte)`VpLxO_Y!r|)Kmx;li5(smz**qOPnJ6{BGUL5s2@v%1j9V$$Q{ z-k8L%oi$28>^K;;US-Y6%|O}$H@u$OQe0?QGf3jSG71h8nWVM zYw2Yeenr^>mp8&PbD=_y! zJp*>#aYabvzGe}$G@`^h@gBLN^<$(6JhZAS6|EbbkYv1HMSLxibWe}q|2o0*;UFz4c$}>XyF(y9zVOE6LmU47 z4Q*P!8qaZFv%!_D7Aeo2^93#(f8Z}#eSDrr|6Mk{kWhB7Ns{8pR|M^sdXTP&3k_xA zHy^sboBzfoIl0GmRsxYRscJj*Tf)4avgm<_=IAHi^~30 zqz?|<<+Yw=Ob#xzF$d=sk`d!dpVd;{n8u)ct{CmzG`@{etUVu^FH()z39SDd!~c{F zp`x`~mEC1BayyS!5-1(bA3-Q~2<6}Hl9$6L!7;e)s2YGbaPFQ32jIWm)lNt0NeWPt zycy4uG)NQ>k|V#N?nuivwIqzMeY%y`mJD<(A&n6^k=&zBD&zps&JDjkfk$_oVgi+SRQ!mVBl~Cp=W_oLZacTjv`Ze+2c**;H})S~IE?qX*m?YHOc2HK|JAH?2`s*Um0{Ta@VF+RL@lyIiU!Epk3qNf1Bo|MY8r zj!$EKW3;=*_C8QxG^VsSM7AL05}r?ae??o-;$lb6m)PCZTaG^$Z0=u7X`(Z(3>EZ^ zwyYA#d^-RPjINOnLld?o<()f~Q!Xb!3&BL6amExK)k;4sFpyq$?d6P^^Ari~KdO5h z-u;mP(VgAh2LIb+t^6~c51R6BrlQtE1fq^-u97u*G=82mwkiUTPz=A+1HvwxU8NS! z8OgujZd8Twj&pm+YCs`%nWTg3P*uuF&ZuKrvGuA`u>9XCzR;`&m4BA(Go2$wW0^!R zlbeKWN|oWcZqMq0JASK4#!-c_4<#-qJ<8fZud)W2&}Gr4%sOFEJ82u?J%JIww}Cj` zn`CTMH=2yXX|c8%Fkr;BHIY$R2&s8Zbv$)d*7$R{7U7^aqa>ij#r202VA!Mf!i?26 z{T-3$MrpN@pBn`mb6~W5RP-4|H7q5n8Xo2C8g&Sr7Ru3(CA)n%{gbF4nV=q2aY@D0qm0Twwh+xqAW0=rZvpTle zonIpYf>Su&qYiE6>TSCyROEg@m$gbp;^7cxV zH7tQocQQkysApj2kjFLQ=AcX39XvNaQ+=uL@5ETnwwdt?Ay>i_HH)gN#H)OFFyyG& zy#i4xNyzfUQ_t}nOYM}wSJouu?w<2KW(=AVzgyM$boar5@M;tdA=Ywai+*Kjg5zLz z98|m|)KoJxZ1Z28&m{%#!}uJTy8uy`Rj+#&Z=8XSN6G3HsLjH_a7v52O&%N5?Wm;I zF|y8$&4-s?V4syxCtR~KNt-z(c1YWRYg!GQ=0jF3g&8wazW&=o=1&^ViyAztFzRR> zHhH2-0aTSC#c2MV_F8vp;<+o{_JFg>OVJ6& z?wpoGaiL#;{CM@Z)-yXh8{~NW?lG`7e;%LcaMcDSHIjEw(dh+-wytH~<1?8R({&|b zgHrcL5>AMTSXniEb+U!K9i5PjF*a^YydnEkt7DXEP78lb+&kSpEa_WTUd{(vSBOhI z-%$QTX&js8@D!$-@tb# zsqWF0rzq=!^8e&d76EP-m0XcwIZ^RwlgMI)5eE7 zh8F#3sFdZdaJyA0GI5!YkBSelRqO@ga`GrjPQ>9XxT z1Q{m8yz@};{jMMFx8w#Az~_P~CA80su7tAk&NWbaveh~v)8QDa$d>SS6RG-sZ{_-u z(q57AqPRHL>{4o<2n3wo_gKT&ddDVtk2yydGw#HJpI7u(i~el*mZ@80(U+@ z+bsVz7`Wu_v;1m27Y?lnUtu?FRaCPpiiy3!xtD)BPgJ@*tkI-u$D!35`!pOQBW-_n zwYLzfxnLrHhzK z2_FWL*z9bz%vOtsv!E7=jfpu2h({ynTxoQF{c1%_pY|0zM@N}5&sJC0ijo$_@?O1^ z0lv(E`~iE5sXCC#{aRreBSTVS1BqH*!!Bh(-F+t+1t z>_3W%9=MQRNL))Sv5h@EtuaVxVfc;62(rt z&>LnO4nmz#pc06w1=Y?Zq-r+r^6KjqD?$WhwP#0zD3C*2b(a$@=SvUldxpZLgGHTg zWQtBa4oPBf&g{D;pyrVZ1}7k_B7(mEH?Mf}3)}A9SQM-VX*nc-sB)apZE-=P~pLpFVv8kx=)_lVcZQ(j;-Ba4vtA94AJ!jF=APAfhe}&^M&i zNU#&3axu^sRL?VD+#Uz#o^?FAxg{^Nu1iMG&wBpk+)5!d0;Muv%l;&rTHwCIjt=fB zH5HWum`H`KDoLu5(C1STY&}_K0fL5dE-nQS#kS&B1D{$uMk%H_AV;d4Gmv#6wp+sV zl?x#%4nEX!|HUVrzc*{;e)j_d1FH|+9V(nFD42ZD(0TVZC8n2Q<%%3) zYzVJq&qXMvV0<|_Ik#=y3R0VC3_ca5rE=%ay+4X?s;%8Z%_}GbW+_2qgIW^&0dn7&RmU zng90f6{}V(H|Z$Q^yM|I<$!_Cn|pP<>SI=sRs#e|TK0yXVJGoD5h4Myc|+|4yPZ*BZ20I&M~b5p$|`(Y!X7Gj#&GM6jEulV_}*h=PmvX! zI_v8n5y69L20a1}!!Pe1H0Rl;0>d2I8lORa5)WdgWSY09XD!guI9A1xTQW=(pw189 zuc)XP4a*Hn#=sJP_wGuQ_B>EKfTr_-=5#5Lg7x(FRyH?(8F*Oj&qF6RNMkTH#a_Gj zVRTEpl>o;WA8g}-b%wV#f^MdDD_4Mosr>o#_Zo59JQ!v^BY(v1($wVkDT(s+-IA>O z@(dKxp^fd5jEr|U~!ipG|*L^@L;fv(ugr9>{?}C}BDu5JR-@ptC8|?XQpqBAu5~LAW(? zXd=tJOHocP652y$I)?iDZ^XpL-Ye*U8(IVN5YzOl*zEJQz+-chX_X&8D(dOQHYRHv zz%UD1M1#8qZt6Y^w3s=>e|q$2Lw8RPV%r+#%g2(U2TyQ_oE6s@+8P@6ZcE~4j4l~moc>2V>Zvu zhfp}Q8JvZ(w6xcN<5^wZDA@lGj%USOeAY}@!tZqn({CKl&N-sr-sNcDTVL3*_oA3Y zT4tuymZ8m?$JuHcBBP;;mUPWE6;DDZb-!%*jaE7$IGUEVViS4xV`MOxnnMNCDeKm+ z7tbuDG-?QeW;KxgT0S34f=k?HRVSS@)e|Bt64bU?6;ds^1?q{6or%mS{Y)?o|B2N* zZ{2}?`_Dn#5b2CzCN4t*Sh&ayAuMCHw zEk`EKUpascKWqej(ar^(W1MA&uliA32K^OefJXnDNUknsc_WZi+oa1+^yiRLpkhrv zKm+=+iiL#*VN)KsviBt=2rFRQz`}Aaz>C!-y`5$n>w>B_@$O?(_7XjH7vl^5?_|E)#>_(G_L2e2!g<;o8v2h z@{)lZBEcI-umuRIQ`MBRZ5VLf~U?xQ&CL z1yA_xzXKi$JMTg<=e3N=mXDYZH7F>A{FFs~S1X~q`}%U>pZ8j)#5!)%m&uR+wi9sA zTD{b}_#_hE+&d52`QbmCOLSSQVih*W{)h3d0F{Eea^fpcbLA2g{6xH76!TAF`#mdR zo_`*CX6JHnjiFjAEd}JVM7q z@-3u7B)5+-vwbm@LL$@lGgkk z9NhQaLjS;FKmu7eTC-{veXZr&7oK$43-oPv9@qOm-23H8FTe;cwE(vYTNi-%-z-iM zsoQ6B$#1_F7-@&&cGR?h>kNR5I44I4uE~k~e~#?^#s;+wx)p0)-1u~CsfX{`wB`ip zfbIMTwfzA~L$YR1pFUk;z5iV1ntkP49pR_F__GMc_$_fT=8Ln-iW<-n0R>#4rQYJ7 z?)%@a1KMlwvWxHaKgTC47%Xu)@fgH!@ zY4wY5yz)u|V`Ice_?DyoKq$X*i)s0>W5<^G>T|bRTQ4uQAv)jzmss(Ib2op&i-teH z4O{Bs6&Igh>PgR@KYz3MhkiTZ6+r0iCjaT6vEVkqjmG)=Z;K16yY6=Clon6wg{I$J z?D>TF(%F@&APm0v#w4w&uC4tZYWd|7M#sb;mAXYwC-}qfz05Crfda!NmSABq&G^6y zJ$vz@dhx}8T{khYUTW8E23v`Xo0IhmPfF{_N|J_ym zSL_G~fJ<&U$i^>uol8qg7d>44ZcNJ^by!cBoq~F{49~xw^t+3_`Dqi~di5oD_1kr0 z1|xgv(xvayuV40e<%;D?Y|mz$>G6R@U-w^Y|IHHSI1tZg!YBYa0H+W^@So)NdxJb* zj5)Ca%zJZM>5}_x0PyM(w|vXit&46%6CKxz@ggG=6D0%$Eb=1z$BS-@&Q9ts@%`T& zPo3e;*GsI#pSP-dmo!R{#xLRRemS1lAXBRlI7fz?5FCB&=~>cWV=$OS_Z0kfQLJ>x5}WXSWbjhA zWXUJNu$ZNu1T%rW7T>X?HJg~2x))nkggO7tSTDTZeFMOJ%~mR}R0Tzsr5;hu$oTy2 z+Gqc??0w(aH6Q(Qag{G+hc3uM!}}6;2i;At)%w*l1hea80F&;Ma1Sp;jBdQv#iP4C%d}f`eApbQ6oVPj4JnHyE>jy9jWlHPI;i7@3bVBKU|_cu0DlU1p&B}O?d~{{(wqGK?$s)kFNDpvazr1R12VB6@T0`| z_-YsnbS4v`Ya^fyLHpFmQCDp#sfUoG2s@qih2rO8pAvR3Zim^QkVv z*S6{Ze-3xToS7t0`Ua|BRDAqFa8sxk`_*A7kfD;4mBqu1B@WC#_z|opP~k!3iI*Gb zl`e(aFaz)sq^JnsJ7gE*L8Vfwr;0(|;{O)>w7K952yVJ*n-Jiokl!FS8U8?o)1R2K zs;VFk7A--fPgcO zjVvcF0ig!rrW}SdlOqs~1_5{pocUzvmH>SYj(WfjBn@t+gV}~jOp?&X9RLJ-Xsk({Rz+PhbrgvSMOcro)y0(c5}SRiOOU@#Aq0plMZw}FnHgD^^pOk?{8_Z$fH z_c!P+3qTPSAc06V>(#4cm|7VAR^Qs1q?v1j@rAyp=TMu#LRAMo8k%PeJoIQTu7qjU zH7jAjQ7q&_pdMuigPYU=9OE$j@b@314*9IX+&nz^S=uyAhr_`;19Ew1^+tIJ+n_=M z5F}up#xP0A)61&?s>06!WpWReh7nHAg(Y6Q<2*grHPqKzQC;-)^#QTsgEs~!V7&iR zl!)W_-C&XLA47zJn&d^eb;yw@lpc>CpoT zv|+XypLqn5m!ChepdftWzRL{T=E2_iO2>rz#QoU`^xHL{iTgEC5?C&3{h_;vBBTcee^^;V=#P#psR$ERVe^py7JTTy7}P-GV`BD^?I9yX zsO-_WO{M^}!h{+NXh1mhE58xM6>&0ue5|MlsGss&2_8Fgy8j(0*zDUE^Hzcb2C6L@ zx)K%Wl7Pkmz;GWFyf(Cfiau_iRG>leqg@OsAJse<(pUBWj!o!Ig819OU`?3Dp;ZE= z+OXr0l>_*X3o*^X0KV#T-kT)>elYFp0Wmo(EiD4C_<}P!yB$nwAdhy6Wu@Q&^3=@K z*glMkrgZ?agS24}F+~7@3ups3(WwxE?kFUq1UE!5{aOiVskJWV6-un>l90SotaG;r zo6lguv}qfVKk~CUZHS0I0cg5_-Ot z`1#)lgOTDMO9q2bH(4qeEb?X*{vX0l)Q={^^E~rc=JrA*;lHh?YNlz}k^y0T{ny1} z&hMo8e+zmngz^nw^`J|6(HH4=Tje}jdSHpQSUBwqKq-_Z7JuDuNe#;8UqCPM5<9){ zGtKyb6(3mlkYJ*jZxz*I_CRhytKvt#@4tC?d)}v|PKUvN%fq*9S#(SP$-|9}6`+u@ zRN@20wEyu0zP>}hWHJj?s3lhY&mt%-F8aDVb-tbTZzs^2QtM?}S6?3uHOECa`=2xf zsjM%)jR3`{s4Sg8f)!gVO5-4oN@V^g6ZsOkAD)IYC}oaQuT@jo08fZKGytt_=r z-!!Oy1K|`Q^!|nYnBM>Jq#uqVj88mx=uqimFaqk5KzIh|`7N3fqU#Cu#3hXoG!88K zWea%5!Zv5^0^9;(^d*w6?{Nz#k1cU>&Mg}sxxFZE0Ry?L7IQLJZhilhVsF+Zk|3@X zP_SL}lpurz0=^|-QYX=f#SW?WFYvr~x#dEMiu?k4auI5GSR>=(Qr2S4$`y;QJ*pEZ zD+ez2`1zX9>qftvvWsvI*;~qOzo&Ve=_j z>_tiz0qo~F^2;{W{cL7SiD=eI5P2;{b5c_PrDT3&v0{^`vLn*-JX)Cdm2z`;-2If&Nr$iz^2<8xX3x z|J!=PK;0$Lnx;;M`H14@_H$&R&%QDIn;`g4G5nXSwSiS=$@<>XHLLHJ^1oF#CQ)@cPK?&WXe0;RLJfXwzkSQYz!07(*(l;{CP zoO#~yV!106(j}lq4YQCJ+s!ah;Ub6rmnFPbgiDYOA|?vb`L2_)_z2j-qW7Nqbr~tU zH@ptAOlu%_CZ?q+A{ioJp#VQ9#yQrn=xr3s3elVYLAkZrNJIg{9)YTWQ25jvn zuaKz+g4frH=>R{!y3ga#&;)B=IX>|pbO3lt(Vb+TEr3Ow|A@am_^s36hiW1N?P+-b zgImtPuQ@gd3`Zb4YKEDz!@y}e>_V7ChyZFj4rW(xoppx47Vz}xAt2KMlZ_!$MOm3s zKmb2J(Q0188Z74OLKr)>=NZsVHtziE6q`?+1iqyXZVlhH51W7pDe~v|*qMcaZ9ft% z?q3ePZV1z_0rC+}=;-KOt%~T4z$HW6Cry}d?0cIoL7>A)0cL2Y(nNsv1bEPZnTEQ$ zIwLO47FZXn(%{8Xmr16G{cx=hL!F`2L_Ig+R|7@#gTm&q&8c-7a31v6*3 zaMcNV&K9>Q6r8Ij5NWAIS=U7HT++e3nDsl3^ryWPUie$o0#U&t-)Q@>9@a7c*Rvcz z!~&Ui{Wt}VsUQd7oX!En z!;rjF1{AX0pyUKfGO^VrWH*4a`e1Z5BHwKTxD9kv3}7O2(A~xI6=1_uqmE=vb@K% zPn(FhDXoD&FN^2U{ty#h=<;PjX*|g;LLVNY`?uS<|}M=c;pu3b2Trm`~aFjQ%Or0Qj@;z4My@ z)(QTgBh7{8plMZej+IS<{V`YTbjcq)1(VN3sf4|-n$|f!qI=dm%fkt`hO6u@Ix$UC z1`e_H9Bo24i!eZRveu$#%i z3R1@v&gLMIJ~e@MyQu4GoC^#9wQ6prs@ep|0;yMZb(8fY zoHGOh*U)R4y6ROGg;p^X@|DcQS#JC+6%o3W+`95re9U- z&ce-8gSZ-zOL57pB0~hx{==F%^}g3k+Hx9R9i|dwy&J$~3!`>gH2|`epr|DH{N`8`+Ylh107Dj(PuZhj7-htkYl*IS}^kn+B0L;NI4R>mQ#S z1yDXN$mJ_yLay3zxjK^Wf#{z6nbGoKA?w6~@pRlFVPOrBc4-88gIzg=d3l#1cw%M7 zNJ}@s-8775KQWM2NA3x^X#})Y-)KZ~Mtt^4u{kFU+3l@&oI5<8iY2-yZS2RKJ*{Yhn%w}wPXrttos?mql@o!Ajt2)g-{DXm$X&Nlp!M04Cp(cOeI!Hw zY(?bb54O_-nOI=V0IB$KRj(iLcR@@PTYe<>qA!8&kCM7G=gzsmN==2iD@;r=Ko@l& z>H}tJ-C~kStd-(X0yZQ1K6D$-ojXSa?WPy`5u$sJ7OnWR4BSl|)DrUNW~XI%RviJB zzD=85mzM*Lf(HtDFWI8=4A7jAY%Som2Qa}W?O%cGnIAv_iiZ|<)@IFPRbn`?*%2Qk zz7##uRbmX43e-B_9~30PzD!0Ei~%$wbHOx5kcbe@XUQt^Wa1)!*y{s?b48TLbr-!q z;r@aVRQdd?dP!CRFQDQFh__H0=Lsm%*i&H!LFBoI+YNyN64_#DX^F}LsQ=CGWImY@ zh!2WjvdJ$>eysOwc%cQ^Q3bb&m*!8k)z=X0TT!)4~c-!6iWIB*MO+i#H;JzOtAGPjN zL5I$^zcm753nF0y zQkOKq$ zP@v7qZ!o);&5K1)9#o;JrLr#1p~8#enaU$|$E%t}`#$oQ6+p zT2}D+5~2zX4GgM*rEw-AC@2V>X!LHNLf5n%5F)IgWe-84gBloa_hl_uGDCDqiv&#Lv%l+tPQNRi1~Y8Z-P@}HL6|c55!5;)n9p(Ilko>q2n z$O-uSZyimG(tVACZMk6@ye!{bg)YI8TEvCRy9(Dp4I;KisBtFEQE_v9X0CfSy+964 z6^xWJFvYB4mDC}Wu{B61NJIAD=q1o@fL;KINsUlqOoI0>QI?XDA{-^xxE@wd`!KHW z4WV4wtE4XDE1*Z$2y!JxP+UZX$41z2lYx%H4maHDwu*WQd;6)Qu6mkj266;J!d?q^ z$GH>g|F{k7EHBYcpqTWc)zW_te7&PWI#G@uNVpQjFE0p21=`__H1q;aV44vvZp zPfSZ%Mn*V!s11t9GGa6bYbDH;KPK#b#8-K1HCb1G+76ExVY-3dLwYq9|K*01k|z(DbAS* zpnZKpa&Slz5t_1$z#nF4OMtlmB1bY*1h^mu_uDUB`^J5}_=Mw0Hq^oc+7p=&`=XCv z9dq0Q9d*F_g>NM|c1(w(upmg=Ma@?cM4jQ3%7d00D#W9Qd+ipmnWUlq$XS|ro#?(NPqBPGE{b?+U?8-7!L zN*^kn)Ye|nIMf>h@w$4JStM{C_-qGT8LCFm{y>*d4gBb5z&Uwin;2=qX8m2XHUnzIkXF9VA$0sKzs~>Blpfw%p2PpUf;s{)f z+8qVMtKiIFd{L~6v<@Kwt1AoOnwO&tcN`=RI|XUlJVkr})c^6`lPQNb-U4g{=T;>B zMPsA`v}RMyk?gNqP#T5Oq@eHv5Q3qBMCX0jl8sZc&m(?@f`T8KfMp2XK&TrNHTt36 z(YHe(u!N4?8DOtn$Iqh}jeT~Q9)!*=fGb)fQu5*P zUdlu5nKST!MnX-L34WqndsZH09|a>`Hx0;y8i!6ddeoeUoP}>RoDT?35Mu*uBCsgd zP4j64#z5FBjw_%hicVx#8A8jm!&G+=1?gJ}bv-(ccpS75AJ_WIFkuRWI+%3~wGfR( z!HamH#xm2a&-Y{`yb$LDt_{*(p1WXAL6{ z?SjfUtz@WZ8v!SURaEAh4)m@g5A8e*Q>#tqI`1&@nzm+b)v^S?k1Q)VYB(S6rtCmE z0uVL*TB&cf37V4t{0(v3acJ6On2=&Er1y9Pyc&QjQnefh_?)N8wKLk5u`gTJxe108 zFMFo_1OmMyx&6mPsDvn1Q?cp(ROmfS)}HyY1JQa=cQU9(&-G>CodM87-)RJ<5@ol@ zR-p^yJKvEC9e1cb7%BKe>oE!3UIYA>yRgIc49=32oY{wxQIs|#KLH(Ocyw<;DBPw6 ze+->LqLg-RW&wp8Mcz~pqPBu_Z451k)Y_l|4ouGHd%M$sT7gu#>S2p0AssWZZyPo- z!nOeNL4>Ha5&8&Gv$Fy8R7OUEqsyzE&=76{e~hkX%%Wu`&%cLOwQ#e&w25m2)Z zSh*LNZ9lB~44j$)G=MrlT1o*(g6sHk1>Z9g(!iIPpLqvMgls) Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 13/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 1b421e6f334..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -19,8 +19,8 @@ var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray_head", - Image: "my_image", + Name: "ray-head", + Image: "blah", VolumeMounts: []v1.VolumeMount{ { Name: "hostPath", From ba8bb562dd326f2ac66394a35201fbc74d59b3c5 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 10:58:43 +0100 Subject: [PATCH 14/56] rebased to current --- apiserver/pkg/model/volumes_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..a094224e777 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,11 +15,12 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head", + Name: "ray-head-test", Image: "blah", VolumeMounts: []v1.VolumeMount{ { From 262b3f4cc9f27113474327e0cc0242ee272e9d91 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 17 Aug 2023 15:15:07 +0100 Subject: [PATCH 15/56] restoring PB numbers --- proto/cluster.proto | 16 ++++++++-------- proto/go_client/cluster.pb.go | 32 ++++++++++++++++---------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 827577b38e1..9ef278f63d0 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -221,22 +221,22 @@ message HeadGroupSpec { string service_type = 3; // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 4; + bool enableIngress = 11; // Optional. The ray start params of head node group. - map ray_start_params = 5; + map ray_start_params = 4; // Optional. The volumes mount to head pod - repeated Volume volumes = 6; + repeated Volume volumes = 5; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 7; + string service_account = 6; // Optional. image pull secret used by head pod - string image_pull_secret = 8; + string image_pull_secret = 7; // Optional. Environment variables for head pod - map environment = 9; + map environment = 8; // Optional. Annotations for the head pod - map annotations = 10; + map annotations = 9; // Optional. Labels for the head pod - map labels = 11; + map labels = 10; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 9f5952e72f8..6b8e12c4da7 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -970,22 +970,22 @@ type HeadGroupSpec struct { ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + EnableIngress bool `protobuf:"varint,11,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` // Optional. The ray start params of head node group. - RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // 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"` + ImagePullSecret string `protobuf:"bytes,7,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 map[string]string `protobuf:"bytes,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // 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"` + Annotations map[string]string `protobuf:"bytes,9,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 - Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -1530,32 +1530,32 @@ var file_cluster_proto_rawDesc = []byte{ 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, + 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 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, + 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 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, + 0x65, 0x73, 0x18, 0x05, 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, + 0x75, 0x6e, 0x74, 0x18, 0x06, 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, + 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x07, 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, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x08, 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, + 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 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, + 0x73, 0x18, 0x0a, 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, From 06ccd0974eddd21e9daf6380eea698e26863aafe Mon Sep 17 00:00:00 2001 From: Laurentiu Bradin <109964136+z103cb@users.noreply.github.com> Date: Thu, 17 Aug 2023 21:51:28 +0300 Subject: [PATCH 16/56] Bump the golangci-lint version in the api server makefile (#1342) Bump the golangci-lint version in the api server makefile --- apiserver/Makefile | 2 +- apiserver/pkg/util/cluster.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/apiserver/Makefile b/apiserver/Makefile index 0462d9257cd..b02a76e2a96 100644 --- a/apiserver/Makefile +++ b/apiserver/Makefile @@ -122,7 +122,7 @@ KIND ?= $(REPO_ROOT_BIN)/kind KUSTOMIZE_VERSION ?= v3.8.7 GOFUMPT_VERSION ?= v0.3.1 GOIMPORTS_VERSION ?= latest -GOLANGCI_LINT_VERSION ?= v1.50.1 +GOLANGCI_LINT_VERSION ?= v1.54.1 KIND_VERSION ?= v0.19.0 KUSTOMIZE_INSTALL_SCRIPT ?= "https://raw.githubusercontent.com/kubernetes-sigs/kustomize/master/hack/install_kustomize.sh" diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index b251b9b647e..a99c676fd7a 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -597,8 +597,8 @@ func NewComputeTemplate(runtime *api.ComputeTemplate) (*v1.ConfigMap, error) { func GetNodeHostIP(node *v1.Node) (net.IP, error) { addresses := node.Status.Addresses addressMap := make(map[v1.NodeAddressType][]v1.NodeAddress) - for i := range addresses { - addressMap[addresses[i].Type] = append(addressMap[addresses[i].Type], addresses[i]) + for _, nodeAddress := range addresses { + addressMap[nodeAddress.Type] = append(addressMap[nodeAddress.Type], nodeAddress) } if addresses, ok := addressMap[v1.NodeInternalIP]; ok { return net.ParseIP(addresses[0].Address), nil From 1cbac51a77cf0c2729d1b29cb2a6356c6a951e2e Mon Sep 17 00:00:00 2001 From: Praveen Date: Thu, 17 Aug 2023 16:01:13 -0700 Subject: [PATCH 17/56] Documentation and example for running simple NLP service on kuberay (#1340) * add service yaml for nlp * Documentation fixes * Fix instructions * Apply suggestions from code review Co-authored-by: Kai-Hsun Chen Signed-off-by: Praveen * Fix tolerations comment * review comments * Update docs/guidance/stable-diffusion-rayservice.md Signed-off-by: Kai-Hsun Chen --------- Signed-off-by: Praveen Signed-off-by: Kai-Hsun Chen Co-authored-by: Kai-Hsun Chen --- docs/guidance/aws-eks-gpu-cluster.md | 2 +- docs/guidance/stable-diffusion-rayservice.md | 10 ++- docs/guidance/text-summarizer-rayservice.md | 69 ++++++++++++++++ .../samples/ray-service.text-sumarizer.yaml | 79 +++++++++++++++++++ 4 files changed, 158 insertions(+), 2 deletions(-) create mode 100644 docs/guidance/text-summarizer-rayservice.md create mode 100644 ray-operator/config/samples/ray-service.text-sumarizer.yaml diff --git a/docs/guidance/aws-eks-gpu-cluster.md b/docs/guidance/aws-eks-gpu-cluster.md index e77470403d2..f18998b3ae2 100644 --- a/docs/guidance/aws-eks-gpu-cluster.md +++ b/docs/guidance/aws-eks-gpu-cluster.md @@ -32,7 +32,7 @@ Create a GPU node group for Ray GPU workers. > **Note:** If you encounter permission issues with `kubectl`, follow "Step 2: Configure your computer to communicate with your cluster" in the [AWS documentation](https://docs.aws.amazon.com/eks/latest/userguide/getting-started-console.html#). -2. Please install the NVIDIA device plugin. +2. Please install the NVIDIA device plugin. Note: You don't need this if you used `BOTTLEROCKET_x86_64_NVIDIA` image in above step * Install the DaemonSet for NVIDIA device plugin to run GPU enabled containers in your Amazon EKS cluster. You can refer to the [Amazon EKS optimized accelerated Amazon Linux AMIs](https://docs.aws.amazon.com/eks/latest/userguide/eks-optimized-ami.html#gpu-ami) or [NVIDIA/k8s-device-plugin](https://github.com/NVIDIA/k8s-device-plugin) repository for more details. * If the GPU nodes have taints, add `tolerations` to `nvidia-device-plugin.yml` to enable the DaemonSet to schedule Pods on the GPU nodes. diff --git a/docs/guidance/stable-diffusion-rayservice.md b/docs/guidance/stable-diffusion-rayservice.md index 5421660476e..94532531972 100644 --- a/docs/guidance/stable-diffusion-rayservice.md +++ b/docs/guidance/stable-diffusion-rayservice.md @@ -21,7 +21,7 @@ kubectl apply -f ray-service.stable-diffusion.yaml This RayService configuration contains some important settings: -* Its `tolerations` for workers match the taints on the GPU node group. Without the tolerations, worker Pods won't be scheduled on GPU nodes. +* The `tolerations` for workers allow them to be scheduled on nodes without any taints or on nodes with specific taints. However, workers will only be scheduled on GPU nodes because we set `nvidia.com/gpu: 1` in the Pod's resource configurations. ```yaml # Please add the following taints to the GPU node. tolerations: @@ -34,6 +34,14 @@ This RayService configuration contains some important settings: ## Step 4: Forward the port of Serve +First get the service name from this command. + +```sh +kubectl get services +``` + +Then, port forward to the serve. + ```sh kubectl port-forward svc/stable-diffusion-serve-svc 8000 ``` diff --git a/docs/guidance/text-summarizer-rayservice.md b/docs/guidance/text-summarizer-rayservice.md new file mode 100644 index 00000000000..ca758de7687 --- /dev/null +++ b/docs/guidance/text-summarizer-rayservice.md @@ -0,0 +1,69 @@ +# Serve a text summarizer using RayService + +> **Note:** The Python files for the Ray Serve application and its client are in the [ray-project/serve_config_examples](https://github.com/ray-project/serve_config_examples) repo. + +## Step 1: Create a Kubernetes cluster with GPUs + +Follow [aws-eks-gpu-cluster.md](./aws-eks-gpu-cluster.md) or [gcp-gke-gpu-cluster.md](./gcp-gke-gpu-cluster.md) to create a Kubernetes cluster with 1 CPU node and 1 GPU node. + +## Step 2: Install KubeRay operator + +Follow [this document](../../helm-chart/kuberay-operator/README.md) to install the latest stable KubeRay operator via Helm repository. +Please note that the YAML file in this example uses `serveConfigV2`, which is supported starting from KubeRay v0.6.0. + +## Step 3: Install a RayService + +```sh +# path: ray-operator/config/samples/ +kubectl apply -f ray-service.text-sumarizer.yaml +``` + +This RayService configuration contains some important settings: + +* The `tolerations`` for workers allow them to be scheduled on nodes without any taints or on nodes with specific taints. However, workers will only be scheduled on GPU nodes because we set `nvidia.com/gpu: 1` in the Pod's resource configurations. + ```yaml + # Please add the following taints to the GPU node. + tolerations: + - key: "ray.io/node-type" + operator: "Equal" + value: "worker" + effect: "NoSchedule" + ``` + +## Step 4: Forward the port of Serve + +First get the service name from this command. + +```sh +kubectl get services +``` + +Then, port forward to the serve. + +```sh +kubectl port-forward svc/text-summarizer-serve-svc 8000 +``` + +Note that the RayService's Kubernetes service will be created after the Serve applications are ready and running. This process may take approximately 1 minute after all Pods in the RayCluster are running. + +## Step 5: Send a request to the text_summarizer model + +```sh +# Step 5.1: Download `text_summarizer_req.py` +curl -LO https://raw.githubusercontent.com/ray-project/serve_config_examples/master/text_summarizer/text_summarizer_req.py + +# Step 5.2: Send a request to the Summarizer model. +python text_summarizer_req.py +# Check printed to console +``` + +## Step 6: Delete your service + +```sh +# path: ray-operator/config/samples/ +kubectl delete -f ray-service.text-sumarizer.yaml +``` + +## Step 7: Uninstall your kuberay operator + +Follow [this document](../../helm-chart/kuberay-operator/README.md) to uninstall the latest stable KubeRay operator via Helm repository. \ No newline at end of file diff --git a/ray-operator/config/samples/ray-service.text-sumarizer.yaml b/ray-operator/config/samples/ray-service.text-sumarizer.yaml new file mode 100644 index 00000000000..fbf9d3e6464 --- /dev/null +++ b/ray-operator/config/samples/ray-service.text-sumarizer.yaml @@ -0,0 +1,79 @@ +apiVersion: ray.io/v1alpha1 +kind: RayService +metadata: + name: text-summarizer +spec: + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. + serveConfigV2: | + applications: + - name: text_summarizer + import_path: text_summarizer.text_summarizer:deployment + runtime_env: + working_dir: "https://github.com/ray-project/serve_config_examples/archive/refs/heads/master.zip" + rayClusterConfig: + rayVersion: '2.6.3' # Should match the Ray version in the image of the containers + ######################headGroupSpecs################################# + # Ray head pod template. + 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: + dashboard-host: '0.0.0.0' + # Pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray-ml:2.6.3 + ports: + - containerPort: 6379 + name: gcs + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + - containerPort: 8000 + name: serve + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + resources: + limits: + cpu: "2" + memory: "8G" + requests: + cpu: "2" + memory: "8G" + volumes: + - name: ray-logs + emptyDir: {} + workerGroupSpecs: + # The pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 10 + groupName: gpu-group + rayStartParams: {} + # Pod template + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray-ml:2.6.3 + resources: + limits: + cpu: 4 + memory: "16G" + nvidia.com/gpu: 1 + requests: + cpu: 3 + memory: "12G" + nvidia.com/gpu: 1 + # Please add the following taints to the GPU node. + tolerations: + - key: "ray.io/node-type" + operator: "Equal" + value: "worker" + effect: "NoSchedule" From 8be0a21dd1f0f4ce7298b54f4191e98a7ca9a558 Mon Sep 17 00:00:00 2001 From: Laurentiu Bradin <109964136+z103cb@users.noreply.github.com> Date: Sat, 19 Aug 2023 01:44:36 +0300 Subject: [PATCH 18/56] Removed use of the of BUILD_FLAGS in apiserver makefile (#1336) Removed use of the of BUILD_FLAGS in apiserver makefile --- apiserver/Makefile | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/apiserver/Makefile b/apiserver/Makefile index b02a76e2a96..7745dd129e5 100644 --- a/apiserver/Makefile +++ b/apiserver/Makefile @@ -62,17 +62,8 @@ lint: golangci-lint fmt vet fumpt imports ## Run the linter. ##@ Build -COMMIT := $(shell git rev-parse --short HEAD) -VERSION := $(shell git describe --tags $(shell git rev-list --tags --max-count=1)) -DATE=$(shell date -u +'%Y-%m-%dT%H:%M:%SZ') -REPO=github.com/ray-project/kuberay - -BUILD_FLAGS ?= -ldflags="-X '${REPO}/apiserver/cmd/version.Version=$(VERSION)' \ - -X '${REPO}/apiserver/cmd/version.gitCommit=$(COMMIT)' \ - -X '${REPO}/apiserver/cmd/version.buildDate=$(DATE)'" - build: fmt vet fumpt imports lint ## Build api server binary. - go build $(BUILD_FLAGS) -o ${REPO_ROOT_BIN}/kuberay-apiserver cmd/main.go + go build -o ${REPO_ROOT_BIN}/kuberay-apiserver cmd/main.go run: fmt vet fumpt imports lint ## Run the api server from your host. go run -race cmd/main.go From e79e0b9c429a5c8a1b206c7504c9ed8a1a608dfc Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 21 Aug 2023 13:46:20 -0700 Subject: [PATCH 19/56] [GCS FT][Refactor] Redefine the behavior for deleting Pods and stop listening to Kubernetes events (#1341) Redefine the behavior for deleting Pods and stop listening to Kubernetes events --- .github/workflows/test-job.yaml | 6 +- apiserver/DEVELOPMENT.md | 2 +- .../controllers/ray/raycluster_controller.go | 191 +++------ .../ray/raycluster_controller_fake_test.go | 385 ++++++++++-------- tests/compatibility-test.py | 68 ++-- tests/config/ray-cluster.ray-ft.yaml.template | 4 +- tests/framework/utils.py | 4 + tests/kuberay_utils/utils.py | 107 ++--- 8 files changed, 358 insertions(+), 409 deletions(-) diff --git a/.github/workflows/test-job.yaml b/.github/workflows/test-job.yaml index 65e7e428969..7b73d784a17 100644 --- a/.github/workflows/test-job.yaml +++ b/.github/workflows/test-job.yaml @@ -41,7 +41,7 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version - version: v1.50.1 + version: v1.54.1 # Optional: working directory, useful for monorepos working-directory: ./ray-operator @@ -65,7 +65,7 @@ jobs: - name: Run linter against apiserver uses: golangci/golangci-lint-action@v2 with: - version: v1.50.1 + version: v1.54.1 working-directory: ./apiserver args: --timeout=3m skip-go-installation: true @@ -74,7 +74,7 @@ jobs: - name: Run linter against cli uses: golangci/golangci-lint-action@v2 with: - version: v1.50.1 + version: v1.54.1 working-directory: ./cli args: --timeout=3m skip-go-installation: true diff --git a/apiserver/DEVELOPMENT.md b/apiserver/DEVELOPMENT.md index c3b3862aa61..15318bcbd0e 100644 --- a/apiserver/DEVELOPMENT.md +++ b/apiserver/DEVELOPMENT.md @@ -21,7 +21,7 @@ Typing `make dev-tools` will download and install all of them. The `make clean-d | Software | Version | Link | | :------- | :------: | -----------------------------------------------------------------------:| | kind | v0.19.0 | [Install](https://kind.sigs.k8s.io/docs/user/quick-start/#installation) | -| golangci-lint | v1.50.1 | [Install](https://golangci-lint.run/usage/install/) | +| golangci-lint | v1.54.1 | [Install](https://golangci-lint.run/usage/install/) | | kustomize | v3.8.7 | [install](https://kubectl.docs.kubernetes.io/installation/kustomize/) | | gofumpt | v0.3.1 | To install `go install mvdan.cc/gofumpt@v0.3.1` | | goimports | latest | To install `go install golang.org/x/tools/cmd/goimports@latest` | diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 12d82eb75b2..d36600d839f 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -9,8 +9,6 @@ import ( "strings" "time" - "sigs.k8s.io/controller-runtime/pkg/event" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler" @@ -35,11 +33,9 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" controller "sigs.k8s.io/controller-runtime/pkg/controller" "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil" - "sigs.k8s.io/controller-runtime/pkg/handler" "sigs.k8s.io/controller-runtime/pkg/manager" "sigs.k8s.io/controller-runtime/pkg/predicate" "sigs.k8s.io/controller-runtime/pkg/reconcile" - "sigs.k8s.io/controller-runtime/pkg/source" ) var ( @@ -104,12 +100,6 @@ type RayClusterReconciler struct { func (r *RayClusterReconciler) Reconcile(ctx context.Context, request ctrl.Request) (ctrl.Result, error) { var err error - // Try to fetch the Event instance - event := &corev1.Event{} - if err = r.Get(ctx, request.NamespacedName, event); err == nil { - return r.eventReconcile(ctx, request, event) - } - // Try to fetch the RayCluster instance instance := &rayv1alpha1.RayCluster{} if err = r.Get(ctx, request.NamespacedName, instance); err == nil { @@ -126,77 +116,6 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, request ctrl.Reque return ctrl.Result{}, client.IgnoreNotFound(err) } -func (r *RayClusterReconciler) eventReconcile(ctx context.Context, request ctrl.Request, event *corev1.Event) (ctrl.Result, error) { - var unhealthyPod *corev1.Pod - pods := corev1.PodList{} - - // we only care about pod events - if event.InvolvedObject.Kind != "Pod" || event.Type != "Warning" || event.Reason != "Unhealthy" || - !strings.Contains(event.Message, "Readiness probe failed") { - // This is not supposed to happen since we already filter events in the watch - msg := fmt.Sprintf("unexpected event, we should have already filtered these conditions: %v", event) - r.Log.Error(fmt.Errorf(msg), msg, "event", event) - return ctrl.Result{}, nil - } - - _ = r.Log.WithValues("event", request.NamespacedName) - - options := []client.ListOption{ - client.MatchingFields(map[string]string{podUIDIndexField: string(event.InvolvedObject.UID)}), - client.InNamespace(event.InvolvedObject.Namespace), - client.MatchingLabels(map[string]string{common.RayNodeLabelKey: "yes"}), - } - - if err := r.List(ctx, &pods, options...); err != nil { - return ctrl.Result{}, err - } - - if len(pods.Items) == 0 { - r.Log.Info("no ray node pod found for event", "event", event) - return ctrl.Result{}, nil - } else if len(pods.Items) > 1 { - // This happens when we use fake client - r.Log.Info("are you running in test mode?") - for _, pod := range pods.Items { - if pod.Name == event.InvolvedObject.Name { - unhealthyPod = &pod - break - } - } - } else { - r.Log.Info("found unhealthy ray node", "pod name", event.InvolvedObject.Name) - unhealthyPod = &pods.Items[0] - } - - if unhealthyPod.Annotations == nil { - r.Log.Info("The unhealthy ray node not found", "pod name", event.InvolvedObject.Name) - return ctrl.Result{}, nil - } - - if enabledString, ok := unhealthyPod.Annotations[common.RayFTEnabledAnnotationKey]; ok { - if strings.ToLower(enabledString) != "true" { - r.Log.Info("FT not enabled skipping event reconcile for pod.", "pod name", unhealthyPod.Name) - return ctrl.Result{}, nil - } - } else { - r.Log.Info("HAEnabled annotation not found", "pod name", unhealthyPod.Name) - return ctrl.Result{}, nil - } - - if !utils.IsRunningAndReady(unhealthyPod) { - if v, ok := unhealthyPod.Annotations[common.RayNodeHealthStateAnnotationKey]; !ok || v != common.PodUnhealthy { - updatedPod := unhealthyPod.DeepCopy() - updatedPod.Annotations[common.RayNodeHealthStateAnnotationKey] = common.PodUnhealthy - r.Log.Info("mark pod unhealthy and need for a rebuild", "pod", unhealthyPod) - if err := r.Update(ctx, updatedPod); err != nil { - return ctrl.Result{}, err - } - } - } - - return ctrl.Result{}, nil -} - func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request ctrl.Request, instance *rayv1alpha1.RayCluster) (ctrl.Result, error) { // Please do NOT modify `originalRayClusterInstance` in the following code. originalRayClusterInstance := instance.DeepCopy() @@ -416,22 +335,37 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv // Reconcile head Pod if len(headPods.Items) == 1 { headPod := headPods.Items[0] - r.Log.Info("reconcilePods", "head pod found", headPod.Name) + r.Log.Info("reconcilePods", "Found 1 head Pod", headPod.Name) + // TODO (kevin85421): Consider deleting a head Pod if its Ray container restarts excessively, as this + // might suggest an unhealthy Kubernetes node. Deleting and then recreating the head Pod might allow + // it to be scheduled on a different node. However, it's aggressive to delete a head Pod that is not + // in a terminated state (i.e., `Failed` or `Succeeded`). We should only delete a head Pod when GCS + // fault tolerance is enabled, and drain the head Pod before deleting it. if headPod.Status.Phase == corev1.PodRunning || headPod.Status.Phase == corev1.PodPending { - r.Log.Info("reconcilePods", "head pod is up and running... checking workers", headPod.Name) - } else if headPod.Status.Phase == corev1.PodFailed && strings.Contains(headPod.Status.Reason, "Evicted") { - // Handle evicted pod - r.Log.Info("reconcilePods", "head pod has been evicted and controller needs to replace the pod", headPod.Name) + r.Log.Info("reconcilePods", "The head pod is Running or Pending... checking workers", headPod.Name) + } else { + if headPod.Spec.RestartPolicy == corev1.RestartPolicyAlways { + // Based on my observation, a Pod with `RestartPolicy: Always` will never be in the terminated states (i.e., `Failed` or `Succeeded`). + // However, I couldn't find any well-defined behavior in the Kubernetes documentation, so I can't guarantee that the status transition + // from `Running` to `Failed / Succeeded` and back to `Running` won't occur when we kill the main process (i.e., `ray start` in KubeRay) + // in the head Pod. Therefore, I've added this check as a safeguard. + message := fmt.Sprintf( + "The status of the head Pod %s is %s. However, KubeRay will not delete the Pod because its restartPolicy is set to 'Always' "+ + "and it should be able to restart automatically.", headPod.Name, headPod.Status.Phase) + r.Log.Info(message) + return fmt.Errorf(message) + } + message := fmt.Sprintf("The status of the head Pod %s is %s which is a terminal state. It is not expected that the head pod ever be in a terminal state, so KubeRay will delete the Pod and recreate the head Pod in the next reconciliation.", headPod.Name, headPod.Status.Phase) + r.Log.Info(message) if err := r.Delete(ctx, &headPod); err != nil { return err } - } else { - return fmt.Errorf("head pod %s is not running nor pending", headPod.Name) + r.Recorder.Eventf(instance, corev1.EventTypeNormal, "Deleted", "Deleted head Pod %s; status: %s", headPod.Name, headPod.Status.Phase) + return fmt.Errorf(message) } - } - if len(headPods.Items) == 0 || headPods.Items == nil { - // create head pod - r.Log.Info("reconcilePods", "creating head pod for cluster", instance.Name) + } else if len(headPods.Items) == 0 { + // Create head Pod if it does not exist. + r.Log.Info("reconcilePods", "Found 0 head Pods; creating a head Pod for the RayCluster.", instance.Name) common.CreatedClustersCounterInc(instance.Namespace) if err := r.createHeadPod(ctx, *instance); err != nil { common.FailedClustersCounterInc(instance.Namespace) @@ -439,11 +373,12 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv } common.SuccessfulClustersCounterInc(instance.Namespace) } else if len(headPods.Items) > 1 { - r.Log.Info("reconcilePods", "more than 1 head pod found for cluster", instance.Name) + r.Log.Info("reconcilePods", fmt.Sprintf("Found %d head Pods; deleting extra head Pods.", len(headPods.Items)), instance.Name) + // TODO (kevin85421): In-place update may not be a good idea. itemLength := len(headPods.Items) for index := 0; index < itemLength; index++ { if headPods.Items[index].Status.Phase == corev1.PodRunning || headPods.Items[index].Status.Phase == corev1.PodPending { - // Remove the healthy pod at index i from the list of pods to delete + // Remove the healthy pod at index i from the list of pods to delete headPods.Items[index] = headPods.Items[len(headPods.Items)-1] // replace last element with the healthy head. headPods.Items = headPods.Items[:len(headPods.Items)-1] // Truncate slice. itemLength-- @@ -455,18 +390,6 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv return err } } - } else { - // we have exactly one head pod running - if headPods.Items[0].Annotations != nil { - if v, ok := headPods.Items[0].Annotations[common.RayNodeHealthStateAnnotationKey]; ok && v == common.PodUnhealthy { - if err := r.Delete(ctx, &headPods.Items[0]); err != nil { - return err - } - r.Log.Info(fmt.Sprintf("need to delete unhealthy head pod %s", headPods.Items[0].Name)) - // we are deleting the head pod now, let's reconcile again later - return nil - } - } } if ForcedClusterUpgrade { @@ -531,21 +454,36 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv return err } - // delete the worker pod if it is marked unhealthy + // Delete unhealthy worker Pods + numDeletedUnhealthyWorkerPods := 0 for _, workerPod := range workerPods.Items { - if workerPod.Annotations == nil { - continue - } - if v, ok := workerPod.Annotations[common.RayNodeHealthStateAnnotationKey]; ok && v == common.PodUnhealthy { - r.Log.Info(fmt.Sprintf("deleting unhealthy worker pod %s", workerPod.Name)) + // TODO (kevin85421): Consider deleting a worker Pod if its Ray container restarts excessively, + // as this could suggest an unhealthy Kubernetes node. Deleting and then recreating the worker Pod + // might allow it to be scheduled on a different node. Compared to deleting a head Pod, removing a + // worker Pod is less aggressive and aligns more closely with the behavior of the Ray Autoscaler. + // Nevertheless, we should still carefully drain the node before deleting the worker Pod. Enabling + // GCS fault tolerance might not be necessary when deleting worker Pods. Note that the Ray Autoscaler + // will not delete any worker Pods that have never been registered with the Ray cluster. Therefore, + // we may need to address the Ray Autoscaler's blind spots. + + // TODO (kevin85421): We may need to allow users to configure how many `Failed` or `Succeeded` Pods should be kept for debugging purposes. + if workerPod.Spec.RestartPolicy != corev1.RestartPolicyAlways && !isPodRunningOrPendingAndNotDeleting(workerPod) { + // If the Pod's status is `Failed` or `Succeeded`, the Pod will not restart and we can safely delete it. + numDeletedUnhealthyWorkerPods++ + r.Log.Info(fmt.Sprintf("The worker Pod %s status is %s. KubeRay will delete the Pod because the status is not Running or Pending. ", workerPod.Name, workerPod.Status.Phase)) if err := r.Delete(ctx, &workerPod); err != nil { return err + } else { + r.Recorder.Eventf(instance, corev1.EventTypeNormal, "Deleted", "Deleted worker Pod %s; status: %s", workerPod.Name, workerPod.Status.Phase) } - // we are deleting one worker pod now, let's reconcile again later - return nil } } + // If we delete unhealthy Pods, we will not create new Pods in this reconciliation. + if numDeletedUnhealthyWorkerPods > 0 { + return fmt.Errorf("Delete %d unhealthy worker Pods.", numDeletedUnhealthyWorkerPods) + } + // Always remove the specified WorkersToDelete - regardless of the value of Replicas. // Essentially WorkersToDelete has to be deleted to meet the expectations of the Autoscaler. deletedWorkers := make(map[string]struct{}) @@ -813,31 +751,6 @@ func (r *RayClusterReconciler) SetupWithManager(mgr ctrl.Manager, reconcileConcu predicate.LabelChangedPredicate{}, predicate.AnnotationChangedPredicate{}, ))). - Watches(&source.Kind{Type: &corev1.Event{}}, - &handler.EnqueueRequestForObject{}, - builder.WithPredicates(predicate.Funcs{ - CreateFunc: func(e event.CreateEvent) bool { - if eventObj, ok := e.Object.(*corev1.Event); ok { - if eventObj.InvolvedObject.Kind != "Pod" || eventObj.Type != "Warning" || - eventObj.Reason != "Unhealthy" || !strings.Contains(eventObj.Message, "Readiness probe failed") { - // only care about pod unhealthy events - return false - } - return true - } - return false - }, - UpdateFunc: func(e event.UpdateEvent) bool { - return false - }, - DeleteFunc: func(e event.DeleteEvent) bool { - return false - }, - GenericFunc: func(e event.GenericEvent) bool { - return false - }, - }), - ). Owns(&corev1.Pod{}). Owns(&corev1.Service{}) diff --git a/ray-operator/controllers/ray/raycluster_controller_fake_test.go b/ray-operator/controllers/ray/raycluster_controller_fake_test.go index bb3e1728159..dc06c1f345e 100644 --- a/ray-operator/controllers/ray/raycluster_controller_fake_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_fake_test.go @@ -17,7 +17,6 @@ package ray import ( "context" - "fmt" "os" "testing" "time" @@ -359,158 +358,6 @@ func setupTest(t *testing.T) { workerSelector = labels.NewSelector().Add(*instanceReq).Add(*groupNameReq) } -// TestReconcile_UnhealthyEvent tests the case where we have unhealthy events -// and we want to update the corresponding pods. -func TestReconcile_UnhealthyEvent(t *testing.T) { - setupTest(t) - - testPodName := "eventPod" - - // testPodEventName is the name of the event that will be created for testPodName - // The name of the event is generated by concatenating the pod name and a - // meaningless random string - testPodEventName := fmt.Sprintf("%s.15f0c0c5c5c5c5c5", testPodName) - - // add a pod in a different namespace - newPods := []runtime.Object{ - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodName, - UID: types.UID(testPodName), - Namespace: "ns2", - Labels: map[string]string{ - common.RayNodeLabelKey: "yes", - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: "newPod2", - UID: types.UID("newPod2"), - Namespace: namespaceStr, - Labels: map[string]string{ - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodName, - UID: types.UID(testPodName), - Namespace: namespaceStr, - Labels: map[string]string{ - common.RayNodeLabelKey: "yes", - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - Annotations: map[string]string{ - common.RayFTEnabledAnnotationKey: "true", - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - } - - testPods = append(testPods, newPods...) - - fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(testPods...).Build() - ctx := context.Background() - - podList := corev1.PodList{} - err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) - - assert.Nil(t, err, "Fail to get pod list") - assert.Equal(t, len(testPods)-1, len(podList.Items), "Init pod list len is wrong") - - testRayClusterReconciler := &RayClusterReconciler{ - Client: fakeClient, - Recorder: &record.FakeRecorder{}, - Scheme: scheme.Scheme, - Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), - } - - // add event for reconcile - err = fakeClient.Create(ctx, &corev1.Event{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodEventName, - Namespace: namespaceStr, - }, - InvolvedObject: corev1.ObjectReference{ - Kind: "Pod", - Name: testPodName, - Namespace: namespaceStr, - }, - Reason: "Unhealthy", - Type: "Warning", - Message: "Readiness probe failed", - }) - assert.Nil(t, err, "Fail to create event") - - if _, err := testRayClusterReconciler.Reconcile(ctx, ctrl.Request{ - NamespacedName: types.NamespacedName{ - Namespace: namespaceStr, - Name: testPodEventName, - }, - }); err != nil { - assert.Nil(t, err, "Fail to reconcile") - } - - err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) - assert.Nil(t, err, "Fail to get pod list") - - for _, pod := range podList.Items { - if pod.Name == testPodName && pod.Namespace == namespaceStr { - assert.Equal(t, pod.Annotations[common.RayNodeHealthStateAnnotationKey], - common.PodUnhealthy, "Pod annotation is wrong") - } - } - - for _, pod := range podList.Items { - assert.Equal(t, corev1.PodRunning, pod.Status.Phase, "Pod phase is wrong") - } -} - func TestReconcile_RemoveWorkersToDelete_RandomDelete(t *testing.T) { setupTest(t) @@ -862,7 +709,7 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate the deletion of 2 worker Pods. After the deletion, the number of worker Pods should be 4. + // Simulate the deletion of 1 worker Pod. After the deletion, the number of worker Pods should be 4. err = fakeClient.Delete(ctx, &podList.Items[3]) assert.Nil(t, err, "Fail to delete pod") @@ -874,8 +721,8 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), } - // Since the desired state of the workerGroup is 3 replicas, - // the controller will delete a worker Pod to reach the goal state. + // Since the desired state of the workerGroup is 3 replicas, the controller + // will delete a worker Pod randomly to reach the goal state. err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) assert.Nil(t, err, "Fail to reconcile Pods") @@ -890,7 +737,7 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { "Replica number is wrong after reconcile expect %d actual %d", expectReplicaNum, len(podList.Items)) } -func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { +func TestReconcile_Diff0_WorkersToDelete_OK(t *testing.T) { setupTest(t) // TODO (kevin85421): The tests in this file are not independent. As a workaround, @@ -922,15 +769,6 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate 2 Pod fails. Because the workersToDelete also contains pod3 and pod4, the controller will - // delete these two Pods. After the deletion, the number of worker Pods should be 3. - podList.Items[3].Status.Phase = corev1.PodFailed - podList.Items[4].Status.Phase = corev1.PodFailed - err = fakeClient.Update(ctx, &podList.Items[3]) - assert.Nil(t, err, "Fail to get update pod status") - err = fakeClient.Update(ctx, &podList.Items[4]) - assert.Nil(t, err, "Fail to get update pod status") - // Initialize a new RayClusterReconciler. testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -939,8 +777,8 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), } - // Since the desired state of the workerGroup is 3 replicas, the controller - // will not create or delete any worker Pods. + // Pod3 and Pod4 should be deleted because of the workersToDelete. + // Hence, no failed Pods should exist in `podList`. err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) assert.Nil(t, err, "Fail to reconcile Pods") @@ -950,8 +788,6 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { }) assert.Nil(t, err, "Fail to get pod list after reconcile") - // Failed Pods (pod3, pod4) should be deleted because of the workersToDelete. - // Hence, no failed Pods should exist in `podList`. assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) assert.Equal(t, expectedNumWorkerPods, getNotFailedPodItemNum(podList), "Replica number is wrong after reconcile expect %d actual %d", expectReplicaNum, getNotFailedPodItemNum(podList)) @@ -1006,12 +842,6 @@ func TestReconcile_PodCrash_DiffLess0_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate 1 Pod fails. Because the workersToDelete also contains pod3, the controller will - // delete pod3. After the deletion, the number of worker Pods should be 4. - podList.Items[3].Status.Phase = corev1.PodFailed - err = fakeClient.Update(ctx, &podList.Items[3]) - assert.Nil(t, err, "Fail to get update pod status") - // Initialize a new RayClusterReconciler. testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -1071,7 +901,7 @@ func TestReconcile_PodEvicted_DiffLess0_OK(t *testing.T) { podList.Items[0].Status.Phase = corev1.PodFailed podList.Items[0].Status.Reason = "Evicted" err = fakeClient.Update(ctx, &podList.Items[0]) - assert.Nil(t, err, "Fail to get update pod status") + assert.Nil(t, err, "Fail to update head Pod status") testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -1081,7 +911,10 @@ func TestReconcile_PodEvicted_DiffLess0_OK(t *testing.T) { } err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) - assert.Nil(t, err, "Fail to reconcile Pods") + // The head Pod with the status `Failed` will be deleted, and the function will return an + // error to requeue the request with a short delay. If the function returns nil, the controller + // will requeue the request after RAYCLUSTER_DEFAULT_REQUEUE_SECONDS_ENV (default: 300) seconds. + assert.NotNil(t, err) // Filter head pod err = fakeClient.List(ctx, &podList, &client.ListOptions{ @@ -1725,3 +1558,199 @@ func TestCalculateStatus(t *testing.T) { assert.Equal(t, headNodeIP, newInstance.Status.Head.PodIP) assert.Equal(t, headServiceIP, newInstance.Status.Head.ServiceIP) } + +func Test_TerminatedWorkers_NoAutoscaler(t *testing.T) { + setupTest(t) + + // TODO (kevin85421): The tests in this file are not independent. As a workaround, + // I added the assertion to prevent the test logic from being affected by other changes. + // However, we should refactor the tests in the future. + + // This test makes some assumptions about the testRayCluster object. + // (1) 1 workerGroup + // (2) The goal state of the workerGroup is 3 replicas. + // (3) Set the `WorkersToDelete` field to an empty slice. + // (4) Disable autoscaling. + assert.Equal(t, 1, len(testRayCluster.Spec.WorkerGroupSpecs), "This test assumes only one worker group.") + expectedNumWorkerPods := int(*testRayCluster.Spec.WorkerGroupSpecs[0].Replicas) + assert.Equal(t, 3, expectedNumWorkerPods, "This test assumes the expected number of worker pods is 3.") + testRayCluster.Spec.WorkerGroupSpecs[0].ScaleStrategy.WorkersToDelete = []string{} + testRayCluster.Spec.EnableInTreeAutoscaling = nil + + // This test makes some assumptions about the testPods object. + // `testPods` contains 6 pods, including 1 head pod and 5 worker pods. + assert.Equal(t, 6, len(testPods), "This test assumes the testPods object contains 6 pods.") + numHeadPods := 1 + oldNumWorkerPods := len(testPods) - numHeadPods + + // Initialize a fake client with newScheme and runtimeObjects. + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(testPods...).Build() + ctx := context.Background() + + // Get the pod list from the fake client. + podList := corev1.PodList{} + err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") + + // Make sure all worker Pods are running. + for _, pod := range podList.Items { + pod.Status.Phase = corev1.PodRunning + err = fakeClient.Status().Update(ctx, &pod) + assert.Nil(t, err, "Fail to update pod status") + } + + // Initialize a new RayClusterReconciler. + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: scheme.Scheme, + Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), + } + + // Since the desired state of the workerGroup is 3 replicas, the controller + // will delete 2 worker Pods. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err, "Fail to reconcile Pods") + + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) + + // Update 1 worker Pod to Failed (a terminate state) state. + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Status().Update(ctx, &podList.Items[0]) + assert.Nil(t, err, "Fail to update Pod status") + + // Reconcile again, and the Failed worker Pod should be deleted even if the goal state of the workerGroup specifies 3 replicas. + // The function will return an error to requeue the request after a brief delay. Moreover, if there are unhealthy worker + // Pods to be deleted, the controller won't create new worker Pods during the same reconcile loop. As a result, the number of worker + // Pods will be (expectedNumWorkerPods - 1) after the reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods-1, len(podList.Items)) + + // Reconcile again, and the controller will create a new worker Pod to reach the goal state of the workerGroup. + // Note that the status of new worker Pod created by the fake client is empty, so we need to set all worker + // Pods to running state manually to avoid the new Pod being deleted in the next `reconcilePods` call. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) + for _, pod := range podList.Items { + pod.Status.Phase = corev1.PodRunning + err = fakeClient.Status().Update(ctx, &pod) + assert.Nil(t, err, "Fail to update pod status") + } + + // Update 1 worker Pod to Succeeded (a terminate state) state. + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + podList.Items[0].Status.Phase = corev1.PodSucceeded + err = fakeClient.Status().Update(ctx, &podList.Items[0]) + assert.Nil(t, err, "Fail to update Pod status") + + // Reconcile again, and the Succeeded worker Pod should be deleted even if the goal state of the workerGroup specifies 3 replicas. + // The function will return an error to requeue the request after a brief delay. Moreover, if there are unhealthy worker + // Pods to be deleted, the controller won't create new worker Pods during the same reconcile loop. As a result, the number of worker + // Pods will be (expectedNumWorkerPods - 1) after the reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods-1, len(podList.Items)) + + // Reconcile again, and the controller will create a new worker Pod to reach the goal state of the workerGroup. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) +} + +func Test_TerminatedHead_RestartPolicy_Always(t *testing.T) { + setupTest(t) + + // Create a new scheme with CRDs, Pod, Service schemes. + newScheme := runtime.NewScheme() + _ = rayv1alpha1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) + + // Only one head Pod and no worker Pods in the RayCluster. + runtimeObjects := testPods[0:1] + cluster := testRayCluster.DeepCopy() + cluster.Spec.WorkerGroupSpecs = nil + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(runtimeObjects...).Build() + ctx := context.Background() + + // Get the pod list from the fake client. + podList := corev1.PodList{} + err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) + assert.Equal(t, "headNode", podList.Items[0].Name) + + // Make sure the head Pod's restart policy is `Always` and status is `Failed`. + podList.Items[0].Spec.RestartPolicy = corev1.RestartPolicyAlways + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Update(ctx, &podList.Items[0]) + assert.Nil(t, err) + + // Initialize a new RayClusterReconciler. + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: newScheme, + Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), + } + + // The head Pod will not be deleted because the restart policy is `Always`, + // and the controller will return an error to requeue the request after a brief delay. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) + + // Make sure the head Pod's restart policy is `Never` and status is `Failed`. + podList.Items[0].Spec.RestartPolicy = corev1.RestartPolicyNever + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Update(ctx, &podList.Items[0]) + assert.Nil(t, err) + + // The head Pod will be deleted and the controller will return an error + // instead of creating a new head Pod in the same reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 0, len(podList.Items)) + + // The new head Pod will be created in the this reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) +} diff --git a/tests/compatibility-test.py b/tests/compatibility-test.py index 203eb3de6f0..35e8960b431 100755 --- a/tests/compatibility-test.py +++ b/tests/compatibility-test.py @@ -90,26 +90,6 @@ def setUpClass(cls): operator_manager.prepare_operator() utils.create_ray_cluster(RayFTTestCase.cluster_template, ray_version, ray_image) - @unittest.skip("Skip test_kill_head due to its flakiness.") - def test_kill_head(self): - # This test will delete head node and wait for a new replacement to - # come up. - shell_subprocess_run( - 'kubectl delete pod $(kubectl get pods -A | grep -e "-head" | awk "{print \$2}")') - - # wait for new head node to start - time.sleep(80) - shell_subprocess_run('kubectl get pods -A') - - # make sure the new head is ready - # shell_assert_success('kubectl wait --for=condition=Ready pod/$(kubectl get pods -A | grep -e "-head" | awk "{print \$2}") --timeout=900s') - # make sure both head and worker pods are ready - rtn = shell_subprocess_run( - 'kubectl wait --for=condition=ready pod -l rayCluster=raycluster-compatibility-test --all --timeout=900s', check = False) - if rtn != 0: - show_cluster_info("default") - raise Exception(f"Nonzero return code {rtn} in test_kill_head()") - def test_ray_serve(self): """Kill GCS process on the head Pod and then test a deployed Ray Serve model.""" if not utils.is_feature_supported(ray_version, CONST.RAY_SERVE_FT): @@ -127,9 +107,7 @@ def test_ray_serve(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_ray_serve_1.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_ray_serve_1.py. The exit code is {exit_code}.") old_head_pod = get_head_pod(RayFTTestCase.ray_cluster_ns) old_head_pod_name = old_head_pod.metadata.name @@ -141,7 +119,7 @@ def test_ray_serve(self): pod_exec_command(old_head_pod_name, RayFTTestCase.ray_cluster_ns, "pkill gcs_server") # Waiting for all pods become ready and running. - utils.wait_for_new_head(old_head_pod_name, restart_count, + utils.wait_for_new_head(CONST.KILL_GCS_SERVER, old_head_pod_name, restart_count, RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) # Try to connect to the deployed model again @@ -154,9 +132,7 @@ def test_ray_serve(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_ray_serve_2.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_ray_serve_2.py. The exit code is {exit_code}.") def test_detached_actor(self): """Kill GCS process on the head Pod and then test a detached actor.""" @@ -175,21 +151,20 @@ def test_detached_actor(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_detached_actor_1.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_detached_actor_1.py. The exit code is {exit_code}.") old_head_pod = get_head_pod(RayFTTestCase.ray_cluster_ns) old_head_pod_name = old_head_pod.metadata.name restart_count = old_head_pod.status.container_statuses[0].restart_count + # [Test 1: Kill GCS process to "restart" the head Pod] # Kill the gcs_server process on head node. The head node will crash after 20 seconds # because the value of `RAY_gcs_rpc_server_reconnect_timeout_s` is "20" in the # `ray-cluster.ray-ft.yaml.template` file. pod_exec_command(old_head_pod_name, RayFTTestCase.ray_cluster_ns, "pkill gcs_server") # Waiting for all pods become ready and running. - utils.wait_for_new_head(old_head_pod_name, restart_count, + utils.wait_for_new_head(CONST.KILL_GCS_SERVER, old_head_pod_name, restart_count, RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) # Try to connect to the detached actor again. @@ -198,16 +173,39 @@ def test_detached_actor(self): # connection succeeds. headpod = get_head_pod(RayFTTestCase.ray_cluster_ns) headpod_name = headpod.metadata.name + expected_output = 3 exit_code = pod_exec_command(headpod_name, RayFTTestCase.ray_cluster_ns, - f" python samples/test_detached_actor_2.py {ray_namespace}", + f" python samples/test_detached_actor_2.py {ray_namespace} {expected_output}", check = False ) if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}.") + + # [Test 2: Delete the head Pod and wait for a new head Pod] + # Delete the head Pod. The `kubectl delete pod` command has a default flag `--wait=true`, + # which waits for resources to be gone before returning. + shell_subprocess_run( + f'kubectl delete pod {headpod_name} -n {RayFTTestCase.ray_cluster_ns}') + restart_count = headpod.status.container_statuses[0].restart_count + + # Waiting for all pods become ready and running. + utils.wait_for_new_head(CONST.KILL_HEAD_POD, headpod_name, restart_count, + RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) + + # Try to connect to the detached actor again. + headpod = get_head_pod(RayFTTestCase.ray_cluster_ns) + headpod_name = headpod.metadata.name + expected_output = 4 + exit_code = pod_exec_command(headpod_name, RayFTTestCase.ray_cluster_ns, + f" python samples/test_detached_actor_2.py {ray_namespace} {expected_output}", + check = False + ) + + if exit_code != 0: + show_cluster_info(RayFTTestCase.ray_cluster_ns) + self.fail(f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}.") class RayServiceTestCase(unittest.TestCase): """Integration tests for RayService""" diff --git a/tests/config/ray-cluster.ray-ft.yaml.template b/tests/config/ray-cluster.ray-ft.yaml.template index ccad180470d..b00beb7588f 100644 --- a/tests/config/ray-cluster.ray-ft.yaml.template +++ b/tests/config/ray-cluster.ray-ft.yaml.template @@ -267,6 +267,4 @@ data: val = retry_with_timeout(lambda: ray.get(tc.increment.remote())) print(f"val: {val}") - # The actual value should be 1 rather than 2. Ray will launch all registered actors when - # the ray cluster restarts, but the internal state of the state will not be restored. - assert(val == 3) + assert(val == int(sys.argv[2])) diff --git a/tests/framework/utils.py b/tests/framework/utils.py index 079903ecf30..c553bc4221d 100644 --- a/tests/framework/utils.py +++ b/tests/framework/utils.py @@ -53,6 +53,10 @@ class CONST: RAY_SERVICE_CRD = "RayService" RAY_JOB_CRD = "RayJob" + # Failures + KILL_GCS_SERVER = "KILL_GCS_SERVER" + KILL_HEAD_POD = "KILL_HEAD_POD" + CONST = CONST() diff --git a/tests/kuberay_utils/utils.py b/tests/kuberay_utils/utils.py index 92a1d05d77d..ef81e4c04f9 100644 --- a/tests/kuberay_utils/utils.py +++ b/tests/kuberay_utils/utils.py @@ -14,7 +14,6 @@ from framework.utils import ( get_head_pod, CONST, - K8S_CLUSTER_MANAGER ) logger = logging.getLogger(__name__) @@ -60,7 +59,7 @@ def create_ray_cluster(template_name, ray_version, ray_image): ray_cluster_add_event = RayClusterAddCREvent( custom_resource_object = context['cr'], rulesets = [], - timeout = 90, + timeout = 180, namespace='default', filepath = context['filepath'] ) @@ -123,71 +122,79 @@ def wait_for_condition( message += f" Last exception: {last_ex}" raise RuntimeError(message) -def wait_for_new_head(old_head_pod_name, old_restart_count, namespace, timeout, retry_interval_ms): +def wait_for_new_head(mode, old_head_pod_name, old_restart_count, namespace, timeout, retry_interval_ms): """ - `wait_for_new_head` is used to wait for new head is ready and running. For example, `test_detached_actor` kills - the gcs_server process on the head pod. It takes nearly 1 min to kill the head pod, and the head pod will still - be in 'Running' and 'Ready' in that minute. - - Hence, we need to check `restart_count` or `new_head_pod_name`. - (1) `restart_count`: If the pod is restarted by the restartPolicy of a Pod, `restart_count` will increase by 1. - If the pod is deleted by KubeRay and the reconciler creates a new one, `restart_count` will be 0. - (2) `new_head_pod_name`: If the reconciler creates a new head pod, `new_head_pod_name` will be different from - `old_head_pod_name`. - - Next, we check the status of pods to ensure all pods should be "Running" and "Ready". + `wait_for_new_head` is used to wait for the head Pod is ready and running. - After the cluster state converges (all pods are "Running" and "Ready"), ray processes still need tens of seconds to - become ready to serve new connections from ray clients. So, users need to retry until a Ray client connection succeeds. + [Case 1] + KILL_GCS_SERVER: The head Pod should be restarted rather than creating a new head Pod. + [Case 2] + KILL_HEAD_POD: The new head Pod should be created. Args: + mode: KILL_GCS_SERVER or KILL_HEAD_POD. old_head_pod_name: Name of the old head pod. - old_restart_count: If the Pod is restarted by Kubernetes Pod RestartPolicy, the restart_count will increase by 1. + old_restart_count: The restart count of the old head pod. namespace: Namespace that the head pod is running in. timeout: Same as `wait_for_condition`. retry_interval_ms: Same as `wait_for_condition`. Raises: - RuntimeError: If the condition is not met before the timeout expires, raise the RuntimeError. + RuntimeError: Raise a RuntimeError if a timeout occurs. """ - k8s_v1_api = K8S_CLUSTER_MANAGER.k8s_client_dict[CONST.K8S_V1_CLIENT_KEY] - def check_status(old_head_pod_name, old_restart_count, namespace) -> bool: - all_pods = k8s_v1_api.list_namespaced_pod(namespace = namespace) + def check_status(mode, old_head_pod_name, old_restart_count, namespace) -> bool: headpod = get_head_pod(namespace) + if headpod is None: + logger.info( + "There is no head Pod. We will only check the following conditions " + + "after the head Pod is created." + ) + return False new_head_pod_name = headpod.metadata.name new_restart_count = headpod.status.container_statuses[0].restart_count - # The default container restartPolicy of a Pod is `Always`. Hence, when GCS server is killed, - # the head pod will restart the old one rather than create a new one. - if new_head_pod_name != old_head_pod_name: - logger.info(f'If GCS server is killed, the head pod will restart the old one rather than create a new one.' + - f' new_head_pod_name: {new_head_pod_name}, old_head_pod_name: {old_head_pod_name}') - # TODO (kevin85421): We should `return False` here, but currently ray:nightly has a high possibility to create - # a new head pod instead of restarting the old one. - - # When GCS server is killed, it takes nearly 1 min to kill the head pod. In the minute, the head - # pod will still be in 'Running' and 'Ready'. Hence, we need to check `restart_count`. - else: - # TODO (kevin85421): We should remove `else` in the future. Currently, ray:nightly has a high possibility to - # create a new head pod instead of restarting the old one. The new pod's `restart_count` - # is 0. + + logger.info("Failure mode: %s", mode) + if mode == CONST.KILL_GCS_SERVER: + if new_head_pod_name != old_head_pod_name: + logger.warning( + "GCS server process is killed. The head Pod should be restarted " + "rather than creating a new head Pod. There is something wrong. " + "new_head_pod_name: %s, old_head_pod_name: %s", + new_head_pod_name, old_head_pod_name + ) + return False if new_restart_count != old_restart_count + 1: - logger.info(f'new_restart_count != old_restart_count + 1 => new_restart_count: {new_restart_count},' + - f' old_restart_count: {old_restart_count}') + logger.info( + "new_restart_count != old_restart_count + 1 =>" + "new_restart_count: %s; old_restart_count: %s", + new_restart_count, old_restart_count + ) + return False + elif mode == CONST.KILL_HEAD_POD: + if new_head_pod_name == old_head_pod_name: + logger.info("The old head Pod %s is not killed.", old_head_pod_name) return False - # All pods should be "Running" and "Ready". This check is an overkill. We added this check due to - # the buggy behaviors of Ray HA. To elaborate, when GCS server is killed, the head pod should restart, - # but worker pods should not. However, currently, worker pods will also restart. - # See https://github.com/ray-project/kuberay/issues/634 for more details. - for pod in all_pods.items: - if pod.status.phase != 'Running': - logger.info(f'Pod {pod.metadata.name} is not Running.') + else: + raise ValueError(f"Invalid failure mode: {mode}") + + if headpod.status.phase != "Running": + logger.info( + "The head Pod %s is not running. The status is %s", + headpod.metadata.name, headpod.status.phase + ) + return False + for container_status in headpod.status.container_statuses: + if not container_status.ready: + logger.info( + "The container %s is not ready. The status is %s", + container_status.name, container_status.ready + ) return False - for c in pod.status.container_statuses: - if not c.ready: - logger.info(f'Container {c.name} in {pod.metadata.name} is not ready.') - return False return True - wait_for_condition(check_status, timeout=timeout, retry_interval_ms=retry_interval_ms, - old_head_pod_name=old_head_pod_name, old_restart_count=old_restart_count, namespace=namespace) + wait_for_condition( + check_status, timeout=timeout, retry_interval_ms=retry_interval_ms, + mode=mode, old_head_pod_name=old_head_pod_name, old_restart_count=old_restart_count, + namespace=namespace + ) # After the cluster state converges, ray processes still need tens of seconds to become ready. # TODO (kevin85421): Make ray processes become ready when pods are "Ready" and "Running". From d6e4a5986f9b47c82ed4ccc6c4e5c4c3a9fa29ff Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 20/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 3 +-- proto/cluster.proto | 16 +++++++-------- proto/go_client/cluster.pb.go | 32 ++++++++++++++--------------- 3 files changed, 25 insertions(+), 26 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index a094224e777..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,12 +15,11 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head-test", + Name: "ray-head", Image: "blah", VolumeMounts: []v1.VolumeMount{ { diff --git a/proto/cluster.proto b/proto/cluster.proto index 9ef278f63d0..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -221,22 +221,22 @@ message HeadGroupSpec { string service_type = 3; // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 11; + bool enableIngress = 4; // Optional. The ray start params of head node group. - map ray_start_params = 4; + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; + map labels = 11; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 6b8e12c4da7..9f5952e72f8 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -970,22 +970,22 @@ type HeadGroupSpec struct { ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - EnableIngress bool `protobuf:"varint,11,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` // Optional. The ray start params of head node group. - RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // Optional. image pull secret used by head pod - ImagePullSecret string `protobuf:"bytes,7,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` + 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,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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"` // Optional. Annotations for the head pod - Annotations map[string]string `protobuf:"bytes,9,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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 - Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -1530,32 +1530,32 @@ var file_cluster_proto_rawDesc = []byte{ 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, 0x0b, + 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, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, + 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, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 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, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, + 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, 0x07, + 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, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x72, 0x6f, + 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, 0x09, 0x20, + 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, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 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, From c2a5efca4a39cd6e78fdf888cf694e6301d8d91c Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 08:41:50 +0100 Subject: [PATCH 21/56] minor fixes --- proto/cluster.proto | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 827577b38e1..9838ce671f0 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -201,7 +201,7 @@ message Volume { BIDIRECTIONAL = 2; } MountPropagationMode mount_propagation_mode = 7; - // If indicate ephemeral, we need to let user specify volumeClaimTemplate + // If indicate ephemeral, we need to let user specify volumeClaimTemplate parameters string storageClassName = 8; // If not defined, default is used enum AccessMode { RWO = 0; // ReadWriteOnce @@ -219,24 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. Enable Ingress - // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 4; - // Optional. The ray start params of head node group. - map ray_start_params = 5; + // Optional. The ray start params of head node group. + map ray_start_params = 4; // Optional. The volumes mount to head pod - repeated Volume volumes = 6; + repeated Volume volumes = 5; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 7; + string service_account = 6; // Optional. image pull secret used by head pod - string image_pull_secret = 8; + string image_pull_secret = 7; // Optional. Environment variables for head pod - map environment = 9; + map environment = 8; // Optional. Annotations for the head pod - map annotations = 10; + map annotations = 9; // Optional. Labels for the head pod - map labels = 11; + map labels = 10; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 11; } message WorkerGroupSpec { From 81b4187246e25cfec66d5de2eefe567c2060e482 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 08:44:03 +0100 Subject: [PATCH 22/56] minor fixes --- proto/cluster.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 9838ce671f0..782d9a0c000 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -234,7 +234,7 @@ message HeadGroupSpec { map annotations = 9; // Optional. Labels for the head pod map labels = 10; - // Optional. Enable Ingress + // Optional. Enable cluster Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class bool enableIngress = 11; } From 405fddb952e8222bf89ca1e1502b13bdf6748144 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 10:52:20 +0100 Subject: [PATCH 23/56] minor fixes --- apiserver/pkg/model/volumes_test.go | 1 + apiserver/pkg/util/cluster.go | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index db5b237cbb7..f3c4a6c210a 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -651,8 +651,8 @@ func NewComputeTemplate(runtime *api.ComputeTemplate) (*v1.ConfigMap, error) { func GetNodeHostIP(node *v1.Node) (net.IP, error) { addresses := node.Status.Addresses addressMap := make(map[v1.NodeAddressType][]v1.NodeAddress) - for i := range addresses { - addressMap[addresses[i].Type] = append(addressMap[addresses[i].Type], addresses[i]) + for _, nodeAddress := range addresses { + addressMap[nodeAddress.Type] = append(addressMap[nodeAddress.Type], nodeAddress) } if addresses, ok := addressMap[v1.NodeInternalIP]; ok { return net.ParseIP(addresses[0].Address), nil From 5de4a42c7b6de51de9728e1fdb54309be9d26fac Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Tue, 22 Aug 2023 12:58:34 -0700 Subject: [PATCH 24/56] [RayJob] Add runtime env YAML field (#1338) Adds the field RuntimeEnvYAML to the RayJob CRD which accepts a multi-line YAML string. This format is preferred for two reasons: Consistency with the ServeConfigV2 format, which is also a Ray configuration specified as a multi-line YAML string (Related to above) Allows using snake_case fields without modification We preserve the older field RuntimeEnv which accepts a base64-encoded string of the runtime env. We mark it as deprecated in the documentation. We raise an error if both fields are specified. Related issue number Closes #1195 --------- Signed-off-by: Archit Kulkarni --- .buildkite/test-sample-yamls.yml | 24 ++- docs/guidance/rayjob.md | 3 +- .../kuberay-operator/crds/ray.io_rayjobs.yaml | 7 +- .../apis/ray/v1alpha1/rayjob_types.go | 4 + .../config/crd/bases/ray.io_rayjobs.yaml | 7 +- .../samples/ray-job.custom-head-svc.yaml | 12 +- .../ray_v1alpha1_rayjob.batch-inference.yaml | 179 ++++++++++++++++++ .../samples/ray_v1alpha1_rayjob.shutdown.yaml | 24 ++- .../config/samples/ray_v1alpha1_rayjob.yaml | 23 ++- ray-operator/controllers/ray/common/job.go | 41 +++- .../controllers/ray/common/job_test.go | 101 ++++++++++ .../controllers/ray/rayjob_controller.go | 5 + ray-operator/go.mod | 2 +- 13 files changed, 393 insertions(+), 39 deletions(-) create mode 100644 ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml diff --git a/.buildkite/test-sample-yamls.yml b/.buildkite/test-sample-yamls.yml index ef415f6d526..0a06e9715f3 100644 --- a/.buildkite/test-sample-yamls.yml +++ b/.buildkite/test-sample-yamls.yml @@ -5,6 +5,10 @@ image: golang:1.19 commands: - ./.buildkite/setup-env.sh + # Build nightly KubeRay operator image + - pushd ray-operator + - IMG=kuberay/operator:nightly make docker-image + - popd # Use nightly KubeRay operator image - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:nightly python3 tests/test_sample_raycluster_yamls.py @@ -21,13 +25,19 @@ image: golang:1.19 commands: - ./.buildkite/setup-env.sh + # Build nightly KubeRay operator image + - pushd ray-operator + - IMG=kuberay/operator:nightly make docker-image + - popd # Use nightly KubeRay operator image - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:nightly python3 tests/test_sample_rayjob_yamls.py -- label: 'Test RayJob Sample YAMLs (latest release)' - instance_size: large - image: golang:1.19 - commands: - - ./.buildkite/setup-env.sh - # Use KubeRay operator image from the latest release - - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:v0.6.0 python3 tests/test_sample_rayjob_yamls.py +# Temporarily skip due to adding new `RuntimeEnvYAML` field in sample YAMLs. +# TODO(architkulkarni): Reenable after 1.0 release +# - label: 'Test RayJob Sample YAMLs (latest release)' +# instance_size: large +# image: golang:1.19 +# commands: +# - ./.buildkite/setup-env.sh +# # Use KubeRay operator image from the latest release +# - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:v0.6.0 python3 tests/test_sample_rayjob_yamls.py diff --git a/docs/guidance/rayjob.md b/docs/guidance/rayjob.md index e4e664420a1..dbabd4c9b49 100644 --- a/docs/guidance/rayjob.md +++ b/docs/guidance/rayjob.md @@ -54,10 +54,11 @@ $ kubectl get pod * `rayClusterSpec` - The spec for the Ray cluster to run the job on. * `jobId` - _(Optional)_ Job ID to specify for the job. If not provided, one will be generated. * `metadata` - _(Optional)_ Arbitrary user-provided metadata for the job. -* `runtimeEnv` - _(Optional)_ base64-encoded string of the runtime env json string. +* `runtimeEnvYAML` - _(Optional)_ The runtime environment configuration provided as a multi-line YAML string. _(New in KubeRay version 1.0.)_ * `shutdownAfterJobFinishes` - _(Optional)_ whether to recycle the cluster after the job finishes. Defaults to false. * `ttlSecondsAfterFinished` - _(Optional)_ TTL to clean up the cluster. This only works if `shutdownAfterJobFinishes` is set. * `submitterPodTemplate` - _(Optional)_ Pod template spec for the pod that runs `ray job submit` against the Ray cluster. +* `runtimeEnv` - [DEPRECATED] _(Optional)_ base64-encoded string of the runtime env json string. ## RayJob Observability diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml index 2d6c5eb9e6e..1645ec4032d 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml @@ -12090,7 +12090,12 @@ spec: - headGroupSpec type: object runtimeEnv: - description: RuntimeEnv is base64 encoded. + description: RuntimeEnv is base64 encoded. This field is deprecated, + please use RuntimeEnvYAML instead. + type: string + runtimeEnvYAML: + description: RuntimeEnvYAML represents the runtime environment configuration + provided as a multi-line YAML string type: string shutdownAfterJobFinishes: description: ShutdownAfterJobFinishes will determine whether to delete diff --git a/ray-operator/apis/ray/v1alpha1/rayjob_types.go b/ray-operator/apis/ray/v1alpha1/rayjob_types.go index 670b1764795..8c791e740af 100644 --- a/ray-operator/apis/ray/v1alpha1/rayjob_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayjob_types.go @@ -52,7 +52,11 @@ type RayJobSpec struct { // Metadata is data to store along with this job. Metadata map[string]string `json:"metadata,omitempty"` // RuntimeEnv is base64 encoded. + // This field is deprecated, please use RuntimeEnvYAML instead. RuntimeEnv string `json:"runtimeEnv,omitempty"` + // RuntimeEnvYAML represents the runtime environment configuration + // provided as a multi-line YAML string. + RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"` // If jobId is not set, a new jobId will be auto-generated. JobId string `json:"jobId,omitempty"` // ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. diff --git a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml index 2d6c5eb9e6e..1645ec4032d 100644 --- a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml @@ -12090,7 +12090,12 @@ spec: - headGroupSpec type: object runtimeEnv: - description: RuntimeEnv is base64 encoded. + description: RuntimeEnv is base64 encoded. This field is deprecated, + please use RuntimeEnvYAML instead. + type: string + runtimeEnvYAML: + description: RuntimeEnvYAML represents the runtime environment configuration + provided as a multi-line YAML string type: string shutdownAfterJobFinishes: description: ShutdownAfterJobFinishes will determine whether to delete diff --git a/ray-operator/config/samples/ray-job.custom-head-svc.yaml b/ray-operator/config/samples/ray-job.custom-head-svc.yaml index 31ba15a74b8..72e09c4d3d6 100644 --- a/ray-operator/config/samples/ray-job.custom-head-svc.yaml +++ b/ray-operator/config/samples/ray-job.custom-head-svc.yaml @@ -7,7 +7,17 @@ metadata: name: rayjob-sample spec: entrypoint: python /home/ray/samples/sample_code.py - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.6.3' # should match the Ray version in the image of the containers diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml new file mode 100644 index 00000000000..8fdc51a647d --- /dev/null +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml @@ -0,0 +1,179 @@ +apiVersion: ray.io/v1alpha1 +kind: RayJob +metadata: + name: rayjob-sample +spec: + entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. + # shutdownAfterJobFinishes: false + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. + # ttlSecondsAfterFinished: 10 + # Runtime env decoded to { + # { + # "pip": [ + # "torch", + # "torchvision", + # "Pillow", + # "transformers" + # ] + # } + runtimeEnv: ewogICJwaXAiOiBbCiAgICAidG9yY2giLAogICAgInRvcmNodmlzaW9uIiwKICAgICJQaWxsb3ciLAogICAgInRyYW5zZm9ybWVycyIKICBdCn0= + # Suspend specifies whether the RayJob controller should create a RayCluster instance. + # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. + # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. + # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. + rayClusterSpec: + rayVersion: '2.6.3' # should match the Ray version in the image of the containers + # Ray head pod template + 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: + dashboard-host: '0.0.0.0' + #pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray-ml:2.6.3-gpu + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 # Ray dashboard + name: dashboard + - containerPort: 10001 + name: client + resources: + limits: + cpu: 2 + memory: 8Gi + requests: + cpu: 2 + memory: 8Gi + volumeMounts: + - mountPath: /home/ray/samples + name: code-sample + volumes: + # You set volumes at the Pod level, then mount them into containers inside that Pod + - name: code-sample + configMap: + # Provide the name of the ConfigMap you want to mount. + name: ray-job-code-sample + # An array of keys from the ConfigMap to create as files + items: + - key: sample_code.py + path: sample_code.py + workerGroupSpecs: + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + # logical group name, for this called small-group, also can be functional + 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: + resources: '"{\"accelerator_type_cpu\": 48, \"accelerator_type_a10\": 2, \"accelerator_type_a100\": 2}"' + #pod template + template: + spec: + containers: + - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + image: rayproject/ray-ml:2.6.3-gpu + lifecycle: + preStop: + exec: + command: [ "/bin/sh","-c","ray stop" ] + resources: + limits: + cpu: "48" + memory: "192G" + nvidia.com/gpu: 4 + requests: + cpu: "36" + memory: "128G" + nvidia.com/gpu: 4 + nodeSelector: + cloud.google.com/gke-accelerator: nvidia-tesla-t4 + # SubmitterPodTemplate is the template for the pod that will run the `ray job submit` command against the RayCluster. + # If SubmitterPodTemplate is specified, the first container is assumed to be the submitter container. + # submitterPodTemplate: + # spec: + # restartPolicy: Never + # containers: + # - name: my-custom-rayjob-submitter-pod + # image: rayproject/ray:2.6.3 + # # If Command is not specified, the correct command will be supplied at runtime using the RayJob spec `entrypoint` field. + # # Specifying Command is not recommended. + # # command: ["ray job submit --address=http://rayjob-sample-raycluster-v6qcq-head-svc.default.svc.cluster.local:8265 -- echo hello world"] + + +######################Ray code sample################################# +# this sample is from https://docs.ray.io/en/latest/cluster/job-submission.html#quick-start-example +# it is mounted into the container and executed to show the Ray job at work +--- +apiVersion: v1 +kind: ConfigMap +metadata: + name: ray-job-code-sample +data: + sample_code.py: | + import ray + + s3_uri = "s3://anonymous@air-example-data-2/imagenette2/val/" + + ds = ray.data.read_images( + s3_uri, mode="RGB" + ) + ds + # TODO(archit) need to install Pillow, pytorch or tf or flax (pip install torch torchvision torchaudio) + from typing import Dict + import numpy as np + + from transformers import pipeline + from PIL import Image + + # Pick the largest batch size that can fit on our GPUs + BATCH_SIZE = 1024 + + # TODO(archit) basic step + + # single_batch = ds.take_batch(10) + + # from PIL import Image + + # img = Image.fromarray(single_batch["image"][0]) + # # display image + # img.show() + # from transformers import pipeline + # from PIL import Image + + # # If doing CPU inference, set device="cpu" instead. + # classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") + # outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) + # del classifier # Delete the classifier to free up GPU memory. + # print(outputs) + + @ray.remote(num_gpus=1) + def do_single_batch(): + single_batch = ds.take_batch(10) + + from PIL import Image + + img = Image.fromarray(single_batch["image"][0]) + # display image + img.show() + from transformers import pipeline + from PIL import Image + + # If doing CPU inference, set device="cpu" instead. + classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") + outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) + del classifier # Delete the classifier to free up GPU memory. + print(outputs) + return outputs + + print(ray.get(do_single_batch.remote())) \ No newline at end of file diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml index f4a2425bb5e..cddfb5a0e3c 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml @@ -4,24 +4,28 @@ metadata: name: rayjob-sample-shutdown spec: entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. shutdownAfterJobFinishes: true + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. ttlSecondsAfterFinished: 10 - # runtimeEnv decoded to '{ - # "pip": [ - # "requests==2.26.0", - # "pendulum==2.1.2" - # ], - # "env_vars": { - # "counter_name": "test_counter" - # } - # }' - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # Suspend specifies whether the RayJob controller should create a RayCluster instance. # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.5.0' # should match the Ray version in the image of the containers diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml index a922e06f2de..445ee6bf5fb 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml @@ -6,22 +6,25 @@ spec: entrypoint: python /home/ray/samples/sample_code.py # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. # shutdownAfterJobFinishes: false + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. # ttlSecondsAfterFinished: 10 - # runtimeEnv decoded to '{ - # "pip": [ - # "requests==2.26.0", - # "pendulum==2.1.2" - # ], - # "env_vars": { - # "counter_name": "test_counter" - # } - #}' - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # Suspend specifies whether the RayJob controller should create a RayCluster instance. # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.5.0' # should match the Ray version in the image of the containers diff --git a/ray-operator/controllers/ray/common/job.go b/ray-operator/controllers/ray/common/job.go index f7a67139910..5f2093b52a1 100644 --- a/ray-operator/controllers/ray/common/job.go +++ b/ray-operator/controllers/ray/common/job.go @@ -10,6 +10,7 @@ import ( "github.com/google/shlex" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" v1 "k8s.io/api/core/v1" + "sigs.k8s.io/yaml" ) // GetDecodedRuntimeEnv decodes the runtime environment for the Ray job from a base64-encoded string. @@ -21,6 +22,33 @@ func GetDecodedRuntimeEnv(runtimeEnv string) (string, error) { return string(decodedBytes), nil } +// GetRuntimeEnvJson returns the JSON string of the runtime environment for the Ray job. +func getRuntimeEnvJson(rayJobInstance *rayv1alpha1.RayJob) (string, error) { + runtimeEnv := rayJobInstance.Spec.RuntimeEnv + runtimeEnvYAML := rayJobInstance.Spec.RuntimeEnvYAML + + // Check if both runtimeEnv and RuntimeEnvYAML are specified. + if len(runtimeEnv) > 0 && len(runtimeEnvYAML) > 0 { + return "", fmt.Errorf("Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.") + } + + if len(runtimeEnv) > 0 { + return GetDecodedRuntimeEnv(runtimeEnv) + } + + if len(runtimeEnvYAML) > 0 { + // Convert YAML to JSON + jsonData, err := yaml.YAMLToJSON([]byte(runtimeEnvYAML)) + if err != nil { + return "", err + } + // We return the JSON as a string + return string(jsonData), nil + } + + return "", nil +} + // GetBaseRayJobCommand returns the first part of the Ray Job command up to and including the address, e.g. "ray job submit --address http://..." func GetBaseRayJobCommand(address string) []string { // add http:// if needed @@ -54,19 +82,18 @@ func GetMetadataJson(metadata map[string]string, rayVersion string) (string, err // GetK8sJobCommand builds the K8s job command for the Ray job. func GetK8sJobCommand(rayJobInstance *rayv1alpha1.RayJob) ([]string, error) { address := rayJobInstance.Status.DashboardURL - runtimeEnv := rayJobInstance.Spec.RuntimeEnv metadata := rayJobInstance.Spec.Metadata jobId := rayJobInstance.Status.JobId entrypoint := rayJobInstance.Spec.Entrypoint k8sJobCommand := GetBaseRayJobCommand(address) - if len(runtimeEnv) > 0 { - runtimeEnvDecoded, err := GetDecodedRuntimeEnv(runtimeEnv) - if err != nil { - return nil, err - } - k8sJobCommand = append(k8sJobCommand, "--runtime-env-json", runtimeEnvDecoded) + runtimeEnvJson, err := getRuntimeEnvJson(rayJobInstance) + if err != nil { + return nil, err + } + if len(runtimeEnvJson) > 0 { + k8sJobCommand = append(k8sJobCommand, "--runtime-env-json", runtimeEnvJson) } if len(metadata) > 0 { diff --git a/ray-operator/controllers/ray/common/job_test.go b/ray-operator/controllers/ray/common/job_test.go index e5cd782b520..431b1ae314e 100644 --- a/ray-operator/controllers/ray/common/job_test.go +++ b/ray-operator/controllers/ray/common/job_test.go @@ -1,6 +1,7 @@ package common import ( + "encoding/json" "testing" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" @@ -30,6 +31,49 @@ func TestGetDecodedRuntimeEnv(t *testing.T) { assert.Equal(t, `{"test":"test"}`, decoded) } +func TestGetRuntimeEnvJsonFromBase64(t *testing.T) { + expected := `{"test":"test"}` + jsonOutput, err := getRuntimeEnvJson(testRayJob) + assert.NoError(t, err) + assert.Equal(t, expected, jsonOutput) +} + +func TestGetRuntimeEnvJsonFromYAML(t *testing.T) { + rayJobWithYAML := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnvYAML: ` +working_dir: "https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip" +pip: ["python-multipart==0.0.6"] +`, + }, + } + expectedJSON := `{"working_dir":"https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip","pip":["python-multipart==0.0.6"]}` + jsonOutput, err := getRuntimeEnvJson(rayJobWithYAML) + assert.NoError(t, err) + + var expectedMap map[string]interface{} + var actualMap map[string]interface{} + + // Convert the JSON strings into map types to avoid errors due to ordering + assert.NoError(t, json.Unmarshal([]byte(expectedJSON), &expectedMap)) + assert.NoError(t, json.Unmarshal([]byte(jsonOutput), &actualMap)) + + // Now compare the maps + assert.Equal(t, expectedMap, actualMap) +} + +func TestGetRuntimeEnvJsonErrorWithBothFields(t *testing.T) { + rayJobWithBoth := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnv: "eyJ0ZXN0IjoidGVzdCJ9", + RuntimeEnvYAML: `pip: ["python-multipart==0.0.6"]`, + }, + } + _, err := getRuntimeEnvJson(rayJobWithBoth) + assert.Error(t, err) + assert.Contains(t, err.Error(), "Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.") +} + func TestGetBaseRayJobCommand(t *testing.T) { expected := []string{"ray", "job", "submit", "--address", "http://127.0.0.1:8265"} command := GetBaseRayJobCommand(testRayJob.Status.DashboardURL) @@ -57,6 +101,63 @@ func TestGetK8sJobCommand(t *testing.T) { assert.Equal(t, expected, command) } +func TestGetK8sJobCommandWithYAML(t *testing.T) { + rayJobWithYAML := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnvYAML: ` +working_dir: "https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip" +pip: ["python-multipart==0.0.6"] +`, + Metadata: map[string]string{ + "testKey": "testValue", + }, + RayClusterSpec: &rayv1alpha1.RayClusterSpec{ + RayVersion: "2.6.0", + }, + Entrypoint: "echo hello", + }, + Status: rayv1alpha1.RayJobStatus{ + DashboardURL: "http://127.0.0.1:8265", + JobId: "testJobId", + }, + } + expected := []string{ + "ray", "job", "submit", "--address", "http://127.0.0.1:8265", + "--runtime-env-json", `{"working_dir":"https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip","pip":["python-multipart==0.0.6"]}`, + "--metadata-json", `{"testKey":"testValue"}`, + "--submission-id", "testJobId", + "--", + "echo", "hello", + } + command, err := GetK8sJobCommand(rayJobWithYAML) + assert.NoError(t, err) + + // Ensure the slices are the same length. + assert.Equal(t, len(expected), len(command)) + + for i := 0; i < len(expected); i++ { + if expected[i] == "--runtime-env-json" { + // Decode the JSON string from the next element. + var expectedMap, actualMap map[string]interface{} + err1 := json.Unmarshal([]byte(expected[i+1]), &expectedMap) + err2 := json.Unmarshal([]byte(command[i+1]), &actualMap) + + // If there's an error decoding either JSON string, it's an error in the test. + assert.NoError(t, err1) + assert.NoError(t, err2) + + // Compare the maps directly to avoid errors due to ordering. + assert.Equal(t, expectedMap, actualMap) + + // Skip the next element because we've just checked it. + i++ + } else { + // For non-JSON elements, compare them directly. + assert.Equal(t, expected[i], command[i]) + } + } +} + func TestMetadataRaisesErrorBeforeRay26(t *testing.T) { rayJob := &rayv1alpha1.RayJob{ Spec: rayv1alpha1.RayJobSpec{ diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index d0a190c4232..f587219fd85 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -360,6 +360,11 @@ func (r *RayJobReconciler) getSubmitterTemplate(rayJobInstance *rayv1alpha1.RayJ // If the command in the submitter pod template isn't set, use the default command. if len(submitterTemplate.Spec.Containers[0].Command) == 0 { + // Check for deprecated 'runtimeEnv' field usage and log a warning. + if len(rayJobInstance.Spec.RuntimeEnv) > 0 { + r.Log.Info("Warning: The 'runtimeEnv' field is deprecated. Please use 'runtimeEnvYAML' instead.") + } + k8sJobCommand, err := common.GetK8sJobCommand(rayJobInstance) if err != nil { return v1.PodTemplateSpec{}, err diff --git a/ray-operator/go.mod b/ray-operator/go.mod index a5ecf326420..907d99ad4dd 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -25,6 +25,7 @@ require ( k8s.io/code-generator v0.23.0 k8s.io/utils v0.0.0-20210930125809-cb0fa318a74b sigs.k8s.io/controller-runtime v0.11.1 + sigs.k8s.io/yaml v1.3.0 volcano.sh/apis v1.6.0-alpha.0.0.20221012070524-685db38b4fae ) @@ -84,5 +85,4 @@ require ( k8s.io/kube-openapi v0.0.0-20211115234752-e816edb12b65 // indirect sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 // indirect sigs.k8s.io/structured-merge-diff/v4 v4.2.0 // indirect - sigs.k8s.io/yaml v1.3.0 // indirect ) From 3a7a17fcddc9e6933b72f8263e7aaa27c98b45e3 Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Wed, 23 Aug 2023 10:08:53 -0700 Subject: [PATCH 25/56] Delete ray_v1alpha1_rayjob.batch-inference.yaml (#1360) Accidentally committed WIP file Signed-off-by: Archit Kulkarni --- .../ray_v1alpha1_rayjob.batch-inference.yaml | 179 ------------------ 1 file changed, 179 deletions(-) delete mode 100644 ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml deleted file mode 100644 index 8fdc51a647d..00000000000 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml +++ /dev/null @@ -1,179 +0,0 @@ -apiVersion: ray.io/v1alpha1 -kind: RayJob -metadata: - name: rayjob-sample -spec: - entrypoint: python /home/ray/samples/sample_code.py - # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. - # shutdownAfterJobFinishes: false - # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. - # ttlSecondsAfterFinished: 10 - # Runtime env decoded to { - # { - # "pip": [ - # "torch", - # "torchvision", - # "Pillow", - # "transformers" - # ] - # } - runtimeEnv: ewogICJwaXAiOiBbCiAgICAidG9yY2giLAogICAgInRvcmNodmlzaW9uIiwKICAgICJQaWxsb3ciLAogICAgInRyYW5zZm9ybWVycyIKICBdCn0= - # Suspend specifies whether the RayJob controller should create a RayCluster instance. - # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. - # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. - # suspend: false - # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. - rayClusterSpec: - rayVersion: '2.6.3' # should match the Ray version in the image of the containers - # Ray head pod template - 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: - dashboard-host: '0.0.0.0' - #pod template - template: - spec: - containers: - - name: ray-head - image: rayproject/ray-ml:2.6.3-gpu - ports: - - containerPort: 6379 - name: gcs-server - - containerPort: 8265 # Ray dashboard - name: dashboard - - containerPort: 10001 - name: client - resources: - limits: - cpu: 2 - memory: 8Gi - requests: - cpu: 2 - memory: 8Gi - volumeMounts: - - mountPath: /home/ray/samples - name: code-sample - volumes: - # You set volumes at the Pod level, then mount them into containers inside that Pod - - name: code-sample - configMap: - # Provide the name of the ConfigMap you want to mount. - name: ray-job-code-sample - # An array of keys from the ConfigMap to create as files - items: - - key: sample_code.py - path: sample_code.py - workerGroupSpecs: - # the pod replicas in this group typed worker - - replicas: 1 - minReplicas: 1 - maxReplicas: 5 - # logical group name, for this called small-group, also can be functional - 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: - resources: '"{\"accelerator_type_cpu\": 48, \"accelerator_type_a10\": 2, \"accelerator_type_a100\": 2}"' - #pod template - template: - spec: - containers: - - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' - image: rayproject/ray-ml:2.6.3-gpu - lifecycle: - preStop: - exec: - command: [ "/bin/sh","-c","ray stop" ] - resources: - limits: - cpu: "48" - memory: "192G" - nvidia.com/gpu: 4 - requests: - cpu: "36" - memory: "128G" - nvidia.com/gpu: 4 - nodeSelector: - cloud.google.com/gke-accelerator: nvidia-tesla-t4 - # SubmitterPodTemplate is the template for the pod that will run the `ray job submit` command against the RayCluster. - # If SubmitterPodTemplate is specified, the first container is assumed to be the submitter container. - # submitterPodTemplate: - # spec: - # restartPolicy: Never - # containers: - # - name: my-custom-rayjob-submitter-pod - # image: rayproject/ray:2.6.3 - # # If Command is not specified, the correct command will be supplied at runtime using the RayJob spec `entrypoint` field. - # # Specifying Command is not recommended. - # # command: ["ray job submit --address=http://rayjob-sample-raycluster-v6qcq-head-svc.default.svc.cluster.local:8265 -- echo hello world"] - - -######################Ray code sample################################# -# this sample is from https://docs.ray.io/en/latest/cluster/job-submission.html#quick-start-example -# it is mounted into the container and executed to show the Ray job at work ---- -apiVersion: v1 -kind: ConfigMap -metadata: - name: ray-job-code-sample -data: - sample_code.py: | - import ray - - s3_uri = "s3://anonymous@air-example-data-2/imagenette2/val/" - - ds = ray.data.read_images( - s3_uri, mode="RGB" - ) - ds - # TODO(archit) need to install Pillow, pytorch or tf or flax (pip install torch torchvision torchaudio) - from typing import Dict - import numpy as np - - from transformers import pipeline - from PIL import Image - - # Pick the largest batch size that can fit on our GPUs - BATCH_SIZE = 1024 - - # TODO(archit) basic step - - # single_batch = ds.take_batch(10) - - # from PIL import Image - - # img = Image.fromarray(single_batch["image"][0]) - # # display image - # img.show() - # from transformers import pipeline - # from PIL import Image - - # # If doing CPU inference, set device="cpu" instead. - # classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") - # outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) - # del classifier # Delete the classifier to free up GPU memory. - # print(outputs) - - @ray.remote(num_gpus=1) - def do_single_batch(): - single_batch = ds.take_batch(10) - - from PIL import Image - - img = Image.fromarray(single_batch["image"][0]) - # display image - img.show() - from transformers import pipeline - from PIL import Image - - # If doing CPU inference, set device="cpu" instead. - classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") - outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) - del classifier # Delete the classifier to free up GPU memory. - print(outputs) - return outputs - - print(ray.get(do_single_batch.remote())) \ No newline at end of file From de8bc269f3615ab525704b53a28106893735d241 Mon Sep 17 00:00:00 2001 From: Darren Date: Wed, 23 Aug 2023 23:50:38 -0700 Subject: [PATCH 26/56] [Feature] Allow RayCluster Helm chart to specify different images for different worker groups (#1352) Allow RayCluster Helm chart to specify different images for different worker groups --- .../templates/raycluster-cluster.yaml | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/helm-chart/ray-cluster/templates/raycluster-cluster.yaml b/helm-chart/ray-cluster/templates/raycluster-cluster.yaml index e282d77a61d..0c13e86eafd 100644 --- a/helm-chart/ray-cluster/templates/raycluster-cluster.yaml +++ b/helm-chart/ray-cluster/templates/raycluster-cluster.yaml @@ -48,8 +48,13 @@ spec: containers: - volumeMounts: {{- toYaml .Values.head.volumeMounts | nindent 12 }} name: ray-head + {{- if .Values.head.image }} + image: {{ .Values.head.image.repository }}:{{ .Values.head.image.tag }} + imagePullPolicy: {{ .Values.head.image.pullPolicy }} + {{- else }} image: {{ .Values.image.repository }}:{{ .Values.image.tag }} imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml .Values.head.resources | nindent 14 }} securityContext: {{- toYaml .Values.head.securityContext | nindent 14 }} @@ -120,8 +125,13 @@ spec: containers: - volumeMounts: {{- toYaml $values.volumeMounts | nindent 12 }} name: ray-worker - image: {{ $.Values.image.repository }}:{{ $.Values.image.tag }} - imagePullPolicy: {{ $.Values.image.pullPolicy }} + {{- if $values.image }} + image: {{ $values.image.repository }}:{{ $values.image.tag }} + imagePullPolicy: {{ $values.image.pullPolicy }} + {{- else }} + image: {{ .Values.image.repository }}:{{ .Values.image.tag }} + imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml $values.resources | nindent 14 }} securityContext: {{- toYaml $values.securityContext | nindent 14 }} @@ -190,8 +200,13 @@ spec: containers: - volumeMounts: {{- toYaml .Values.worker.volumeMounts | nindent 12 }} name: ray-worker + {{- if .Values.worker.image }} + image: {{ .Values.worker.image.repository }}:{{ .Values.worker.image.tag }} + imagePullPolicy: {{ .Values.worker.image.pullPolicy }} + {{- else }} image: {{ .Values.image.repository }}:{{ .Values.image.tag }} imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml .Values.worker.resources | nindent 14 }} securityContext: {{- toYaml .Values.worker.securityContext | nindent 14 }} From 106490e1d1d70f9f8800f25a81d961da28cbdfe3 Mon Sep 17 00:00:00 2001 From: Laurentiu Bradin <109964136+z103cb@users.noreply.github.com> Date: Thu, 17 Aug 2023 21:51:28 +0300 Subject: [PATCH 27/56] Bump the golangci-lint version in the api server makefile (#1342) Bump the golangci-lint version in the api server makefile --- apiserver/Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apiserver/Makefile b/apiserver/Makefile index 0462d9257cd..b02a76e2a96 100644 --- a/apiserver/Makefile +++ b/apiserver/Makefile @@ -122,7 +122,7 @@ KIND ?= $(REPO_ROOT_BIN)/kind KUSTOMIZE_VERSION ?= v3.8.7 GOFUMPT_VERSION ?= v0.3.1 GOIMPORTS_VERSION ?= latest -GOLANGCI_LINT_VERSION ?= v1.50.1 +GOLANGCI_LINT_VERSION ?= v1.54.1 KIND_VERSION ?= v0.19.0 KUSTOMIZE_INSTALL_SCRIPT ?= "https://raw.githubusercontent.com/kubernetes-sigs/kustomize/master/hack/install_kustomize.sh" From 0d3d696087a959dac33b7d30c942770b77388d5f Mon Sep 17 00:00:00 2001 From: Praveen Date: Thu, 17 Aug 2023 16:01:13 -0700 Subject: [PATCH 28/56] Documentation and example for running simple NLP service on kuberay (#1340) * add service yaml for nlp * Documentation fixes * Fix instructions * Apply suggestions from code review Co-authored-by: Kai-Hsun Chen Signed-off-by: Praveen * Fix tolerations comment * review comments * Update docs/guidance/stable-diffusion-rayservice.md Signed-off-by: Kai-Hsun Chen --------- Signed-off-by: Praveen Signed-off-by: Kai-Hsun Chen Co-authored-by: Kai-Hsun Chen --- docs/guidance/aws-eks-gpu-cluster.md | 2 +- docs/guidance/stable-diffusion-rayservice.md | 10 ++- docs/guidance/text-summarizer-rayservice.md | 69 ++++++++++++++++ .../samples/ray-service.text-sumarizer.yaml | 79 +++++++++++++++++++ 4 files changed, 158 insertions(+), 2 deletions(-) create mode 100644 docs/guidance/text-summarizer-rayservice.md create mode 100644 ray-operator/config/samples/ray-service.text-sumarizer.yaml diff --git a/docs/guidance/aws-eks-gpu-cluster.md b/docs/guidance/aws-eks-gpu-cluster.md index e77470403d2..f18998b3ae2 100644 --- a/docs/guidance/aws-eks-gpu-cluster.md +++ b/docs/guidance/aws-eks-gpu-cluster.md @@ -32,7 +32,7 @@ Create a GPU node group for Ray GPU workers. > **Note:** If you encounter permission issues with `kubectl`, follow "Step 2: Configure your computer to communicate with your cluster" in the [AWS documentation](https://docs.aws.amazon.com/eks/latest/userguide/getting-started-console.html#). -2. Please install the NVIDIA device plugin. +2. Please install the NVIDIA device plugin. Note: You don't need this if you used `BOTTLEROCKET_x86_64_NVIDIA` image in above step * Install the DaemonSet for NVIDIA device plugin to run GPU enabled containers in your Amazon EKS cluster. You can refer to the [Amazon EKS optimized accelerated Amazon Linux AMIs](https://docs.aws.amazon.com/eks/latest/userguide/eks-optimized-ami.html#gpu-ami) or [NVIDIA/k8s-device-plugin](https://github.com/NVIDIA/k8s-device-plugin) repository for more details. * If the GPU nodes have taints, add `tolerations` to `nvidia-device-plugin.yml` to enable the DaemonSet to schedule Pods on the GPU nodes. diff --git a/docs/guidance/stable-diffusion-rayservice.md b/docs/guidance/stable-diffusion-rayservice.md index 5421660476e..94532531972 100644 --- a/docs/guidance/stable-diffusion-rayservice.md +++ b/docs/guidance/stable-diffusion-rayservice.md @@ -21,7 +21,7 @@ kubectl apply -f ray-service.stable-diffusion.yaml This RayService configuration contains some important settings: -* Its `tolerations` for workers match the taints on the GPU node group. Without the tolerations, worker Pods won't be scheduled on GPU nodes. +* The `tolerations` for workers allow them to be scheduled on nodes without any taints or on nodes with specific taints. However, workers will only be scheduled on GPU nodes because we set `nvidia.com/gpu: 1` in the Pod's resource configurations. ```yaml # Please add the following taints to the GPU node. tolerations: @@ -34,6 +34,14 @@ This RayService configuration contains some important settings: ## Step 4: Forward the port of Serve +First get the service name from this command. + +```sh +kubectl get services +``` + +Then, port forward to the serve. + ```sh kubectl port-forward svc/stable-diffusion-serve-svc 8000 ``` diff --git a/docs/guidance/text-summarizer-rayservice.md b/docs/guidance/text-summarizer-rayservice.md new file mode 100644 index 00000000000..ca758de7687 --- /dev/null +++ b/docs/guidance/text-summarizer-rayservice.md @@ -0,0 +1,69 @@ +# Serve a text summarizer using RayService + +> **Note:** The Python files for the Ray Serve application and its client are in the [ray-project/serve_config_examples](https://github.com/ray-project/serve_config_examples) repo. + +## Step 1: Create a Kubernetes cluster with GPUs + +Follow [aws-eks-gpu-cluster.md](./aws-eks-gpu-cluster.md) or [gcp-gke-gpu-cluster.md](./gcp-gke-gpu-cluster.md) to create a Kubernetes cluster with 1 CPU node and 1 GPU node. + +## Step 2: Install KubeRay operator + +Follow [this document](../../helm-chart/kuberay-operator/README.md) to install the latest stable KubeRay operator via Helm repository. +Please note that the YAML file in this example uses `serveConfigV2`, which is supported starting from KubeRay v0.6.0. + +## Step 3: Install a RayService + +```sh +# path: ray-operator/config/samples/ +kubectl apply -f ray-service.text-sumarizer.yaml +``` + +This RayService configuration contains some important settings: + +* The `tolerations`` for workers allow them to be scheduled on nodes without any taints or on nodes with specific taints. However, workers will only be scheduled on GPU nodes because we set `nvidia.com/gpu: 1` in the Pod's resource configurations. + ```yaml + # Please add the following taints to the GPU node. + tolerations: + - key: "ray.io/node-type" + operator: "Equal" + value: "worker" + effect: "NoSchedule" + ``` + +## Step 4: Forward the port of Serve + +First get the service name from this command. + +```sh +kubectl get services +``` + +Then, port forward to the serve. + +```sh +kubectl port-forward svc/text-summarizer-serve-svc 8000 +``` + +Note that the RayService's Kubernetes service will be created after the Serve applications are ready and running. This process may take approximately 1 minute after all Pods in the RayCluster are running. + +## Step 5: Send a request to the text_summarizer model + +```sh +# Step 5.1: Download `text_summarizer_req.py` +curl -LO https://raw.githubusercontent.com/ray-project/serve_config_examples/master/text_summarizer/text_summarizer_req.py + +# Step 5.2: Send a request to the Summarizer model. +python text_summarizer_req.py +# Check printed to console +``` + +## Step 6: Delete your service + +```sh +# path: ray-operator/config/samples/ +kubectl delete -f ray-service.text-sumarizer.yaml +``` + +## Step 7: Uninstall your kuberay operator + +Follow [this document](../../helm-chart/kuberay-operator/README.md) to uninstall the latest stable KubeRay operator via Helm repository. \ No newline at end of file diff --git a/ray-operator/config/samples/ray-service.text-sumarizer.yaml b/ray-operator/config/samples/ray-service.text-sumarizer.yaml new file mode 100644 index 00000000000..fbf9d3e6464 --- /dev/null +++ b/ray-operator/config/samples/ray-service.text-sumarizer.yaml @@ -0,0 +1,79 @@ +apiVersion: ray.io/v1alpha1 +kind: RayService +metadata: + name: text-summarizer +spec: + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. + serveConfigV2: | + applications: + - name: text_summarizer + import_path: text_summarizer.text_summarizer:deployment + runtime_env: + working_dir: "https://github.com/ray-project/serve_config_examples/archive/refs/heads/master.zip" + rayClusterConfig: + rayVersion: '2.6.3' # Should match the Ray version in the image of the containers + ######################headGroupSpecs################################# + # Ray head pod template. + 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: + dashboard-host: '0.0.0.0' + # Pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray-ml:2.6.3 + ports: + - containerPort: 6379 + name: gcs + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + - containerPort: 8000 + name: serve + volumeMounts: + - mountPath: /tmp/ray + name: ray-logs + resources: + limits: + cpu: "2" + memory: "8G" + requests: + cpu: "2" + memory: "8G" + volumes: + - name: ray-logs + emptyDir: {} + workerGroupSpecs: + # The pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 10 + groupName: gpu-group + rayStartParams: {} + # Pod template + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray-ml:2.6.3 + resources: + limits: + cpu: 4 + memory: "16G" + nvidia.com/gpu: 1 + requests: + cpu: 3 + memory: "12G" + nvidia.com/gpu: 1 + # Please add the following taints to the GPU node. + tolerations: + - key: "ray.io/node-type" + operator: "Equal" + value: "worker" + effect: "NoSchedule" From 1e93a67dcad40e7d3598aca0ffe7bef2c3c0b435 Mon Sep 17 00:00:00 2001 From: Laurentiu Bradin <109964136+z103cb@users.noreply.github.com> Date: Sat, 19 Aug 2023 01:44:36 +0300 Subject: [PATCH 29/56] Removed use of the of BUILD_FLAGS in apiserver makefile (#1336) Removed use of the of BUILD_FLAGS in apiserver makefile --- apiserver/Makefile | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/apiserver/Makefile b/apiserver/Makefile index b02a76e2a96..7745dd129e5 100644 --- a/apiserver/Makefile +++ b/apiserver/Makefile @@ -62,17 +62,8 @@ lint: golangci-lint fmt vet fumpt imports ## Run the linter. ##@ Build -COMMIT := $(shell git rev-parse --short HEAD) -VERSION := $(shell git describe --tags $(shell git rev-list --tags --max-count=1)) -DATE=$(shell date -u +'%Y-%m-%dT%H:%M:%SZ') -REPO=github.com/ray-project/kuberay - -BUILD_FLAGS ?= -ldflags="-X '${REPO}/apiserver/cmd/version.Version=$(VERSION)' \ - -X '${REPO}/apiserver/cmd/version.gitCommit=$(COMMIT)' \ - -X '${REPO}/apiserver/cmd/version.buildDate=$(DATE)'" - build: fmt vet fumpt imports lint ## Build api server binary. - go build $(BUILD_FLAGS) -o ${REPO_ROOT_BIN}/kuberay-apiserver cmd/main.go + go build -o ${REPO_ROOT_BIN}/kuberay-apiserver cmd/main.go run: fmt vet fumpt imports lint ## Run the api server from your host. go run -race cmd/main.go From 40fe9d472f900a2ef056f02be8ea8428d46272cf Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 21 Aug 2023 13:46:20 -0700 Subject: [PATCH 30/56] [GCS FT][Refactor] Redefine the behavior for deleting Pods and stop listening to Kubernetes events (#1341) Redefine the behavior for deleting Pods and stop listening to Kubernetes events --- .github/workflows/test-job.yaml | 6 +- apiserver/DEVELOPMENT.md | 2 +- .../controllers/ray/raycluster_controller.go | 191 +++------ .../ray/raycluster_controller_fake_test.go | 385 ++++++++++-------- tests/compatibility-test.py | 68 ++-- tests/config/ray-cluster.ray-ft.yaml.template | 4 +- tests/framework/utils.py | 4 + tests/kuberay_utils/utils.py | 107 ++--- 8 files changed, 358 insertions(+), 409 deletions(-) diff --git a/.github/workflows/test-job.yaml b/.github/workflows/test-job.yaml index 65e7e428969..7b73d784a17 100644 --- a/.github/workflows/test-job.yaml +++ b/.github/workflows/test-job.yaml @@ -41,7 +41,7 @@ jobs: uses: golangci/golangci-lint-action@v2 with: # Optional: version of golangci-lint to use in form of v1.2 or v1.2.3 or `latest` to use the latest version - version: v1.50.1 + version: v1.54.1 # Optional: working directory, useful for monorepos working-directory: ./ray-operator @@ -65,7 +65,7 @@ jobs: - name: Run linter against apiserver uses: golangci/golangci-lint-action@v2 with: - version: v1.50.1 + version: v1.54.1 working-directory: ./apiserver args: --timeout=3m skip-go-installation: true @@ -74,7 +74,7 @@ jobs: - name: Run linter against cli uses: golangci/golangci-lint-action@v2 with: - version: v1.50.1 + version: v1.54.1 working-directory: ./cli args: --timeout=3m skip-go-installation: true diff --git a/apiserver/DEVELOPMENT.md b/apiserver/DEVELOPMENT.md index 7a7611b8012..5bc18a400aa 100644 --- a/apiserver/DEVELOPMENT.md +++ b/apiserver/DEVELOPMENT.md @@ -21,7 +21,7 @@ Typing `make dev-tools` will download and install all of them. The `make clean-d | Software | Version | Link | | :------- | :------: | -----------------------------------------------------------------------:| | kind | v0.19.0 | [Install](https://kind.sigs.k8s.io/docs/user/quick-start/#installation) | -| golangci-lint | v1.50.1 | [Install](https://golangci-lint.run/usage/install/) | +| golangci-lint | v1.54.1 | [Install](https://golangci-lint.run/usage/install/) | | kustomize | v3.8.7 | [install](https://kubectl.docs.kubernetes.io/installation/kustomize/) | | gofumpt | v0.3.1 | To install `go install mvdan.cc/gofumpt@v0.3.1` | | goimports | latest | To install `go install golang.org/x/tools/cmd/goimports@latest` | diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 12d82eb75b2..d36600d839f 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -9,8 +9,6 @@ import ( "strings" "time" - "sigs.k8s.io/controller-runtime/pkg/event" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler" @@ -35,11 +33,9 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" controller "sigs.k8s.io/controller-runtime/pkg/controller" "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil" - "sigs.k8s.io/controller-runtime/pkg/handler" "sigs.k8s.io/controller-runtime/pkg/manager" "sigs.k8s.io/controller-runtime/pkg/predicate" "sigs.k8s.io/controller-runtime/pkg/reconcile" - "sigs.k8s.io/controller-runtime/pkg/source" ) var ( @@ -104,12 +100,6 @@ type RayClusterReconciler struct { func (r *RayClusterReconciler) Reconcile(ctx context.Context, request ctrl.Request) (ctrl.Result, error) { var err error - // Try to fetch the Event instance - event := &corev1.Event{} - if err = r.Get(ctx, request.NamespacedName, event); err == nil { - return r.eventReconcile(ctx, request, event) - } - // Try to fetch the RayCluster instance instance := &rayv1alpha1.RayCluster{} if err = r.Get(ctx, request.NamespacedName, instance); err == nil { @@ -126,77 +116,6 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, request ctrl.Reque return ctrl.Result{}, client.IgnoreNotFound(err) } -func (r *RayClusterReconciler) eventReconcile(ctx context.Context, request ctrl.Request, event *corev1.Event) (ctrl.Result, error) { - var unhealthyPod *corev1.Pod - pods := corev1.PodList{} - - // we only care about pod events - if event.InvolvedObject.Kind != "Pod" || event.Type != "Warning" || event.Reason != "Unhealthy" || - !strings.Contains(event.Message, "Readiness probe failed") { - // This is not supposed to happen since we already filter events in the watch - msg := fmt.Sprintf("unexpected event, we should have already filtered these conditions: %v", event) - r.Log.Error(fmt.Errorf(msg), msg, "event", event) - return ctrl.Result{}, nil - } - - _ = r.Log.WithValues("event", request.NamespacedName) - - options := []client.ListOption{ - client.MatchingFields(map[string]string{podUIDIndexField: string(event.InvolvedObject.UID)}), - client.InNamespace(event.InvolvedObject.Namespace), - client.MatchingLabels(map[string]string{common.RayNodeLabelKey: "yes"}), - } - - if err := r.List(ctx, &pods, options...); err != nil { - return ctrl.Result{}, err - } - - if len(pods.Items) == 0 { - r.Log.Info("no ray node pod found for event", "event", event) - return ctrl.Result{}, nil - } else if len(pods.Items) > 1 { - // This happens when we use fake client - r.Log.Info("are you running in test mode?") - for _, pod := range pods.Items { - if pod.Name == event.InvolvedObject.Name { - unhealthyPod = &pod - break - } - } - } else { - r.Log.Info("found unhealthy ray node", "pod name", event.InvolvedObject.Name) - unhealthyPod = &pods.Items[0] - } - - if unhealthyPod.Annotations == nil { - r.Log.Info("The unhealthy ray node not found", "pod name", event.InvolvedObject.Name) - return ctrl.Result{}, nil - } - - if enabledString, ok := unhealthyPod.Annotations[common.RayFTEnabledAnnotationKey]; ok { - if strings.ToLower(enabledString) != "true" { - r.Log.Info("FT not enabled skipping event reconcile for pod.", "pod name", unhealthyPod.Name) - return ctrl.Result{}, nil - } - } else { - r.Log.Info("HAEnabled annotation not found", "pod name", unhealthyPod.Name) - return ctrl.Result{}, nil - } - - if !utils.IsRunningAndReady(unhealthyPod) { - if v, ok := unhealthyPod.Annotations[common.RayNodeHealthStateAnnotationKey]; !ok || v != common.PodUnhealthy { - updatedPod := unhealthyPod.DeepCopy() - updatedPod.Annotations[common.RayNodeHealthStateAnnotationKey] = common.PodUnhealthy - r.Log.Info("mark pod unhealthy and need for a rebuild", "pod", unhealthyPod) - if err := r.Update(ctx, updatedPod); err != nil { - return ctrl.Result{}, err - } - } - } - - return ctrl.Result{}, nil -} - func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request ctrl.Request, instance *rayv1alpha1.RayCluster) (ctrl.Result, error) { // Please do NOT modify `originalRayClusterInstance` in the following code. originalRayClusterInstance := instance.DeepCopy() @@ -416,22 +335,37 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv // Reconcile head Pod if len(headPods.Items) == 1 { headPod := headPods.Items[0] - r.Log.Info("reconcilePods", "head pod found", headPod.Name) + r.Log.Info("reconcilePods", "Found 1 head Pod", headPod.Name) + // TODO (kevin85421): Consider deleting a head Pod if its Ray container restarts excessively, as this + // might suggest an unhealthy Kubernetes node. Deleting and then recreating the head Pod might allow + // it to be scheduled on a different node. However, it's aggressive to delete a head Pod that is not + // in a terminated state (i.e., `Failed` or `Succeeded`). We should only delete a head Pod when GCS + // fault tolerance is enabled, and drain the head Pod before deleting it. if headPod.Status.Phase == corev1.PodRunning || headPod.Status.Phase == corev1.PodPending { - r.Log.Info("reconcilePods", "head pod is up and running... checking workers", headPod.Name) - } else if headPod.Status.Phase == corev1.PodFailed && strings.Contains(headPod.Status.Reason, "Evicted") { - // Handle evicted pod - r.Log.Info("reconcilePods", "head pod has been evicted and controller needs to replace the pod", headPod.Name) + r.Log.Info("reconcilePods", "The head pod is Running or Pending... checking workers", headPod.Name) + } else { + if headPod.Spec.RestartPolicy == corev1.RestartPolicyAlways { + // Based on my observation, a Pod with `RestartPolicy: Always` will never be in the terminated states (i.e., `Failed` or `Succeeded`). + // However, I couldn't find any well-defined behavior in the Kubernetes documentation, so I can't guarantee that the status transition + // from `Running` to `Failed / Succeeded` and back to `Running` won't occur when we kill the main process (i.e., `ray start` in KubeRay) + // in the head Pod. Therefore, I've added this check as a safeguard. + message := fmt.Sprintf( + "The status of the head Pod %s is %s. However, KubeRay will not delete the Pod because its restartPolicy is set to 'Always' "+ + "and it should be able to restart automatically.", headPod.Name, headPod.Status.Phase) + r.Log.Info(message) + return fmt.Errorf(message) + } + message := fmt.Sprintf("The status of the head Pod %s is %s which is a terminal state. It is not expected that the head pod ever be in a terminal state, so KubeRay will delete the Pod and recreate the head Pod in the next reconciliation.", headPod.Name, headPod.Status.Phase) + r.Log.Info(message) if err := r.Delete(ctx, &headPod); err != nil { return err } - } else { - return fmt.Errorf("head pod %s is not running nor pending", headPod.Name) + r.Recorder.Eventf(instance, corev1.EventTypeNormal, "Deleted", "Deleted head Pod %s; status: %s", headPod.Name, headPod.Status.Phase) + return fmt.Errorf(message) } - } - if len(headPods.Items) == 0 || headPods.Items == nil { - // create head pod - r.Log.Info("reconcilePods", "creating head pod for cluster", instance.Name) + } else if len(headPods.Items) == 0 { + // Create head Pod if it does not exist. + r.Log.Info("reconcilePods", "Found 0 head Pods; creating a head Pod for the RayCluster.", instance.Name) common.CreatedClustersCounterInc(instance.Namespace) if err := r.createHeadPod(ctx, *instance); err != nil { common.FailedClustersCounterInc(instance.Namespace) @@ -439,11 +373,12 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv } common.SuccessfulClustersCounterInc(instance.Namespace) } else if len(headPods.Items) > 1 { - r.Log.Info("reconcilePods", "more than 1 head pod found for cluster", instance.Name) + r.Log.Info("reconcilePods", fmt.Sprintf("Found %d head Pods; deleting extra head Pods.", len(headPods.Items)), instance.Name) + // TODO (kevin85421): In-place update may not be a good idea. itemLength := len(headPods.Items) for index := 0; index < itemLength; index++ { if headPods.Items[index].Status.Phase == corev1.PodRunning || headPods.Items[index].Status.Phase == corev1.PodPending { - // Remove the healthy pod at index i from the list of pods to delete + // Remove the healthy pod at index i from the list of pods to delete headPods.Items[index] = headPods.Items[len(headPods.Items)-1] // replace last element with the healthy head. headPods.Items = headPods.Items[:len(headPods.Items)-1] // Truncate slice. itemLength-- @@ -455,18 +390,6 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv return err } } - } else { - // we have exactly one head pod running - if headPods.Items[0].Annotations != nil { - if v, ok := headPods.Items[0].Annotations[common.RayNodeHealthStateAnnotationKey]; ok && v == common.PodUnhealthy { - if err := r.Delete(ctx, &headPods.Items[0]); err != nil { - return err - } - r.Log.Info(fmt.Sprintf("need to delete unhealthy head pod %s", headPods.Items[0].Name)) - // we are deleting the head pod now, let's reconcile again later - return nil - } - } } if ForcedClusterUpgrade { @@ -531,21 +454,36 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv return err } - // delete the worker pod if it is marked unhealthy + // Delete unhealthy worker Pods + numDeletedUnhealthyWorkerPods := 0 for _, workerPod := range workerPods.Items { - if workerPod.Annotations == nil { - continue - } - if v, ok := workerPod.Annotations[common.RayNodeHealthStateAnnotationKey]; ok && v == common.PodUnhealthy { - r.Log.Info(fmt.Sprintf("deleting unhealthy worker pod %s", workerPod.Name)) + // TODO (kevin85421): Consider deleting a worker Pod if its Ray container restarts excessively, + // as this could suggest an unhealthy Kubernetes node. Deleting and then recreating the worker Pod + // might allow it to be scheduled on a different node. Compared to deleting a head Pod, removing a + // worker Pod is less aggressive and aligns more closely with the behavior of the Ray Autoscaler. + // Nevertheless, we should still carefully drain the node before deleting the worker Pod. Enabling + // GCS fault tolerance might not be necessary when deleting worker Pods. Note that the Ray Autoscaler + // will not delete any worker Pods that have never been registered with the Ray cluster. Therefore, + // we may need to address the Ray Autoscaler's blind spots. + + // TODO (kevin85421): We may need to allow users to configure how many `Failed` or `Succeeded` Pods should be kept for debugging purposes. + if workerPod.Spec.RestartPolicy != corev1.RestartPolicyAlways && !isPodRunningOrPendingAndNotDeleting(workerPod) { + // If the Pod's status is `Failed` or `Succeeded`, the Pod will not restart and we can safely delete it. + numDeletedUnhealthyWorkerPods++ + r.Log.Info(fmt.Sprintf("The worker Pod %s status is %s. KubeRay will delete the Pod because the status is not Running or Pending. ", workerPod.Name, workerPod.Status.Phase)) if err := r.Delete(ctx, &workerPod); err != nil { return err + } else { + r.Recorder.Eventf(instance, corev1.EventTypeNormal, "Deleted", "Deleted worker Pod %s; status: %s", workerPod.Name, workerPod.Status.Phase) } - // we are deleting one worker pod now, let's reconcile again later - return nil } } + // If we delete unhealthy Pods, we will not create new Pods in this reconciliation. + if numDeletedUnhealthyWorkerPods > 0 { + return fmt.Errorf("Delete %d unhealthy worker Pods.", numDeletedUnhealthyWorkerPods) + } + // Always remove the specified WorkersToDelete - regardless of the value of Replicas. // Essentially WorkersToDelete has to be deleted to meet the expectations of the Autoscaler. deletedWorkers := make(map[string]struct{}) @@ -813,31 +751,6 @@ func (r *RayClusterReconciler) SetupWithManager(mgr ctrl.Manager, reconcileConcu predicate.LabelChangedPredicate{}, predicate.AnnotationChangedPredicate{}, ))). - Watches(&source.Kind{Type: &corev1.Event{}}, - &handler.EnqueueRequestForObject{}, - builder.WithPredicates(predicate.Funcs{ - CreateFunc: func(e event.CreateEvent) bool { - if eventObj, ok := e.Object.(*corev1.Event); ok { - if eventObj.InvolvedObject.Kind != "Pod" || eventObj.Type != "Warning" || - eventObj.Reason != "Unhealthy" || !strings.Contains(eventObj.Message, "Readiness probe failed") { - // only care about pod unhealthy events - return false - } - return true - } - return false - }, - UpdateFunc: func(e event.UpdateEvent) bool { - return false - }, - DeleteFunc: func(e event.DeleteEvent) bool { - return false - }, - GenericFunc: func(e event.GenericEvent) bool { - return false - }, - }), - ). Owns(&corev1.Pod{}). Owns(&corev1.Service{}) diff --git a/ray-operator/controllers/ray/raycluster_controller_fake_test.go b/ray-operator/controllers/ray/raycluster_controller_fake_test.go index bb3e1728159..dc06c1f345e 100644 --- a/ray-operator/controllers/ray/raycluster_controller_fake_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_fake_test.go @@ -17,7 +17,6 @@ package ray import ( "context" - "fmt" "os" "testing" "time" @@ -359,158 +358,6 @@ func setupTest(t *testing.T) { workerSelector = labels.NewSelector().Add(*instanceReq).Add(*groupNameReq) } -// TestReconcile_UnhealthyEvent tests the case where we have unhealthy events -// and we want to update the corresponding pods. -func TestReconcile_UnhealthyEvent(t *testing.T) { - setupTest(t) - - testPodName := "eventPod" - - // testPodEventName is the name of the event that will be created for testPodName - // The name of the event is generated by concatenating the pod name and a - // meaningless random string - testPodEventName := fmt.Sprintf("%s.15f0c0c5c5c5c5c5", testPodName) - - // add a pod in a different namespace - newPods := []runtime.Object{ - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodName, - UID: types.UID(testPodName), - Namespace: "ns2", - Labels: map[string]string{ - common.RayNodeLabelKey: "yes", - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: "newPod2", - UID: types.UID("newPod2"), - Namespace: namespaceStr, - Labels: map[string]string{ - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - &corev1.Pod{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodName, - UID: types.UID(testPodName), - Namespace: namespaceStr, - Labels: map[string]string{ - common.RayNodeLabelKey: "yes", - common.RayClusterLabelKey: instanceName, - common.RayNodeGroupLabelKey: groupNameStr, - }, - Annotations: map[string]string{ - common.RayFTEnabledAnnotationKey: "true", - }, - }, - Spec: corev1.PodSpec{ - Containers: []corev1.Container{ - { - Name: "ray-worker", - Image: "rayproject/ray:2.2.0", - Command: []string{"echo"}, - Args: []string{"Hello Ray"}, - }, - }, - }, - Status: corev1.PodStatus{ - Phase: corev1.PodRunning, - }, - }, - } - - testPods = append(testPods, newPods...) - - fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(testPods...).Build() - ctx := context.Background() - - podList := corev1.PodList{} - err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) - - assert.Nil(t, err, "Fail to get pod list") - assert.Equal(t, len(testPods)-1, len(podList.Items), "Init pod list len is wrong") - - testRayClusterReconciler := &RayClusterReconciler{ - Client: fakeClient, - Recorder: &record.FakeRecorder{}, - Scheme: scheme.Scheme, - Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), - } - - // add event for reconcile - err = fakeClient.Create(ctx, &corev1.Event{ - ObjectMeta: metav1.ObjectMeta{ - Name: testPodEventName, - Namespace: namespaceStr, - }, - InvolvedObject: corev1.ObjectReference{ - Kind: "Pod", - Name: testPodName, - Namespace: namespaceStr, - }, - Reason: "Unhealthy", - Type: "Warning", - Message: "Readiness probe failed", - }) - assert.Nil(t, err, "Fail to create event") - - if _, err := testRayClusterReconciler.Reconcile(ctx, ctrl.Request{ - NamespacedName: types.NamespacedName{ - Namespace: namespaceStr, - Name: testPodEventName, - }, - }); err != nil { - assert.Nil(t, err, "Fail to reconcile") - } - - err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) - assert.Nil(t, err, "Fail to get pod list") - - for _, pod := range podList.Items { - if pod.Name == testPodName && pod.Namespace == namespaceStr { - assert.Equal(t, pod.Annotations[common.RayNodeHealthStateAnnotationKey], - common.PodUnhealthy, "Pod annotation is wrong") - } - } - - for _, pod := range podList.Items { - assert.Equal(t, corev1.PodRunning, pod.Status.Phase, "Pod phase is wrong") - } -} - func TestReconcile_RemoveWorkersToDelete_RandomDelete(t *testing.T) { setupTest(t) @@ -862,7 +709,7 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate the deletion of 2 worker Pods. After the deletion, the number of worker Pods should be 4. + // Simulate the deletion of 1 worker Pod. After the deletion, the number of worker Pods should be 4. err = fakeClient.Delete(ctx, &podList.Items[3]) assert.Nil(t, err, "Fail to delete pod") @@ -874,8 +721,8 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), } - // Since the desired state of the workerGroup is 3 replicas, - // the controller will delete a worker Pod to reach the goal state. + // Since the desired state of the workerGroup is 3 replicas, the controller + // will delete a worker Pod randomly to reach the goal state. err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) assert.Nil(t, err, "Fail to reconcile Pods") @@ -890,7 +737,7 @@ func TestReconcile_PodDeleted_DiffLess0_OK(t *testing.T) { "Replica number is wrong after reconcile expect %d actual %d", expectReplicaNum, len(podList.Items)) } -func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { +func TestReconcile_Diff0_WorkersToDelete_OK(t *testing.T) { setupTest(t) // TODO (kevin85421): The tests in this file are not independent. As a workaround, @@ -922,15 +769,6 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate 2 Pod fails. Because the workersToDelete also contains pod3 and pod4, the controller will - // delete these two Pods. After the deletion, the number of worker Pods should be 3. - podList.Items[3].Status.Phase = corev1.PodFailed - podList.Items[4].Status.Phase = corev1.PodFailed - err = fakeClient.Update(ctx, &podList.Items[3]) - assert.Nil(t, err, "Fail to get update pod status") - err = fakeClient.Update(ctx, &podList.Items[4]) - assert.Nil(t, err, "Fail to get update pod status") - // Initialize a new RayClusterReconciler. testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -939,8 +777,8 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), } - // Since the desired state of the workerGroup is 3 replicas, the controller - // will not create or delete any worker Pods. + // Pod3 and Pod4 should be deleted because of the workersToDelete. + // Hence, no failed Pods should exist in `podList`. err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) assert.Nil(t, err, "Fail to reconcile Pods") @@ -950,8 +788,6 @@ func TestReconcile_PodCrash_Diff0_WorkersToDelete_OK(t *testing.T) { }) assert.Nil(t, err, "Fail to get pod list after reconcile") - // Failed Pods (pod3, pod4) should be deleted because of the workersToDelete. - // Hence, no failed Pods should exist in `podList`. assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) assert.Equal(t, expectedNumWorkerPods, getNotFailedPodItemNum(podList), "Replica number is wrong after reconcile expect %d actual %d", expectReplicaNum, getNotFailedPodItemNum(podList)) @@ -1006,12 +842,6 @@ func TestReconcile_PodCrash_DiffLess0_OK(t *testing.T) { assert.Nil(t, err, "Fail to get pod list") assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") - // Simulate 1 Pod fails. Because the workersToDelete also contains pod3, the controller will - // delete pod3. After the deletion, the number of worker Pods should be 4. - podList.Items[3].Status.Phase = corev1.PodFailed - err = fakeClient.Update(ctx, &podList.Items[3]) - assert.Nil(t, err, "Fail to get update pod status") - // Initialize a new RayClusterReconciler. testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -1071,7 +901,7 @@ func TestReconcile_PodEvicted_DiffLess0_OK(t *testing.T) { podList.Items[0].Status.Phase = corev1.PodFailed podList.Items[0].Status.Reason = "Evicted" err = fakeClient.Update(ctx, &podList.Items[0]) - assert.Nil(t, err, "Fail to get update pod status") + assert.Nil(t, err, "Fail to update head Pod status") testRayClusterReconciler := &RayClusterReconciler{ Client: fakeClient, @@ -1081,7 +911,10 @@ func TestReconcile_PodEvicted_DiffLess0_OK(t *testing.T) { } err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) - assert.Nil(t, err, "Fail to reconcile Pods") + // The head Pod with the status `Failed` will be deleted, and the function will return an + // error to requeue the request with a short delay. If the function returns nil, the controller + // will requeue the request after RAYCLUSTER_DEFAULT_REQUEUE_SECONDS_ENV (default: 300) seconds. + assert.NotNil(t, err) // Filter head pod err = fakeClient.List(ctx, &podList, &client.ListOptions{ @@ -1725,3 +1558,199 @@ func TestCalculateStatus(t *testing.T) { assert.Equal(t, headNodeIP, newInstance.Status.Head.PodIP) assert.Equal(t, headServiceIP, newInstance.Status.Head.ServiceIP) } + +func Test_TerminatedWorkers_NoAutoscaler(t *testing.T) { + setupTest(t) + + // TODO (kevin85421): The tests in this file are not independent. As a workaround, + // I added the assertion to prevent the test logic from being affected by other changes. + // However, we should refactor the tests in the future. + + // This test makes some assumptions about the testRayCluster object. + // (1) 1 workerGroup + // (2) The goal state of the workerGroup is 3 replicas. + // (3) Set the `WorkersToDelete` field to an empty slice. + // (4) Disable autoscaling. + assert.Equal(t, 1, len(testRayCluster.Spec.WorkerGroupSpecs), "This test assumes only one worker group.") + expectedNumWorkerPods := int(*testRayCluster.Spec.WorkerGroupSpecs[0].Replicas) + assert.Equal(t, 3, expectedNumWorkerPods, "This test assumes the expected number of worker pods is 3.") + testRayCluster.Spec.WorkerGroupSpecs[0].ScaleStrategy.WorkersToDelete = []string{} + testRayCluster.Spec.EnableInTreeAutoscaling = nil + + // This test makes some assumptions about the testPods object. + // `testPods` contains 6 pods, including 1 head pod and 5 worker pods. + assert.Equal(t, 6, len(testPods), "This test assumes the testPods object contains 6 pods.") + numHeadPods := 1 + oldNumWorkerPods := len(testPods) - numHeadPods + + // Initialize a fake client with newScheme and runtimeObjects. + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(testPods...).Build() + ctx := context.Background() + + // Get the pod list from the fake client. + podList := corev1.PodList{} + err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, oldNumWorkerPods+numHeadPods, len(podList.Items), "Init pod list len is wrong") + + // Make sure all worker Pods are running. + for _, pod := range podList.Items { + pod.Status.Phase = corev1.PodRunning + err = fakeClient.Status().Update(ctx, &pod) + assert.Nil(t, err, "Fail to update pod status") + } + + // Initialize a new RayClusterReconciler. + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: scheme.Scheme, + Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), + } + + // Since the desired state of the workerGroup is 3 replicas, the controller + // will delete 2 worker Pods. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err, "Fail to reconcile Pods") + + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) + + // Update 1 worker Pod to Failed (a terminate state) state. + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Status().Update(ctx, &podList.Items[0]) + assert.Nil(t, err, "Fail to update Pod status") + + // Reconcile again, and the Failed worker Pod should be deleted even if the goal state of the workerGroup specifies 3 replicas. + // The function will return an error to requeue the request after a brief delay. Moreover, if there are unhealthy worker + // Pods to be deleted, the controller won't create new worker Pods during the same reconcile loop. As a result, the number of worker + // Pods will be (expectedNumWorkerPods - 1) after the reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods-1, len(podList.Items)) + + // Reconcile again, and the controller will create a new worker Pod to reach the goal state of the workerGroup. + // Note that the status of new worker Pod created by the fake client is empty, so we need to set all worker + // Pods to running state manually to avoid the new Pod being deleted in the next `reconcilePods` call. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) + for _, pod := range podList.Items { + pod.Status.Phase = corev1.PodRunning + err = fakeClient.Status().Update(ctx, &pod) + assert.Nil(t, err, "Fail to update pod status") + } + + // Update 1 worker Pod to Succeeded (a terminate state) state. + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + podList.Items[0].Status.Phase = corev1.PodSucceeded + err = fakeClient.Status().Update(ctx, &podList.Items[0]) + assert.Nil(t, err, "Fail to update Pod status") + + // Reconcile again, and the Succeeded worker Pod should be deleted even if the goal state of the workerGroup specifies 3 replicas. + // The function will return an error to requeue the request after a brief delay. Moreover, if there are unhealthy worker + // Pods to be deleted, the controller won't create new worker Pods during the same reconcile loop. As a result, the number of worker + // Pods will be (expectedNumWorkerPods - 1) after the reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods-1, len(podList.Items)) + + // Reconcile again, and the controller will create a new worker Pod to reach the goal state of the workerGroup. + err = testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, &client.ListOptions{ + LabelSelector: workerSelector, + Namespace: namespaceStr, + }) + assert.Nil(t, err, "Fail to get Pod list after reconcile") + assert.Equal(t, expectedNumWorkerPods, len(podList.Items)) +} + +func Test_TerminatedHead_RestartPolicy_Always(t *testing.T) { + setupTest(t) + + // Create a new scheme with CRDs, Pod, Service schemes. + newScheme := runtime.NewScheme() + _ = rayv1alpha1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) + + // Only one head Pod and no worker Pods in the RayCluster. + runtimeObjects := testPods[0:1] + cluster := testRayCluster.DeepCopy() + cluster.Spec.WorkerGroupSpecs = nil + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(runtimeObjects...).Build() + ctx := context.Background() + + // Get the pod list from the fake client. + podList := corev1.PodList{} + err := fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) + assert.Equal(t, "headNode", podList.Items[0].Name) + + // Make sure the head Pod's restart policy is `Always` and status is `Failed`. + podList.Items[0].Spec.RestartPolicy = corev1.RestartPolicyAlways + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Update(ctx, &podList.Items[0]) + assert.Nil(t, err) + + // Initialize a new RayClusterReconciler. + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: newScheme, + Log: ctrl.Log.WithName("controllers").WithName("RayCluster"), + } + + // The head Pod will not be deleted because the restart policy is `Always`, + // and the controller will return an error to requeue the request after a brief delay. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) + + // Make sure the head Pod's restart policy is `Never` and status is `Failed`. + podList.Items[0].Spec.RestartPolicy = corev1.RestartPolicyNever + podList.Items[0].Status.Phase = corev1.PodFailed + err = fakeClient.Update(ctx, &podList.Items[0]) + assert.Nil(t, err) + + // The head Pod will be deleted and the controller will return an error + // instead of creating a new head Pod in the same reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.NotNil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 0, len(podList.Items)) + + // The new head Pod will be created in the this reconcile loop. + err = testRayClusterReconciler.reconcilePods(ctx, cluster) + assert.Nil(t, err) + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + assert.Nil(t, err, "Fail to get pod list") + assert.Equal(t, 1, len(podList.Items)) +} diff --git a/tests/compatibility-test.py b/tests/compatibility-test.py index 203eb3de6f0..35e8960b431 100755 --- a/tests/compatibility-test.py +++ b/tests/compatibility-test.py @@ -90,26 +90,6 @@ def setUpClass(cls): operator_manager.prepare_operator() utils.create_ray_cluster(RayFTTestCase.cluster_template, ray_version, ray_image) - @unittest.skip("Skip test_kill_head due to its flakiness.") - def test_kill_head(self): - # This test will delete head node and wait for a new replacement to - # come up. - shell_subprocess_run( - 'kubectl delete pod $(kubectl get pods -A | grep -e "-head" | awk "{print \$2}")') - - # wait for new head node to start - time.sleep(80) - shell_subprocess_run('kubectl get pods -A') - - # make sure the new head is ready - # shell_assert_success('kubectl wait --for=condition=Ready pod/$(kubectl get pods -A | grep -e "-head" | awk "{print \$2}") --timeout=900s') - # make sure both head and worker pods are ready - rtn = shell_subprocess_run( - 'kubectl wait --for=condition=ready pod -l rayCluster=raycluster-compatibility-test --all --timeout=900s', check = False) - if rtn != 0: - show_cluster_info("default") - raise Exception(f"Nonzero return code {rtn} in test_kill_head()") - def test_ray_serve(self): """Kill GCS process on the head Pod and then test a deployed Ray Serve model.""" if not utils.is_feature_supported(ray_version, CONST.RAY_SERVE_FT): @@ -127,9 +107,7 @@ def test_ray_serve(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_ray_serve_1.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_ray_serve_1.py. The exit code is {exit_code}.") old_head_pod = get_head_pod(RayFTTestCase.ray_cluster_ns) old_head_pod_name = old_head_pod.metadata.name @@ -141,7 +119,7 @@ def test_ray_serve(self): pod_exec_command(old_head_pod_name, RayFTTestCase.ray_cluster_ns, "pkill gcs_server") # Waiting for all pods become ready and running. - utils.wait_for_new_head(old_head_pod_name, restart_count, + utils.wait_for_new_head(CONST.KILL_GCS_SERVER, old_head_pod_name, restart_count, RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) # Try to connect to the deployed model again @@ -154,9 +132,7 @@ def test_ray_serve(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_ray_serve_2.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_ray_serve_2.py. The exit code is {exit_code}.") def test_detached_actor(self): """Kill GCS process on the head Pod and then test a detached actor.""" @@ -175,21 +151,20 @@ def test_detached_actor(self): if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_detached_actor_1.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_detached_actor_1.py. The exit code is {exit_code}.") old_head_pod = get_head_pod(RayFTTestCase.ray_cluster_ns) old_head_pod_name = old_head_pod.metadata.name restart_count = old_head_pod.status.container_statuses[0].restart_count + # [Test 1: Kill GCS process to "restart" the head Pod] # Kill the gcs_server process on head node. The head node will crash after 20 seconds # because the value of `RAY_gcs_rpc_server_reconnect_timeout_s` is "20" in the # `ray-cluster.ray-ft.yaml.template` file. pod_exec_command(old_head_pod_name, RayFTTestCase.ray_cluster_ns, "pkill gcs_server") # Waiting for all pods become ready and running. - utils.wait_for_new_head(old_head_pod_name, restart_count, + utils.wait_for_new_head(CONST.KILL_GCS_SERVER, old_head_pod_name, restart_count, RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) # Try to connect to the detached actor again. @@ -198,16 +173,39 @@ def test_detached_actor(self): # connection succeeds. headpod = get_head_pod(RayFTTestCase.ray_cluster_ns) headpod_name = headpod.metadata.name + expected_output = 3 exit_code = pod_exec_command(headpod_name, RayFTTestCase.ray_cluster_ns, - f" python samples/test_detached_actor_2.py {ray_namespace}", + f" python samples/test_detached_actor_2.py {ray_namespace} {expected_output}", check = False ) if exit_code != 0: show_cluster_info(RayFTTestCase.ray_cluster_ns) - raise Exception( - f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}." - ) + self.fail(f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}.") + + # [Test 2: Delete the head Pod and wait for a new head Pod] + # Delete the head Pod. The `kubectl delete pod` command has a default flag `--wait=true`, + # which waits for resources to be gone before returning. + shell_subprocess_run( + f'kubectl delete pod {headpod_name} -n {RayFTTestCase.ray_cluster_ns}') + restart_count = headpod.status.container_statuses[0].restart_count + + # Waiting for all pods become ready and running. + utils.wait_for_new_head(CONST.KILL_HEAD_POD, headpod_name, restart_count, + RayFTTestCase.ray_cluster_ns, timeout=300, retry_interval_ms=1000) + + # Try to connect to the detached actor again. + headpod = get_head_pod(RayFTTestCase.ray_cluster_ns) + headpod_name = headpod.metadata.name + expected_output = 4 + exit_code = pod_exec_command(headpod_name, RayFTTestCase.ray_cluster_ns, + f" python samples/test_detached_actor_2.py {ray_namespace} {expected_output}", + check = False + ) + + if exit_code != 0: + show_cluster_info(RayFTTestCase.ray_cluster_ns) + self.fail(f"Fail to execute test_detached_actor_2.py. The exit code is {exit_code}.") class RayServiceTestCase(unittest.TestCase): """Integration tests for RayService""" diff --git a/tests/config/ray-cluster.ray-ft.yaml.template b/tests/config/ray-cluster.ray-ft.yaml.template index ccad180470d..b00beb7588f 100644 --- a/tests/config/ray-cluster.ray-ft.yaml.template +++ b/tests/config/ray-cluster.ray-ft.yaml.template @@ -267,6 +267,4 @@ data: val = retry_with_timeout(lambda: ray.get(tc.increment.remote())) print(f"val: {val}") - # The actual value should be 1 rather than 2. Ray will launch all registered actors when - # the ray cluster restarts, but the internal state of the state will not be restored. - assert(val == 3) + assert(val == int(sys.argv[2])) diff --git a/tests/framework/utils.py b/tests/framework/utils.py index 079903ecf30..c553bc4221d 100644 --- a/tests/framework/utils.py +++ b/tests/framework/utils.py @@ -53,6 +53,10 @@ class CONST: RAY_SERVICE_CRD = "RayService" RAY_JOB_CRD = "RayJob" + # Failures + KILL_GCS_SERVER = "KILL_GCS_SERVER" + KILL_HEAD_POD = "KILL_HEAD_POD" + CONST = CONST() diff --git a/tests/kuberay_utils/utils.py b/tests/kuberay_utils/utils.py index 92a1d05d77d..ef81e4c04f9 100644 --- a/tests/kuberay_utils/utils.py +++ b/tests/kuberay_utils/utils.py @@ -14,7 +14,6 @@ from framework.utils import ( get_head_pod, CONST, - K8S_CLUSTER_MANAGER ) logger = logging.getLogger(__name__) @@ -60,7 +59,7 @@ def create_ray_cluster(template_name, ray_version, ray_image): ray_cluster_add_event = RayClusterAddCREvent( custom_resource_object = context['cr'], rulesets = [], - timeout = 90, + timeout = 180, namespace='default', filepath = context['filepath'] ) @@ -123,71 +122,79 @@ def wait_for_condition( message += f" Last exception: {last_ex}" raise RuntimeError(message) -def wait_for_new_head(old_head_pod_name, old_restart_count, namespace, timeout, retry_interval_ms): +def wait_for_new_head(mode, old_head_pod_name, old_restart_count, namespace, timeout, retry_interval_ms): """ - `wait_for_new_head` is used to wait for new head is ready and running. For example, `test_detached_actor` kills - the gcs_server process on the head pod. It takes nearly 1 min to kill the head pod, and the head pod will still - be in 'Running' and 'Ready' in that minute. - - Hence, we need to check `restart_count` or `new_head_pod_name`. - (1) `restart_count`: If the pod is restarted by the restartPolicy of a Pod, `restart_count` will increase by 1. - If the pod is deleted by KubeRay and the reconciler creates a new one, `restart_count` will be 0. - (2) `new_head_pod_name`: If the reconciler creates a new head pod, `new_head_pod_name` will be different from - `old_head_pod_name`. - - Next, we check the status of pods to ensure all pods should be "Running" and "Ready". + `wait_for_new_head` is used to wait for the head Pod is ready and running. - After the cluster state converges (all pods are "Running" and "Ready"), ray processes still need tens of seconds to - become ready to serve new connections from ray clients. So, users need to retry until a Ray client connection succeeds. + [Case 1] + KILL_GCS_SERVER: The head Pod should be restarted rather than creating a new head Pod. + [Case 2] + KILL_HEAD_POD: The new head Pod should be created. Args: + mode: KILL_GCS_SERVER or KILL_HEAD_POD. old_head_pod_name: Name of the old head pod. - old_restart_count: If the Pod is restarted by Kubernetes Pod RestartPolicy, the restart_count will increase by 1. + old_restart_count: The restart count of the old head pod. namespace: Namespace that the head pod is running in. timeout: Same as `wait_for_condition`. retry_interval_ms: Same as `wait_for_condition`. Raises: - RuntimeError: If the condition is not met before the timeout expires, raise the RuntimeError. + RuntimeError: Raise a RuntimeError if a timeout occurs. """ - k8s_v1_api = K8S_CLUSTER_MANAGER.k8s_client_dict[CONST.K8S_V1_CLIENT_KEY] - def check_status(old_head_pod_name, old_restart_count, namespace) -> bool: - all_pods = k8s_v1_api.list_namespaced_pod(namespace = namespace) + def check_status(mode, old_head_pod_name, old_restart_count, namespace) -> bool: headpod = get_head_pod(namespace) + if headpod is None: + logger.info( + "There is no head Pod. We will only check the following conditions " + + "after the head Pod is created." + ) + return False new_head_pod_name = headpod.metadata.name new_restart_count = headpod.status.container_statuses[0].restart_count - # The default container restartPolicy of a Pod is `Always`. Hence, when GCS server is killed, - # the head pod will restart the old one rather than create a new one. - if new_head_pod_name != old_head_pod_name: - logger.info(f'If GCS server is killed, the head pod will restart the old one rather than create a new one.' + - f' new_head_pod_name: {new_head_pod_name}, old_head_pod_name: {old_head_pod_name}') - # TODO (kevin85421): We should `return False` here, but currently ray:nightly has a high possibility to create - # a new head pod instead of restarting the old one. - - # When GCS server is killed, it takes nearly 1 min to kill the head pod. In the minute, the head - # pod will still be in 'Running' and 'Ready'. Hence, we need to check `restart_count`. - else: - # TODO (kevin85421): We should remove `else` in the future. Currently, ray:nightly has a high possibility to - # create a new head pod instead of restarting the old one. The new pod's `restart_count` - # is 0. + + logger.info("Failure mode: %s", mode) + if mode == CONST.KILL_GCS_SERVER: + if new_head_pod_name != old_head_pod_name: + logger.warning( + "GCS server process is killed. The head Pod should be restarted " + "rather than creating a new head Pod. There is something wrong. " + "new_head_pod_name: %s, old_head_pod_name: %s", + new_head_pod_name, old_head_pod_name + ) + return False if new_restart_count != old_restart_count + 1: - logger.info(f'new_restart_count != old_restart_count + 1 => new_restart_count: {new_restart_count},' + - f' old_restart_count: {old_restart_count}') + logger.info( + "new_restart_count != old_restart_count + 1 =>" + "new_restart_count: %s; old_restart_count: %s", + new_restart_count, old_restart_count + ) + return False + elif mode == CONST.KILL_HEAD_POD: + if new_head_pod_name == old_head_pod_name: + logger.info("The old head Pod %s is not killed.", old_head_pod_name) return False - # All pods should be "Running" and "Ready". This check is an overkill. We added this check due to - # the buggy behaviors of Ray HA. To elaborate, when GCS server is killed, the head pod should restart, - # but worker pods should not. However, currently, worker pods will also restart. - # See https://github.com/ray-project/kuberay/issues/634 for more details. - for pod in all_pods.items: - if pod.status.phase != 'Running': - logger.info(f'Pod {pod.metadata.name} is not Running.') + else: + raise ValueError(f"Invalid failure mode: {mode}") + + if headpod.status.phase != "Running": + logger.info( + "The head Pod %s is not running. The status is %s", + headpod.metadata.name, headpod.status.phase + ) + return False + for container_status in headpod.status.container_statuses: + if not container_status.ready: + logger.info( + "The container %s is not ready. The status is %s", + container_status.name, container_status.ready + ) return False - for c in pod.status.container_statuses: - if not c.ready: - logger.info(f'Container {c.name} in {pod.metadata.name} is not ready.') - return False return True - wait_for_condition(check_status, timeout=timeout, retry_interval_ms=retry_interval_ms, - old_head_pod_name=old_head_pod_name, old_restart_count=old_restart_count, namespace=namespace) + wait_for_condition( + check_status, timeout=timeout, retry_interval_ms=retry_interval_ms, + mode=mode, old_head_pod_name=old_head_pod_name, old_restart_count=old_restart_count, + namespace=namespace + ) # After the cluster state converges, ray processes still need tens of seconds to become ready. # TODO (kevin85421): Make ray processes become ready when pods are "Ready" and "Running". From ed370ba2737c954a21a54f2645baebd8927bce5a Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Tue, 22 Aug 2023 12:58:34 -0700 Subject: [PATCH 31/56] [RayJob] Add runtime env YAML field (#1338) Adds the field RuntimeEnvYAML to the RayJob CRD which accepts a multi-line YAML string. This format is preferred for two reasons: Consistency with the ServeConfigV2 format, which is also a Ray configuration specified as a multi-line YAML string (Related to above) Allows using snake_case fields without modification We preserve the older field RuntimeEnv which accepts a base64-encoded string of the runtime env. We mark it as deprecated in the documentation. We raise an error if both fields are specified. Related issue number Closes #1195 --------- Signed-off-by: Archit Kulkarni --- .buildkite/test-sample-yamls.yml | 24 ++- docs/guidance/rayjob.md | 3 +- .../kuberay-operator/crds/ray.io_rayjobs.yaml | 7 +- .../apis/ray/v1alpha1/rayjob_types.go | 4 + .../config/crd/bases/ray.io_rayjobs.yaml | 7 +- .../samples/ray-job.custom-head-svc.yaml | 12 +- .../ray_v1alpha1_rayjob.batch-inference.yaml | 179 ++++++++++++++++++ .../samples/ray_v1alpha1_rayjob.shutdown.yaml | 24 ++- .../config/samples/ray_v1alpha1_rayjob.yaml | 23 ++- ray-operator/controllers/ray/common/job.go | 41 +++- .../controllers/ray/common/job_test.go | 101 ++++++++++ .../controllers/ray/rayjob_controller.go | 5 + ray-operator/go.mod | 2 +- 13 files changed, 393 insertions(+), 39 deletions(-) create mode 100644 ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml diff --git a/.buildkite/test-sample-yamls.yml b/.buildkite/test-sample-yamls.yml index ef415f6d526..0a06e9715f3 100644 --- a/.buildkite/test-sample-yamls.yml +++ b/.buildkite/test-sample-yamls.yml @@ -5,6 +5,10 @@ image: golang:1.19 commands: - ./.buildkite/setup-env.sh + # Build nightly KubeRay operator image + - pushd ray-operator + - IMG=kuberay/operator:nightly make docker-image + - popd # Use nightly KubeRay operator image - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:nightly python3 tests/test_sample_raycluster_yamls.py @@ -21,13 +25,19 @@ image: golang:1.19 commands: - ./.buildkite/setup-env.sh + # Build nightly KubeRay operator image + - pushd ray-operator + - IMG=kuberay/operator:nightly make docker-image + - popd # Use nightly KubeRay operator image - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:nightly python3 tests/test_sample_rayjob_yamls.py -- label: 'Test RayJob Sample YAMLs (latest release)' - instance_size: large - image: golang:1.19 - commands: - - ./.buildkite/setup-env.sh - # Use KubeRay operator image from the latest release - - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:v0.6.0 python3 tests/test_sample_rayjob_yamls.py +# Temporarily skip due to adding new `RuntimeEnvYAML` field in sample YAMLs. +# TODO(architkulkarni): Reenable after 1.0 release +# - label: 'Test RayJob Sample YAMLs (latest release)' +# instance_size: large +# image: golang:1.19 +# commands: +# - ./.buildkite/setup-env.sh +# # Use KubeRay operator image from the latest release +# - source .venv/bin/activate && BUILDKITE_ENV=true RAY_IMAGE=rayproject/ray:2.6.3 OPERATOR_IMAGE=kuberay/operator:v0.6.0 python3 tests/test_sample_rayjob_yamls.py diff --git a/docs/guidance/rayjob.md b/docs/guidance/rayjob.md index e4e664420a1..dbabd4c9b49 100644 --- a/docs/guidance/rayjob.md +++ b/docs/guidance/rayjob.md @@ -54,10 +54,11 @@ $ kubectl get pod * `rayClusterSpec` - The spec for the Ray cluster to run the job on. * `jobId` - _(Optional)_ Job ID to specify for the job. If not provided, one will be generated. * `metadata` - _(Optional)_ Arbitrary user-provided metadata for the job. -* `runtimeEnv` - _(Optional)_ base64-encoded string of the runtime env json string. +* `runtimeEnvYAML` - _(Optional)_ The runtime environment configuration provided as a multi-line YAML string. _(New in KubeRay version 1.0.)_ * `shutdownAfterJobFinishes` - _(Optional)_ whether to recycle the cluster after the job finishes. Defaults to false. * `ttlSecondsAfterFinished` - _(Optional)_ TTL to clean up the cluster. This only works if `shutdownAfterJobFinishes` is set. * `submitterPodTemplate` - _(Optional)_ Pod template spec for the pod that runs `ray job submit` against the Ray cluster. +* `runtimeEnv` - [DEPRECATED] _(Optional)_ base64-encoded string of the runtime env json string. ## RayJob Observability diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml index 2d6c5eb9e6e..1645ec4032d 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml @@ -12090,7 +12090,12 @@ spec: - headGroupSpec type: object runtimeEnv: - description: RuntimeEnv is base64 encoded. + description: RuntimeEnv is base64 encoded. This field is deprecated, + please use RuntimeEnvYAML instead. + type: string + runtimeEnvYAML: + description: RuntimeEnvYAML represents the runtime environment configuration + provided as a multi-line YAML string type: string shutdownAfterJobFinishes: description: ShutdownAfterJobFinishes will determine whether to delete diff --git a/ray-operator/apis/ray/v1alpha1/rayjob_types.go b/ray-operator/apis/ray/v1alpha1/rayjob_types.go index 670b1764795..8c791e740af 100644 --- a/ray-operator/apis/ray/v1alpha1/rayjob_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayjob_types.go @@ -52,7 +52,11 @@ type RayJobSpec struct { // Metadata is data to store along with this job. Metadata map[string]string `json:"metadata,omitempty"` // RuntimeEnv is base64 encoded. + // This field is deprecated, please use RuntimeEnvYAML instead. RuntimeEnv string `json:"runtimeEnv,omitempty"` + // RuntimeEnvYAML represents the runtime environment configuration + // provided as a multi-line YAML string. + RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"` // If jobId is not set, a new jobId will be auto-generated. JobId string `json:"jobId,omitempty"` // ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. diff --git a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml index 2d6c5eb9e6e..1645ec4032d 100644 --- a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml @@ -12090,7 +12090,12 @@ spec: - headGroupSpec type: object runtimeEnv: - description: RuntimeEnv is base64 encoded. + description: RuntimeEnv is base64 encoded. This field is deprecated, + please use RuntimeEnvYAML instead. + type: string + runtimeEnvYAML: + description: RuntimeEnvYAML represents the runtime environment configuration + provided as a multi-line YAML string type: string shutdownAfterJobFinishes: description: ShutdownAfterJobFinishes will determine whether to delete diff --git a/ray-operator/config/samples/ray-job.custom-head-svc.yaml b/ray-operator/config/samples/ray-job.custom-head-svc.yaml index 31ba15a74b8..72e09c4d3d6 100644 --- a/ray-operator/config/samples/ray-job.custom-head-svc.yaml +++ b/ray-operator/config/samples/ray-job.custom-head-svc.yaml @@ -7,7 +7,17 @@ metadata: name: rayjob-sample spec: entrypoint: python /home/ray/samples/sample_code.py - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.6.3' # should match the Ray version in the image of the containers diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml new file mode 100644 index 00000000000..8fdc51a647d --- /dev/null +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml @@ -0,0 +1,179 @@ +apiVersion: ray.io/v1alpha1 +kind: RayJob +metadata: + name: rayjob-sample +spec: + entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. + # shutdownAfterJobFinishes: false + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. + # ttlSecondsAfterFinished: 10 + # Runtime env decoded to { + # { + # "pip": [ + # "torch", + # "torchvision", + # "Pillow", + # "transformers" + # ] + # } + runtimeEnv: ewogICJwaXAiOiBbCiAgICAidG9yY2giLAogICAgInRvcmNodmlzaW9uIiwKICAgICJQaWxsb3ciLAogICAgInRyYW5zZm9ybWVycyIKICBdCn0= + # Suspend specifies whether the RayJob controller should create a RayCluster instance. + # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. + # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. + # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. + rayClusterSpec: + rayVersion: '2.6.3' # should match the Ray version in the image of the containers + # Ray head pod template + 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: + dashboard-host: '0.0.0.0' + #pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray-ml:2.6.3-gpu + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 # Ray dashboard + name: dashboard + - containerPort: 10001 + name: client + resources: + limits: + cpu: 2 + memory: 8Gi + requests: + cpu: 2 + memory: 8Gi + volumeMounts: + - mountPath: /home/ray/samples + name: code-sample + volumes: + # You set volumes at the Pod level, then mount them into containers inside that Pod + - name: code-sample + configMap: + # Provide the name of the ConfigMap you want to mount. + name: ray-job-code-sample + # An array of keys from the ConfigMap to create as files + items: + - key: sample_code.py + path: sample_code.py + workerGroupSpecs: + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + # logical group name, for this called small-group, also can be functional + 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: + resources: '"{\"accelerator_type_cpu\": 48, \"accelerator_type_a10\": 2, \"accelerator_type_a100\": 2}"' + #pod template + template: + spec: + containers: + - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + image: rayproject/ray-ml:2.6.3-gpu + lifecycle: + preStop: + exec: + command: [ "/bin/sh","-c","ray stop" ] + resources: + limits: + cpu: "48" + memory: "192G" + nvidia.com/gpu: 4 + requests: + cpu: "36" + memory: "128G" + nvidia.com/gpu: 4 + nodeSelector: + cloud.google.com/gke-accelerator: nvidia-tesla-t4 + # SubmitterPodTemplate is the template for the pod that will run the `ray job submit` command against the RayCluster. + # If SubmitterPodTemplate is specified, the first container is assumed to be the submitter container. + # submitterPodTemplate: + # spec: + # restartPolicy: Never + # containers: + # - name: my-custom-rayjob-submitter-pod + # image: rayproject/ray:2.6.3 + # # If Command is not specified, the correct command will be supplied at runtime using the RayJob spec `entrypoint` field. + # # Specifying Command is not recommended. + # # command: ["ray job submit --address=http://rayjob-sample-raycluster-v6qcq-head-svc.default.svc.cluster.local:8265 -- echo hello world"] + + +######################Ray code sample################################# +# this sample is from https://docs.ray.io/en/latest/cluster/job-submission.html#quick-start-example +# it is mounted into the container and executed to show the Ray job at work +--- +apiVersion: v1 +kind: ConfigMap +metadata: + name: ray-job-code-sample +data: + sample_code.py: | + import ray + + s3_uri = "s3://anonymous@air-example-data-2/imagenette2/val/" + + ds = ray.data.read_images( + s3_uri, mode="RGB" + ) + ds + # TODO(archit) need to install Pillow, pytorch or tf or flax (pip install torch torchvision torchaudio) + from typing import Dict + import numpy as np + + from transformers import pipeline + from PIL import Image + + # Pick the largest batch size that can fit on our GPUs + BATCH_SIZE = 1024 + + # TODO(archit) basic step + + # single_batch = ds.take_batch(10) + + # from PIL import Image + + # img = Image.fromarray(single_batch["image"][0]) + # # display image + # img.show() + # from transformers import pipeline + # from PIL import Image + + # # If doing CPU inference, set device="cpu" instead. + # classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") + # outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) + # del classifier # Delete the classifier to free up GPU memory. + # print(outputs) + + @ray.remote(num_gpus=1) + def do_single_batch(): + single_batch = ds.take_batch(10) + + from PIL import Image + + img = Image.fromarray(single_batch["image"][0]) + # display image + img.show() + from transformers import pipeline + from PIL import Image + + # If doing CPU inference, set device="cpu" instead. + classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") + outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) + del classifier # Delete the classifier to free up GPU memory. + print(outputs) + return outputs + + print(ray.get(do_single_batch.remote())) \ No newline at end of file diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml index f4a2425bb5e..cddfb5a0e3c 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.shutdown.yaml @@ -4,24 +4,28 @@ metadata: name: rayjob-sample-shutdown spec: entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. shutdownAfterJobFinishes: true + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. ttlSecondsAfterFinished: 10 - # runtimeEnv decoded to '{ - # "pip": [ - # "requests==2.26.0", - # "pendulum==2.1.2" - # ], - # "env_vars": { - # "counter_name": "test_counter" - # } - # }' - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # Suspend specifies whether the RayJob controller should create a RayCluster instance. # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.5.0' # should match the Ray version in the image of the containers diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml index a922e06f2de..445ee6bf5fb 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayjob.yaml @@ -6,22 +6,25 @@ spec: entrypoint: python /home/ray/samples/sample_code.py # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. # shutdownAfterJobFinishes: false + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. # ttlSecondsAfterFinished: 10 - # runtimeEnv decoded to '{ - # "pip": [ - # "requests==2.26.0", - # "pendulum==2.1.2" - # ], - # "env_vars": { - # "counter_name": "test_counter" - # } - #}' - runtimeEnv: ewogICAgInBpcCI6IFsKICAgICAgICAicmVxdWVzdHM9PTIuMjYuMCIsCiAgICAgICAgInBlbmR1bHVtPT0yLjEuMiIKICAgIF0sCiAgICAiZW52X3ZhcnMiOiB7ImNvdW50ZXJfbmFtZSI6ICJ0ZXN0X2NvdW50ZXIifQp9Cg== + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + # Suspend specifies whether the RayJob controller should create a RayCluster instance. # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. # suspend: false + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: rayVersion: '2.5.0' # should match the Ray version in the image of the containers diff --git a/ray-operator/controllers/ray/common/job.go b/ray-operator/controllers/ray/common/job.go index f7a67139910..5f2093b52a1 100644 --- a/ray-operator/controllers/ray/common/job.go +++ b/ray-operator/controllers/ray/common/job.go @@ -10,6 +10,7 @@ import ( "github.com/google/shlex" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" v1 "k8s.io/api/core/v1" + "sigs.k8s.io/yaml" ) // GetDecodedRuntimeEnv decodes the runtime environment for the Ray job from a base64-encoded string. @@ -21,6 +22,33 @@ func GetDecodedRuntimeEnv(runtimeEnv string) (string, error) { return string(decodedBytes), nil } +// GetRuntimeEnvJson returns the JSON string of the runtime environment for the Ray job. +func getRuntimeEnvJson(rayJobInstance *rayv1alpha1.RayJob) (string, error) { + runtimeEnv := rayJobInstance.Spec.RuntimeEnv + runtimeEnvYAML := rayJobInstance.Spec.RuntimeEnvYAML + + // Check if both runtimeEnv and RuntimeEnvYAML are specified. + if len(runtimeEnv) > 0 && len(runtimeEnvYAML) > 0 { + return "", fmt.Errorf("Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.") + } + + if len(runtimeEnv) > 0 { + return GetDecodedRuntimeEnv(runtimeEnv) + } + + if len(runtimeEnvYAML) > 0 { + // Convert YAML to JSON + jsonData, err := yaml.YAMLToJSON([]byte(runtimeEnvYAML)) + if err != nil { + return "", err + } + // We return the JSON as a string + return string(jsonData), nil + } + + return "", nil +} + // GetBaseRayJobCommand returns the first part of the Ray Job command up to and including the address, e.g. "ray job submit --address http://..." func GetBaseRayJobCommand(address string) []string { // add http:// if needed @@ -54,19 +82,18 @@ func GetMetadataJson(metadata map[string]string, rayVersion string) (string, err // GetK8sJobCommand builds the K8s job command for the Ray job. func GetK8sJobCommand(rayJobInstance *rayv1alpha1.RayJob) ([]string, error) { address := rayJobInstance.Status.DashboardURL - runtimeEnv := rayJobInstance.Spec.RuntimeEnv metadata := rayJobInstance.Spec.Metadata jobId := rayJobInstance.Status.JobId entrypoint := rayJobInstance.Spec.Entrypoint k8sJobCommand := GetBaseRayJobCommand(address) - if len(runtimeEnv) > 0 { - runtimeEnvDecoded, err := GetDecodedRuntimeEnv(runtimeEnv) - if err != nil { - return nil, err - } - k8sJobCommand = append(k8sJobCommand, "--runtime-env-json", runtimeEnvDecoded) + runtimeEnvJson, err := getRuntimeEnvJson(rayJobInstance) + if err != nil { + return nil, err + } + if len(runtimeEnvJson) > 0 { + k8sJobCommand = append(k8sJobCommand, "--runtime-env-json", runtimeEnvJson) } if len(metadata) > 0 { diff --git a/ray-operator/controllers/ray/common/job_test.go b/ray-operator/controllers/ray/common/job_test.go index e5cd782b520..431b1ae314e 100644 --- a/ray-operator/controllers/ray/common/job_test.go +++ b/ray-operator/controllers/ray/common/job_test.go @@ -1,6 +1,7 @@ package common import ( + "encoding/json" "testing" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" @@ -30,6 +31,49 @@ func TestGetDecodedRuntimeEnv(t *testing.T) { assert.Equal(t, `{"test":"test"}`, decoded) } +func TestGetRuntimeEnvJsonFromBase64(t *testing.T) { + expected := `{"test":"test"}` + jsonOutput, err := getRuntimeEnvJson(testRayJob) + assert.NoError(t, err) + assert.Equal(t, expected, jsonOutput) +} + +func TestGetRuntimeEnvJsonFromYAML(t *testing.T) { + rayJobWithYAML := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnvYAML: ` +working_dir: "https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip" +pip: ["python-multipart==0.0.6"] +`, + }, + } + expectedJSON := `{"working_dir":"https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip","pip":["python-multipart==0.0.6"]}` + jsonOutput, err := getRuntimeEnvJson(rayJobWithYAML) + assert.NoError(t, err) + + var expectedMap map[string]interface{} + var actualMap map[string]interface{} + + // Convert the JSON strings into map types to avoid errors due to ordering + assert.NoError(t, json.Unmarshal([]byte(expectedJSON), &expectedMap)) + assert.NoError(t, json.Unmarshal([]byte(jsonOutput), &actualMap)) + + // Now compare the maps + assert.Equal(t, expectedMap, actualMap) +} + +func TestGetRuntimeEnvJsonErrorWithBothFields(t *testing.T) { + rayJobWithBoth := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnv: "eyJ0ZXN0IjoidGVzdCJ9", + RuntimeEnvYAML: `pip: ["python-multipart==0.0.6"]`, + }, + } + _, err := getRuntimeEnvJson(rayJobWithBoth) + assert.Error(t, err) + assert.Contains(t, err.Error(), "Both runtimeEnv and RuntimeEnvYAML are specified. Please specify only one of the fields.") +} + func TestGetBaseRayJobCommand(t *testing.T) { expected := []string{"ray", "job", "submit", "--address", "http://127.0.0.1:8265"} command := GetBaseRayJobCommand(testRayJob.Status.DashboardURL) @@ -57,6 +101,63 @@ func TestGetK8sJobCommand(t *testing.T) { assert.Equal(t, expected, command) } +func TestGetK8sJobCommandWithYAML(t *testing.T) { + rayJobWithYAML := &rayv1alpha1.RayJob{ + Spec: rayv1alpha1.RayJobSpec{ + RuntimeEnvYAML: ` +working_dir: "https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip" +pip: ["python-multipart==0.0.6"] +`, + Metadata: map[string]string{ + "testKey": "testValue", + }, + RayClusterSpec: &rayv1alpha1.RayClusterSpec{ + RayVersion: "2.6.0", + }, + Entrypoint: "echo hello", + }, + Status: rayv1alpha1.RayJobStatus{ + DashboardURL: "http://127.0.0.1:8265", + JobId: "testJobId", + }, + } + expected := []string{ + "ray", "job", "submit", "--address", "http://127.0.0.1:8265", + "--runtime-env-json", `{"working_dir":"https://github.com/ray-project/serve_config_examples/archive/b393e77bbd6aba0881e3d94c05f968f05a387b96.zip","pip":["python-multipart==0.0.6"]}`, + "--metadata-json", `{"testKey":"testValue"}`, + "--submission-id", "testJobId", + "--", + "echo", "hello", + } + command, err := GetK8sJobCommand(rayJobWithYAML) + assert.NoError(t, err) + + // Ensure the slices are the same length. + assert.Equal(t, len(expected), len(command)) + + for i := 0; i < len(expected); i++ { + if expected[i] == "--runtime-env-json" { + // Decode the JSON string from the next element. + var expectedMap, actualMap map[string]interface{} + err1 := json.Unmarshal([]byte(expected[i+1]), &expectedMap) + err2 := json.Unmarshal([]byte(command[i+1]), &actualMap) + + // If there's an error decoding either JSON string, it's an error in the test. + assert.NoError(t, err1) + assert.NoError(t, err2) + + // Compare the maps directly to avoid errors due to ordering. + assert.Equal(t, expectedMap, actualMap) + + // Skip the next element because we've just checked it. + i++ + } else { + // For non-JSON elements, compare them directly. + assert.Equal(t, expected[i], command[i]) + } + } +} + func TestMetadataRaisesErrorBeforeRay26(t *testing.T) { rayJob := &rayv1alpha1.RayJob{ Spec: rayv1alpha1.RayJobSpec{ diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index d0a190c4232..f587219fd85 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -360,6 +360,11 @@ func (r *RayJobReconciler) getSubmitterTemplate(rayJobInstance *rayv1alpha1.RayJ // If the command in the submitter pod template isn't set, use the default command. if len(submitterTemplate.Spec.Containers[0].Command) == 0 { + // Check for deprecated 'runtimeEnv' field usage and log a warning. + if len(rayJobInstance.Spec.RuntimeEnv) > 0 { + r.Log.Info("Warning: The 'runtimeEnv' field is deprecated. Please use 'runtimeEnvYAML' instead.") + } + k8sJobCommand, err := common.GetK8sJobCommand(rayJobInstance) if err != nil { return v1.PodTemplateSpec{}, err diff --git a/ray-operator/go.mod b/ray-operator/go.mod index a5ecf326420..907d99ad4dd 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -25,6 +25,7 @@ require ( k8s.io/code-generator v0.23.0 k8s.io/utils v0.0.0-20210930125809-cb0fa318a74b sigs.k8s.io/controller-runtime v0.11.1 + sigs.k8s.io/yaml v1.3.0 volcano.sh/apis v1.6.0-alpha.0.0.20221012070524-685db38b4fae ) @@ -84,5 +85,4 @@ require ( k8s.io/kube-openapi v0.0.0-20211115234752-e816edb12b65 // indirect sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 // indirect sigs.k8s.io/structured-merge-diff/v4 v4.2.0 // indirect - sigs.k8s.io/yaml v1.3.0 // indirect ) From 6613d2123c044bfdee9d1f2636c84be56e93f5d0 Mon Sep 17 00:00:00 2001 From: Archit Kulkarni Date: Wed, 23 Aug 2023 10:08:53 -0700 Subject: [PATCH 32/56] Delete ray_v1alpha1_rayjob.batch-inference.yaml (#1360) Accidentally committed WIP file Signed-off-by: Archit Kulkarni --- .../ray_v1alpha1_rayjob.batch-inference.yaml | 179 ------------------ 1 file changed, 179 deletions(-) delete mode 100644 ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml diff --git a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml b/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml deleted file mode 100644 index 8fdc51a647d..00000000000 --- a/ray-operator/config/samples/ray_v1alpha1_rayjob.batch-inference.yaml +++ /dev/null @@ -1,179 +0,0 @@ -apiVersion: ray.io/v1alpha1 -kind: RayJob -metadata: - name: rayjob-sample -spec: - entrypoint: python /home/ray/samples/sample_code.py - # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. - # shutdownAfterJobFinishes: false - # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. - # ttlSecondsAfterFinished: 10 - # Runtime env decoded to { - # { - # "pip": [ - # "torch", - # "torchvision", - # "Pillow", - # "transformers" - # ] - # } - runtimeEnv: ewogICJwaXAiOiBbCiAgICAidG9yY2giLAogICAgInRvcmNodmlzaW9uIiwKICAgICJQaWxsb3ciLAogICAgInRyYW5zZm9ybWVycyIKICBdCn0= - # Suspend specifies whether the RayJob controller should create a RayCluster instance. - # If a job is applied with the suspend field set to true, the RayCluster will not be created and we will wait for the transition to false. - # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluste rwill be created. - # suspend: false - # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. - rayClusterSpec: - rayVersion: '2.6.3' # should match the Ray version in the image of the containers - # Ray head pod template - 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: - dashboard-host: '0.0.0.0' - #pod template - template: - spec: - containers: - - name: ray-head - image: rayproject/ray-ml:2.6.3-gpu - ports: - - containerPort: 6379 - name: gcs-server - - containerPort: 8265 # Ray dashboard - name: dashboard - - containerPort: 10001 - name: client - resources: - limits: - cpu: 2 - memory: 8Gi - requests: - cpu: 2 - memory: 8Gi - volumeMounts: - - mountPath: /home/ray/samples - name: code-sample - volumes: - # You set volumes at the Pod level, then mount them into containers inside that Pod - - name: code-sample - configMap: - # Provide the name of the ConfigMap you want to mount. - name: ray-job-code-sample - # An array of keys from the ConfigMap to create as files - items: - - key: sample_code.py - path: sample_code.py - workerGroupSpecs: - # the pod replicas in this group typed worker - - replicas: 1 - minReplicas: 1 - maxReplicas: 5 - # logical group name, for this called small-group, also can be functional - 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: - resources: '"{\"accelerator_type_cpu\": 48, \"accelerator_type_a10\": 2, \"accelerator_type_a100\": 2}"' - #pod template - template: - spec: - containers: - - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' - image: rayproject/ray-ml:2.6.3-gpu - lifecycle: - preStop: - exec: - command: [ "/bin/sh","-c","ray stop" ] - resources: - limits: - cpu: "48" - memory: "192G" - nvidia.com/gpu: 4 - requests: - cpu: "36" - memory: "128G" - nvidia.com/gpu: 4 - nodeSelector: - cloud.google.com/gke-accelerator: nvidia-tesla-t4 - # SubmitterPodTemplate is the template for the pod that will run the `ray job submit` command against the RayCluster. - # If SubmitterPodTemplate is specified, the first container is assumed to be the submitter container. - # submitterPodTemplate: - # spec: - # restartPolicy: Never - # containers: - # - name: my-custom-rayjob-submitter-pod - # image: rayproject/ray:2.6.3 - # # If Command is not specified, the correct command will be supplied at runtime using the RayJob spec `entrypoint` field. - # # Specifying Command is not recommended. - # # command: ["ray job submit --address=http://rayjob-sample-raycluster-v6qcq-head-svc.default.svc.cluster.local:8265 -- echo hello world"] - - -######################Ray code sample################################# -# this sample is from https://docs.ray.io/en/latest/cluster/job-submission.html#quick-start-example -# it is mounted into the container and executed to show the Ray job at work ---- -apiVersion: v1 -kind: ConfigMap -metadata: - name: ray-job-code-sample -data: - sample_code.py: | - import ray - - s3_uri = "s3://anonymous@air-example-data-2/imagenette2/val/" - - ds = ray.data.read_images( - s3_uri, mode="RGB" - ) - ds - # TODO(archit) need to install Pillow, pytorch or tf or flax (pip install torch torchvision torchaudio) - from typing import Dict - import numpy as np - - from transformers import pipeline - from PIL import Image - - # Pick the largest batch size that can fit on our GPUs - BATCH_SIZE = 1024 - - # TODO(archit) basic step - - # single_batch = ds.take_batch(10) - - # from PIL import Image - - # img = Image.fromarray(single_batch["image"][0]) - # # display image - # img.show() - # from transformers import pipeline - # from PIL import Image - - # # If doing CPU inference, set device="cpu" instead. - # classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") - # outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) - # del classifier # Delete the classifier to free up GPU memory. - # print(outputs) - - @ray.remote(num_gpus=1) - def do_single_batch(): - single_batch = ds.take_batch(10) - - from PIL import Image - - img = Image.fromarray(single_batch["image"][0]) - # display image - img.show() - from transformers import pipeline - from PIL import Image - - # If doing CPU inference, set device="cpu" instead. - classifier = pipeline("image-classification", model="google/vit-base-patch16-224", device="cuda:0") - outputs = classifier([Image.fromarray(image_array) for image_array in single_batch["image"]], top_k=1, batch_size=10) - del classifier # Delete the classifier to free up GPU memory. - print(outputs) - return outputs - - print(ray.get(do_single_batch.remote())) \ No newline at end of file From 4438d401c4409edc11ee9c67361f0a78cf95f530 Mon Sep 17 00:00:00 2001 From: Darren Date: Wed, 23 Aug 2023 23:50:38 -0700 Subject: [PATCH 33/56] [Feature] Allow RayCluster Helm chart to specify different images for different worker groups (#1352) Allow RayCluster Helm chart to specify different images for different worker groups --- .../templates/raycluster-cluster.yaml | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/helm-chart/ray-cluster/templates/raycluster-cluster.yaml b/helm-chart/ray-cluster/templates/raycluster-cluster.yaml index e282d77a61d..0c13e86eafd 100644 --- a/helm-chart/ray-cluster/templates/raycluster-cluster.yaml +++ b/helm-chart/ray-cluster/templates/raycluster-cluster.yaml @@ -48,8 +48,13 @@ spec: containers: - volumeMounts: {{- toYaml .Values.head.volumeMounts | nindent 12 }} name: ray-head + {{- if .Values.head.image }} + image: {{ .Values.head.image.repository }}:{{ .Values.head.image.tag }} + imagePullPolicy: {{ .Values.head.image.pullPolicy }} + {{- else }} image: {{ .Values.image.repository }}:{{ .Values.image.tag }} imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml .Values.head.resources | nindent 14 }} securityContext: {{- toYaml .Values.head.securityContext | nindent 14 }} @@ -120,8 +125,13 @@ spec: containers: - volumeMounts: {{- toYaml $values.volumeMounts | nindent 12 }} name: ray-worker - image: {{ $.Values.image.repository }}:{{ $.Values.image.tag }} - imagePullPolicy: {{ $.Values.image.pullPolicy }} + {{- if $values.image }} + image: {{ $values.image.repository }}:{{ $values.image.tag }} + imagePullPolicy: {{ $values.image.pullPolicy }} + {{- else }} + image: {{ .Values.image.repository }}:{{ .Values.image.tag }} + imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml $values.resources | nindent 14 }} securityContext: {{- toYaml $values.securityContext | nindent 14 }} @@ -190,8 +200,13 @@ spec: containers: - volumeMounts: {{- toYaml .Values.worker.volumeMounts | nindent 12 }} name: ray-worker + {{- if .Values.worker.image }} + image: {{ .Values.worker.image.repository }}:{{ .Values.worker.image.tag }} + imagePullPolicy: {{ .Values.worker.image.pullPolicy }} + {{- else }} image: {{ .Values.image.repository }}:{{ .Values.image.tag }} imagePullPolicy: {{ .Values.image.pullPolicy }} + {{- end }} resources: {{- toYaml .Values.worker.resources | nindent 14 }} securityContext: {{- toYaml .Values.worker.securityContext | nindent 14 }} From 140e595b6aab28bb24d986535c1f37c9640db3b6 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 34/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 1 - proto/cluster.proto | 24 ++++++++++++------------ 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 62a351c5130..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,7 +15,6 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ diff --git a/proto/cluster.proto b/proto/cluster.proto index 782d9a0c000..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -201,7 +201,7 @@ message Volume { BIDIRECTIONAL = 2; } MountPropagationMode mount_propagation_mode = 7; - // If indicate ephemeral, we need to let user specify volumeClaimTemplate parameters + // If indicate ephemeral, we need to let user specify volumeClaimTemplate string storageClassName = 8; // If not defined, default is used enum AccessMode { RWO = 0; // ReadWriteOnce @@ -219,24 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. The ray start params of head node group. - map ray_start_params = 4; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 4; + // Optional. The ray start params of head node group. + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; - // Optional. Enable cluster Ingress - // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 11; + map labels = 11; } message WorkerGroupSpec { From 5d8c341a68abaacd5dcf1ce749b7f577fc0444dd Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:05:05 +0100 Subject: [PATCH 35/56] Fixed error handling --- apiserver/go.mod | 3 ++ apiserver/go.sum | 3 +- apiserver/pkg/manager/resource_manager.go | 20 +++++-- apiserver/pkg/model/converter.go | 2 +- apiserver/pkg/util/cluster.go | 63 +++++++++++++++++------ apiserver/pkg/util/cluster_test.go | 13 +++-- apiserver/pkg/util/job.go | 20 ++++--- apiserver/pkg/util/service.go | 23 ++++++--- 8 files changed, 102 insertions(+), 45 deletions(-) diff --git a/apiserver/go.mod b/apiserver/go.mod index e644fc0a1d6..3fe266f1659 100644 --- a/apiserver/go.mod +++ b/apiserver/go.mod @@ -25,6 +25,8 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 ) +require github.com/pmezard/go-difflib v1.0.0 // indirect + require ( github.com/asaskevich/govalidator v0.0.0-20200428143746-21a406dcc535 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -51,6 +53,7 @@ require ( github.com/prometheus/common v0.28.0 // indirect github.com/prometheus/procfs v0.6.0 // indirect github.com/spf13/pflag v1.0.5 // indirect + github.com/stretchr/testify v1.8.4 go.mongodb.org/mongo-driver v1.3.4 // indirect golang.org/x/net v0.12.0 // indirect golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f // indirect diff --git a/apiserver/go.sum b/apiserver/go.sum index 758563e20a1..a99ff2b6e89 100644 --- a/apiserver/go.sum +++ b/apiserver/go.sum @@ -433,7 +433,8 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tidwall/pretty v1.0.0 h1:HsD+QiTn7sK6flMKIvNmpqz1qrpP3Ps6jOKIKMooyg4= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 37f5a936953..a71483a3102 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -91,7 +91,10 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu } // convert *api.Cluster to v1alpha1.RayCluster - rayCluster := util.NewRayCluster(apiCluster, computeTemplateDict) + rayCluster, err := util.NewRayCluster(apiCluster, computeTemplateDict) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to create a Ray cluster for (%s/%s)", apiCluster.Namespace, apiCluster.Name) + } // set our own fields. clusterAt := r.clientManager.Time().Now().String() @@ -216,7 +219,10 @@ func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob) (*v } // convert *api.Cluster to v1alpha1.RayCluster - rayJob := util.NewRayJob(apiJob, computeTemplateMap) + rayJob, err := util.NewRayJob(apiJob, computeTemplateMap) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to create a Ray Job for (%s/%s)", apiJob.Namespace, apiJob.Name) + } newRayJob, err := r.getRayJobClient(apiJob.Namespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) if err != nil { @@ -303,7 +309,10 @@ func (r *ResourceManager) CreateService(ctx context.Context, apiService *api.Ray if err != nil { return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiService.Namespace, apiService.Name) } - rayService := util.NewRayService(apiService, computeTemplateDict) + rayService, err := util.NewRayService(apiService, computeTemplateDict) + if err != nil { + return nil, err + } createdAt := r.clientManager.Time().Now().String() rayService.Annotations["ray.io/creation-timestamp"] = createdAt newRayService, err := r.getRayServiceClient(apiService.Namespace).Create(ctx, rayService.Get(), metav1.CreateOptions{}) @@ -327,7 +336,10 @@ func (r *ResourceManager) UpdateRayService(ctx context.Context, apiService *api. if err != nil { return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiService.Namespace, apiService.Name) } - rayService := util.NewRayService(apiService, computeTemplateDict) + rayService, err := util.NewRayService(apiService, computeTemplateDict) + if err != nil { + return nil, err + } rayService.Annotations["ray.io/update-timestamp"] = r.clientManager.Time().Now().String() rayService.ResourceVersion = oldService.DeepCopy().ResourceVersion newRayService, err := client.Update(ctx, rayService.Get(), metav1.UpdateOptions{}) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index b9c86c158df..d02f1681da6 100755 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -167,7 +167,7 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { headNodeSpec.Labels = spec.Template.Labels } - if *spec.EnableIngress { + if spec.EnableIngress != nil && *spec.EnableIngress { headNodeSpec.EnableIngress = true } diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index f3c4a6c210a..d77da86af78 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -2,6 +2,7 @@ package util import ( "encoding/json" + "errors" "fmt" "net" "strconv" @@ -21,7 +22,13 @@ type RayCluster struct { // NewRayCluster creates a RayCluster. // func NewRayCluster(apiCluster *api.Cluster, clusterRuntime *api.ClusterRuntime, computeRuntime *api.ComputeRuntime) *RayCluster { -func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) *RayCluster { +func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) (*RayCluster, error) { + // Build cluster spec + spec, err := buildRayClusterSpec(apiCluster.Version, apiCluster.Envs, apiCluster.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } + // Build cluster rayCluster := &rayalphaapi.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Name: apiCluster.Name, @@ -29,10 +36,10 @@ func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.C Labels: buildRayClusterLabels(apiCluster), Annotations: buildRayClusterAnnotations(apiCluster), }, - Spec: *buildRayClusterSpec(apiCluster.Version, apiCluster.Envs, apiCluster.ClusterSpec, computeTemplateMap), + Spec: *spec, } - return &RayCluster{rayCluster} + return &RayCluster{rayCluster}, nil } // Build cluster labels @@ -57,15 +64,18 @@ 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 { +func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayClusterSpec, error) { computeTemplate := computeTemplateMap[clusterSpec.HeadGroupSpec.ComputeTemplate] - headPodTemplate := buildHeadPodTemplate(imageVersion, envs, clusterSpec.HeadGroupSpec, computeTemplate) + headPodTemplate, err := buildHeadPodTemplate(imageVersion, envs, clusterSpec.HeadGroupSpec, computeTemplate) + if err != nil { + return nil, err + } headReplicas := int32(1) rayClusterSpec := &rayalphaapi.RayClusterSpec{ RayVersion: imageVersion, HeadGroupSpec: rayalphaapi.HeadGroupSpec{ ServiceType: v1.ServiceType(clusterSpec.HeadGroupSpec.ServiceType), - Template: headPodTemplate, + Template: *headPodTemplate, Replicas: &headReplicas, RayStartParams: clusterSpec.HeadGroupSpec.RayStartParams, }, @@ -79,7 +89,10 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] - workerPodTemplate := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) + workerPodTemplate, err := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) + if err != nil { + return nil, err + } minReplicas := spec.Replicas maxReplicas := spec.Replicas @@ -96,13 +109,13 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe MaxReplicas: intPointer(maxReplicas), Replicas: intPointer(spec.Replicas), RayStartParams: spec.RayStartParams, - Template: workerPodTemplate, + Template: *workerPodTemplate, } rayClusterSpec.WorkerGroupSpecs = append(rayClusterSpec.WorkerGroupSpecs, workerNodeSpec) } - return rayClusterSpec + return rayClusterSpec, nil } // Annotations common to both head and worker nodes @@ -114,7 +127,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 { +func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) if len(spec.Image) != 0 { image = spec.Image @@ -126,7 +139,10 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api // build volume and volumeMounts volMounts := buildVolumeMounts(spec.Volumes) - vols := buildVols(spec.Volumes) + vols, err := buildVols(spec.Volumes) + if err != nil { + return nil, err + } podTemplateSpec := v1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -253,7 +269,7 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api } } - return podTemplateSpec + return &podTemplateSpec, nil } // Convert Toleration operator from string @@ -281,7 +297,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 { +func buildWorkerPodTemplate(imageVersion string, envs map[string]string, 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) @@ -295,7 +311,10 @@ func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *a // build volume and volumeMounts volMounts := buildVolumeMounts(spec.Volumes) - vols := buildVols(spec.Volumes) + vols, err := buildVols(spec.Volumes) + if err != nil { + return nil, err + } podTemplateSpec := v1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -471,7 +490,7 @@ func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *a } } - return podTemplateSpec + return &podTemplateSpec, nil } // Build Volume mounts @@ -506,7 +525,7 @@ func newHostPathType(pathType string) *v1.HostPathType { } // Build volumes -func buildVols(apiVolumes []*api.Volume) []v1.Volume { +func buildVols(apiVolumes []*api.Volume) ([]v1.Volume, error) { var vols []v1.Volume for _, rayVol := range apiVolumes { if rayVol.VolumeType == api.Volume_HOST_PATH { @@ -541,6 +560,16 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { vols = append(vols, vol) } if rayVol.VolumeType == api.Volume_EPHEMERAL { + // Make sure that at least the storage size is defined + if rayVol.Storage == "" { + // Storage size is not defined + return nil, errors.New("storage for ephemeral volume is empty") + } + // Ensure that storage size is formatted correctly + _, err := resource.ParseQuantity(rayVol.Storage) + if err != nil { + return nil, errors.New("storage for ephemeral volume is not specified correctly") + } vol := v1.Volume{ Name: rayVol.Name, VolumeSource: v1.VolumeSource{ @@ -590,7 +619,7 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { } } - return vols + return vols, nil } // Init pointer diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index de321065b97..6df93be43c5 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -5,6 +5,7 @@ import ( "testing" api "github.com/ray-project/kuberay/proto/go_client" + "github.com/stretchr/testify/assert" v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -209,7 +210,8 @@ func TestBuildVolumes(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got := buildVols(tt.apiVolume) + got, err := buildVols(tt.apiVolume) + assert.Nil(t, err) if !reflect.DeepEqual(got, tt.expect) { t.Errorf("failed for %s ..., got %v, expected %v", tt.name, got, tt.expect) } @@ -268,7 +270,8 @@ func TestBuildVolumeMounts(t *testing.T) { } func TestBuildHeadPodTemplate(t *testing.T) { - podSpec := buildHeadPodTemplate("2.4", make(map[string]string), &headGroup, &template) + podSpec, err := buildHeadPodTemplate("2.4", make(map[string]string), &headGroup, &template) + assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { t.Errorf("failed to propagate service account") @@ -295,7 +298,8 @@ func TestBuildHeadPodTemplate(t *testing.T) { } func TestBuildRayCluster(t *testing.T) { - cluster := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + cluster, err := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + assert.Nil(t, err) if len(cluster.ObjectMeta.Annotations) != 1 { t.Errorf("failed to propagate annotations") } @@ -305,7 +309,8 @@ func TestBuildRayCluster(t *testing.T) { } func TestBuilWorkerPodTemplate(t *testing.T) { - podSpec := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) + podSpec, err := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) + assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { t.Errorf("failed to propagate service account") diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 60dd20e55fa..b66ba10bb48 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -15,13 +15,7 @@ type RayJob struct { const rayJobDefaultVersion = "1.13" // NewRayJob creates a RayJob. -func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTemplate) *RayJob { - var clusterSpec *rayalphaapi.RayClusterSpec - - if apiJob.ClusterSpec != nil { - clusterSpec = buildRayClusterSpec(rayJobDefaultVersion, nil, apiJob.ClusterSpec, computeTemplateMap) - } - +func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTemplate) (*RayJob, error) { // transfer json to runtimeEnv encodedText := base64.StdEncoding.EncodeToString([]byte(apiJob.RuntimeEnv)) @@ -39,14 +33,18 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, TTLSecondsAfterFinished: &apiJob.TtlSecondsAfterFinished, JobId: apiJob.JobId, - RayClusterSpec: clusterSpec, + RayClusterSpec: nil, ClusterSelector: apiJob.ClusterSelector, }, } - - return &RayJob{ - rayJob, + if apiJob.ClusterSpec != nil { + clusterSpec, err := buildRayClusterSpec(rayJobDefaultVersion, nil, apiJob.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } + rayJob.Spec.RayClusterSpec = clusterSpec } + return &RayJob{rayJob}, nil } func (j *RayJob) Get() *rayalphaapi.RayJob { diff --git a/apiserver/pkg/util/service.go b/apiserver/pkg/util/service.go index 0f3c813cc2b..ec0cd88ebe4 100644 --- a/apiserver/pkg/util/service.go +++ b/apiserver/pkg/util/service.go @@ -24,7 +24,13 @@ func (s *RayService) Get() *rayalphaapi.RayService { return s.RayService } -func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) *RayService { +func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) (*RayService, error) { + // Build the spec + spec, err := buildRayServiceSpec(apiService, computeTemplateMap) + if err != nil { + return nil, err + } + // Build Ray service rayService := &rayalphaapi.RayService{ ObjectMeta: metav1.ObjectMeta{ Name: apiService.Name, @@ -32,9 +38,9 @@ func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*ap Labels: buildRayServiceLabels(apiService), Annotations: buildRayServiceAnnotations(apiService), }, - Spec: *buildRayServiceSpec(apiService, computeTemplateMap), + Spec: *spec, } - return &RayService{rayService} + return &RayService{rayService}, nil } func buildRayServiceLabels(apiService *api.RayService) map[string]string { @@ -52,7 +58,7 @@ func buildRayServiceAnnotations(apiService *api.RayService) map[string]string { return annotations } -func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) *rayalphaapi.RayServiceSpec { +func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayServiceSpec, error) { serveConfigSpecs := make([]rayalphaapi.ServeConfigSpec, 0) for _, serveConfig := range apiService.ServeDeploymentGraphSpec.ServeConfigs { serveConfigSpec := rayalphaapi.ServeConfigSpec{ @@ -74,7 +80,10 @@ func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[stri } serveConfigSpecs = append(serveConfigSpecs, serveConfigSpec) } - newRayClusterSpec := *buildRayClusterSpec(rayServiceDefaultVersion, nil, apiService.ClusterSpec, computeTemplateMap) + newRayClusterSpec, err := buildRayClusterSpec(rayServiceDefaultVersion, nil, apiService.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } newRayClusterSpec.HeadGroupSpec.Template.Spec.Containers[0].Ports = append(newRayClusterSpec.HeadGroupSpec.Template.Spec.Containers[0].Ports, v1.ContainerPort{ Name: defaultServePortName, ContainerPort: defaultServePort, @@ -85,8 +94,8 @@ func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[stri RuntimeEnv: base64.StdEncoding.EncodeToString([]byte(apiService.ServeDeploymentGraphSpec.RuntimeEnv)), ServeConfigSpecs: serveConfigSpecs, }, - RayClusterSpec: newRayClusterSpec, - } + RayClusterSpec: *newRayClusterSpec, + }, nil } func UpdateRayServiceWorkerGroupSpecs(updateSpecs []*api.WorkerGroupUpdateSpec, workerGroupSpecs []rayalphaapi.WorkerGroupSpec) []rayalphaapi.WorkerGroupSpec { From b512c40637554ec236e7265b53f850254ff9dada Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:06:39 +0100 Subject: [PATCH 36/56] Fixed error handling --- apiserver/pkg/util/cluster.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index d77da86af78..c5046b85235 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -87,6 +87,7 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe rayClusterSpec.HeadGroupSpec.EnableIngress = &clusterSpec.HeadGroupSpec.EnableIngress } + // Build worker groups for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] workerPodTemplate, err := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) From 1a2674de2be4a7e757f85ef000b1d02148198f1c Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:14:24 +0100 Subject: [PATCH 37/56] Fixed error handling --- apiserver/pkg/model/volumes_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From 34b5135b9de4656d8111e2fa89a5fc9d51f10e4f Mon Sep 17 00:00:00 2001 From: blublinsky Date: Sun, 27 Aug 2023 14:27:41 +0100 Subject: [PATCH 38/56] Fixed error handling --- apiserver/pkg/manager/resource_manager.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index a71483a3102..584b9efb61d 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -93,7 +93,7 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu // convert *api.Cluster to v1alpha1.RayCluster rayCluster, err := util.NewRayCluster(apiCluster, computeTemplateDict) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a Ray cluster for (%s/%s)", apiCluster.Namespace, apiCluster.Name) + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray cluster") } // set our own fields. @@ -221,7 +221,7 @@ func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob) (*v // convert *api.Cluster to v1alpha1.RayCluster rayJob, err := util.NewRayJob(apiJob, computeTemplateMap) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a Ray Job for (%s/%s)", apiJob.Namespace, apiJob.Name) + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray Job") } newRayJob, err := r.getRayJobClient(apiJob.Namespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) @@ -311,7 +311,7 @@ func (r *ResourceManager) CreateService(ctx context.Context, apiService *api.Ray } rayService, err := util.NewRayService(apiService, computeTemplateDict) if err != nil { - return nil, err + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray Service") } createdAt := r.clientManager.Time().Now().String() rayService.Annotations["ray.io/creation-timestamp"] = createdAt From e208f06b68ee536b6e97915a7d9f7848ecdc6d8d Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 39/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/converter.go | 10 + apiserver/pkg/model/converter_test.go | 28 +- apiserver/pkg/model/volumes.go | 103 ++++ apiserver/pkg/model/volumes_test.go | 127 ++++ apiserver/pkg/util/cluster.go | 60 +- apiserver/pkg/util/cluster_test.go | 65 +++ proto/cluster.proto | 49 +- proto/go_client/cluster.pb.go | 796 +++++++++++++++----------- proto/kuberay_api.swagger.json | 49 +- proto/swagger/cluster.swagger.json | 49 +- proto/swagger/job.swagger.json | 28 +- proto/swagger/serve.swagger.json | 28 +- 12 files changed, 1012 insertions(+), 380 deletions(-) create mode 100644 apiserver/pkg/model/volumes.go create mode 100644 apiserver/pkg/model/volumes_test.go diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index 3f5b12ede86..b9c86c158df 100755 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -107,6 +107,10 @@ func FromCrdToApiCluster(cluster *v1alpha1.RayCluster, events []v1.Event) *api.C ClusterState: string(cluster.Status.State), } + if len(cluster.ObjectMeta.Annotations) > 0 { + pbCluster.Annotations = cluster.ObjectMeta.Annotations + } + // loop container and find the resource pbCluster.ClusterSpec = PopulateRayClusterSpec(cluster.Spec) @@ -146,6 +150,7 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { ServiceType: string(spec.ServiceType), Image: spec.Template.Annotations[util.RayClusterImageAnnotationKey], ComputeTemplate: spec.Template.Annotations[util.RayClusterComputeTemplateAnnotationKey], + Volumes: PopulateVolumes(&spec.Template), } for _, annotation := range getNodeDefaultAnnotations() { @@ -162,6 +167,10 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { headNodeSpec.Labels = spec.Template.Labels } + if *spec.EnableIngress { + headNodeSpec.EnableIngress = true + } + // 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) @@ -196,6 +205,7 @@ func PopulateWorkerNodeSpec(specs []v1alpha1.WorkerGroupSpec) []*api.WorkerGroup GroupName: spec.GroupName, Image: spec.Template.Annotations[util.RayClusterImageAnnotationKey], ComputeTemplate: spec.Template.Annotations[util.RayClusterComputeTemplateAnnotationKey], + Volumes: PopulateVolumes(&spec.Template), } for _, annotation := range getNodeDefaultAnnotations() { diff --git a/apiserver/pkg/model/converter_test.go b/apiserver/pkg/model/converter_test.go index e4e7805bfb5..63250f08bd3 100644 --- a/apiserver/pkg/model/converter_test.go +++ b/apiserver/pkg/model/converter_test.go @@ -11,7 +11,7 @@ import ( ) var ( - enableIngress = false + enableIngress = true headNodeReplicas int32 = 1 workerReplicas int32 = 5 ) @@ -195,6 +195,22 @@ var workerSpecTest = v1alpha1.WorkerGroupSpec{ }, } +var ClusterSpecTest = v1alpha1.RayCluster{ + ObjectMeta: metav1.ObjectMeta{ + Name: "raycluster-sample", + Namespace: "default", + Annotations: map[string]string{ + "kubernetes.io/ingress.class": "nginx", + }, + }, + Spec: v1alpha1.RayClusterSpec{ + HeadGroupSpec: headSpecTest, + WorkerGroupSpecs: []v1alpha1.WorkerGroupSpec{ + workerSpecTest, + }, + }, +} + var expectedAnnotations = map[string]string{ "custom": "value", } @@ -220,6 +236,9 @@ func TestPopulateHeadNodeSpec(t *testing.T) { if groupSpec.ServiceAccount != "account" { t.Errorf("failed to convert service account") } + if groupSpec.EnableIngress != *headSpecTest.EnableIngress { + t.Errorf("failed to convert enableIngress") + } if groupSpec.ImagePullSecret != "foo" { t.Errorf("failed to convert image pull secret") } @@ -254,6 +273,13 @@ func TestPopulateWorkerNodeSpec(t *testing.T) { } } +func TestPopulateRayClusterSpec(t *testing.T) { + cluster := FromCrdToApiCluster(&ClusterSpecTest, []v1.Event{}) + if len(cluster.Annotations) != 1 { + t.Errorf("failed to convert cluster's annotations") + } +} + func TestPopulateTemplate(t *testing.T) { template := FromKubeToAPIComputeTemplate(&configMapWithoutTolerations) if len(template.Tolerations) != 0 { diff --git a/apiserver/pkg/model/volumes.go b/apiserver/pkg/model/volumes.go new file mode 100644 index 00000000000..344e11b698b --- /dev/null +++ b/apiserver/pkg/model/volumes.go @@ -0,0 +1,103 @@ +package model + +import ( + api "github.com/ray-project/kuberay/proto/go_client" + v1 "k8s.io/api/core/v1" +) + +func PopulateVolumes(podTemplate *v1.PodTemplateSpec) []*api.Volume { + if len(podTemplate.Spec.Volumes) == 0 { + return nil + } + var volumes []*api.Volume + for _, vol := range podTemplate.Spec.Volumes { + mount := GetVolumeMount(podTemplate, vol.Name) + if vol.VolumeSource.HostPath != nil { + // Host Path + volumes = append(volumes, &api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + Source: vol.VolumeSource.HostPath.Path, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_VolumeType(api.Volume_HOSTTOCONTAINER), + HostPathType: GetVolumeHostPathType(&vol), + }) + continue + + } + if vol.VolumeSource.PersistentVolumeClaim != nil { + // PVC + volumes = append(volumes, &api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_PERSISTENT_VOLUME_CLAIM, + ReadOnly: vol.VolumeSource.PersistentVolumeClaim.ReadOnly, + }) + continue + } + if vol.VolumeSource.Ephemeral != nil { + // Ephimeral + request := vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.Resources.Requests[v1.ResourceStorage] + volume := api.Volume{ + Name: vol.Name, + MountPath: mount.MountPath, + MountPropagationMode: GetVolumeMountPropagation(mount), + VolumeType: api.Volume_EPHEMERAL, + AccessMode: GetAccessMode(&vol), + Storage: request.String(), + } + if vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName != nil { + volume.StorageClassName = *vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName + } + volumes = append(volumes, &volume) + continue + } + } + return volumes +} + +func GetVolumeMount(podTemplate *v1.PodTemplateSpec, vol string) *v1.VolumeMount { + for _, container := range podTemplate.Spec.Containers { + for _, mount := range container.VolumeMounts { + if mount.Name == vol { + return &mount + } + } + } + return nil +} + +func GetVolumeMountPropagation(mount *v1.VolumeMount) api.Volume_MountPropagationMode { + if mount.MountPropagation == nil { + return api.Volume_NONE + } + if *mount.MountPropagation == v1.MountPropagationHostToContainer { + return api.Volume_HOSTTOCONTAINER + } + if *mount.MountPropagation == v1.MountPropagationBidirectional { + return api.Volume_BIDIRECTIONAL + } + return api.Volume_NONE +} + +func GetVolumeHostPathType(vol *v1.Volume) api.Volume_HostPathType { + if *vol.VolumeSource.HostPath.Type == v1.HostPathFile { + return api.Volume_FILE + } + return api.Volume_DIRECTORY +} + +func GetAccessMode(vol *v1.Volume) api.Volume_AccessMode { + modes := vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes + if len(modes) == 0 { + return api.Volume_RWO + } + if modes[0] == v1.ReadOnlyMany { + return api.Volume_ROX + } + if modes[0] == v1.ReadWriteMany { + return api.Volume_RWX + } + return api.Volume_RWO +} diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go new file mode 100644 index 00000000000..42efea7b787 --- /dev/null +++ b/apiserver/pkg/model/volumes_test.go @@ -0,0 +1,127 @@ +package model + +import ( + "fmt" + "reflect" + "testing" + + api "github.com/ray-project/kuberay/proto/go_client" + v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +var ( + hostToContainer = v1.MountPropagationHostToContainer + bidirectonal = v1.MountPropagationBidirectional +) +var podTemplateTest = v1.PodTemplateSpec{ + Spec: v1.PodSpec{ + Containers: []v1.Container{ + { + Name: "ray-head", + Image: "blah", + VolumeMounts: []v1.VolumeMount{ + { + Name: "hostPath", + MountPath: "/tmp/hostPath", + MountPropagation: &hostToContainer, + }, + { + Name: "pvc", + MountPath: "/tmp/pvc", + MountPropagation: &bidirectonal, + }, + { + Name: "ephemeral", + MountPath: "/tmp/ephemeral", + }, + }, + }, + }, + Volumes: []v1.Volume{ + { + Name: "hostPath", + VolumeSource: v1.VolumeSource{ + HostPath: &v1.HostPathVolumeSource{ + Path: "/tmp", + Type: newHostPathType(string(v1.HostPathDirectory)), + }, + }, + }, + { + Name: "pvc", + VolumeSource: v1.VolumeSource{ + PersistentVolumeClaim: &v1.PersistentVolumeClaimVolumeSource{ + ClaimName: "pvc", + ReadOnly: false, + }, + }, + }, + { + Name: "ephemeral", + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse("5Gi"), + }, + }, + }, + }, + }, + }, + }, + }, + }, +} + +var expectedVolumes = []*api.Volume{ + { + Name: "hostPath", + Source: "/tmp", + MountPath: "/tmp/hostPath", + VolumeType: api.Volume_HOST_PATH, + HostPathType: api.Volume_DIRECTORY, + MountPropagationMode: api.Volume_HOSTTOCONTAINER, + }, + { + Name: "pvc", + MountPath: "/tmp/pvc", + VolumeType: api.Volume_PERSISTENT_VOLUME_CLAIM, + MountPropagationMode: api.Volume_BIDIRECTIONAL, + ReadOnly: false, + }, + { + Name: "ephemeral", + MountPath: "/tmp/ephemeral", + MountPropagationMode: api.Volume_NONE, + VolumeType: api.Volume_EPHEMERAL, + Storage: "5Gi", + AccessMode: api.Volume_RWO, + }, +} + +// Build host path +func newHostPathType(pathType string) *v1.HostPathType { + hostPathType := new(v1.HostPathType) + *hostPathType = v1.HostPathType(pathType) + return hostPathType +} + +func TestPopulateVolumes(t *testing.T) { + volumes := PopulateVolumes(&podTemplateTest) + for i, vol := range volumes { + fmt.Printf("volume = %#v\n", vol) + if !reflect.DeepEqual(vol, expectedVolumes[i]) { + t.Errorf("failed volumes conversion, got %v, expected %v", volumes, expectedVolumes) + } + } +} diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index a99c676fd7a..f3c4a6c210a 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -49,9 +49,10 @@ func buildRayClusterLabels(cluster *api.Cluster) map[string]string { // Build cluster annotations func buildRayClusterAnnotations(cluster *api.Cluster) map[string]string { - annotations := map[string]string{} - // TODO: Add optional annotations - return annotations + if cluster.Annotations == nil { + return map[string]string{} + } + return cluster.Annotations } // TODO(Basasuya & MissionToMars): The job spec depends on ClusterSpec which not all cluster-related configs are included, @@ -71,6 +72,11 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe WorkerGroupSpecs: []rayalphaapi.WorkerGroupSpec{}, } + // If enable ingress is specified, add it to the head node spec. + if clusterSpec.HeadGroupSpec.EnableIngress { + rayClusterSpec.HeadGroupSpec.EnableIngress = &clusterSpec.HeadGroupSpec.EnableIngress + } + for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] workerPodTemplate := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) @@ -534,6 +540,54 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { } vols = append(vols, vol) } + if rayVol.VolumeType == api.Volume_EPHEMERAL { + vol := v1.Volume{ + Name: rayVol.Name, + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse(rayVol.Storage), + }, + }, + }, + }, + }, + }, + } + if len(rayVol.StorageClassName) > 0 { + // Populate storage class, if defined + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.StorageClassName = &rayVol.StorageClassName + } + + // Populate access mode if defined + switch rayVol.AccessMode { + case api.Volume_RWO: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + } + case api.Volume_RWX: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteMany, + } + case api.Volume_ROX: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadOnlyMany, + } + default: + vol.VolumeSource.Ephemeral.VolumeClaimTemplate.Spec.AccessModes = []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + } + } + vols = append(vols, vol) + } } return vols diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index 1a88a42a537..de321065b97 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -6,6 +6,8 @@ import ( api "github.com/ray-project/kuberay/proto/go_client" v1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/resource" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) var testVolume = &api.Volume{ @@ -35,6 +37,13 @@ var testPVCVolume = &api.Volume{ ReadOnly: true, } +var testEphemeralVolume = &api.Volume{ + Name: "test-ephemeral", + VolumeType: api.Volume_EPHEMERAL, + MountPath: "/ephimeral/dir", + Storage: "10Gi", +} + // Spec for testing var headGroup = api.HeadGroupSpec{ ComputeTemplate: "foo", @@ -47,6 +56,7 @@ var headGroup = api.HeadGroupSpec{ }, ServiceAccount: "account", ImagePullSecret: "foo", + EnableIngress: true, Environment: map[string]string{ "foo": "bar", }, @@ -81,6 +91,20 @@ var workerGroup = api.WorkerGroupSpec{ }, } +var rayCluster = api.Cluster{ + Name: "test_cluster", + Namespace: "foo", + Annotations: map[string]string{ + "kubernetes.io/ingress.class": "nginx", + }, + ClusterSpec: &api.ClusterSpec{ + HeadGroupSpec: &headGroup, + WorkerGroupSpec: []*api.WorkerGroupSpec{ + &workerGroup, + }, + }, +} + var template = api.ComputeTemplate{ Name: "", Namespace: "", @@ -134,6 +158,32 @@ func TestBuildVolumes(t *testing.T) { }, }, } + + targetEphemeralVolume := v1.Volume{ + Name: testEphemeralVolume.Name, + VolumeSource: v1.VolumeSource{ + Ephemeral: &v1.EphemeralVolumeSource{ + VolumeClaimTemplate: &v1.PersistentVolumeClaimTemplate{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "app.kubernetes.io/managed-by": "kuberay-apiserver", + }, + }, + Spec: v1.PersistentVolumeClaimSpec{ + AccessModes: []v1.PersistentVolumeAccessMode{ + v1.ReadWriteOnce, + }, + Resources: v1.ResourceRequirements{ + Requests: v1.ResourceList{ + v1.ResourceStorage: resource.MustParse(testEphemeralVolume.Storage), + }, + }, + }, + }, + }, + }, + } + tests := []struct { name string apiVolume []*api.Volume @@ -151,6 +201,11 @@ func TestBuildVolumes(t *testing.T) { []*api.Volume{testPVCVolume}, []v1.Volume{targetPVCVolume}, }, + { + "ephemeral test", + []*api.Volume{testEphemeralVolume}, + []v1.Volume{targetEphemeralVolume}, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { @@ -239,6 +294,16 @@ func TestBuildHeadPodTemplate(t *testing.T) { } } +func TestBuildRayCluster(t *testing.T) { + cluster := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + if len(cluster.ObjectMeta.Annotations) != 1 { + t.Errorf("failed to propagate annotations") + } + if !(*cluster.Spec.HeadGroupSpec.EnableIngress) { + t.Errorf("failed to propagate create Ingress") + } +} + func TestBuilWorkerPodTemplate(t *testing.T) { podSpec := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) diff --git a/proto/cluster.proto b/proto/cluster.proto index e28bd809eb0..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -146,24 +146,26 @@ message Cluster { // Required field. This field indicates ray cluster configuration ClusterSpec cluster_spec = 6; + // Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class + map annotations = 7; + + // Optional input field. Container environment variables from user. + map envs = 8; // Output. The time that the cluster created. - google.protobuf.Timestamp created_at = 7; + google.protobuf.Timestamp created_at = 9; // Output. The time that the cluster deleted. - google.protobuf.Timestamp deleted_at = 8; + google.protobuf.Timestamp deleted_at = 10; // Output. The status to show the cluster status.state - string cluster_state = 9; + string cluster_state = 11; // Output. The list related to the cluster. - repeated ClusterEvent events = 10; + repeated ClusterEvent events = 12; // Output. The service endpoint of the cluster - map service_endpoint = 11; - - // Optional input field. Container environment variables from user. - map envs = 12; + map service_endpoint = 13; } message ClusterSpec { @@ -178,6 +180,7 @@ message Volume { enum VolumeType { PERSISTENT_VOLUME_CLAIM = 0; HOST_PATH = 1; + EPHEMERAL = 2; // Ephemeral (https://kubernetes.io/docs/concepts/storage/ephemeral-volumes/#generic-ephemeral-volumes) } VolumeType volume_type = 2; string name = 3; @@ -197,7 +200,16 @@ message Volume { HOSTTOCONTAINER = 1; BIDIRECTIONAL = 2; } - MountPropagationMode mount_propagation_mode = 7; + MountPropagationMode mount_propagation_mode = 7; + // If indicate ephemeral, we need to let user specify volumeClaimTemplate + string storageClassName = 8; // If not defined, default is used + enum AccessMode { + RWO = 0; // ReadWriteOnce + ROX = 1; // ReadOnlyMany + RWX = 2; // ReadWriteMany + } + AccessMode accessMode = 9; + string storage = 10; // For ephemeral - required storage, GB } message HeadGroupSpec { @@ -207,21 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. The ray start params of head node group - map ray_start_params = 4; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 4; + // Optional. The ray start params of head node group. + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; + map labels = 11; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 9f5ed4618c5..9f5952e72f8 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -82,6 +82,7 @@ type Volume_VolumeType int32 const ( Volume_PERSISTENT_VOLUME_CLAIM Volume_VolumeType = 0 Volume_HOST_PATH Volume_VolumeType = 1 + Volume_EPHEMERAL Volume_VolumeType = 2 // Ephemeral (https://kubernetes.io/docs/concepts/storage/ephemeral-volumes/#generic-ephemeral-volumes) ) // Enum value maps for Volume_VolumeType. @@ -89,10 +90,12 @@ var ( Volume_VolumeType_name = map[int32]string{ 0: "PERSISTENT_VOLUME_CLAIM", 1: "HOST_PATH", + 2: "EPHEMERAL", } Volume_VolumeType_value = map[string]int32{ "PERSISTENT_VOLUME_CLAIM": 0, "HOST_PATH": 1, + "EPHEMERAL": 2, } ) @@ -220,6 +223,55 @@ func (Volume_MountPropagationMode) EnumDescriptor() ([]byte, []int) { return file_cluster_proto_rawDescGZIP(), []int{9, 2} } +type Volume_AccessMode int32 + +const ( + Volume_RWO Volume_AccessMode = 0 // ReadWriteOnce + Volume_ROX Volume_AccessMode = 1 // ReadOnlyMany + Volume_RWX Volume_AccessMode = 2 // ReadWriteMany +) + +// Enum value maps for Volume_AccessMode. +var ( + Volume_AccessMode_name = map[int32]string{ + 0: "RWO", + 1: "ROX", + 2: "RWX", + } + Volume_AccessMode_value = map[string]int32{ + "RWO": 0, + "ROX": 1, + "RWX": 2, + } +) + +func (x Volume_AccessMode) Enum() *Volume_AccessMode { + p := new(Volume_AccessMode) + *p = x + return p +} + +func (x Volume_AccessMode) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Volume_AccessMode) Descriptor() protoreflect.EnumDescriptor { + return file_cluster_proto_enumTypes[4].Descriptor() +} + +func (Volume_AccessMode) Type() protoreflect.EnumType { + return &file_cluster_proto_enumTypes[4] +} + +func (x Volume_AccessMode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Volume_AccessMode.Descriptor instead. +func (Volume_AccessMode) EnumDescriptor() ([]byte, []int) { + return file_cluster_proto_rawDescGZIP(), []int{9, 3} +} + type CreateClusterRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -589,18 +641,20 @@ type Cluster struct { Environment Cluster_Environment `protobuf:"varint,5,opt,name=environment,proto3,enum=proto.Cluster_Environment" json:"environment,omitempty"` // Required field. This field indicates ray cluster configuration ClusterSpec *ClusterSpec `protobuf:"bytes,6,opt,name=cluster_spec,json=clusterSpec,proto3" json:"cluster_spec,omitempty"` + // Optional. Annotations, for example, IngressClassAnnotationKey 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"` // Output. The time that the cluster created. - CreatedAt *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` + CreatedAt *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` // Output. The time that the cluster deleted. - DeletedAt *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=deleted_at,json=deletedAt,proto3" json:"deleted_at,omitempty"` + DeletedAt *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=deleted_at,json=deletedAt,proto3" json:"deleted_at,omitempty"` // Output. The status to show the cluster status.state - ClusterState string `protobuf:"bytes,9,opt,name=cluster_state,json=clusterState,proto3" json:"cluster_state,omitempty"` + ClusterState string `protobuf:"bytes,11,opt,name=cluster_state,json=clusterState,proto3" json:"cluster_state,omitempty"` // Output. The list related to the cluster. - Events []*ClusterEvent `protobuf:"bytes,10,rep,name=events,proto3" json:"events,omitempty"` + Events []*ClusterEvent `protobuf:"bytes,12,rep,name=events,proto3" json:"events,omitempty"` // Output. The service endpoint of the cluster - ServiceEndpoint map[string]string `protobuf:"bytes,11,rep,name=service_endpoint,json=serviceEndpoint,proto3" json:"service_endpoint,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,12,rep,name=envs,proto3" json:"envs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ServiceEndpoint map[string]string `protobuf:"bytes,13,rep,name=service_endpoint,json=serviceEndpoint,proto3" json:"service_endpoint,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *Cluster) Reset() { @@ -677,6 +731,20 @@ func (x *Cluster) GetClusterSpec() *ClusterSpec { return nil } +func (x *Cluster) GetAnnotations() map[string]string { + if x != nil { + return x.Annotations + } + return nil +} + +func (x *Cluster) GetEnvs() map[string]string { + if x != nil { + return x.Envs + } + return nil +} + func (x *Cluster) GetCreatedAt() *timestamppb.Timestamp { if x != nil { return x.CreatedAt @@ -712,13 +780,6 @@ func (x *Cluster) GetServiceEndpoint() map[string]string { return nil } -func (x *Cluster) GetEnvs() map[string]string { - if x != nil { - return x.Envs - } - return nil -} - type ClusterSpec struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -788,6 +849,10 @@ type Volume struct { ReadOnly bool `protobuf:"varint,5,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` HostPathType Volume_HostPathType `protobuf:"varint,6,opt,name=host_path_type,json=hostPathType,proto3,enum=proto.Volume_HostPathType" json:"host_path_type,omitempty"` MountPropagationMode Volume_MountPropagationMode `protobuf:"varint,7,opt,name=mount_propagation_mode,json=mountPropagationMode,proto3,enum=proto.Volume_MountPropagationMode" json:"mount_propagation_mode,omitempty"` + // If indicate ephemeral, we need to let user specify volumeClaimTemplate + StorageClassName string `protobuf:"bytes,8,opt,name=storageClassName,proto3" json:"storageClassName,omitempty"` // If not defined, default is used + AccessMode Volume_AccessMode `protobuf:"varint,9,opt,name=accessMode,proto3,enum=proto.Volume_AccessMode" json:"accessMode,omitempty"` + Storage string `protobuf:"bytes,10,opt,name=storage,proto3" json:"storage,omitempty"` // For ephemeral - required storage, GB } func (x *Volume) Reset() { @@ -871,6 +936,27 @@ func (x *Volume) GetMountPropagationMode() Volume_MountPropagationMode { return Volume_NONE } +func (x *Volume) GetStorageClassName() string { + if x != nil { + return x.StorageClassName + } + return "" +} + +func (x *Volume) GetAccessMode() Volume_AccessMode { + if x != nil { + return x.AccessMode + } + return Volume_RWO +} + +func (x *Volume) GetStorage() string { + if x != nil { + return x.Storage + } + return "" +} + type HeadGroupSpec struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -882,21 +968,24 @@ type HeadGroupSpec struct { Image string `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"` // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` - // Optional. The ray start params of head node group - RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + // Optional. The ray start params of head node group. + RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // Optional. image pull secret used by head pod - ImagePullSecret string `protobuf:"bytes,7,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` + 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,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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"` // Optional. Annotations for the head pod - Annotations map[string]string `protobuf:"bytes,9,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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 - Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -952,6 +1041,13 @@ func (x *HeadGroupSpec) GetServiceType() string { return "" } +func (x *HeadGroupSpec) GetEnableIngress() bool { + if x != nil { + return x.EnableIngress + } + return false +} + func (x *HeadGroupSpec) GetRayStartParams() map[string]string { if x != nil { return x.RayStartParams @@ -1319,7 +1415,7 @@ 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, 0xe3, 0x05, + 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, @@ -1333,252 +1429,274 @@ var file_cluster_proto_rawDesc = []byte{ 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, 0x39, - 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x07, 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, 0x08, 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, 0x09, 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, 0x0a, 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, 0x0b, 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, 0x12, 0x2c, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x0c, - 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, 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, 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, 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, 0xf4, 0x03, 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, 0x22, - 0x38, 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, 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, 0x8f, 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, 0x52, 0x0a, 0x10, 0x72, 0x61, 0x79, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 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, 0x05, 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, 0x06, 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, 0x07, 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, 0x08, 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, 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, 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, 0x0a, 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, + 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, 0xac, 0x05, 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, 0x22, 0x47, 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, 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, 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, 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, + 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, 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, 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, + 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 ( @@ -1593,83 +1711,87 @@ func file_cluster_proto_rawDescGZIP() []byte { return file_cluster_proto_rawDescData } -var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 23) +var file_cluster_proto_enumTypes = make([]protoimpl.EnumInfo, 5) +var file_cluster_proto_msgTypes = make([]protoimpl.MessageInfo, 24) 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 - (*CreateClusterRequest)(nil), // 4: proto.CreateClusterRequest - (*GetClusterRequest)(nil), // 5: proto.GetClusterRequest - (*ListClustersRequest)(nil), // 6: proto.ListClustersRequest - (*ListClustersResponse)(nil), // 7: proto.ListClustersResponse - (*ListAllClustersRequest)(nil), // 8: proto.ListAllClustersRequest - (*ListAllClustersResponse)(nil), // 9: proto.ListAllClustersResponse - (*DeleteClusterRequest)(nil), // 10: proto.DeleteClusterRequest - (*Cluster)(nil), // 11: proto.Cluster - (*ClusterSpec)(nil), // 12: proto.ClusterSpec - (*Volume)(nil), // 13: proto.Volume - (*HeadGroupSpec)(nil), // 14: proto.HeadGroupSpec - (*WorkerGroupSpec)(nil), // 15: proto.WorkerGroupSpec - (*ClusterEvent)(nil), // 16: proto.ClusterEvent - nil, // 17: proto.Cluster.ServiceEndpointEntry - nil, // 18: proto.Cluster.EnvsEntry - nil, // 19: proto.HeadGroupSpec.RayStartParamsEntry - nil, // 20: proto.HeadGroupSpec.EnvironmentEntry - nil, // 21: proto.HeadGroupSpec.AnnotationsEntry - nil, // 22: proto.HeadGroupSpec.LabelsEntry - nil, // 23: proto.WorkerGroupSpec.RayStartParamsEntry - nil, // 24: proto.WorkerGroupSpec.EnvironmentEntry - nil, // 25: proto.WorkerGroupSpec.AnnotationsEntry - nil, // 26: proto.WorkerGroupSpec.LabelsEntry - (*timestamppb.Timestamp)(nil), // 27: google.protobuf.Timestamp - (*emptypb.Empty)(nil), // 28: google.protobuf.Empty + (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.HeadGroupSpec.RayStartParamsEntry + nil, // 22: proto.HeadGroupSpec.EnvironmentEntry + nil, // 23: proto.HeadGroupSpec.AnnotationsEntry + nil, // 24: proto.HeadGroupSpec.LabelsEntry + nil, // 25: proto.WorkerGroupSpec.RayStartParamsEntry + nil, // 26: proto.WorkerGroupSpec.EnvironmentEntry + nil, // 27: proto.WorkerGroupSpec.AnnotationsEntry + nil, // 28: proto.WorkerGroupSpec.LabelsEntry + (*timestamppb.Timestamp)(nil), // 29: google.protobuf.Timestamp + (*emptypb.Empty)(nil), // 30: google.protobuf.Empty } var file_cluster_proto_depIdxs = []int32{ - 11, // 0: proto.CreateClusterRequest.cluster:type_name -> proto.Cluster - 11, // 1: proto.ListClustersResponse.clusters:type_name -> proto.Cluster - 11, // 2: proto.ListAllClustersResponse.clusters:type_name -> proto.Cluster + 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 - 12, // 4: proto.Cluster.cluster_spec:type_name -> proto.ClusterSpec - 27, // 5: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp - 27, // 6: proto.Cluster.deleted_at:type_name -> google.protobuf.Timestamp - 16, // 7: proto.Cluster.events:type_name -> proto.ClusterEvent - 17, // 8: proto.Cluster.service_endpoint:type_name -> proto.Cluster.ServiceEndpointEntry - 18, // 9: proto.Cluster.envs:type_name -> proto.Cluster.EnvsEntry - 14, // 10: proto.ClusterSpec.head_group_spec:type_name -> proto.HeadGroupSpec - 15, // 11: proto.ClusterSpec.worker_group_spec:type_name -> proto.WorkerGroupSpec - 1, // 12: proto.Volume.volume_type:type_name -> proto.Volume.VolumeType - 2, // 13: proto.Volume.host_path_type:type_name -> proto.Volume.HostPathType - 3, // 14: proto.Volume.mount_propagation_mode:type_name -> proto.Volume.MountPropagationMode - 19, // 15: proto.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry - 13, // 16: proto.HeadGroupSpec.volumes:type_name -> proto.Volume - 20, // 17: proto.HeadGroupSpec.environment:type_name -> proto.HeadGroupSpec.EnvironmentEntry - 21, // 18: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry - 22, // 19: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry - 23, // 20: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry - 13, // 21: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume - 24, // 22: proto.WorkerGroupSpec.environment:type_name -> proto.WorkerGroupSpec.EnvironmentEntry - 25, // 23: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry - 26, // 24: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry - 27, // 25: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp - 27, // 26: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp - 27, // 27: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp - 4, // 28: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest - 5, // 29: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest - 6, // 30: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest - 8, // 31: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest - 10, // 32: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest - 11, // 33: proto.ClusterService.CreateCluster:output_type -> proto.Cluster - 11, // 34: proto.ClusterService.GetCluster:output_type -> proto.Cluster - 7, // 35: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse - 9, // 36: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse - 28, // 37: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty - 33, // [33:38] is the sub-list for method output_type - 28, // [28:33] is the sub-list for method input_type - 28, // [28:28] is the sub-list for extension type_name - 28, // [28:28] is the sub-list for extension extendee - 0, // [0:28] is the sub-list for field type_name + 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 + 29, // 7: proto.Cluster.created_at:type_name -> google.protobuf.Timestamp + 29, // 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.HeadGroupSpec.ray_start_params:type_name -> proto.HeadGroupSpec.RayStartParamsEntry + 14, // 18: proto.HeadGroupSpec.volumes:type_name -> proto.Volume + 22, // 19: proto.HeadGroupSpec.environment:type_name -> proto.HeadGroupSpec.EnvironmentEntry + 23, // 20: proto.HeadGroupSpec.annotations:type_name -> proto.HeadGroupSpec.AnnotationsEntry + 24, // 21: proto.HeadGroupSpec.labels:type_name -> proto.HeadGroupSpec.LabelsEntry + 25, // 22: proto.WorkerGroupSpec.ray_start_params:type_name -> proto.WorkerGroupSpec.RayStartParamsEntry + 14, // 23: proto.WorkerGroupSpec.volumes:type_name -> proto.Volume + 26, // 24: proto.WorkerGroupSpec.environment:type_name -> proto.WorkerGroupSpec.EnvironmentEntry + 27, // 25: proto.WorkerGroupSpec.annotations:type_name -> proto.WorkerGroupSpec.AnnotationsEntry + 28, // 26: proto.WorkerGroupSpec.labels:type_name -> proto.WorkerGroupSpec.LabelsEntry + 29, // 27: proto.ClusterEvent.created_at:type_name -> google.protobuf.Timestamp + 29, // 28: proto.ClusterEvent.first_timestamp:type_name -> google.protobuf.Timestamp + 29, // 29: proto.ClusterEvent.last_timestamp:type_name -> google.protobuf.Timestamp + 5, // 30: proto.ClusterService.CreateCluster:input_type -> proto.CreateClusterRequest + 6, // 31: proto.ClusterService.GetCluster:input_type -> proto.GetClusterRequest + 7, // 32: proto.ClusterService.ListCluster:input_type -> proto.ListClustersRequest + 9, // 33: proto.ClusterService.ListAllClusters:input_type -> proto.ListAllClustersRequest + 11, // 34: proto.ClusterService.DeleteCluster:input_type -> proto.DeleteClusterRequest + 12, // 35: proto.ClusterService.CreateCluster:output_type -> proto.Cluster + 12, // 36: proto.ClusterService.GetCluster:output_type -> proto.Cluster + 8, // 37: proto.ClusterService.ListCluster:output_type -> proto.ListClustersResponse + 10, // 38: proto.ClusterService.ListAllClusters:output_type -> proto.ListAllClustersResponse + 30, // 39: proto.ClusterService.DeleteCluster:output_type -> google.protobuf.Empty + 35, // [35:40] is the sub-list for method output_type + 30, // [30:35] is the sub-list for method input_type + 30, // [30:30] is the sub-list for extension type_name + 30, // [30:30] is the sub-list for extension extendee + 0, // [0:30] is the sub-list for field type_name } func init() { file_cluster_proto_init() } @@ -1840,8 +1962,8 @@ func file_cluster_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_cluster_proto_rawDesc, - NumEnums: 4, - NumMessages: 23, + NumEnums: 5, + NumMessages: 24, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/kuberay_api.swagger.json b/proto/kuberay_api.swagger.json index e236d882f5a..bd6e85a685c 100644 --- a/proto/kuberay_api.swagger.json +++ b/proto/kuberay_api.swagger.json @@ -992,6 +992,15 @@ "default": "DEV", "description": "Optional field." }, + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -1014,7 +1023,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -1062,6 +1072,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "Required field. This field indicates ray cluster configuration" }, + "annotations": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class" + }, + "envs": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Optional input field. Container environment variables from user." + }, "createdAt": { "type": "string", "format": "date-time", @@ -1089,13 +1113,6 @@ "type": "string" }, "title": "Output. The service endpoint of the cluster" - }, - "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Optional input field. Container environment variables from user." } } }, @@ -1175,12 +1192,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -1267,6 +1288,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, diff --git a/proto/swagger/cluster.swagger.json b/proto/swagger/cluster.swagger.json index 6c9b5475c3c..9409bb30677 100644 --- a/proto/swagger/cluster.swagger.json +++ b/proto/swagger/cluster.swagger.json @@ -202,6 +202,15 @@ "default": "DEV", "description": "Optional field." }, + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -224,7 +233,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -272,6 +282,20 @@ "$ref": "#/definitions/protoClusterSpec", "title": "Required field. This field indicates ray cluster configuration" }, + "annotations": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "title": "Optional. Annotations, for example, IngressClassAnnotationKey to define Ingress class" + }, + "envs": { + "type": "object", + "additionalProperties": { + "type": "string" + }, + "description": "Optional input field. Container environment variables from user." + }, "createdAt": { "type": "string", "format": "date-time", @@ -299,13 +323,6 @@ "type": "string" }, "title": "Output. The service endpoint of the cluster" - }, - "envs": { - "type": "object", - "additionalProperties": { - "type": "string" - }, - "description": "Optional input field. Container environment variables from user." } } }, @@ -385,12 +402,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -477,6 +498,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, diff --git a/proto/swagger/job.swagger.json b/proto/swagger/job.swagger.json index c3d67eeac5b..394f7362a70 100644 --- a/proto/swagger/job.swagger.json +++ b/proto/swagger/job.swagger.json @@ -191,6 +191,15 @@ } }, "definitions": { + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -213,7 +222,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -266,12 +276,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -435,6 +449,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, diff --git a/proto/swagger/serve.swagger.json b/proto/swagger/serve.swagger.json index 1428d52b417..854a3cd2e22 100644 --- a/proto/swagger/serve.swagger.json +++ b/proto/swagger/serve.swagger.json @@ -317,6 +317,15 @@ } }, "definitions": { + "VolumeAccessMode": { + "type": "string", + "enum": [ + "RWO", + "ROX", + "RWX" + ], + "default": "RWO" + }, "VolumeHostPathType": { "type": "string", "enum": [ @@ -339,7 +348,8 @@ "type": "string", "enum": [ "PERSISTENT_VOLUME_CLAIM", - "HOST_PATH" + "HOST_PATH", + "EPHEMERAL" ], "default": "PERSISTENT_VOLUME_CLAIM" }, @@ -430,12 +440,16 @@ "type": "string", "title": "Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node" }, + "enableIngress": { + "type": "boolean", + "title": "Optional. Enable Ingress\nif Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class" + }, "rayStartParams": { "type": "object", "additionalProperties": { "type": "string" }, - "title": "Optional. The ray start params of head node group" + "description": "Optional. The ray start params of head node group." }, "volumes": { "type": "array", @@ -783,6 +797,16 @@ }, "mountPropagationMode": { "$ref": "#/definitions/VolumeMountPropagationMode" + }, + "storageClassName": { + "type": "string", + "title": "If indicate ephemeral, we need to let user specify volumeClaimTemplate" + }, + "accessMode": { + "$ref": "#/definitions/VolumeAccessMode" + }, + "storage": { + "type": "string" } } }, From 13488008bff597a38d3b4c78e1a9596c3f6f774d Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 11 Aug 2023 08:35:37 +0100 Subject: [PATCH 40/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From c933c3719b94314b45d20cad285c1c3f27764e30 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 41/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 62a351c5130..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,7 +15,6 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From c6dc652508d2aa2b9c0d47e83a7a988ff3fc35c8 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 08:37:49 +0100 Subject: [PATCH 42/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..95d8fa2e0e0 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -20,7 +20,7 @@ var podTemplateTest = v1.PodTemplateSpec{ Containers: []v1.Container{ { Name: "ray-head", - Image: "blah", + Image: "my_image", VolumeMounts: []v1.VolumeMount{ { Name: "hostPath", From 522ac3e43458ee3e73848b4f3f90d5864b22a896 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 08:40:53 +0100 Subject: [PATCH 43/56] Fixed formatting --- apiserver/pkg/model/volumes_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 95d8fa2e0e0..1b421e6f334 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -19,7 +19,7 @@ var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head", + Name: "ray_head", Image: "my_image", VolumeMounts: []v1.VolumeMount{ { From e42042da5dc56c36ea540caf744c5f50c79e43f3 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 44/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 1b421e6f334..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -19,8 +19,8 @@ var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray_head", - Image: "my_image", + Name: "ray-head", + Image: "blah", VolumeMounts: []v1.VolumeMount{ { Name: "hostPath", From 6557b592e06aa60a742604fbcc78bb42f468d28c Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 15 Aug 2023 10:58:43 +0100 Subject: [PATCH 45/56] rebased to current --- apiserver/pkg/model/volumes_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..a094224e777 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,11 +15,12 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head", + Name: "ray-head-test", Image: "blah", VolumeMounts: []v1.VolumeMount{ { From 06b299245e50733c62670f5146e38b0d59d78e93 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 17 Aug 2023 15:15:07 +0100 Subject: [PATCH 46/56] restoring PB numbers --- proto/cluster.proto | 16 ++++++++-------- proto/go_client/cluster.pb.go | 32 ++++++++++++++++---------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 827577b38e1..9ef278f63d0 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -221,22 +221,22 @@ message HeadGroupSpec { string service_type = 3; // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 4; + bool enableIngress = 11; // Optional. The ray start params of head node group. - map ray_start_params = 5; + map ray_start_params = 4; // Optional. The volumes mount to head pod - repeated Volume volumes = 6; + repeated Volume volumes = 5; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 7; + string service_account = 6; // Optional. image pull secret used by head pod - string image_pull_secret = 8; + string image_pull_secret = 7; // Optional. Environment variables for head pod - map environment = 9; + map environment = 8; // Optional. Annotations for the head pod - map annotations = 10; + map annotations = 9; // Optional. Labels for the head pod - map labels = 11; + map labels = 10; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 9f5952e72f8..6b8e12c4da7 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -970,22 +970,22 @@ type HeadGroupSpec struct { ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + EnableIngress bool `protobuf:"varint,11,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` // Optional. The ray start params of head node group. - RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // 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"` + ImagePullSecret string `protobuf:"bytes,7,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 map[string]string `protobuf:"bytes,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // 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"` + Annotations map[string]string `protobuf:"bytes,9,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 - Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -1530,32 +1530,32 @@ var file_cluster_proto_rawDesc = []byte{ 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, + 0x0d, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 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, + 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 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, + 0x65, 0x73, 0x18, 0x05, 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, + 0x75, 0x6e, 0x74, 0x18, 0x06, 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, + 0x67, 0x65, 0x5f, 0x70, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x07, 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, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x08, 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, + 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 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, + 0x73, 0x18, 0x0a, 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, From cd4690d1eec14613e4126b754bafa54da9f18e20 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 47/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 3 +-- proto/cluster.proto | 16 +++++++-------- proto/go_client/cluster.pb.go | 32 ++++++++++++++--------------- 3 files changed, 25 insertions(+), 26 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index a094224e777..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,12 +15,11 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ { - Name: "ray-head-test", + Name: "ray-head", Image: "blah", VolumeMounts: []v1.VolumeMount{ { diff --git a/proto/cluster.proto b/proto/cluster.proto index 9ef278f63d0..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -221,22 +221,22 @@ message HeadGroupSpec { string service_type = 3; // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 11; + bool enableIngress = 4; // Optional. The ray start params of head node group. - map ray_start_params = 4; + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; + map labels = 11; } message WorkerGroupSpec { diff --git a/proto/go_client/cluster.pb.go b/proto/go_client/cluster.pb.go index 6b8e12c4da7..9f5952e72f8 100644 --- a/proto/go_client/cluster.pb.go +++ b/proto/go_client/cluster.pb.go @@ -970,22 +970,22 @@ type HeadGroupSpec struct { ServiceType string `protobuf:"bytes,3,opt,name=service_type,json=serviceType,proto3" json:"service_type,omitempty"` // Optional. Enable Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - EnableIngress bool `protobuf:"varint,11,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` + EnableIngress bool `protobuf:"varint,4,opt,name=enableIngress,proto3" json:"enableIngress,omitempty"` // Optional. The ray start params of head node group. - RayStartParams map[string]string `protobuf:"bytes,4,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RayStartParams map[string]string `protobuf:"bytes,5,rep,name=ray_start_params,json=rayStartParams,proto3" json:"ray_start_params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Optional. The volumes mount to head pod - Volumes []*Volume `protobuf:"bytes,5,rep,name=volumes,proto3" json:"volumes,omitempty"` + Volumes []*Volume `protobuf:"bytes,6,rep,name=volumes,proto3" json:"volumes,omitempty"` // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - ServiceAccount string `protobuf:"bytes,6,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + ServiceAccount string `protobuf:"bytes,7,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` // Optional. image pull secret used by head pod - ImagePullSecret string `protobuf:"bytes,7,opt,name=image_pull_secret,json=imagePullSecret,proto3" json:"image_pull_secret,omitempty"` + 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,8,rep,name=environment,proto3" json:"environment,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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"` // Optional. Annotations for the head pod - Annotations map[string]string `protobuf:"bytes,9,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + 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 - Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *HeadGroupSpec) Reset() { @@ -1530,32 +1530,32 @@ var file_cluster_proto_rawDesc = []byte{ 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, 0x0b, + 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, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, + 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, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 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, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x65, 0x72, 0x76, 0x69, + 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, 0x07, + 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, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x72, 0x6f, + 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, 0x09, 0x20, + 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, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 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, From c6195bd23f257fac82ae43db1404f046d54c2672 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 08:41:50 +0100 Subject: [PATCH 48/56] minor fixes --- proto/cluster.proto | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 827577b38e1..9838ce671f0 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -201,7 +201,7 @@ message Volume { BIDIRECTIONAL = 2; } MountPropagationMode mount_propagation_mode = 7; - // If indicate ephemeral, we need to let user specify volumeClaimTemplate + // If indicate ephemeral, we need to let user specify volumeClaimTemplate parameters string storageClassName = 8; // If not defined, default is used enum AccessMode { RWO = 0; // ReadWriteOnce @@ -219,24 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. Enable Ingress - // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 4; - // Optional. The ray start params of head node group. - map ray_start_params = 5; + // Optional. The ray start params of head node group. + map ray_start_params = 4; // Optional. The volumes mount to head pod - repeated Volume volumes = 6; + repeated Volume volumes = 5; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 7; + string service_account = 6; // Optional. image pull secret used by head pod - string image_pull_secret = 8; + string image_pull_secret = 7; // Optional. Environment variables for head pod - map environment = 9; + map environment = 8; // Optional. Annotations for the head pod - map annotations = 10; + map annotations = 9; // Optional. Labels for the head pod - map labels = 11; + map labels = 10; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 11; } message WorkerGroupSpec { From 402f252014c7656b98c21d823a730a5daab07881 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 08:44:03 +0100 Subject: [PATCH 49/56] minor fixes --- proto/cluster.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/proto/cluster.proto b/proto/cluster.proto index 9838ce671f0..782d9a0c000 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -234,7 +234,7 @@ message HeadGroupSpec { map annotations = 9; // Optional. Labels for the head pod map labels = 10; - // Optional. Enable Ingress + // Optional. Enable cluster Ingress // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class bool enableIngress = 11; } From e0910fce6060721a57526aad941a1387a714e0d8 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Tue, 22 Aug 2023 10:52:20 +0100 Subject: [PATCH 50/56] minor fixes --- apiserver/pkg/model/volumes_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From bda0809c1c15d6fc92cead3386807ac3738e0ae6 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Thu, 10 Aug 2023 14:58:33 +0100 Subject: [PATCH 51/56] Added support for ephemeral volumes and ingress creation support --- apiserver/pkg/model/volumes_test.go | 1 - proto/cluster.proto | 24 ++++++++++++------------ 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 62a351c5130..42efea7b787 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,7 +15,6 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) - var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ diff --git a/proto/cluster.proto b/proto/cluster.proto index 782d9a0c000..827577b38e1 100644 --- a/proto/cluster.proto +++ b/proto/cluster.proto @@ -201,7 +201,7 @@ message Volume { BIDIRECTIONAL = 2; } MountPropagationMode mount_propagation_mode = 7; - // If indicate ephemeral, we need to let user specify volumeClaimTemplate parameters + // If indicate ephemeral, we need to let user specify volumeClaimTemplate string storageClassName = 8; // If not defined, default is used enum AccessMode { RWO = 0; // ReadWriteOnce @@ -219,24 +219,24 @@ message HeadGroupSpec { string image = 2; // Optional. The service type (ClusterIP, NodePort, Load balancer) of the head node string service_type = 3; - // Optional. The ray start params of head node group. - map ray_start_params = 4; + // Optional. Enable Ingress + // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class + bool enableIngress = 4; + // Optional. The ray start params of head node group. + map ray_start_params = 5; // Optional. The volumes mount to head pod - repeated Volume volumes = 5; + repeated Volume volumes = 6; // Optional. ServiceAccount used by head pod // Note that the service account has to be created prior to usage here - string service_account = 6; + string service_account = 7; // Optional. image pull secret used by head pod - string image_pull_secret = 7; + string image_pull_secret = 8; // Optional. Environment variables for head pod - map environment = 8; + map environment = 9; // Optional. Annotations for the head pod - map annotations = 9; + map annotations = 10; // Optional. Labels for the head pod - map labels = 10; - // Optional. Enable cluster Ingress - // if Ingress is enabled, we might have to specify annotation IngressClassAnnotationKey, for the cluster itself, defining Ingress class - bool enableIngress = 11; + map labels = 11; } message WorkerGroupSpec { From 52bac982968c1ccadc05cd3f131cf587c37f8f41 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:05:05 +0100 Subject: [PATCH 52/56] Fixed error handling --- apiserver/go.mod | 3 ++ apiserver/go.sum | 3 +- apiserver/pkg/manager/resource_manager.go | 20 +++++-- apiserver/pkg/model/converter.go | 2 +- apiserver/pkg/util/cluster.go | 63 +++++++++++++++++------ apiserver/pkg/util/cluster_test.go | 13 +++-- apiserver/pkg/util/job.go | 20 ++++--- apiserver/pkg/util/service.go | 23 ++++++--- 8 files changed, 102 insertions(+), 45 deletions(-) diff --git a/apiserver/go.mod b/apiserver/go.mod index e644fc0a1d6..3fe266f1659 100644 --- a/apiserver/go.mod +++ b/apiserver/go.mod @@ -25,6 +25,8 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.6.0 ) +require github.com/pmezard/go-difflib v1.0.0 // indirect + require ( github.com/asaskevich/govalidator v0.0.0-20200428143746-21a406dcc535 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -51,6 +53,7 @@ require ( github.com/prometheus/common v0.28.0 // indirect github.com/prometheus/procfs v0.6.0 // indirect github.com/spf13/pflag v1.0.5 // indirect + github.com/stretchr/testify v1.8.4 go.mongodb.org/mongo-driver v1.3.4 // indirect golang.org/x/net v0.12.0 // indirect golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f // indirect diff --git a/apiserver/go.sum b/apiserver/go.sum index 758563e20a1..a99ff2b6e89 100644 --- a/apiserver/go.sum +++ b/apiserver/go.sum @@ -433,7 +433,8 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tidwall/pretty v1.0.0 h1:HsD+QiTn7sK6flMKIvNmpqz1qrpP3Ps6jOKIKMooyg4= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 37f5a936953..a71483a3102 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -91,7 +91,10 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu } // convert *api.Cluster to v1alpha1.RayCluster - rayCluster := util.NewRayCluster(apiCluster, computeTemplateDict) + rayCluster, err := util.NewRayCluster(apiCluster, computeTemplateDict) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to create a Ray cluster for (%s/%s)", apiCluster.Namespace, apiCluster.Name) + } // set our own fields. clusterAt := r.clientManager.Time().Now().String() @@ -216,7 +219,10 @@ func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob) (*v } // convert *api.Cluster to v1alpha1.RayCluster - rayJob := util.NewRayJob(apiJob, computeTemplateMap) + rayJob, err := util.NewRayJob(apiJob, computeTemplateMap) + if err != nil { + return nil, util.NewInternalServerError(err, "Failed to create a Ray Job for (%s/%s)", apiJob.Namespace, apiJob.Name) + } newRayJob, err := r.getRayJobClient(apiJob.Namespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) if err != nil { @@ -303,7 +309,10 @@ func (r *ResourceManager) CreateService(ctx context.Context, apiService *api.Ray if err != nil { return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiService.Namespace, apiService.Name) } - rayService := util.NewRayService(apiService, computeTemplateDict) + rayService, err := util.NewRayService(apiService, computeTemplateDict) + if err != nil { + return nil, err + } createdAt := r.clientManager.Time().Now().String() rayService.Annotations["ray.io/creation-timestamp"] = createdAt newRayService, err := r.getRayServiceClient(apiService.Namespace).Create(ctx, rayService.Get(), metav1.CreateOptions{}) @@ -327,7 +336,10 @@ func (r *ResourceManager) UpdateRayService(ctx context.Context, apiService *api. if err != nil { return nil, util.NewInternalServerError(err, "Failed to populate compute template for (%s/%s)", apiService.Namespace, apiService.Name) } - rayService := util.NewRayService(apiService, computeTemplateDict) + rayService, err := util.NewRayService(apiService, computeTemplateDict) + if err != nil { + return nil, err + } rayService.Annotations["ray.io/update-timestamp"] = r.clientManager.Time().Now().String() rayService.ResourceVersion = oldService.DeepCopy().ResourceVersion newRayService, err := client.Update(ctx, rayService.Get(), metav1.UpdateOptions{}) diff --git a/apiserver/pkg/model/converter.go b/apiserver/pkg/model/converter.go index b9c86c158df..d02f1681da6 100755 --- a/apiserver/pkg/model/converter.go +++ b/apiserver/pkg/model/converter.go @@ -167,7 +167,7 @@ func PopulateHeadNodeSpec(spec v1alpha1.HeadGroupSpec) *api.HeadGroupSpec { headNodeSpec.Labels = spec.Template.Labels } - if *spec.EnableIngress { + if spec.EnableIngress != nil && *spec.EnableIngress { headNodeSpec.EnableIngress = true } diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index f3c4a6c210a..d77da86af78 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -2,6 +2,7 @@ package util import ( "encoding/json" + "errors" "fmt" "net" "strconv" @@ -21,7 +22,13 @@ type RayCluster struct { // NewRayCluster creates a RayCluster. // func NewRayCluster(apiCluster *api.Cluster, clusterRuntime *api.ClusterRuntime, computeRuntime *api.ComputeRuntime) *RayCluster { -func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) *RayCluster { +func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.ComputeTemplate) (*RayCluster, error) { + // Build cluster spec + spec, err := buildRayClusterSpec(apiCluster.Version, apiCluster.Envs, apiCluster.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } + // Build cluster rayCluster := &rayalphaapi.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Name: apiCluster.Name, @@ -29,10 +36,10 @@ func NewRayCluster(apiCluster *api.Cluster, computeTemplateMap map[string]*api.C Labels: buildRayClusterLabels(apiCluster), Annotations: buildRayClusterAnnotations(apiCluster), }, - Spec: *buildRayClusterSpec(apiCluster.Version, apiCluster.Envs, apiCluster.ClusterSpec, computeTemplateMap), + Spec: *spec, } - return &RayCluster{rayCluster} + return &RayCluster{rayCluster}, nil } // Build cluster labels @@ -57,15 +64,18 @@ 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 { +func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpec *api.ClusterSpec, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayClusterSpec, error) { computeTemplate := computeTemplateMap[clusterSpec.HeadGroupSpec.ComputeTemplate] - headPodTemplate := buildHeadPodTemplate(imageVersion, envs, clusterSpec.HeadGroupSpec, computeTemplate) + headPodTemplate, err := buildHeadPodTemplate(imageVersion, envs, clusterSpec.HeadGroupSpec, computeTemplate) + if err != nil { + return nil, err + } headReplicas := int32(1) rayClusterSpec := &rayalphaapi.RayClusterSpec{ RayVersion: imageVersion, HeadGroupSpec: rayalphaapi.HeadGroupSpec{ ServiceType: v1.ServiceType(clusterSpec.HeadGroupSpec.ServiceType), - Template: headPodTemplate, + Template: *headPodTemplate, Replicas: &headReplicas, RayStartParams: clusterSpec.HeadGroupSpec.RayStartParams, }, @@ -79,7 +89,10 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] - workerPodTemplate := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) + workerPodTemplate, err := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) + if err != nil { + return nil, err + } minReplicas := spec.Replicas maxReplicas := spec.Replicas @@ -96,13 +109,13 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe MaxReplicas: intPointer(maxReplicas), Replicas: intPointer(spec.Replicas), RayStartParams: spec.RayStartParams, - Template: workerPodTemplate, + Template: *workerPodTemplate, } rayClusterSpec.WorkerGroupSpecs = append(rayClusterSpec.WorkerGroupSpecs, workerNodeSpec) } - return rayClusterSpec + return rayClusterSpec, nil } // Annotations common to both head and worker nodes @@ -114,7 +127,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 { +func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api.HeadGroupSpec, computeRuntime *api.ComputeTemplate) (*v1.PodTemplateSpec, error) { image := constructRayImage(RayClusterDefaultImageRepository, imageVersion) if len(spec.Image) != 0 { image = spec.Image @@ -126,7 +139,10 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api // build volume and volumeMounts volMounts := buildVolumeMounts(spec.Volumes) - vols := buildVols(spec.Volumes) + vols, err := buildVols(spec.Volumes) + if err != nil { + return nil, err + } podTemplateSpec := v1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -253,7 +269,7 @@ func buildHeadPodTemplate(imageVersion string, envs map[string]string, spec *api } } - return podTemplateSpec + return &podTemplateSpec, nil } // Convert Toleration operator from string @@ -281,7 +297,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 { +func buildWorkerPodTemplate(imageVersion string, envs map[string]string, 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) @@ -295,7 +311,10 @@ func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *a // build volume and volumeMounts volMounts := buildVolumeMounts(spec.Volumes) - vols := buildVols(spec.Volumes) + vols, err := buildVols(spec.Volumes) + if err != nil { + return nil, err + } podTemplateSpec := v1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -471,7 +490,7 @@ func buildWorkerPodTemplate(imageVersion string, envs map[string]string, spec *a } } - return podTemplateSpec + return &podTemplateSpec, nil } // Build Volume mounts @@ -506,7 +525,7 @@ func newHostPathType(pathType string) *v1.HostPathType { } // Build volumes -func buildVols(apiVolumes []*api.Volume) []v1.Volume { +func buildVols(apiVolumes []*api.Volume) ([]v1.Volume, error) { var vols []v1.Volume for _, rayVol := range apiVolumes { if rayVol.VolumeType == api.Volume_HOST_PATH { @@ -541,6 +560,16 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { vols = append(vols, vol) } if rayVol.VolumeType == api.Volume_EPHEMERAL { + // Make sure that at least the storage size is defined + if rayVol.Storage == "" { + // Storage size is not defined + return nil, errors.New("storage for ephemeral volume is empty") + } + // Ensure that storage size is formatted correctly + _, err := resource.ParseQuantity(rayVol.Storage) + if err != nil { + return nil, errors.New("storage for ephemeral volume is not specified correctly") + } vol := v1.Volume{ Name: rayVol.Name, VolumeSource: v1.VolumeSource{ @@ -590,7 +619,7 @@ func buildVols(apiVolumes []*api.Volume) []v1.Volume { } } - return vols + return vols, nil } // Init pointer diff --git a/apiserver/pkg/util/cluster_test.go b/apiserver/pkg/util/cluster_test.go index de321065b97..6df93be43c5 100644 --- a/apiserver/pkg/util/cluster_test.go +++ b/apiserver/pkg/util/cluster_test.go @@ -5,6 +5,7 @@ import ( "testing" api "github.com/ray-project/kuberay/proto/go_client" + "github.com/stretchr/testify/assert" v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -209,7 +210,8 @@ func TestBuildVolumes(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got := buildVols(tt.apiVolume) + got, err := buildVols(tt.apiVolume) + assert.Nil(t, err) if !reflect.DeepEqual(got, tt.expect) { t.Errorf("failed for %s ..., got %v, expected %v", tt.name, got, tt.expect) } @@ -268,7 +270,8 @@ func TestBuildVolumeMounts(t *testing.T) { } func TestBuildHeadPodTemplate(t *testing.T) { - podSpec := buildHeadPodTemplate("2.4", make(map[string]string), &headGroup, &template) + podSpec, err := buildHeadPodTemplate("2.4", make(map[string]string), &headGroup, &template) + assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { t.Errorf("failed to propagate service account") @@ -295,7 +298,8 @@ func TestBuildHeadPodTemplate(t *testing.T) { } func TestBuildRayCluster(t *testing.T) { - cluster := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + cluster, err := NewRayCluster(&rayCluster, map[string]*api.ComputeTemplate{"foo": &template}) + assert.Nil(t, err) if len(cluster.ObjectMeta.Annotations) != 1 { t.Errorf("failed to propagate annotations") } @@ -305,7 +309,8 @@ func TestBuildRayCluster(t *testing.T) { } func TestBuilWorkerPodTemplate(t *testing.T) { - podSpec := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) + podSpec, err := buildWorkerPodTemplate("2.4", make(map[string]string), &workerGroup, &template) + assert.Nil(t, err) if podSpec.Spec.ServiceAccountName != "account" { t.Errorf("failed to propagate service account") diff --git a/apiserver/pkg/util/job.go b/apiserver/pkg/util/job.go index 60dd20e55fa..b66ba10bb48 100644 --- a/apiserver/pkg/util/job.go +++ b/apiserver/pkg/util/job.go @@ -15,13 +15,7 @@ type RayJob struct { const rayJobDefaultVersion = "1.13" // NewRayJob creates a RayJob. -func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTemplate) *RayJob { - var clusterSpec *rayalphaapi.RayClusterSpec - - if apiJob.ClusterSpec != nil { - clusterSpec = buildRayClusterSpec(rayJobDefaultVersion, nil, apiJob.ClusterSpec, computeTemplateMap) - } - +func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTemplate) (*RayJob, error) { // transfer json to runtimeEnv encodedText := base64.StdEncoding.EncodeToString([]byte(apiJob.RuntimeEnv)) @@ -39,14 +33,18 @@ func NewRayJob(apiJob *api.RayJob, computeTemplateMap map[string]*api.ComputeTem ShutdownAfterJobFinishes: apiJob.ShutdownAfterJobFinishes, TTLSecondsAfterFinished: &apiJob.TtlSecondsAfterFinished, JobId: apiJob.JobId, - RayClusterSpec: clusterSpec, + RayClusterSpec: nil, ClusterSelector: apiJob.ClusterSelector, }, } - - return &RayJob{ - rayJob, + if apiJob.ClusterSpec != nil { + clusterSpec, err := buildRayClusterSpec(rayJobDefaultVersion, nil, apiJob.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } + rayJob.Spec.RayClusterSpec = clusterSpec } + return &RayJob{rayJob}, nil } func (j *RayJob) Get() *rayalphaapi.RayJob { diff --git a/apiserver/pkg/util/service.go b/apiserver/pkg/util/service.go index 0f3c813cc2b..ec0cd88ebe4 100644 --- a/apiserver/pkg/util/service.go +++ b/apiserver/pkg/util/service.go @@ -24,7 +24,13 @@ func (s *RayService) Get() *rayalphaapi.RayService { return s.RayService } -func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) *RayService { +func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) (*RayService, error) { + // Build the spec + spec, err := buildRayServiceSpec(apiService, computeTemplateMap) + if err != nil { + return nil, err + } + // Build Ray service rayService := &rayalphaapi.RayService{ ObjectMeta: metav1.ObjectMeta{ Name: apiService.Name, @@ -32,9 +38,9 @@ func NewRayService(apiService *api.RayService, computeTemplateMap map[string]*ap Labels: buildRayServiceLabels(apiService), Annotations: buildRayServiceAnnotations(apiService), }, - Spec: *buildRayServiceSpec(apiService, computeTemplateMap), + Spec: *spec, } - return &RayService{rayService} + return &RayService{rayService}, nil } func buildRayServiceLabels(apiService *api.RayService) map[string]string { @@ -52,7 +58,7 @@ func buildRayServiceAnnotations(apiService *api.RayService) map[string]string { return annotations } -func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) *rayalphaapi.RayServiceSpec { +func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[string]*api.ComputeTemplate) (*rayalphaapi.RayServiceSpec, error) { serveConfigSpecs := make([]rayalphaapi.ServeConfigSpec, 0) for _, serveConfig := range apiService.ServeDeploymentGraphSpec.ServeConfigs { serveConfigSpec := rayalphaapi.ServeConfigSpec{ @@ -74,7 +80,10 @@ func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[stri } serveConfigSpecs = append(serveConfigSpecs, serveConfigSpec) } - newRayClusterSpec := *buildRayClusterSpec(rayServiceDefaultVersion, nil, apiService.ClusterSpec, computeTemplateMap) + newRayClusterSpec, err := buildRayClusterSpec(rayServiceDefaultVersion, nil, apiService.ClusterSpec, computeTemplateMap) + if err != nil { + return nil, err + } newRayClusterSpec.HeadGroupSpec.Template.Spec.Containers[0].Ports = append(newRayClusterSpec.HeadGroupSpec.Template.Spec.Containers[0].Ports, v1.ContainerPort{ Name: defaultServePortName, ContainerPort: defaultServePort, @@ -85,8 +94,8 @@ func buildRayServiceSpec(apiService *api.RayService, computeTemplateMap map[stri RuntimeEnv: base64.StdEncoding.EncodeToString([]byte(apiService.ServeDeploymentGraphSpec.RuntimeEnv)), ServeConfigSpecs: serveConfigSpecs, }, - RayClusterSpec: newRayClusterSpec, - } + RayClusterSpec: *newRayClusterSpec, + }, nil } func UpdateRayServiceWorkerGroupSpecs(updateSpecs []*api.WorkerGroupUpdateSpec, workerGroupSpecs []rayalphaapi.WorkerGroupSpec) []rayalphaapi.WorkerGroupSpec { From 300844eb878d1ee893b8552999ba6eefe0024923 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:06:39 +0100 Subject: [PATCH 53/56] Fixed error handling --- apiserver/pkg/util/cluster.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/util/cluster.go b/apiserver/pkg/util/cluster.go index d77da86af78..c5046b85235 100755 --- a/apiserver/pkg/util/cluster.go +++ b/apiserver/pkg/util/cluster.go @@ -87,6 +87,7 @@ func buildRayClusterSpec(imageVersion string, envs map[string]string, clusterSpe rayClusterSpec.HeadGroupSpec.EnableIngress = &clusterSpec.HeadGroupSpec.EnableIngress } + // Build worker groups for _, spec := range clusterSpec.WorkerGroupSpec { computeTemplate = computeTemplateMap[spec.ComputeTemplate] workerPodTemplate, err := buildWorkerPodTemplate(imageVersion, envs, spec, computeTemplate) From 8434ea94c98477fa6bba3e9a2df3bb352d79e56b Mon Sep 17 00:00:00 2001 From: blublinsky Date: Fri, 25 Aug 2023 16:14:24 +0100 Subject: [PATCH 54/56] Fixed error handling --- apiserver/pkg/model/volumes_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/model/volumes_test.go b/apiserver/pkg/model/volumes_test.go index 42efea7b787..62a351c5130 100644 --- a/apiserver/pkg/model/volumes_test.go +++ b/apiserver/pkg/model/volumes_test.go @@ -15,6 +15,7 @@ var ( hostToContainer = v1.MountPropagationHostToContainer bidirectonal = v1.MountPropagationBidirectional ) + var podTemplateTest = v1.PodTemplateSpec{ Spec: v1.PodSpec{ Containers: []v1.Container{ From 806d66d576171baabda0a59bd73d92f6a356196c Mon Sep 17 00:00:00 2001 From: blublinsky Date: Sun, 27 Aug 2023 14:27:41 +0100 Subject: [PATCH 55/56] Fixed error handling --- apiserver/pkg/manager/resource_manager.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index a71483a3102..584b9efb61d 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -93,7 +93,7 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu // convert *api.Cluster to v1alpha1.RayCluster rayCluster, err := util.NewRayCluster(apiCluster, computeTemplateDict) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a Ray cluster for (%s/%s)", apiCluster.Namespace, apiCluster.Name) + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray cluster") } // set our own fields. @@ -221,7 +221,7 @@ func (r *ResourceManager) CreateJob(ctx context.Context, apiJob *api.RayJob) (*v // convert *api.Cluster to v1alpha1.RayCluster rayJob, err := util.NewRayJob(apiJob, computeTemplateMap) if err != nil { - return nil, util.NewInternalServerError(err, "Failed to create a Ray Job for (%s/%s)", apiJob.Namespace, apiJob.Name) + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray Job") } newRayJob, err := r.getRayJobClient(apiJob.Namespace).Create(ctx, rayJob.Get(), metav1.CreateOptions{}) @@ -311,7 +311,7 @@ func (r *ResourceManager) CreateService(ctx context.Context, apiService *api.Ray } rayService, err := util.NewRayService(apiService, computeTemplateDict) if err != nil { - return nil, err + return nil, util.NewInvalidInputErrorWithDetails(err, "Failed to create a Ray Service") } createdAt := r.clientManager.Time().Now().String() rayService.Annotations["ray.io/creation-timestamp"] = createdAt From ca1d7a344e077d2fdf5e33bdda4a5e6eeebdf242 Mon Sep 17 00:00:00 2001 From: blublinsky Date: Mon, 28 Aug 2023 08:45:52 +0100 Subject: [PATCH 56/56] Fixed error handling --- apiserver/pkg/manager/resource_manager.go | 1 + 1 file changed, 1 insertion(+) diff --git a/apiserver/pkg/manager/resource_manager.go b/apiserver/pkg/manager/resource_manager.go index 584b9efb61d..dfa5bb3b81e 100644 --- a/apiserver/pkg/manager/resource_manager.go +++ b/apiserver/pkg/manager/resource_manager.go @@ -108,6 +108,7 @@ func (r *ResourceManager) CreateCluster(ctx context.Context, apiCluster *api.Clu return newRayCluster, nil } +// Compute template func (r *ResourceManager) populateComputeTemplate(ctx context.Context, clusterSpec *api.ClusterSpec, nameSpace string) (map[string]*api.ComputeTemplate, error) { dict := map[string]*api.ComputeTemplate{} // populate head compute template