Description
Describe the issue:
When using a dask operator deployment in k8s with the role/rolebinding defined at the namespace level (rbac.cluster: false
), the creation of a daskclusters.kubernetes.dask.org
by a service account (dask
in the example) inside a pod within a namespace (myns
in the example) leads to the following error:
Short Error Message:
User "system:serviceaccount:myns:dask" cannot create resource "daskclusters" in API group "kubernetes.dask.org" in the namespace "default"
...
User "system:serviceaccount:myns:dask" cannot list resource "daskclusters" in API group "kubernetes.dask.org" in the namespace "default"
Full Stacktrace:
Traceback (most recent call last):
File "/usr/local/lib/python3.11/site-packages/kr8s/_api.py", line 168, in call_api
response.raise_for_status()
File "/usr/local/lib/python3.11/site-packages/httpx/_models.py", line 829, in raise_for_status
raise HTTPStatusError(message, request=request, response=self)
httpx.HTTPStatusError: Client error '403 Forbidden' for url 'https://.../apis/kubernetes.dask.org/v1/namespaces/default/daskclusters'
For more information check: https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/403
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/usr/local/lib/python3.11/site-packages/prefect/engine.py", line 42, in <module>
run_flow(flow, flow_run=flow_run)
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 1453, in run_flow
return run_flow_sync(**kwargs)
^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 1333, in run_flow_sync
return engine.state if return_type == "state" else engine.result()
^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 313, in result
raise self._raised
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 721, in run_context
yield self
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 1331, in run_flow_sync
engine.call_flow_fn()
File "/usr/local/lib/python3.11/site-packages/prefect/flow_engine.py", line 744, in call_flow_fn
result = call_with_parameters(self.flow.fn, self.parameters)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/prefect/utilities/callables.py", line 206, in call_with_parameters
return fn(*args, **kwargs)
^^^^^^^^^^^^^^^^^^^
File "/workdir/examples/cs/flows/misc/run_on_dask/flow.py", line 43, in run_on_dask
cluster = KubeCluster(
^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 282, in __init__
self.sync(self._start)
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 363, in sync
return sync(
^^^^^
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 439, in sync
raise error
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 413, in f
result = yield future
^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/tornado/gen.py", line 766, in run
value = future.result()
^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 322, in _start
await self._create_cluster()
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 361, in _create_cluster
await cluster.create()
File "/usr/local/lib/python3.11/site-packages/kr8s/_objects.py", line 320, in create
async with self.api.call_api(
File "/usr/local/lib/python3.11/contextlib.py", line 210, in __aenter__
return await anext(self.gen)
^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/kr8s/_api.py", line 186, in call_api
raise ServerError(
kr8s._exceptions.ServerError: daskclusters.kubernetes.dask.org is forbidden: User "system:serviceaccount:myns:dask" cannot create resource "daskclusters" in API group "kubernetes.dask.org" in the namespace "default"
Exception ignored in atexit callback: <function reap_clusters at 0x7afb85dafe20>
Traceback (most recent call last):
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 1033, in reap_clusters
asyncio.run(_reap_clusters())
File "/usr/local/lib/python3.11/asyncio/runners.py", line 190, in run
return runner.run(main)
^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/asyncio/runners.py", line 118, in run
return self._loop.run_until_complete(task)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/asyncio/base_events.py", line 654, in run_until_complete
return future.result()
^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 1031, in _reap_clusters
cluster.close(timeout=10)
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 700, in close
return self.sync(self._close, timeout=timeout)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 363, in sync
return sync(
^^^^^
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 439, in sync
raise error
File "/usr/local/lib/python3.11/site-packages/distributed/utils.py", line 413, in f
result = yield future
^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/tornado/gen.py", line 766, in run
value = future.result()
^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/dask_kubernetes/operator/kubecluster/kubecluster.py", line 706, in _close
cluster = await DaskCluster.get(self.name, namespace=self.namespace)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/kr8s/_objects.py", line 265, in get
raise e
File "/usr/local/lib/python3.11/site-packages/kr8s/_objects.py", line 255, in get
resources = await api.async_get(
^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/kr8s/_api.py", line 460, in async_get
async with self.async_get_kind(
File "/usr/local/lib/python3.11/contextlib.py", line 210, in __aenter__
return await anext(self.gen)
^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/kr8s/_api.py", line 396, in async_get_kind
async with self.call_api(
File "/usr/local/lib/python3.11/contextlib.py", line 210, in __aenter__
return await anext(self.gen)
^^^^^^^^^^^^^^^^^^^^^
File "/usr/local/lib/python3.11/site-packages/kr8s/_api.py", line 186, in call_api
raise ServerError(
kr8s._exceptions.ServerError: daskclusters.kubernetes.dask.org "test-cluster" is forbidden: User "system:serviceaccount:myns:dask" cannot list resource "daskclusters" in API group "kubernetes.dask.org" in the namespace "default"
Minimal Complete Verifiable Example:
Running this inside a pod:
from dask_kubernetes.operator.kubecluster.kubecluster import KubeCluster, make_cluster_spec
if __name__ == '__main__':
spec = make_cluster_spec(
name="test-cluster",
)
cluster = KubeCluster(
custom_cluster_spec=spec,
)
cluster.adapt(minimum=0, maximum=2)
Anything else we need to know?:
When running the exact same test with 2024.5.0
version, it works fine so I think this is due to an update made in the 2024.8.0
release since it does not work since this version.
To make this work with 2024.8.0
or later, I need to define the namespace
option when instantiating the KubeCluster
(but I don't know the ns in advance in my use case):
Environment:
- Dask version: 2024.11.2
- Python version: 3.11.9
- Operating System: ubuntu 22.04
- Install method (conda, pip, source): pip