ray-project / kuberay

A toolkit to run Ray applications on Kubernetes
Apache License 2.0
1.14k stars 367 forks source link

[Bug] Operator does not remove workers #942

Closed birgerbr closed 1 year ago

birgerbr commented 1 year ago

Search before asking

KubeRay Component

ray-operator

What happened + What you expected to happen

The operator sometimes ends up in a bad state where it is not able to remove workers.

Logs that might be relevant:

monitor.log

2023-03-03 07:20:43,762 INFO node_provider.py:239 -- Listing pods for RayCluster ray in namespace default at pods resource version >= 4144232.
2023-03-03 07:20:43,816 INFO node_provider.py:257 -- Fetched pod data at resource version 4144232.
2023-03-03 07:20:43,820 INFO autoscaler.py:143 -- The autoscaler took 0.106 seconds to fetch the list of non-terminated nodes.
2023-03-03 07:20:43,820 WARNING node_provider.py:328 -- Waiting for operator to remove worker ray-worker-ray-n1-standard-8-ssd-0-gpu-1-9bf4k.
2023-03-03 07:20:43,820 INFO autoscaler.py:396 -- Backing off of autoscaler update. Will try again in 5 seconds.
2023-03-03 07:20:48,928 INFO node_provider.py:239 -- Listing pods for RayCluster ray in namespace default at pods resource version >= 4144304.
2023-03-03 07:20:48,983 INFO node_provider.py:257 -- Fetched pod data at resource version 4144304.
2023-03-03 07:20:48,989 INFO autoscaler.py:143 -- The autoscaler took 0.113 seconds to fetch the list of non-terminated nodes.
2023-03-03 07:20:48,989 WARNING node_provider.py:328 -- Waiting for operator to remove worker ray-worker-ray-n1-standard-8-ssd-0-gpu-1-9bf4k.
2023-03-03 07:20:48,990 INFO autoscaler.py:396 -- Backing off of autoscaler update. Will try again in 5 seconds.

operator logs

2023-03-03T08:11:49.945Z    INFO    controllers.RayCluster  reconciling RayCluster  {"cluster name": "ray"}
2023-03-03T08:11:49.945Z    INFO    controllers.RayCluster  reconcileServices   {"headService service found": "ray-head-svc"}
2023-03-03T08:11:49.946Z    INFO    controllers.RayCluster  reconcilePods   {"head pod found": "ray-head-bmgrb"}
2023-03-03T08:11:49.946Z    INFO    controllers.RayCluster  reconcilePods   {"head pod is up and running... checking workers": "ray-head-bmgrb"}
2023-03-03T08:11:49.946Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n1-standard-8-ssd-0-gpu-0"}
2023-03-03T08:11:49.946Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n1-standard-8-ssd-0-gpu-0"}
2023-03-03T08:11:49.947Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n1-standard-8-ssd-1-gpu-0"}
2023-03-03T08:11:49.947Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n1-standard-8-ssd-1-gpu-0"}
2023-03-03T08:11:49.947Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n1-highmem-8-ssd-1-gpu-0"}
2023-03-03T08:11:49.947Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n1-highmem-8-ssd-1-gpu-0"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-e2-highmem-16-ssd-0-gpu-0"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-e2-highmem-16-ssd-0-gpu-0"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n2-highmem-32-ssd-0-gpu-0"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n2-highmem-32-ssd-0-gpu-0"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n1-standard-8-ssd-0-gpu-1"}
2023-03-03T08:11:49.948Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n1-standard-8-ssd-0-gpu-1"}
2023-03-03T08:11:49.949Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-n1-standard-8-ssd-1-gpu-1"}
2023-03-03T08:11:49.949Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-n1-standard-8-ssd-1-gpu-1"}

Reproduction script

No reproduction script.

Anything else

This has happened only once, but we have only been using KubeRay for a week on our staging cluster.

Are you willing to submit a PR?

igorgad commented 1 year ago

+1 here It seems to happen when any pod exits with an error, at least with error 137 in my case.

harryge00 commented 1 year ago

@igorgad @birgerbr Do you have the raycluster yamls?

igorgad commented 1 year ago

Sharing just a small part of it

# For most use-cases, it makes sense to schedule one Ray pod per Kubernetes node.
# This is a configuration for an autoscaling RayCluster with 1 Ray head pod and 1 Ray worker pod,
# with capacity to scale up to 10 worker pods.
# Each pod requests 54 Gi memory and 14 CPU.
# Each pod can be scheduled on a virtual machine with roughly 64 Gi memory and 16 CPU.
# (AWS: m5.4xlarge, GCP: e2-standard-16, Azure: Standard_D5_v2)
# Optimal resource allocation will depend on your Kubernetes infrastructure and might
# require some experimentation.

# The Ray autoscaler and KubeRay operator scale Ray pod quantities.
# To achieve Kubernetes node autoscaling with this example, we recommend setting up an autoscaling node group/pool with
# - 2 nodes minimum for the Ray head pod and Ray worker pod
# - 11 nodes maximum to accommodate up to 10 Ray worker pods.
apiVersion: ray.io/v1alpha1
kind: RayCluster
metadata:
  labels:
    controller-tools.k8s.io: "1.0"
    # A unique identifier for the head node and workers of this cluster.
  name: lany-cluster
spec:
  # The version of Ray you are using. Make sure all Ray containers are running this version of Ray.
  rayVersion: '2.3.0'
  # If enableInTreeAutoscaling is true, the autoscaler sidecar will be added to the Ray head pod.
  # Ray autoscaler integration is supported only for Ray versions >= 1.11.0
  # Ray autoscaler integration is Beta with KubeRay >= 0.3.0 and Ray >= 2.0.0.
  enableInTreeAutoscaling: true
  # autoscalerOptions is an OPTIONAL field specifying configuration overrides for the Ray autoscaler.
  # The example configuration shown below below represents the DEFAULT values.
  # (You may delete autoscalerOptions if the defaults are suitable.)
  autoscalerOptions:
    # upscalingMode is "Default" or "Aggressive."
    # Conservative: Upscaling is rate-limited; the number of pending worker pods is at most the size of the Ray cluster.
    # Default: Upscaling is not rate-limited.
    # Aggressive: An alias for Default; upscaling is not rate-limited.
    upscalingMode: Conservative
    # idleTimeoutSeconds is the number of seconds to wait before scaling down a worker pod which is not using Ray resources.
    idleTimeoutSeconds: 500
    # image optionally overrides the autoscaler's container image.
    # If instance.spec.rayVersion is at least "2.0.0", the autoscaler will default to the same image as
    # the ray container by. For older Ray versions, the autoscaler will default to using the Ray 2.0.0 image.
    ## image: "my-repo/my-custom-autoscaler-image:tag"
    # imagePullPolicy optionally overrides the autoscaler container's image pull policy.
    imagePullPolicy: Always
    # resources specifies optional resource request and limit overrides for the autoscaler container.
    # For large Ray clusters, we recommend monitoring container resource usage to determine if overriding the defaults is required.
    resources:
      limits:
        cpu: "500m"
        memory: "512Mi"
      requests:
        cpu: "500m"
        memory: "512Mi"
    env:
      - name: AUTOSCALER_MAX_LAUNCH_BATCH
        value: '16'
  ######################headGroupSpec#################################
  # head group template and specs, (perhaps 'group' is not needed in the name)
  headGroupSpec:
    # Kubernetes Service Type, valid values are 'ClusterIP', 'NodePort' and 'LoadBalancer'
    serviceType: ClusterIP
    # logical group name, for this called head-group, also can be functional
    # pod type head or worker
    # rayNodeType: head # Not needed since it is under the headgroup
    # the following params are used to complete the ray start: ray start --head --block --port=6379 ...
    rayStartParams:
      # Flag "no-monitor" will be automatically set when autoscaling is enabled.
      dashboard-host: '0.0.0.0'
      block: 'true'
      # system-config: '"{\"worker_register_timeout_seconds\": 400}"'
      # num-cpus: '0'
      # num-gpus: '0'
      # num-cpus: '14' # can be auto-completed from the limits
      # Use `resources` to optionally specify custom resource annotations for the Ray node.
      # The value of `resources` is a string-integer mapping.
      # Currently, `resources` must be provided in the specific format demonstrated below:
      # resources: '"{\"Custom1\": 1, \"Custom2\": 5}"'
    #pod template
    template:
      spec:
        containers:
        # The Ray head container
        - name: ray-head
          image: custom/ray/image
          imagePullPolicy: Always
          env:
            - name: RAY_RUNTIME_ENV_TEMPORARY_REFERENCE_EXPIRATION_S
              value: '600'
            - name: RAY_memory_monitor_refresh_ms
              value: '0'
            - name: RAY_scheduler_report_pinned_bytes_only
              value: '1'
          # Optimal resource allocation will depend on your Kubernetes infrastructure and might
          # require some experimentation.
          # Setting requests=limits is recommended with Ray. K8s limits are used for Ray-internal
          # resource accounting. K8s requests are not used by Ray.
          resources:
            limits:
              cpu: 5
              memory: 25G
            requests:
              cpu: 5
              memory: 25G
          ports:
          - containerPort: 6379
            name: gcs
          - containerPort: 8265
            name: dashboard
          - containerPort: 10001
            name: client
          lifecycle:
            preStop:
              exec:
                command: ["/bin/sh","-c","ray stop"]
        nodeSelector:
          cloud.google.com/gke-nodepool: cpu

  # A100_SPOT NODES
  - replicas: 0
    minReplicas: 0
    maxReplicas: 16
    groupName: a100spot-group
    rayStartParams:
      block: 'true'
      num-gpus: '4'
      resources: '"{\"A100_SPOT\": 4}"'
    #pod template
    template:
      metadata:
        labels:
          key: value
        # annotations for pod
        annotations:
          key: value
      spec:
        initContainers:
        # the env var $RAY_IP is set by the operator if missing, with the value of the head service name
        - name: init-myservice
          image: busybox:1.28
          command: ['sh', '-c', "until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local; do echo waiting for myservice; sleep 2; done"]
        containers:
        - name: ray
          image: custom/ray/image
          env:
            - name: LD_LIBRARY_PATH
              value: '/usr/local/nvidia/lib64/'
            - name: RAY_RUNTIME_ENV_TEMPORARY_REFERENCE_EXPIRATION_S
              value: '600'
            - name: RAY_memory_monitor_refresh_ms
              value: '0'
            - name: RAY_scheduler_report_pinned_bytes_only
              value: '1'
          resources:
            limits:
              nvidia.com/gpu: 4
              cpu: 46
              memory: 340G
            requests:
              nvidia.com/gpu: 4
              cpu: 46
              memory: 340G
          lifecycle:
            preStop:
              exec:
                command: ["/bin/sh","-c","ray stop"]
        nodeSelector:
          cloud.google.com/gke-nodepool: a100spot
igorgad commented 1 year ago

I just verified that manually deleting pods with an error restores the operator state, allowing it to terminate pods again

birgerbr commented 1 year ago

I guess you do not need it anymore, but here is an example raycluster yaml:

apiVersion: ray.io/v1alpha1
kind: RayCluster
metadata:
  labels:
    controller-tools.k8s.io: '1.0'
  name: ray
spec:
  autoscalerOptions:
    env: []
    envFrom: []
    idleTimeoutSeconds: 60
    imagePullPolicy: Always
    resources:
      limits:
        cpu: 500m
        memory: 512Mi
      requests:
        cpu: 500m
        memory: 512Mi
    securityContext: {}
    upscalingMode: Conservative
  enableInTreeAutoscaling: true
  headGroupSpec:
    rayStartParams:
      block: 'true'
      dashboard-host: 0.0.0.0
      num-cpus: '0'
      object-store-memory: '1073741824'
      tracing-startup-hook: imerso.ray.util:setup_tracing
    serviceType: ClusterIP
    template:
      spec:
        containers:
        - env:
          - name: IMERSO_TEMP_FOLDER
            value: /scratch/imerso_tmp
          image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          imagePullPolicy: Always
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-head
          ports:
          - containerPort: 6379
            name: gcs
          - containerPort: 8265
            name: dashboard
          - containerPort: 10001
            name: client
          resources:
            limits:
              cpu: '1'
              memory: 4500Mi
            requests:
              cpu: '1'
              memory: 4500Mi
        nodeSelector:
          ray-head: 'true'
        tolerations:
        - effect: NoSchedule
          key: ray-head
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  rayVersion: 2.3.0
  workerGroupSpecs:
  - groupName: ray-n1-standard-8-ssd-0-gpu-0
    maxReplicas: 10
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 53687091200}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '7'
              memory: 25G
            requests:
              cpu: '7'
              memory: 25G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          ray-n1-standard-8-ssd-0-gpu-0: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n1-standard-8-ssd-0-gpu-0
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-n1-standard-8-ssd-1-gpu-0
    maxReplicas: 10
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 301721452544}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '7'
              ephemeral-storage: 281Gi
              memory: 25G
            requests:
              cpu: '7'
              ephemeral-storage: 200Gi
              memory: 25G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          cloud.google.com/gke-ephemeral-storage-local-ssd: 'true'
          ray-n1-standard-8-ssd-1-gpu-0: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n1-standard-8-ssd-1-gpu-0
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-n1-highmem-8-ssd-1-gpu-0
    maxReplicas: 5
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 301721452544}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '7'
              ephemeral-storage: 281Gi
              memory: 46G
            requests:
              cpu: '7'
              ephemeral-storage: 200Gi
              memory: 46G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          cloud.google.com/gke-ephemeral-storage-local-ssd: 'true'
          ray-n1-highmem-8-ssd-1-gpu-0: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n1-highmem-8-ssd-1-gpu-0
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-e2-highmem-16-ssd-0-gpu-0
    maxReplicas: 5
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 53687091200}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '15'
              memory: 120G
            requests:
              cpu: '15'
              memory: 120G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          ray-e2-highmem-16-ssd-0-gpu-0: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-e2-highmem-16-ssd-0-gpu-0
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-n2-highmem-32-ssd-0-gpu-0
    maxReplicas: 5
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 53687091200}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '31'
              memory: 120G
            requests:
              cpu: '31'
              memory: 120G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          ray-n2-highmem-32-ssd-0-gpu-0: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n2-highmem-32-ssd-0-gpu-0
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-n1-standard-8-ssd-0-gpu-1
    maxReplicas: 7
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 53687091200}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '7'
              memory: 25G
              nvidia.com/gpu: 1
            requests:
              cpu: '7'
              memory: 25G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          ray-n1-standard-8-ssd-0-gpu-1: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n1-standard-8-ssd-0-gpu-1
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
  - groupName: ray-n1-standard-8-ssd-1-gpu-1
    maxReplicas: 7
    minReplicas: 0
    rayStartParams:
      block: 'true'
      object-store-memory: '1073741824'
      resources: '"{\"local_storage\": 301721452544}"'
    replicas: 0
    template:
      spec:
        containers:
        - image: europe-west2-docker.pkg.dev/imerso-3dscanner-backend/europe-west2-docker/imerso-ray:foo
          lifecycle:
            preStop:
              exec:
                command:
                - /bin/sh
                - -c
                - ray stop
          name: ray-worker
          resources:
            limits:
              cpu: '7'
              ephemeral-storage: 281Gi
              memory: 25G
              nvidia.com/gpu: 1
            requests:
              cpu: '7'
              ephemeral-storage: 200Gi
              memory: 25G
          volumeMounts:
          - mountPath: /scratch
            name: scratch-volume
          - mountPath: /filestore
            name: filestore
        initContainers:
        - command:
          - sh
          - -c
          - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local;
            do echo waiting for K8s Service $RAY_IP; sleep 2; done
          image: busybox:1.28
          name: init
        nodeSelector:
          cloud.google.com/gke-ephemeral-storage-local-ssd: 'true'
          ray-n1-standard-8-ssd-1-gpu-1: 'true'
        serviceAccountName: imerso-services-staging
        tolerations:
        - effect: NoSchedule
          key: cloud.google.com/gke-preemptible
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: imerso-ray-worker
          operator: Equal
          value: 'true'
        - effect: NoSchedule
          key: ray-n1-standard-8-ssd-1-gpu-1
          operator: Equal
          value: 'true'
        volumes:
        - emptyDir: {}
          name: scratch-volume
        - name: filestore
          persistentVolumeClaim:
            claimName: filestore-claim
            readOnly: false
igorgad commented 1 year ago

Current workaround

while true; do kubectl get pods | grep Error | awk '{print $1}' | xargs kubectl delete pod; sleep 10; done

qizzzh commented 1 year ago

I hit the same issue. Is there a resolution?

kevin85421 commented 1 year ago

Here is PR #1139. I would appreciate any feedback! Thanks!