ray-project / kuberay

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

[Bug] Autoscaler deployment fails - reports Forbidden access (403) to Kubernetes API #960

Closed akelloway closed 1 year ago

akelloway commented 1 year ago

Search before asking

KubeRay Component

ray-operator

What happened + What you expected to happen

I am unable to deploy an autoscaling ray cluster to AKS. Our Ops team have deployed the KubeRay operator. We have added the enableInTreeAutoscaling: true to the yaml cluster definition manifest but are unable to deploy successfully (autoscaler sidecar fails and head pod reports CrashLoopBackOff) - we are trying to deploy to a single namespace and our deployment service accounts only have access to that single namespace. We are quite new to both k8s, kuberay, and ray clusters in general so it is entirely possible we have made a mistake along the way. After some digging through the code (here and here) it seems like the service account the autoscaler is using (created by the ray deployment) in the single namespace does not have permission against the following -> https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespace/apm0005738-sb/rayclusters because it cannot do this at the cluster scope. I have accessed the pods directly and confirmed by curl command (having grabed the auth bearer token from /var/run/secrets/kubernetes.io/serviceaccount/token) - which reports:

{
  "kind": "Status",
  "apiVersion": "v1",
  "metadata": {},
  "status": "Failure",
  "message": "apis \"ray.io\" is forbidden: User \"system:serviceaccount:apm0005738-sb:forecasting-raycluster\" cannot get resource \"apis/v1alpha1\" in API group \"\" at the cluster scope",
  "reason": "Forbidden",
  "details": {
    "name": "ray.io",
    "kind": "apis"
  },
  "code": 403,
}

Reproduction script

I believe our Ops team deployed the KubeRay Operator to the namespace scope only but used their deployment service accounts to do so which does have permissions to deploy at the cluster scope which was needed for the CRDs. I believe the ops team ran the following to deploy the kuberay operator: helm install kuberay-operator kuberay/kuberay-operator --version 0.4.0 -namespace apm0005738-sb

The ray cluster manifest we are using is as follows and was deployed using service accounts with only permissions over the namespce:

apiVersion: ray.io/v1alpha1
kind: RayCluster
metadata:
  labels:
    controller-tools.k8s.io: "1.0"   
  name: forecasting-raycluster
  namespace: apm0005738-sb
spec:
  rayVersion: '2.3.0'
  enableInTreeAutoscaling: true
  autoscalerOptions:
    upscalingMode: Default
    idleTimeoutSeconds: 60
    imagePullPolicy: Always
    securityContext: {}
    env: []
    envFrom: []
    resources:
      limits:
        cpu: "500m"
        memory: "512Mi"
      requests:
        cpu: "500m"
        memory: "512Mi"
  # Ray head pod configuration
  headGroupSpec:
    # Kubernetes Service Type. This is an optional field, and the default value is ClusterIP.
    serviceType: NodePort # ClusterIP
    # the following params are used to complete the ray start: ray start --head --block --dashboard-host: '0.0.0.0' ...
    rayStartParams:
      dashboard-host: '0.0.0.0'
      block: 'true'
    # pod template
    template:
      metadata:
        # Custom labels. NOTE: To avoid conflicts with KubeRay operator, do not define custom labels start with `raycluster`.
        # Refer to https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/
        labels: {}
      spec:
        containers:
        - name: ray-head
          image: rayproject/ray:2.3.0 
          ports:
          - containerPort: 6379
            name: gcs
          - containerPort: 8265
            name: dashboard
          - containerPort: 10001
            name: client
          - containerPort: 8000
            name: serve
          - containerPort: 52365
            name: dashboard-agent                                   
          lifecycle:
            preStop:
              exec:
                command: ["/bin/sh","-c","ray stop"]
          volumeMounts:
            - mountPath: /tmp/ray
              name: ray-logs
          resources:
            limits:
              cpu: "4"
              memory: "8G"
            requests: # limits and requests should be equal
              cpu: "4"
              memory: "8G"
        volumes:
          - name: ray-logs
            emptyDir: {}
  workerGroupSpecs:
  # the pod replicas in this group typed worker
  - replicas: 1
    minReplicas: 1
    maxReplicas: 3 
    # logical group name, for this called small-group, also can be functional
    groupName: ray-worker-group
    # the following params are used to complete the ray start: ray start --block
    rayStartParams:
      block: 'true'
    #pod template
    template:
      spec:
        containers:
        - name: ray-worker
          image: rayproject/ray:2.3.0
          lifecycle:
            preStop:
              exec:
                command: ["/bin/sh","-c","ray stop"]
          # use volumeMounts.Optional.
          # Refer to https://kubernetes.io/docs/concepts/storage/volumes/
          volumeMounts:
            - mountPath: /tmp/ray
              name: ray-logs
          # The resource requests and limits in this config are too small for production!
          # For an example with more realistic resource configuration, see
          # ray-cluster.autoscaler.large.yaml.
          # It is better to use a few large Ray pod than many small ones.
          # For production, it is ideal to size each Ray pod to take up the
          # entire Kubernetes node on which it is scheduled.
          resources:
            limits:
              cpu: "4"
              memory: "8G"
            requests:
              cpu: "4"
              memory: "8G"
        initContainers:
        # the env var $RAY_IP is set by the operator if missing, with the value of the head service name
        - name: init
          image: busybox:1.28
          # Change the cluster postfix if you don't have a default setting
          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"]
          # command: ['sh', '-c', "until nslookup $RAY_IP.apm0005738-sb.svc.cluster.local; do echo waiting for K8s Service $RAY_IP; sleep 2; done"]          
          # Special Dow must supply container limits
          resources:
            limits:
              cpu: "1"
              memory: "1G"
            requests:
              cpu: "1"
              memory: "1G"
        # use volumes
        # Refer to https://kubernetes.io/docs/concepts/storage/volumes/
        volumes:
          - name: ray-logs
            emptyDir: {}

Anything else

This issue seems closely related to https://github.com/ray-project/kuberay/issues/924 but our issue is on first deployment rather than when attempting a cluster update/change so we felt it perhaps needed its own issue.

Are you willing to submit a PR?

DmitriGekhtman commented 1 year ago

If you could share a stacktrace from the autoscaler's logs, that would be great!

akelloway commented 1 year ago

I think this is what you are looking for ...

❯ kubectl -n apm0005738-sb logs pod/forecasting-raycluster-head-x7vlv -c autoscaler
E0315 09:20:58.096963   98231 memcache.go:255] couldn't get resource list for external.metrics.k8s.io/v1beta1: Got empty response for: external.metrics.k8s.io/v1beta1
The Ray head is ready. Starting the autoscaler.
2023-03-15 14:20:30,769 INFO monitor.py:167 -- session_name: session_2023-03-15_14-18-06_679719_1
2023-03-15 14:20:30,771 INFO monitor.py:198 -- Starting autoscaler metrics server on port 44217
2023-03-15 14:20:30,772 INFO monitor.py:218 -- Monitor: Started
2023-03-15 14:20:30,797 ERROR monitor.py:503 -- Error in monitor loop
Traceback (most recent call last):
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 703, in urlopen
    httplib_response = self._make_request(
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 386, in _make_request
    self._validate_conn(conn)
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 1042, in _validate_conn
    conn.connect()
  File "/usr/local/lib/python3.10/site-packages/urllib3/connection.py", line 414, in connect
    self.sock = ssl_wrap_socket(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/ssl_.py", line 449, in ssl_wrap_socket
    ssl_sock = _ssl_wrap_socket_impl(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/ssl_.py", line 493, in _ssl_wrap_socket_impl
    return ssl_context.wrap_socket(sock, server_hostname=server_hostname)
  File "/usr/local/lib/python3.10/ssl.py", line 513, in wrap_socket
    return self.sslsocket_class._create(
  File "/usr/local/lib/python3.10/ssl.py", line 1071, in _create
    self.do_handshake()
  File "/usr/local/lib/python3.10/ssl.py", line 1342, in do_handshake
    self._sslobj.do_handshake()
ssl.SSLCertVerificationError: [SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/usr/local/lib/python3.10/site-packages/requests/adapters.py", line 489, in send
    resp = conn.urlopen(
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 787, in urlopen
    retries = retries.increment(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/retry.py", line 592, in increment
    raise MaxRetryError(_pool, url, error or ResponseError(cause))
urllib3.exceptions.MaxRetryError: HTTPSConnectionPool(host='kubernetes.default', port=443): Max retries exceeded with url: /apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster (Caused by SSLError(SSLCertVerificationError(1, '[SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)')))

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/monitor.py", line 547, in run
    self._initialize_autoscaler()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/monitor.py", line 233, in _initialize_autoscaler
    self.autoscaler = StandardAutoscaler(
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 247, in __init__
    self.reset(errors_fatal=True)
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 1107, in reset
    raise e
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 1024, in reset
    new_config = self.config_reader()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 59, in __call__
    ray_cr = self._fetch_ray_cr_from_k8s_with_retries()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 71, in _fetch_ray_cr_from_k8s_with_retries
    return self._fetch_ray_cr_from_k8s()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 85, in _fetch_ray_cr_from_k8s
    result = requests.get(
  File "/usr/local/lib/python3.10/site-packages/requests/api.py", line 73, in get
    return request("get", url, params=params, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/api.py", line 59, in request
    return session.request(method=method, url=url, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/sessions.py", line 587, in request
    resp = self.send(prep, **send_kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/sessions.py", line 701, in send
    r = adapter.send(request, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/adapters.py", line 563, in send
    raise SSLError(e, request=request)
requests.exceptions.SSLError: HTTPSConnectionPool(host='kubernetes.default', port=443): Max retries exceeded with url: /apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster (Caused by SSLError(SSLCertVerificationError(1, '[SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)')))
Traceback (most recent call last):
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 703, in urlopen
    httplib_response = self._make_request(
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 386, in _make_request
    self._validate_conn(conn)
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 1042, in _validate_conn
    conn.connect()
  File "/usr/local/lib/python3.10/site-packages/urllib3/connection.py", line 414, in connect
    self.sock = ssl_wrap_socket(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/ssl_.py", line 449, in ssl_wrap_socket
    ssl_sock = _ssl_wrap_socket_impl(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/ssl_.py", line 493, in _ssl_wrap_socket_impl
    return ssl_context.wrap_socket(sock, server_hostname=server_hostname)
  File "/usr/local/lib/python3.10/ssl.py", line 513, in wrap_socket
    return self.sslsocket_class._create(
  File "/usr/local/lib/python3.10/ssl.py", line 1071, in _create
    self.do_handshake()
  File "/usr/local/lib/python3.10/ssl.py", line 1342, in do_handshake
    self._sslobj.do_handshake()
ssl.SSLCertVerificationError: [SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/usr/local/lib/python3.10/site-packages/requests/adapters.py", line 489, in send
    resp = conn.urlopen(
  File "/usr/local/lib/python3.10/site-packages/urllib3/connectionpool.py", line 787, in urlopen
    retries = retries.increment(
  File "/usr/local/lib/python3.10/site-packages/urllib3/util/retry.py", line 592, in increment
    raise MaxRetryError(_pool, url, error or ResponseError(cause))
urllib3.exceptions.MaxRetryError: HTTPSConnectionPool(host='kubernetes.default', port=443): Max retries exceeded with url: /apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster (Caused by SSLError(SSLCertVerificationError(1, '[SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)')))

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/usr/local/bin/ray", line 8, in <module>
    sys.exit(main())
  File "/usr/local/lib/python3.10/site-packages/ray/scripts/scripts.py", line 2422, in main
    return cli()
  File "/usr/local/lib/python3.10/site-packages/click/core.py", line 1130, in __call__
    return self.main(*args, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/click/core.py", line 1055, in main
    rv = self.invoke(ctx)
  File "/usr/local/lib/python3.10/site-packages/click/core.py", line 1657, in invoke
    return _process_result(sub_ctx.command.invoke(sub_ctx))
  File "/usr/local/lib/python3.10/site-packages/click/core.py", line 1404, in invoke
    return ctx.invoke(self.callback, **ctx.params)
  File "/usr/local/lib/python3.10/site-packages/click/core.py", line 760, in invoke
    return __callback(*args, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/ray/scripts/scripts.py", line 2168, in kuberay_autoscaler
    run_kuberay_autoscaler(cluster_name, cluster_namespace)
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/run_autoscaler.py", line 64, in run_kuberay_autoscaler
    ).run()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/monitor.py", line 547, in run
    self._initialize_autoscaler()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/monitor.py", line 233, in _initialize_autoscaler
    self.autoscaler = StandardAutoscaler(
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 247, in __init__
    self.reset(errors_fatal=True)
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 1107, in reset
    raise e
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/autoscaler.py", line 1024, in reset
    new_config = self.config_reader()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 59, in __call__
    ray_cr = self._fetch_ray_cr_from_k8s_with_retries()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 71, in _fetch_ray_cr_from_k8s_with_retries
    return self._fetch_ray_cr_from_k8s()
  File "/usr/local/lib/python3.10/site-packages/ray/autoscaler/_private/kuberay/autoscaling_config.py", line 85, in _fetch_ray_cr_from_k8s
    result = requests.get(
  File "/usr/local/lib/python3.10/site-packages/requests/api.py", line 73, in get
    return request("get", url, params=params, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/api.py", line 59, in request
    return session.request(method=method, url=url, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/sessions.py", line 587, in request
    resp = self.send(prep, **send_kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/sessions.py", line 701, in send
    r = adapter.send(request, **kwargs)
  File "/usr/local/lib/python3.10/site-packages/requests/adapters.py", line 563, in send
    raise SSLError(e, request=request)
requests.exceptions.SSLError: HTTPSConnectionPool(host='kubernetes.default', port=443): Max retries exceeded with url: /apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster (Caused by SSLError(SSLCertVerificationError(1, '[SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997)')))

Let me know if you need anything else. Thanks!

akelloway commented 1 year ago

FYI exec'ing into the head pod's ray-head container (autoscaler is dead of course) I can execute the following:

❯ kubectl -n apm0005738-sb exec --stdin --tty pod/forecasting-raycluster-head-x7vlv -- /bin/bash
E0315 09:30:36.720567   98599 memcache.go:255] couldn't get resource list for external.metrics.k8s.io/v1beta1: Got empty response for: external.metrics.k8s.io/v1beta1
Defaulted container "ray-head" out of: ray-head, autoscaler
root@forecasting-raycluster-head-x7vlv:/code# TOKEN=$(cat /var/run/secrets/kubernetes.io/serviceaccount/token)
root@forecasting-raycluster-head-x7vlv:/code# curl -H "Authorization: Bearer $TOKEN" --cacert /var/run/secrets/kubernetes.io/serviceaccount/ca.crt https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters
{
  "kind": "Status",
  "apiVersion": "v1",
  "metadata": {},
  "status": "Failure",
  "message": "apis \"ray.io\" is forbidden: User \"system:serviceaccount:apm0005738-sb:forecasting-raycluster\" cannot get resource \"apis/v1alpha1\" in API group \"\" at the cluster scope",
  "reason": "Forbidden",
  "details": {
    "name": "ray.io",
    "kind": "apis"
  },
  "code": 403
}
root@forecasting-raycluster-head-x7vlv:/code#
DmitriGekhtman commented 1 year ago

The autoscaler isn't granted "list" RBAC for RayCluster, but it is granted "get," IIRC.

What happens if you add the name of the Ray cluster in the curl command?https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/<your-ray-cluster's-name>

akelloway commented 1 year ago

@DmitriGekhtman It looks like it gives the same error

root@forecasting-raycluster-head-wxkkx:/code# curl -H "Authorization: Bearer $TOKEN" --cacert /var/run/secrets/kubernetes.io/serviceaccount/ca.crt https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster
{
  "kind": "Status",
  "apiVersion": "v1",
  "metadata": {},
  "status": "Failure",
  "message": "apis \"ray.io\" is forbidden: User \"system:serviceaccount:apm0005738-sb:forecasting-raycluster\" cannot get resource \"apis/v1alpha1\" in API group \"\" at the cluster scope",
  "reason": "Forbidden",
  "details": {
    "name": "ray.io",
    "kind": "apis"
  },
  "code": 403
DmitriGekhtman commented 1 year ago

hmm, what do you get if you run kubectl auth can-i get raycluster -n apm0005738-sb from the head container?

Are you able to fetch from K8s the service account, role, and role binding objects created for the autoscaler?

It might be a good idea to also check the operator's logs...

kevin85421 commented 1 year ago

@iycheng also takes a look at this issue.

akelloway commented 1 year ago

Operator logs when creating cluster

2023-03-16T18:01:27.057Z    INFO    controllers.RayCluster  reconciling RayCluster  {"cluster name": "forecasting-raycluster"}
2023-03-16T18:01:27.074Z    INFO    controllers.RayCluster  Pod ServiceAccount created successfully {"service account name": "forecasting-raycluster"}
2023-03-16T18:01:27.074Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created service account forecasting-raycluster"}
2023-03-16T18:01:27.125Z    INFO    controllers.RayCluster  Role created successfully   {"role name": "forecasting-raycluster"}
2023-03-16T18:01:27.125Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created role forecasting-raycluster"}
2023-03-16T18:01:27.167Z    INFO    controllers.RayCluster  RoleBinding created successfully    {"role binding name": "forecasting-raycluster"}
2023-03-16T18:01:27.167Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created role binding forecasting-raycluster"}
2023-03-16T18:01:27.241Z    INFO    controllers.RayCluster  Pod Service created successfully    {"service name": "forecasting-raycluster-head-svc"}
2023-03-16T18:01:27.241Z    INFO    controllers.RayCluster  reconcilePods   {"creating head pod for cluster": "forecasting-raycluster"}
2023-03-16T18:01:27.241Z    INFO    RayCluster-Controller   Setting pod namespaces  {"namespace": "apm0005738-sb"}
2023-03-16T18:01:27.241Z    INFO    RayCluster-Controller   Head pod container with index 0 identified as Ray container.
2023-03-16T18:01:27.241Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created service forecasting-raycluster-head-svc"}
2023-03-16T18:01:27.241Z    INFO    controllers.RayCluster  head pod labels {"labels": {"app.kubernetes.io/created-by":"kuberay-operator","app.kubernetes.io/name":"kuberay","ray.io/cluster":"forecasting-raycluster","ray.io/cluster-dashboard":"forecasting-raycluster-dashboard","ray.io/group":"headgroup","ray.io/identifier":"forecasting-raycluster-head","ray.io/is-ray-node":"yes","ray.io/node-type":"head"}}
2023-03-16T18:01:27.241Z    INFO    RayCluster-Controller   Head pod container with index 0 identified as Ray container.
2023-03-16T18:01:27.241Z    INFO    RayCluster-Controller   volume already mounted  {"volume": "ray-logs", "path": "/tmp/ray"}
2023-03-16T18:01:27.241Z    INFO    controllers.RayCluster  createHeadPod   {"head pod with name": "forecasting-raycluster-head-"}
2023-03-16T18:01:27.328Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-worker-group"}
2023-03-16T18:01:27.328Z    INFO    controllers.RayCluster  reconcilePods   {"add workers for group": "ray-worker-group"}
2023-03-16T18:01:27.328Z    INFO    controllers.RayCluster  reconcilePods   {"creating worker for group": "ray-worker-group", "index 0": "in total 1"}
2023-03-16T18:01:27.328Z    INFO    RayCluster-Controller   Setting pod namespaces  {"namespace": "apm0005738-sb"}
2023-03-16T18:01:27.328Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created head pod forecasting-raycluster-head-gbvmk"}
2023-03-16T18:01:27.328Z    INFO    RayCluster-Controller   Head pod container with index 0 identified as Ray container.
2023-03-16T18:01:27.388Z    INFO    controllers.RayCluster  Created pod {"Pod ": "forecasting-raycluster-worker-ray-worker-group-"}
2023-03-16T18:01:27.388Z    DEBUG   events  Normal  {"object": {"kind":"RayCluster","namespace":"apm0005738-sb","name":"forecasting-raycluster","uid":"6a4ac7ed-1afd-419d-9be3-dd7db9a78a5c","apiVersion":"ray.io/v1alpha1","resourceVersion":"100669163"}, "reason": "Created", "message": "Created worker pod "}
2023-03-16T18:01:27.405Z    INFO    controllers.RayCluster  reconciling RayCluster  {"cluster name": "forecasting-raycluster"}
2023-03-16T18:01:27.406Z    INFO    controllers.RayCluster  reconcileServices   {"headService service found": "forecasting-raycluster-head-svc"}
2023-03-16T18:01:27.406Z    INFO    controllers.RayCluster  reconcilePods   {"head pod found": "forecasting-raycluster-head-gbvmk"}
2023-03-16T18:01:27.406Z    INFO    controllers.RayCluster  reconcilePods   {"head pod is up and running... checking workers": "forecasting-raycluster-head-gbvmk"}
2023-03-16T18:01:27.406Z    INFO    controllers.RayCluster  reconcilePods   {"removing the pods in the scaleStrategy of": "ray-worker-group"}
2023-03-16T18:01:27.406Z    INFO    controllers.RayCluster  reconcilePods   {"all workers already exist for group": "ray-worker-group"}
2023-03-16T18:01:27.418Z    ERROR   controllers.RayCluster  Update status error {"cluster name": "forecasting-raycluster", "error": "Operation cannot be fulfilled on rayclusters.ray.io \"forecasting-raycluster\": the object has been modified; please apply your changes to the latest version and try again"}
github.com/ray-project/kuberay/ray-operator/controllers/ray.(*RayClusterReconciler).Reconcile
    /workspace/controllers/ray/raycluster_controller.go:102
sigs.k8s.io/controller-runtime/pkg/internal/controller.(*Controller).Reconcile
    /go/pkg/mod/sigs.k8s.io/controller-runtime@v0.11.1/pkg/internal/controller/controller.go:114
sigs.k8s.io/controller-runtime/pkg/internal/controller.(*Controller).reconcileHandler
    /go/pkg/mod/sigs.k8s.io/controller-runtime@v0.11.1/pkg/internal/controller/controller.go:311
sigs.k8s.io/controller-runtime/pkg/internal/controller.(*Controller).processNextWorkItem
    /go/pkg/mod/sigs.k8s.io/controller-runtime@v0.11.1/pkg/internal/controller/controller.go:266
sigs.k8s.io/controller-runtime/pkg/internal/controller.(*Controller).Start.func2.2
    /go/pkg/mod/sigs.k8s.io/controller-runtime@v0.11.1/pkg/internal/controller/controller.go:227
2023-03-16T18:02:10.621Z    INFO    controllers.RayCluster  Read request instance not found error!  {"name": "apm0004419-sb/deployment-restart-job-dev-27983091-wxsv4.174cf4ef75fffe67"}
2023-03-16T18:02:10.621Z    INFO    controllers.RayCluster  Read request instance not found error!  {"name": "apm0004419-sb/deployment-restart-job-dev-27983101-jx2fk.174cf4f0aa98079e"}
akelloway commented 1 year ago

@DmitriGekhtman - looks like kubectl is not installed on the head container.

❯ kubectl -n apm0005738-sb exec --stdin --tty pod/forecasting-raycluster-head-gbvmk -- /bin/bash
E0316 13:07:14.166653   38902 memcache.go:255] couldn't get resource list for external.metrics.k8s.io/v1beta1: Got empty response for: external.metrics.k8s.io/v1beta1
Defaulted container "ray-head" out of: ray-head, autoscaler
root@forecasting-raycluster-head-gbvmk:/code# kubectl auth can-i get raycluster -n apm0005738-sb
bash: kubectl: command not found
root@forecasting-raycluster-head-gbvmk:/code#
DmitriGekhtman commented 1 year ago

operator logs don't show anything out of the ordinary -- the role, service account, and role binding were created without issue

if your networking setup permits it, you could try installing kubectl after exec-ing in

the thing we'd want to troubleshoot is why the role, sa, and role binding generated by the operator are not sufficient to allow the autoscaler to fetch the raycluster object.

ssl.SSLCertVerificationError: [SSL: CERTIFICATE_VERIFY_FAILED] certificate verify failed: unable to get local issuer certificate (_ssl.c:997) might suggest a different kind of error? This doesn't appear related to RBAC. What would cause this error?

DmitriGekhtman commented 1 year ago

Another way to frame this -- confirm that you don't see the issue if you deploy everything on Minikube, then figure out what the difference is between the minikube environment and your AKS environment.

akelloway commented 1 year ago

@DmitriGekhtman - quick update. I did install kubectl on the head node and ran the kubectl auth can-i get raycluster -n apm0005738-sb and the answer is yes. Additionally, I did spin up a Minikube and install both kuberay-operator and the ray cluster inside the same namespace but everything worked as expected. I'll circle back with my Ops team to see if any custom settings are applies to the cluster which might be preventing/causing that SSL error.

Thanks again.

akelloway commented 1 year ago

new strange update (from within the ray head node):

❯  kubectl -n apm0005738-sb exec --stdin --tty pod/forecasting-raycluster-head-r5pll -- /bin/bash
Defaulted container "ray-head" out of: ray-head, autoscaler
root@forecasting-raycluster-head-r5pll:/code# kubectl auth can-i get raycluster -n apm0005738-sb
yes
root@forecasting-raycluster-head-r5pll:/code# kubectl get raycluster -n apm0005738-sb
Error from server (Forbidden): rayclusters.ray.io is forbidden: User "system:serviceaccount:apm0005738-sb:forecasting-raycluster" cannot list resource "rayclusters" in API group "ray.io" in the namespace "apm0005738-sb"
root@forecasting-raycluster-head-r5pll:/code#

So it is saying "auth can-i get raycluster" -> yes but when I actually try and get the raycluster it says I am forbidden?

akelloway commented 1 year ago

as it turns out I was trying an incorrect curl command above and the correct one is here: curl -H "Authorization: Bearer $TOKEN" --cacert /var/run/secrets/kubernetes.io/serviceaccount/ca.crt https://kubernetes.default:443/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster

Previously, I had /api/v1/apis/ray.io/v1alpha1/... but that was wrong and I needed /apis/ray.io/v1alpha1/.... Querying the correct endpoint does return what looks like the correct data (ray cluster manifest)

DmitriGekhtman commented 1 year ago

Iiinteresting. I wonder what the difference is between the curl command and whatever the code is doing. @pcmoritz might have some vague recollection of how these requests are formatted

akelloway commented 1 year ago

I believe I have figured out the issue!

I was not using image: rayproject/ray:2.3.0 but rather a custom image with our required dependencies (including ray). This custom image worked without issue when running against a non-autoscaler cluster but changing to the default image: rayproject/ray:2.3.0 in an auto-scaler cluster everything worked as expected.

Seems like the best path forward is to just start from ray provided base images.

Thanks for all the help.

kevin85421 commented 1 year ago

@DmitriGekhtman It looks like it gives the same error

root@forecasting-raycluster-head-wxkkx:/code# curl -H "Authorization: Bearer $TOKEN" --cacert /var/run/secrets/kubernetes.io/serviceaccount/ca.crt https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster
{
  "kind": "Status",
  "apiVersion": "v1",
  "metadata": {},
  "status": "Failure",
  "message": "apis \"ray.io\" is forbidden: User \"system:serviceaccount:apm0005738-sb:forecasting-raycluster\" cannot get resource \"apis/v1alpha1\" in API group \"\" at the cluster scope",
  "reason": "Forbidden",
  "details": {
    "name": "ray.io",
    "kind": "apis"
  },
  "code": 403

Read this issue today. Take a note to correct the curl command. We should change

https://kubernetes.default:443/api/v1/apis/ray.io/v1alpha1/namespaces/apm0005738-sb/rayclusters/forecasting-raycluster

to (remove api/v1/)

https://kubernetes.default:443/apis/ray.io/v1alpha1/namespaces/$RAYCLUSTER_NAMESPACE/rayclusters/$RAYCLUSTER_CR_NAME