Skip to content

Commit

Permalink
Use dask config rather than daskernetes env variables (#52)
Browse files Browse the repository at this point in the history
  • Loading branch information
mrocklin committed Mar 10, 2018
1 parent a6fc441 commit 5ba08f7
Show file tree
Hide file tree
Showing 5 changed files with 10 additions and 38 deletions.
1 change: 0 additions & 1 deletion dask_kubernetes/__init__.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
from .config import config
from .core import KubeCluster
from .objects import make_pod_spec, make_pod_from_dict

Expand Down
19 changes: 0 additions & 19 deletions dask_kubernetes/config.py

This file was deleted.

12 changes: 5 additions & 7 deletions dask_kubernetes/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,19 +5,17 @@
import time
from urllib.parse import urlparse
import uuid
from weakref import finalize, ref
from weakref import finalize

try:
import yaml
except ImportError:
yaml = False
from tornado import gen
from tornado.ioloop import IOLoop

from distributed.deploy import LocalCluster, Cluster
from distributed.config import config
import kubernetes

from .config import config
from .objects import make_pod_from_dict, clean_pod_template

logger = logging.getLogger(__name__)
Expand Down Expand Up @@ -134,9 +132,9 @@ def __init__(
**kwargs
):
if pod_template is None:
if 'worker-template-path' in config:
if 'kubernetes-worker-template-path' in config:
import yaml
with open(config['worker-template-path']) as f:
with open(config['kubernetes-worker-template-path']) as f:
d = yaml.safe_load(f)
pod_template = make_pod_from_dict(d)
else:
Expand All @@ -158,7 +156,7 @@ def __init__(
namespace = _namespace_default()

if name is None:
worker_name = config.get('worker-name', 'dask-{user}-{uuid}')
worker_name = config.get('kubernetes-worker-name', 'dask-{user}-{uuid}')
name = worker_name.format(user=getpass.getuser(), uuid=str(uuid.uuid4())[:10], **os.environ)

self.pod_template = clean_pod_template(pod_template)
Expand Down
12 changes: 3 additions & 9 deletions dask_kubernetes/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@

import pytest
from distributed.config import set_config
from dask_kubernetes import KubeCluster, make_pod_spec, config
from dask_kubernetes import KubeCluster, make_pod_spec
from dask.distributed import Client, wait
from distributed.utils_test import loop # noqa: F401
from distributed.utils import tmpfile
Expand Down Expand Up @@ -98,12 +98,9 @@ def test_ipython_display(cluster):


def test_dask_worker_name_env_variable(pod_spec, loop, ns):
config['worker-name'] = 'foo-{USER}-{uuid}'
try:
with set_config(**{'kubernetes-worker-name': 'foo-{USER}-{uuid}'}):
with KubeCluster(pod_spec, loop=loop, namespace=ns) as cluster:
assert 'foo-' + getpass.getuser() in cluster.name
finally:
del config['worker-name']


def test_diagnostics_link_env_variable(pod_spec, loop, ns):
Expand Down Expand Up @@ -335,9 +332,6 @@ def test_automatic_startup(image_name, loop, ns):
with tmpfile(extension='yaml') as fn:
with open(fn, mode='w') as f:
yaml.dump(test_yaml, f)
config['worker-template-path'] = fn
try:
with set_config(**{'kubernetes-worker-template-path': fn}):
with KubeCluster(loop=loop, namespace=ns) as cluster:
assert cluster.pod_template.metadata.labels['foo'] == 'bar'
finally:
del config['worker-template-path']
4 changes: 2 additions & 2 deletions doc/source/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -100,8 +100,8 @@ There are a few special environment variables that affect dask-kubernetes behavi
instead of ``dask-{user}-{uuid}``. **Ensure you keep the ``uuid`` somewhere in
the template.**

Any other environment variable starting with ``DASK_KUBERNETES_`` will be placed in
the ``dask_kubernetes.config`` dictionary for general use.
Any other environment variable starting with ``DASK_`` will be placed in
the ``dask.distributed.config`` dictionary for general use.


Docker Images
Expand Down

0 comments on commit 5ba08f7

Please sign in to comment.