Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 1 addition & 2 deletions dask_kubernetes/classic/kubecluster.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import kubernetes_asyncio as kubernetes
from kubernetes_asyncio.client.rest import ApiException

from ..constants import KUBECLUSTER_WORKER_CONTAINER_NAME
from ..common.objects import (
make_pod_from_dict,
make_service_from_dict,
Expand Down Expand Up @@ -112,7 +111,7 @@ async def logs(self):
log = await self.core_api.read_namespaced_pod_log(
self._pod.metadata.name,
self.namespace,
container=KUBECLUSTER_WORKER_CONTAINER_NAME,
container=self.pod_template.spec.containers[0].name,
)
except ApiException as e:
if "waiting to start" in str(e):
Expand Down
28 changes: 18 additions & 10 deletions dask_kubernetes/classic/tests/test_async.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
from dask.utils import tmpfile
from distributed.utils_test import captured_logger

from dask_kubernetes.constants import KUBECLUSTER_WORKER_CONTAINER_NAME
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME

TEST_DIR = os.path.abspath(os.path.join(__file__, ".."))
CONFIG_DEMO = os.path.join(TEST_DIR, "config-demo.yaml")
Expand Down Expand Up @@ -217,7 +217,7 @@ async def test_pod_from_yaml(k8s_cluster, docker_image):
],
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -262,7 +262,7 @@ async def test_pod_expand_env_vars(k8s_cluster, docker_image):
],
"image": "${FOO_IMAGE}",
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -296,7 +296,7 @@ async def test_pod_template_dict(docker_image):
"command": None,
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -336,7 +336,7 @@ async def test_pod_template_minimal_dict(k8s_cluster, docker_image):
"command": None,
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
}
Expand All @@ -354,20 +354,28 @@ async def test_pod_template_minimal_dict(k8s_cluster, docker_image):
async def test_pod_template_from_conf(docker_image):
spec = {
"spec": {
"containers": [
{"name": KUBECLUSTER_WORKER_CONTAINER_NAME, "image": docker_image}
]
"containers": [{"name": KUBECLUSTER_CONTAINER_NAME, "image": docker_image}]
}
}

with dask.config.set({"kubernetes.worker-template": spec}):
async with KubeCluster(**cluster_kwargs) as cluster:
assert (
cluster.pod_template.spec.containers[0].name
== KUBECLUSTER_WORKER_CONTAINER_NAME
== KUBECLUSTER_CONTAINER_NAME
)


@pytest.mark.asyncio
async def test_pod_template_with_custom_container_name(docker_image):
container_name = "my-custom-container"
spec = {"spec": {"containers": [{"name": container_name, "image": docker_image}]}}

with dask.config.set({"kubernetes.worker-template": spec}):
async with KubeCluster(**cluster_kwargs) as cluster:
assert cluster.pod_template.spec.containers[0].name == container_name


@pytest.mark.asyncio
async def test_constructor_parameters(k8s_cluster, pod_spec):
env = {"FOO": "BAR", "A": 1}
Expand Down Expand Up @@ -567,7 +575,7 @@ async def test_automatic_startup(k8s_cluster, docker_image):
"1",
],
"image": docker_image,
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down
27 changes: 17 additions & 10 deletions dask_kubernetes/classic/tests/test_sync.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
from dask.utils import tmpfile

from dask_kubernetes import KubeCluster, make_pod_spec
from dask_kubernetes.constants import KUBECLUSTER_WORKER_CONTAINER_NAME
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME

TEST_DIR = os.path.abspath(os.path.join(__file__, ".."))
CONFIG_DEMO = os.path.join(TEST_DIR, "config-demo.yaml")
Expand Down Expand Up @@ -100,7 +100,7 @@ def dont_test_pod_template_yaml(docker_image, loop):
],
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -146,7 +146,7 @@ def test_pod_template_yaml_expand_env_vars(docker_image, loop):
],
"image": "${FOO_IMAGE}",
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -179,7 +179,7 @@ def test_pod_template_dict(docker_image, loop):
"command": None,
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down Expand Up @@ -218,7 +218,7 @@ def test_pod_template_minimal_dict(docker_image, loop):
"command": None,
"image": docker_image,
"imagePullPolicy": "IfNotPresent",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
}
Expand All @@ -235,20 +235,27 @@ def test_pod_template_minimal_dict(docker_image, loop):
def test_pod_template_from_conf(docker_image):
spec = {
"spec": {
"containers": [
{"name": KUBECLUSTER_WORKER_CONTAINER_NAME, "image": docker_image}
]
"containers": [{"name": KUBECLUSTER_CONTAINER_NAME, "image": docker_image}]
}
}

with dask.config.set({"kubernetes.worker-template": spec}):
with KubeCluster() as cluster:
assert (
cluster.pod_template.spec.containers[0].name
== KUBECLUSTER_WORKER_CONTAINER_NAME
== KUBECLUSTER_CONTAINER_NAME
)


def test_pod_template_with_custom_container_name(docker_image):
container_name = "my-custom-container"
spec = {"spec": {"containers": [{"name": container_name, "image": docker_image}]}}

with dask.config.set({"kubernetes.worker-template": spec}):
with KubeCluster() as cluster:
assert cluster.pod_template.spec.containers[0].name == container_name


def test_bad_args():
with pytest.raises(FileNotFoundError) as info:
KubeCluster("myfile.yaml")
Expand Down Expand Up @@ -318,7 +325,7 @@ def test_automatic_startup(docker_image):
"1",
],
"image": docker_image,
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
}
]
},
Expand Down
4 changes: 2 additions & 2 deletions dask_kubernetes/common/objects.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

from kubernetes.client.configuration import Configuration

from dask_kubernetes.constants import KUBECLUSTER_WORKER_CONTAINER_NAME
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME

_FakeResponse = namedtuple("_FakeResponse", ["data"])

Expand Down Expand Up @@ -193,7 +193,7 @@ def make_pod_spec(
restart_policy="Never",
containers=[
client.V1Container(
name=KUBECLUSTER_WORKER_CONTAINER_NAME,
name=KUBECLUSTER_CONTAINER_NAME,
image=image,
args=args,
env=[client.V1EnvVar(name=k, value=v) for k, v in env.items()],
Expand Down
4 changes: 2 additions & 2 deletions dask_kubernetes/common/tests/test_objects.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
from dask_kubernetes.constants import KUBECLUSTER_WORKER_CONTAINER_NAME
from dask_kubernetes.constants import KUBECLUSTER_CONTAINER_NAME
from dask_kubernetes.common.objects import make_pod_from_dict


Expand All @@ -16,7 +16,7 @@ def test_make_pod_from_dict():
"1",
],
"image": "image-name",
"name": KUBECLUSTER_WORKER_CONTAINER_NAME,
"name": KUBECLUSTER_CONTAINER_NAME,
"securityContext": {
"capabilities": {"add": ["SYS_ADMIN"]},
"privileged": True,
Expand Down
2 changes: 1 addition & 1 deletion dask_kubernetes/constants.py
Original file line number Diff line number Diff line change
@@ -1 +1 @@
KUBECLUSTER_WORKER_CONTAINER_NAME = "dask-worker"
KUBECLUSTER_CONTAINER_NAME = "dask-container"
2 changes: 2 additions & 0 deletions doc/source/testing.rst
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
Testing
=======

.. warning:: Tests are not working on aarch64 (Apple M1) at the moment due to an architecture incompatibility between ``pytest-kind-control-plane`` and the docker image built from ``ci/Dockerfile``, similar to `this GitHub issue <https://github.com/kubernetes-sigs/kind/issues/2402>`_.

Running the test suite for ``dask-kubernetes`` doesn't require an existing Kubernetes cluster but does require
`Docker <https://docs.docker.com/get-docker/>`_, `kubectl <https://kubernetes.io/docs/tasks/tools/#kubectl>`_ and `helm <https://helm.sh/docs/intro/install/>`_.

Expand Down